You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

631 lines
19 KiB

// 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 (
"bytes"
"context"
"fmt"
"runtime/trace"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/dgryski/go-farm"
"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/kv/memdb"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)
var (
_ kv.Transaction = (*tikvTxn)(nil)
)
var (
tikvTxnCmdHistogramWithCommit = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblCommit)
tikvTxnCmdHistogramWithRollback = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblRollback)
tikvTxnCmdHistogramWithBatchGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblBatchGet)
tikvTxnCmdHistogramWithGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblGet)
tikvTxnCmdHistogramWithLockKeys = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblLockKeys)
)
// SchemaAmender is used by pessimistic transactions to amend commit mutations for schema change during 2pc.
type SchemaAmender interface {
// AmendTxn is the amend entry, new mutations will be generated based on input mutations using schema change info.
// The returned results are mutations need to prewrite and mutations need to cleanup.
AmendTxn(ctx context.Context, startInfoSchema SchemaVer, change *RelatedSchemaChange, mutations CommitterMutations) (*CommitterMutations, error)
}
// 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
lockKeys [][]byte
lockedMap map[string]bool
mu sync.Mutex // For thread-safe LockKeys function.
setCnt int64
vars *kv.Variables
committer *twoPhaseCommitter
// For data consistency check.
// assertions[:confirmed] is the assertion of current transaction.
// assertions[confirmed:len(assertions)] is the assertions of current statement.
// StmtCommit/StmtRollback may change the confirmed position.
assertions []assertionPair
confirmed int
valid bool
dirty bool
// txnInfoSchema is the infoSchema fetched at startTS.
txnInfoSchema SchemaVer
// SchemaAmender is used amend pessimistic txn commit mutations for schema change
schemaAmender SchemaAmender
// commitCallback is called after current transaction gets committed
commitCallback func(info kv.TxnInfo, err error)
}
func newTiKVTxn(store *tikvStore) (*tikvTxn, error) {
bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil)
startTS, err := store.getTimestampWithRetry(bo)
if err != nil {
return nil, errors.Trace(err)
}
return newTikvTxnWithStartTS(store, startTS, store.nextReplicaReadSeed())
}
// newTikvTxnWithStartTS creates a txn with startTS.
func newTikvTxnWithStartTS(store *tikvStore, startTS uint64, replicaReadSeed uint32) (*tikvTxn, error) {
ver := kv.NewVersion(startTS)
snapshot := newTiKVSnapshot(store, ver, replicaReadSeed)
return &tikvTxn{
snapshot: snapshot,
us: kv.NewUnionStore(snapshot),
lockedMap: make(map[string]bool),
store: store,
startTS: startTS,
startTime: time.Now(),
valid: true,
vars: kv.DefaultVars,
}, nil
}
type assertionPair struct {
key kv.Key
assertion kv.AssertionType
}
func (a assertionPair) String() string {
return fmt.Sprintf("key: %s, assertion type: %d", a.key, a.assertion)
}
// SetSuccess is used to probe if kv variables are set or not. It is ONLY used in test cases.
var SetSuccess = false
func (txn *tikvTxn) SetVars(vars *kv.Variables) {
txn.vars = vars
txn.snapshot.vars = vars
failpoint.Inject("probeSetVars", func(val failpoint.Value) {
if val.(bool) {
SetSuccess = true
}
})
}
func (txn *tikvTxn) GetVars() *kv.Variables {
return txn.vars
}
// tikvTxnStagingBuffer is the staging buffer returned to tikvTxn user.
// Because tikvTxn needs to maintain dirty state when Flush staging data into txn.
type tikvTxnStagingBuffer struct {
kv.MemBuffer
txn *tikvTxn
}
func (buf *tikvTxnStagingBuffer) Flush() (int, error) {
cnt, err := buf.MemBuffer.Flush()
if cnt != 0 {
buf.txn.dirty = true
}
return cnt, err
}
func (txn *tikvTxn) NewStagingBuffer() kv.MemBuffer {
return &tikvTxnStagingBuffer{
MemBuffer: txn.us.NewStagingBuffer(),
txn: txn,
}
}
func (txn *tikvTxn) Flush() (int, error) {
return txn.us.Flush()
}
func (txn *tikvTxn) Discard() {
txn.us.Discard()
}
// Get implements transaction interface.
func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) {
ret, err := txn.us.Get(ctx, k)
if kv.IsErrNotFound(err) {
return nil, err
}
if err != nil {
return nil, errors.Trace(err)
}
return ret, nil
}
func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]byte, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("tikvTxn.BatchGet", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
return kv.NewBufferBatchGetter(txn.GetMemBuffer(), nil, txn.snapshot).BatchGet(ctx, keys)
}
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) {
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) {
return txn.us.IterReverse(k)
}
func (txn *tikvTxn) Delete(k kv.Key) error {
txn.dirty = true
return txn.us.Delete(k)
}
func (txn *tikvTxn) DeleteWithNeedLock(k kv.Key) error {
txn.dirty = true
return txn.us.DeleteWithNeedLock(k)
}
func (txn *tikvTxn) GetFlags(ctx context.Context, k kv.Key) memdb.KeyFlags {
return txn.us.GetFlags(ctx, 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)
case kv.SnapshotTS:
txn.snapshot.setSnapshotTS(val.(uint64))
case kv.CheckExists:
txn.us.SetOption(kv.CheckExists, val.(map[string]struct{}))
case kv.InfoSchema:
txn.txnInfoSchema = val.(SchemaVer)
case kv.SchemaAmender:
txn.schemaAmender = val.(SchemaAmender)
case kv.CommitHook:
txn.commitCallback = val.(func(info kv.TxnInfo, err error))
}
}
func (txn *tikvTxn) DelOption(opt kv.Option) {
txn.us.DelOption(opt)
}
func (txn *tikvTxn) IsPessimistic() bool {
return txn.us.GetOption(kv.Pessimistic) != nil
}
func (txn *tikvTxn) Commit(ctx context.Context) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("tikvTxn.Commit", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
defer trace.StartRegion(ctx, "CommitTxn").End()
if !txn.valid {
return kv.ErrInvalidTxn
}
defer txn.close()
failpoint.Inject("mockCommitError", func(val failpoint.Value) {
if val.(bool) && kv.IsMockCommitErrorEnable() {
kv.MockCommitErrorDisable()
failpoint.Return(errors.New("mock commit error"))
}
})
start := time.Now()
defer func() { tikvTxnCmdHistogramWithCommit.Observe(time.Since(start).Seconds()) }()
// connID is used for log.
var connID uint64
val := ctx.Value(sessionctx.ConnID)
if val != nil {
connID = val.(uint64)
}
var err error
// If the txn use pessimistic lock, committer is initialized.
committer := txn.committer
if committer == nil {
committer, err = newTwoPhaseCommitter(txn, connID)
if err != nil {
return errors.Trace(err)
}
}
defer committer.ttlManager.close()
if err := committer.initKeysAndMutations(); err != nil {
return errors.Trace(err)
}
if committer.mutations.len() == 0 {
return nil
}
defer func() {
ctxValue := ctx.Value(execdetails.CommitDetailCtxKey)
if ctxValue != nil {
commitDetail := ctxValue.(**execdetails.CommitDetails)
if *commitDetail != nil {
(*commitDetail).TxnRetry++
} else {
*commitDetail = committer.getDetail()
}
}
}()
// latches disabled
// pessimistic transaction should also bypass latch.
if txn.store.txnLatches == nil || txn.IsPessimistic() {
err = committer.execute(ctx)
if val == nil || connID > 0 {
txn.onCommitted(err)
}
logutil.Logger(ctx).Debug("[kv] txnLatches disabled, 2pc directly", zap.Error(err))
return errors.Trace(err)
}
// latches enabled
// for transactions which need to acquire latches
start = time.Now()
lock := txn.store.txnLatches.Lock(committer.startTS, committer.mutations.keys)
commitDetail := committer.getDetail()
commitDetail.LocalLatchTime = time.Since(start)
if commitDetail.LocalLatchTime > 0 {
metrics.TiKVLocalLatchWaitTimeHistogram.Observe(commitDetail.LocalLatchTime.Seconds())
}
defer txn.store.txnLatches.UnLock(lock)
if lock.IsStale() {
return kv.ErrWriteConflictInTiDB.FastGenByArgs(txn.startTS)
}
err = committer.execute(ctx)
if val == nil || connID > 0 {
txn.onCommitted(err)
}
if err == nil {
lock.SetCommitTS(committer.commitTS)
}
logutil.Logger(ctx).Debug("[kv] txnLatches enabled while txn retryable", zap.Error(err))
return errors.Trace(err)
}
func (txn *tikvTxn) close() {
txn.valid = false
}
func (txn *tikvTxn) Rollback() error {
if !txn.valid {
return kv.ErrInvalidTxn
}
start := time.Now()
// Clean up pessimistic lock.
if txn.IsPessimistic() && txn.committer != nil {
err := txn.rollbackPessimisticLocks()
txn.committer.ttlManager.close()
if err != nil {
logutil.BgLogger().Error(err.Error())
}
}
txn.close()
logutil.BgLogger().Debug("[kv] rollback txn", zap.Uint64("txnStartTS", txn.StartTS()))
tikvTxnCmdHistogramWithRollback.Observe(time.Since(start).Seconds())
return nil
}
func (txn *tikvTxn) rollbackPessimisticLocks() error {
if len(txn.lockKeys) == 0 {
return nil
}
return txn.committer.pessimisticRollbackMutations(NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars), CommitterMutations{keys: txn.lockKeys})
}
func (txn *tikvTxn) onCommitted(err error) {
if txn.commitCallback != nil {
info := kv.TxnInfo{StartTS: txn.startTS, CommitTS: txn.commitTS}
if err != nil {
info.ErrMsg = err.Error()
}
txn.commitCallback(info, err)
}
}
// lockWaitTime in ms, except that kv.LockAlwaysWait(0) means always wait lock, kv.LockNowait(-1) means nowait lock
func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput ...kv.Key) error {
txn.mu.Lock()
defer txn.mu.Unlock()
// Exclude keys that are already locked.
var err error
keys := make([][]byte, 0, len(keysInput))
startTime := time.Now()
defer func() {
tikvTxnCmdHistogramWithLockKeys.Observe(time.Since(startTime).Seconds())
if err == nil {
if lockCtx.PessimisticLockWaited != nil {
if atomic.LoadInt32(lockCtx.PessimisticLockWaited) > 0 {
timeWaited := time.Since(lockCtx.WaitStartTime)
atomic.StoreInt64(lockCtx.LockKeysDuration, int64(timeWaited))
metrics.TiKVPessimisticLockKeysDuration.Observe(timeWaited.Seconds())
}
}
}
if lockCtx.LockKeysCount != nil {
*lockCtx.LockKeysCount += int32(len(keys))
}
if lockCtx.Stats != nil {
lockCtx.Stats.TotalTime = time.Since(startTime)
ctxValue := ctx.Value(execdetails.LockKeysDetailCtxKey)
if ctxValue != nil {
lockKeysDetail := ctxValue.(**execdetails.LockKeysDetails)
*lockKeysDetail = lockCtx.Stats
}
}
}()
for _, key := range keysInput {
// The value of lockedMap is only used by pessimistic transactions.
valueExist, locked := txn.lockedMap[string(key)]
_, checkKeyExists := lockCtx.CheckKeyExists[string(key)]
if !locked {
keys = append(keys, key)
} else if txn.IsPessimistic() {
if checkKeyExists && valueExist {
existErrInfo := txn.us.GetKeyExistErrInfo(key)
if existErrInfo == nil {
logutil.Logger(ctx).Error("key exist error not found", zap.Uint64("connID", txn.committer.connID),
zap.Stringer("key", key))
return errors.Errorf("conn %d, existErr for key:%s should not be nil", txn.committer.connID, key)
}
return existErrInfo.Err()
}
}
if lockCtx.ReturnValues && locked {
// An already locked key can not return values, we add an entry to let the caller get the value
// in other ways.
lockCtx.Values[string(key)] = kv.ReturnedValue{AlreadyLocked: true}
}
}
if len(keys) == 0 {
return nil
}
keys = deduplicateKeys(keys)
if txn.IsPessimistic() && lockCtx.ForUpdateTS > 0 {
if txn.committer == nil {
// connID is used for log.
var connID uint64
var err error
val := ctx.Value(sessionctx.ConnID)
if val != nil {
connID = val.(uint64)
}
txn.committer, err = newTwoPhaseCommitter(txn, connID)
if err != nil {
return err
}
}
var assignedPrimaryKey bool
if txn.committer.primaryKey == nil {
txn.committer.primaryKey = keys[0]
assignedPrimaryKey = true
}
lockCtx.Stats = &execdetails.LockKeysDetails{
LockKeys: int32(len(keys)),
}
bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars)
txn.committer.forUpdateTS = lockCtx.ForUpdateTS
// If the number of keys greater than 1, it can be on different region,
// concurrently execute on multiple regions may lead to deadlock.
txn.committer.isFirstLock = len(txn.lockKeys) == 0 && len(keys) == 1
err = txn.committer.pessimisticLockMutations(bo, lockCtx, CommitterMutations{keys: keys})
if bo.totalSleep > 0 {
atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.totalSleep)*int64(time.Millisecond))
lockCtx.Stats.Mu.Lock()
lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.types...)
lockCtx.Stats.Mu.Unlock()
}
if lockCtx.Killed != nil {
// If the kill signal is received during waiting for pessimisticLock,
// pessimisticLockKeys would handle the error but it doesn't reset the flag.
// We need to reset the killed flag here.
atomic.CompareAndSwapUint32(lockCtx.Killed, 1, 0)
}
if err != nil {
for _, key := range keys {
txn.us.DeleteKeyExistErrInfo(key)
}
keyMayBeLocked := terror.ErrorNotEqual(kv.ErrWriteConflict, err) && terror.ErrorNotEqual(kv.ErrKeyExists, err)
// If there is only 1 key and lock fails, no need to do pessimistic rollback.
if len(keys) > 1 || keyMayBeLocked {
wg := txn.asyncPessimisticRollback(ctx, keys)
if dl, ok := errors.Cause(err).(*ErrDeadlock); ok && hashInKeys(dl.DeadlockKeyHash, keys) {
dl.IsRetryable = true
// Wait for the pessimistic rollback to finish before we retry the statement.
wg.Wait()
// Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock.
time.Sleep(time.Millisecond * 5)
failpoint.Inject("SingleStmtDeadLockRetrySleep", func() {
time.Sleep(300 * time.Millisecond)
})
}
}
if assignedPrimaryKey {
// unset the primary key if we assigned primary key when failed to lock it.
txn.committer.primaryKey = nil
}
return err
}
if assignedPrimaryKey {
txn.committer.ttlManager.run(txn.committer, lockCtx)
}
}
txn.lockKeys = append(txn.lockKeys, keys...)
for _, key := range keys {
// PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exists.
// For other lock modes, the locked key values always exist.
if lockCtx.ReturnValues {
val, _ := lockCtx.Values[string(key)]
valExists := len(val.Value) > 0
txn.lockedMap[string(key)] = valExists
} else {
txn.lockedMap[string(key)] = true
}
}
txn.dirty = true
return nil
}
// deduplicateKeys deduplicate the keys, it use sort instead of map to avoid memory allocation.
func deduplicateKeys(keys [][]byte) [][]byte {
sort.Slice(keys, func(i, j int) bool {
return bytes.Compare(keys[i], keys[j]) < 0
})
deduped := keys[:1]
for i := 1; i < len(keys); i++ {
if !bytes.Equal(deduped[len(deduped)-1], keys[i]) {
deduped = append(deduped, keys[i])
}
}
return deduped
}
func (txn *tikvTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *sync.WaitGroup {
// Clone a new committer for execute in background.
committer := &twoPhaseCommitter{
store: txn.committer.store,
connID: txn.committer.connID,
startTS: txn.committer.startTS,
forUpdateTS: txn.committer.forUpdateTS,
primaryKey: txn.committer.primaryKey,
}
wg := new(sync.WaitGroup)
wg.Add(1)
go func() {
failpoint.Inject("AsyncRollBackSleep", func(sleepTimeMS failpoint.Value) {
if tmp, ok := sleepTimeMS.(int); ok {
if tmp < 10000 {
logutil.Logger(ctx).Info("[failpoint] sleep before trigger asyncPessimisticRollback", zap.Int("sleep ms", tmp))
time.Sleep(time.Duration(tmp) * time.Millisecond)
} else {
logutil.Logger(ctx).Info("[failpoint] async rollback return")
failpoint.Return()
}
}
})
err := committer.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), CommitterMutations{keys: keys})
if err != nil {
logutil.Logger(ctx).Warn("[kv] pessimisticRollback failed.", zap.Error(err))
}
wg.Done()
}()
return wg
}
func hashInKeys(deadlockKeyHash uint64, keys [][]byte) bool {
for _, key := range keys {
if farm.Fingerprint64(key) == deadlockKeyHash {
return true
}
}
return false
}
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) GetMemBufferSnapshot() kv.MemBuffer {
panic("unsupported operation")
}
func (txn *tikvTxn) GetSnapshot() kv.Snapshot {
return txn.snapshot
}
func (txn *tikvTxn) ResetStmtKeyExistErrs() {
txn.us.ResetStmtKeyExistErrs()
}
func (txn *tikvTxn) MergeStmtKeyExistErrs() {
txn.us.MergeStmtKeyExistErrs()
}