forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
txn.go
273 lines (231 loc) · 6.85 KB
/
txn.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
// Copyright 2016 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 (
"fmt"
"sync"
"time"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
"golang.org/x/net/context"
)
var (
_ kv.Transaction = (*tikvTxn)(nil)
)
// tikvTxn implements kv.Transaction.
type tikvTxn struct {
snapshot *tikvSnapshot
us kv.UnionStore
store *tikvStore // for connection to region.
startTS uint64
startTime time.Time // Monotonic timestamp for recording txn time consuming.
commitTS uint64
valid bool
lockKeys [][]byte
mu sync.Mutex // For thread-safe LockKeys function.
dirty bool
setCnt int64
vars *kv.Variables
}
func newTiKVTxn(store *tikvStore) (*tikvTxn, error) {
bo := NewBackoffer(context.Background(), tsoMaxBackoff)
startTS, err := store.getTimestampWithRetry(bo)
if err != nil {
return nil, errors.Trace(err)
}
return newTikvTxnWithStartTS(store, startTS)
}
// newTikvTxnWithStartTS creates a txn with startTS.
func newTikvTxnWithStartTS(store *tikvStore, startTS uint64) (*tikvTxn, error) {
ver := kv.NewVersion(startTS)
snapshot := newTiKVSnapshot(store, ver)
return &tikvTxn{
snapshot: snapshot,
us: kv.NewUnionStore(snapshot),
store: store,
startTS: startTS,
startTime: time.Now(),
valid: true,
vars: kv.DefaultVars,
}, nil
}
func (txn *tikvTxn) SetVars(vars *kv.Variables) {
txn.vars = vars
txn.snapshot.vars = vars
}
// SetMemBufCap sets the transaction's MemBuffer capability, to reduce memory allocations.
func (txn *tikvTxn) SetCap(cap int) {
txn.us.SetCap(cap)
}
// Reset reset tikvTxn's membuf.
func (txn *tikvTxn) Reset() {
txn.us.Reset()
}
// Get implements transaction interface.
func (txn *tikvTxn) Get(k kv.Key) ([]byte, error) {
metrics.TiKVTxnCmdCounter.WithLabelValues("get").Inc()
start := time.Now()
defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }()
ret, err := txn.us.Get(k)
if err != nil {
return nil, errors.Trace(err)
}
err = txn.store.CheckVisibility(txn.startTS)
if err != nil {
return nil, errors.Trace(err)
}
return ret, nil
}
func (txn *tikvTxn) Set(k kv.Key, v []byte) error {
txn.setCnt++
txn.dirty = true
return txn.us.Set(k, v)
}
func (txn *tikvTxn) String() string {
return fmt.Sprintf("%d", txn.StartTS())
}
func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) {
metrics.TiKVTxnCmdCounter.WithLabelValues("seek").Inc()
start := time.Now()
defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("seek").Observe(time.Since(start).Seconds()) }()
return txn.us.Iter(k, upperBound)
}
// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k.
func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) {
metrics.TiKVTxnCmdCounter.WithLabelValues("seek_reverse").Inc()
start := time.Now()
defer func() {
metrics.TiKVTxnCmdHistogram.WithLabelValues("seek_reverse").Observe(time.Since(start).Seconds())
}()
return txn.us.IterReverse(k)
}
func (txn *tikvTxn) Delete(k kv.Key) error {
metrics.TiKVTxnCmdCounter.WithLabelValues("delete").Inc()
txn.dirty = true
return txn.us.Delete(k)
}
func (txn *tikvTxn) SetOption(opt kv.Option, val interface{}) {
txn.us.SetOption(opt, val)
switch opt {
case kv.Priority:
txn.snapshot.priority = kvPriorityToCommandPri(val.(int))
case kv.NotFillCache:
txn.snapshot.notFillCache = val.(bool)
case kv.SyncLog:
txn.snapshot.syncLog = val.(bool)
case kv.KeyOnly:
txn.snapshot.keyOnly = val.(bool)
}
}
func (txn *tikvTxn) DelOption(opt kv.Option) {
txn.us.DelOption(opt)
}
func (txn *tikvTxn) Commit(ctx context.Context) error {
if !txn.valid {
return kv.ErrInvalidTxn
}
defer txn.close()
metrics.TiKVTxnCmdCounter.WithLabelValues("set").Add(float64(txn.setCnt))
metrics.TiKVTxnCmdCounter.WithLabelValues("commit").Inc()
start := time.Now()
defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("commit").Observe(time.Since(start).Seconds()) }()
if err := txn.us.CheckLazyConditionPairs(); err != nil {
return errors.Trace(err)
}
// connID is used for log.
var connID uint64
val := ctx.Value(sessionctx.ConnID)
if val != nil {
connID = val.(uint64)
}
committer, err := newTwoPhaseCommitter(txn, connID)
if err != nil || committer == nil {
return errors.Trace(err)
}
// latches disabled
if txn.store.txnLatches == nil {
err = committer.executeAndWriteFinishBinlog(ctx)
log.Debug("[kv]", connID, " txnLatches disabled, 2pc directly:", err)
return errors.Trace(err)
}
// latches enabled
var bypassLatch bool
if option := txn.us.GetOption(kv.BypassLatch); option != nil {
bypassLatch = option.(bool)
}
// When bypassLatch flag is true, commit directly.
if bypassLatch {
err = committer.executeAndWriteFinishBinlog(ctx)
log.Debug("[kv]", connID, " txnLatches enabled while txn not retryable, 2pc directly:", err)
return errors.Trace(err)
}
// for transactions which need to acquire latches
lock := txn.store.txnLatches.Lock(committer.startTS, committer.keys)
defer txn.store.txnLatches.UnLock(lock)
if lock.IsStale() {
err = errors.Errorf("startTS %d is stale", txn.startTS)
return errors.Annotate(err, txnRetryableMark)
}
err = committer.executeAndWriteFinishBinlog(ctx)
if err == nil {
lock.SetCommitTS(committer.commitTS)
}
log.Debug("[kv]", connID, " txnLatches enabled while txn retryable:", err)
return errors.Trace(err)
}
func (txn *tikvTxn) close() {
txn.valid = false
}
func (txn *tikvTxn) Rollback() error {
if !txn.valid {
return kv.ErrInvalidTxn
}
txn.close()
log.Debugf("[kv] Rollback txn %d", txn.StartTS())
metrics.TiKVTxnCmdCounter.WithLabelValues("rollback").Inc()
return nil
}
func (txn *tikvTxn) LockKeys(keys ...kv.Key) error {
metrics.TiKVTxnCmdCounter.WithLabelValues("lock_keys").Inc()
txn.mu.Lock()
for _, key := range keys {
txn.lockKeys = append(txn.lockKeys, key)
}
txn.dirty = true
txn.mu.Unlock()
return nil
}
func (txn *tikvTxn) IsReadOnly() bool {
return !txn.dirty
}
func (txn *tikvTxn) StartTS() uint64 {
return txn.startTS
}
func (txn *tikvTxn) Valid() bool {
return txn.valid
}
func (txn *tikvTxn) Len() int {
return txn.us.Len()
}
func (txn *tikvTxn) Size() int {
return txn.us.Size()
}
func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer {
return txn.us.GetMemBuffer()
}
func (txn *tikvTxn) GetSnapshot() kv.Snapshot {
return txn.snapshot
}