forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathprewrite.go
289 lines (262 loc) · 9.5 KB
/
prewrite.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package tikv
import (
"encoding/hex"
"math"
"sync/atomic"
"time"
"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/store/tikv/config"
"github.com/pingcap/tidb/store/tikv/logutil"
"github.com/pingcap/tidb/store/tikv/metrics"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)
type actionPrewrite struct{}
var _ twoPhaseCommitAction = actionPrewrite{}
func (actionPrewrite) String() string {
return "prewrite"
}
func (actionPrewrite) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return metrics.TxnRegionsNumHistogramPrewrite
}
func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize uint64) *tikvrpc.Request {
m := batch.mutations
mutations := make([]*pb.Mutation, m.Len())
isPessimisticLock := make([]bool, m.Len())
for i := 0; i < m.Len(); i++ {
mutations[i] = &pb.Mutation{
Op: m.GetOp(i),
Key: m.GetKey(i),
Value: m.GetValue(i),
}
isPessimisticLock[i] = m.IsPessimisticLock(i)
}
c.mu.Lock()
minCommitTS := c.minCommitTS
c.mu.Unlock()
if c.forUpdateTS > 0 && c.forUpdateTS >= minCommitTS {
minCommitTS = c.forUpdateTS + 1
} else if c.startTS >= minCommitTS {
minCommitTS = c.startTS + 1
}
failpoint.Inject("mockZeroCommitTS", func(val failpoint.Value) {
// Should be val.(uint64) but failpoint doesn't support that.
if tmp, ok := val.(int); ok && uint64(tmp) == c.startTS {
minCommitTS = 0
}
})
ttl := c.lockTTL
if c.sessionID > 0 {
failpoint.Inject("twoPCShortLockTTL", func() {
ttl = 1
keys := make([]string, 0, len(mutations))
for _, m := range mutations {
keys = append(keys, hex.EncodeToString(m.Key))
}
logutil.BgLogger().Info("[failpoint] injected lock ttl = 1 on prewrite",
zap.Uint64("txnStartTS", c.startTS), zap.Strings("keys", keys))
})
}
req := &pb.PrewriteRequest{
Mutations: mutations,
PrimaryLock: c.primary(),
StartVersion: c.startTS,
LockTtl: ttl,
IsPessimisticLock: isPessimisticLock,
ForUpdateTs: c.forUpdateTS,
TxnSize: txnSize,
MinCommitTs: minCommitTS,
MaxCommitTs: c.maxCommitTS,
}
failpoint.Inject("invalidMaxCommitTS", func() {
if req.MaxCommitTs > 0 {
req.MaxCommitTs = minCommitTS - 1
}
})
if c.isAsyncCommit() {
if batch.isPrimary {
req.Secondaries = c.asyncSecondaries()
}
req.UseAsyncCommit = true
}
if c.isOnePC() {
req.TryOnePc = true
}
return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog})
}
func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
// WARNING: This function only tries to send a single request to a single region, so it don't
// need to unset the `useOnePC` flag when it fails. A special case is that when TiKV returns
// regionErr, it's uncertain if the request will be splitted into multiple and sent to multiple
// regions. It invokes `prewriteMutations` recursively here, and the number of batches will be
// checked there.
if c.sessionID > 0 {
failpoint.Inject("prewritePrimaryFail", func() {
if batch.isPrimary {
// Delay to avoid cancelling other normally ongoing prewrite requests.
time.Sleep(time.Millisecond * 50)
logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting primary batch",
zap.Uint64("txnStartTS", c.startTS))
failpoint.Return(errors.New("injected error on prewriting primary batch"))
}
})
}
if c.sessionID > 0 {
failpoint.Inject("prewriteSecondaryFail", func() {
if !batch.isPrimary {
// Delay to avoid cancelling other normally ongoing prewrite requests.
time.Sleep(time.Millisecond * 50)
logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting secondary batch",
zap.Uint64("txnStartTS", c.startTS))
failpoint.Return(errors.New("injected error on prewriting secondary batch"))
}
})
}
txnSize := uint64(c.regionTxnSize[batch.region.id])
// When we retry because of a region miss, we don't know the transaction size. We set the transaction size here
// to MaxUint64 to avoid unexpected "resolve lock lite".
if len(bo.errors) > 0 {
txnSize = math.MaxUint64
}
req := c.buildPrewriteRequest(batch, txnSize)
for {
sender := NewRegionRequestSender(c.store.regionCache, c.store.client)
resp, err := sender.SendReq(bo, req, batch.region, readTimeoutShort)
// If we fail to receive response for async commit prewrite, it will be undetermined whether this
// transaction has been successfully committed.
// If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined
// errors.
if (c.isAsyncCommit() || c.isOnePC()) && sender.rpcError != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 {
c.setUndeterminedErr(errors.Trace(sender.rpcError))
}
if err != nil {
return errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return errors.Trace(err)
}
err = c.prewriteMutations(bo, batch.mutations)
return errors.Trace(err)
}
if resp.Resp == nil {
return errors.Trace(ErrBodyMissing)
}
prewriteResp := resp.Resp.(*pb.PrewriteResponse)
keyErrs := prewriteResp.GetErrors()
if len(keyErrs) == 0 {
if batch.isPrimary {
// After writing the primary key, if the size of the transaction is larger than 32M,
// start the ttlManager. The ttlManager will be closed in tikvTxn.Commit().
// In this case 1PC is not expected to be used, but still check it for safety.
if int64(c.txnSize) > config.GetGlobalConfig().TiKVClient.TTLRefreshedTxnSize &&
prewriteResp.OnePcCommitTs == 0 {
c.run(c, nil)
}
}
if c.isOnePC() {
if prewriteResp.OnePcCommitTs == 0 {
if prewriteResp.MinCommitTs != 0 {
return errors.Trace(errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc"))
}
logutil.Logger(bo.ctx).Warn("1pc failed and fallbacks to normal commit procedure",
zap.Uint64("startTS", c.startTS))
metrics.OnePCTxnCounterFallback.Inc()
c.setOnePC(false)
c.setAsyncCommit(false)
} else {
// For 1PC, there's no racing to access to access `onePCCommmitTS` so it's safe
// not to lock the mutex.
if c.onePCCommitTS != 0 {
logutil.Logger(bo.ctx).Fatal("one pc happened multiple times",
zap.Uint64("startTS", c.startTS))
}
c.onePCCommitTS = prewriteResp.OnePcCommitTs
}
return nil
} else if prewriteResp.OnePcCommitTs != 0 {
logutil.Logger(bo.ctx).Fatal("tikv committed a non-1pc transaction with 1pc protocol",
zap.Uint64("startTS", c.startTS))
}
if c.isAsyncCommit() {
// 0 if the min_commit_ts is not ready or any other reason that async
// commit cannot proceed. The client can then fallback to normal way to
// continue committing the transaction if prewrite are all finished.
if prewriteResp.MinCommitTs == 0 {
if c.testingKnobs.noFallBack {
return nil
}
logutil.Logger(bo.ctx).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+
"fallback to normal path", zap.Uint64("startTS", c.startTS))
c.setAsyncCommit(false)
} else {
c.mu.Lock()
if prewriteResp.MinCommitTs > c.minCommitTS {
c.minCommitTS = prewriteResp.MinCommitTs
}
c.mu.Unlock()
}
}
return nil
}
var locks []*Lock
for _, keyErr := range keyErrs {
// Check already exists error
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
key := alreadyExist.GetKey()
return c.extractKeyExistsErr(key)
}
// Extract lock from key error
lock, err1 := extractLockFromKeyErr(keyErr)
if err1 != nil {
return errors.Trace(err1)
}
logutil.BgLogger().Info("prewrite encounters lock",
zap.Uint64("session", c.sessionID),
zap.Stringer("lock", lock))
locks = append(locks, lock)
}
start := time.Now()
msBeforeExpired, err := c.store.lockResolver.resolveLocksForWrite(bo, c.startTS, locks)
if err != nil {
return errors.Trace(err)
}
atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start)))
if msBeforeExpired > 0 {
err = bo.BackoffWithMaxSleep(BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks)))
if err != nil {
return errors.Trace(err)
}
}
}
}
func (c *twoPhaseCommitter) prewriteMutations(bo *Backoffer, mutations CommitterMutations) error {
if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("twoPhaseCommitter.prewriteMutations", opentracing.ChildOf(span.Context()))
defer span1.Finish()
bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1)
}
// `doActionOnMutations` will unset `useOnePC` if the mutations is splitted into multiple batches.
return c.doActionOnMutations(bo, actionPrewrite{}, mutations)
}