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.
 
 

1820 lines
60 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"
"encoding/hex"
"math"
"sort"
"sync"
"sync/atomic"
"time"
"unsafe"
"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx/binloginfo"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tipb/go-binlog"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)
type twoPhaseCommitAction interface {
handleSingleBatch(*twoPhaseCommitter, *Backoffer, batchMutations) error
tiKVTxnRegionsNumHistogram() prometheus.Observer
String() string
}
type actionPrewrite struct{}
type actionCommit struct{ retry bool }
type actionCleanup struct{}
type actionPessimisticLock struct {
*kv.LockCtx
}
type actionPessimisticRollback struct{}
var (
_ twoPhaseCommitAction = actionPrewrite{}
_ twoPhaseCommitAction = actionCommit{}
_ twoPhaseCommitAction = actionCleanup{}
_ twoPhaseCommitAction = actionPessimisticLock{}
_ twoPhaseCommitAction = actionPessimisticRollback{}
)
var (
tikvSecondaryLockCleanupFailureCounterCommit = metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("commit")
tikvSecondaryLockCleanupFailureCounterRollback = metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback")
tiKVTxnHeartBeatHistogramOK = metrics.TiKVTxnHeartBeatHistogram.WithLabelValues("ok")
tiKVTxnHeartBeatHistogramError = metrics.TiKVTxnHeartBeatHistogram.WithLabelValues("err")
tiKVTxnRegionsNumHistogramPrewrite = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues(metricsTag("prewrite"))
tiKVTxnRegionsNumHistogramCommit = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues(metricsTag("commit"))
tiKVTxnRegionsNumHistogramCleanup = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues(metricsTag("cleanup"))
tiKVTxnRegionsNumHistogramPessimisticLock = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues(metricsTag("pessimistic_lock"))
tiKVTxnRegionsNumHistogramPessimisticRollback = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues(metricsTag("pessimistic_rollback"))
)
// Global variable set by config file.
var (
ManagedLockTTL uint64 = 20000 // 20s
)
func (actionPrewrite) String() string {
return "prewrite"
}
func (actionPrewrite) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return tiKVTxnRegionsNumHistogramPrewrite
}
func (actionCommit) String() string {
return "commit"
}
func (actionCommit) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return tiKVTxnRegionsNumHistogramCommit
}
func (actionCleanup) String() string {
return "cleanup"
}
func (actionCleanup) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return tiKVTxnRegionsNumHistogramCleanup
}
func (actionPessimisticLock) String() string {
return "pessimistic_lock"
}
func (actionPessimisticLock) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return tiKVTxnRegionsNumHistogramPessimisticLock
}
func (actionPessimisticRollback) String() string {
return "pessimistic_rollback"
}
func (actionPessimisticRollback) tiKVTxnRegionsNumHistogram() prometheus.Observer {
return tiKVTxnRegionsNumHistogramPessimisticRollback
}
// metricsTag returns detail tag for metrics.
func metricsTag(action string) string {
return "2pc_" + action
}
// twoPhaseCommitter executes a two-phase commit protocol.
type twoPhaseCommitter struct {
store *tikvStore
txn *tikvTxn
startTS uint64
mutations CommitterMutations
lockTTL uint64
commitTS uint64
priority pb.CommandPri
connID uint64 // connID is used for log.
cleanWg sync.WaitGroup
detail unsafe.Pointer
txnSize int
noNeedCommitKeys map[string]struct{}
primaryKey []byte
forUpdateTS uint64
mu struct {
sync.RWMutex
undeterminedErr error // undeterminedErr saves the rpc error we encounter when commit primary key.
committed bool
}
syncLog bool
// For pessimistic transaction
isPessimistic bool
isFirstLock bool
// regionTxnSize stores the number of keys involved in each region
regionTxnSize map[uint64]int
// Used by pessimistic transaction and large transaction.
ttlManager
testingKnobs struct {
acAfterCommitPrimary chan struct{}
bkAfterCommitPrimary chan struct{}
}
// doingAmend means the amend prewrite is ongoing.
doingAmend bool
}
// CommitterMutations contains transaction operations.
type CommitterMutations struct {
ops []pb.Op
keys [][]byte
values [][]byte
isPessimisticLock []bool
}
// Mutation represents a single transaction operation.
type Mutation struct {
KeyOp pb.Op
Key []byte
Value []byte
IsPessimisticLock bool
}
// NewCommiterMutations creates a CommitterMutations object with sizeHint reserved.
func NewCommiterMutations(sizeHint int) CommitterMutations {
return CommitterMutations{
ops: make([]pb.Op, 0, sizeHint),
keys: make([][]byte, 0, sizeHint),
values: make([][]byte, 0, sizeHint),
isPessimisticLock: make([]bool, 0, sizeHint),
}
}
func (c *CommitterMutations) subRange(from, to int) CommitterMutations {
var res CommitterMutations
res.keys = c.keys[from:to]
if c.ops != nil {
res.ops = c.ops[from:to]
}
if c.values != nil {
res.values = c.values[from:to]
}
if c.isPessimisticLock != nil {
res.isPessimisticLock = c.isPessimisticLock[from:to]
}
return res
}
// Push another mutation into mutations.
func (c *CommitterMutations) Push(op pb.Op, key []byte, value []byte, isPessimisticLock bool) {
c.ops = append(c.ops, op)
c.keys = append(c.keys, key)
c.values = append(c.values, value)
c.isPessimisticLock = append(c.isPessimisticLock, isPessimisticLock)
}
func (c *CommitterMutations) len() int {
return len(c.keys)
}
// batchExecutor is txn controller providing rate control like utils
type batchExecutor struct {
rateLim int // concurrent worker numbers
rateLimiter *rateLimit // rate limiter for concurrency control, maybe more strategies
committer *twoPhaseCommitter // here maybe more different type committer in the future
action twoPhaseCommitAction // the work action type
backoffer *Backoffer // Backoffer
tokenWaitDuration time.Duration // get token wait time
}
// GetKeys returns the keys.
func (c *CommitterMutations) GetKeys() [][]byte {
return c.keys
}
// GetOps returns the key ops.
func (c *CommitterMutations) GetOps() []pb.Op {
return c.ops
}
// GetValues returns the key values.
func (c *CommitterMutations) GetValues() [][]byte {
return c.values
}
// GetPessimisticFlags returns the key pessimistic flags.
func (c *CommitterMutations) GetPessimisticFlags() []bool {
return c.isPessimisticLock
}
// MergeMutations append input mutations into current mutations.
func (c *CommitterMutations) MergeMutations(mutations CommitterMutations) {
c.ops = append(c.ops, mutations.ops...)
c.keys = append(c.keys, mutations.keys...)
c.values = append(c.values, mutations.values...)
c.isPessimisticLock = append(c.isPessimisticLock, mutations.isPessimisticLock...)
}
// AppendMutation merges a single Mutation into the current mutations.
func (c *CommitterMutations) AppendMutation(mutation Mutation) {
c.ops = append(c.ops, mutation.KeyOp)
c.keys = append(c.keys, mutation.Key)
c.values = append(c.values, mutation.Value)
c.isPessimisticLock = append(c.isPessimisticLock, mutation.IsPessimisticLock)
}
// newTwoPhaseCommitter creates a twoPhaseCommitter.
func newTwoPhaseCommitter(txn *tikvTxn, connID uint64) (*twoPhaseCommitter, error) {
return &twoPhaseCommitter{
store: txn.store,
txn: txn,
startTS: txn.StartTS(),
connID: connID,
regionTxnSize: map[uint64]int{},
ttlManager: ttlManager{
ch: make(chan struct{}),
},
isPessimistic: txn.IsPessimistic(),
}, nil
}
func sendTxnHeartBeat(bo *Backoffer, store *tikvStore, primary []byte, startTS, ttl uint64) (uint64, error) {
req := tikvrpc.NewRequest(tikvrpc.CmdTxnHeartBeat, &pb.TxnHeartBeatRequest{
PrimaryLock: primary,
StartVersion: startTS,
AdviseLockTtl: ttl,
})
for {
loc, err := store.GetRegionCache().LocateKey(bo, primary)
if err != nil {
return 0, errors.Trace(err)
}
resp, err := store.SendReq(bo, req, loc.Region, readTimeoutShort)
if err != nil {
return 0, errors.Trace(err)
}
regionErr, err := resp.GetRegionError()
if err != nil {
return 0, errors.Trace(err)
}
if regionErr != nil {
err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String()))
if err != nil {
return 0, errors.Trace(err)
}
continue
}
if resp.Resp == nil {
return 0, errors.Trace(ErrBodyMissing)
}
cmdResp := resp.Resp.(*pb.TxnHeartBeatResponse)
if keyErr := cmdResp.GetError(); keyErr != nil {
return 0, errors.Errorf("txn %d heartbeat fail, primary key = %v, err = %s", startTS, primary, keyErr.Abort)
}
return cmdResp.GetLockTtl(), nil
}
}
func (c *twoPhaseCommitter) initKeysAndMutations() error {
var (
size int
putCnt int
delCnt int
lockCnt int
noNeedCommitKey = make(map[string]struct{})
)
txn := c.txn
sizeHint := len(txn.lockKeys) + txn.us.Len()
mutations := NewCommiterMutations(sizeHint)
c.isPessimistic = txn.IsPessimistic()
// Merge ordered lockKeys and pairs in the memBuffer into the mutations array
sort.Slice(txn.lockKeys, func(i, j int) bool {
return bytes.Compare(txn.lockKeys[i], txn.lockKeys[j]) < 0
})
lockIdx := 0
err := txn.us.WalkBuffer(func(k kv.Key, v []byte) error {
var (
op pb.Op
value []byte
isPessimisticLock bool
)
if len(v) > 0 {
if tablecodec.IsUntouchedIndexKValue(k, v) {
return nil
}
op = pb.Op_Put
if c := txn.us.GetKeyExistErrInfo(k); c != nil {
op = pb.Op_Insert
}
value = v
putCnt++
} else {
if !txn.IsPessimistic() && txn.us.GetKeyExistErrInfo(k) != nil {
// delete-your-writes keys in optimistic txn need check not exists in prewrite-phase
// due to `Op_CheckNotExists` doesn't prewrite lock, so mark those keys should not be used in commit-phase.
op = pb.Op_CheckNotExists
noNeedCommitKey[string(k)] = struct{}{}
} else {
// normal delete keys in optimistic txn can be delete without not exists checking
// delete-your-writes keys in pessimistic txn can ensure must be no exists so can directly delete them
op = pb.Op_Del
delCnt++
}
}
for lockIdx < len(txn.lockKeys) {
lockKey := txn.lockKeys[lockIdx]
ord := bytes.Compare(lockKey, k)
if ord == 0 {
isPessimisticLock = c.isPessimistic
lockIdx++
break
} else if ord > 0 {
break
} else {
mutations.Push(pb.Op_Lock, lockKey, nil, c.isPessimistic)
lockCnt++
size += len(lockKey)
lockIdx++
}
}
mutations.Push(op, k, value, isPessimisticLock)
entrySize := len(k) + len(v)
if entrySize > kv.TxnEntrySizeLimit {
return kv.ErrEntryTooLarge.GenWithStackByArgs(kv.TxnEntrySizeLimit, entrySize)
}
size += entrySize
return nil
})
if err != nil {
return errors.Trace(err)
}
// add the remaining locks to mutations and keys
for _, lockKey := range txn.lockKeys[lockIdx:] {
mutations.Push(pb.Op_Lock, lockKey, nil, c.isPessimistic)
lockCnt++
size += len(lockKey)
}
if mutations.len() == 0 {
return nil
}
c.txnSize = size
if len(c.primaryKey) == 0 {
for i, op := range mutations.ops {
if op != pb.Op_CheckNotExists {
c.primaryKey = mutations.keys[i]
break
}
}
}
if size > int(kv.TxnTotalSizeLimit) {
return kv.ErrTxnTooLarge.GenWithStackByArgs(size)
}
const logEntryCount = 10000
const logSize = 4 * 1024 * 1024 // 4MB
if mutations.len() > logEntryCount || size > logSize {
tableID := tablecodec.DecodeTableID(mutations.keys[0])
logutil.BgLogger().Info("[BIG_TXN]",
zap.Uint64("con", c.connID),
zap.Int64("table ID", tableID),
zap.Int("size", size),
zap.Int("keys", mutations.len()),
zap.Int("puts", putCnt),
zap.Int("dels", delCnt),
zap.Int("locks", lockCnt),
zap.Int("checks", len(noNeedCommitKey)),
zap.Uint64("txnStartTS", txn.startTS))
}
// Sanity check for startTS.
if txn.StartTS() == math.MaxUint64 {
err = errors.Errorf("try to commit with invalid txnStartTS: %d", txn.StartTS())
logutil.BgLogger().Error("commit failed",
zap.Uint64("conn", c.connID),
zap.Error(err))
return errors.Trace(err)
}
commitDetail := &execdetails.CommitDetails{WriteSize: size, WriteKeys: mutations.len()}
metrics.TiKVTxnWriteKVCountHistogram.Observe(float64(commitDetail.WriteKeys))
metrics.TiKVTxnWriteSizeHistogram.Observe(float64(commitDetail.WriteSize))
c.noNeedCommitKeys = noNeedCommitKey
c.mutations = mutations
c.lockTTL = txnLockTTL(txn.startTime, size)
c.priority = getTxnPriority(txn)
c.syncLog = getTxnSyncLog(txn)
c.setDetail(commitDetail)
return nil
}
func (c *twoPhaseCommitter) primary() []byte {
if len(c.primaryKey) == 0 {
return c.mutations.keys[0]
}
return c.primaryKey
}
const bytesPerMiB = 1024 * 1024
func txnLockTTL(startTime time.Time, txnSize int) uint64 {
// Increase lockTTL for large transactions.
// The formula is `ttl = ttlFactor * sqrt(sizeInMiB)`.
// When writeSize is less than 256KB, the base ttl is defaultTTL (3s);
// When writeSize is 1MiB, 4MiB, or 10MiB, ttl is 6s, 12s, 20s correspondingly;
lockTTL := defaultLockTTL
if txnSize >= txnCommitBatchSize {
sizeMiB := float64(txnSize) / bytesPerMiB
lockTTL = uint64(float64(ttlFactor) * math.Sqrt(sizeMiB))
if lockTTL < defaultLockTTL {
lockTTL = defaultLockTTL
}
if lockTTL > ManagedLockTTL {
lockTTL = ManagedLockTTL
}
}
// Increase lockTTL by the transaction's read time.
// When resolving a lock, we compare current ts and startTS+lockTTL to decide whether to clean up. If a txn
// takes a long time to read, increasing its TTL will help to prevent it from been aborted soon after prewrite.
elapsed := time.Since(startTime) / time.Millisecond
return lockTTL + uint64(elapsed)
}
var preSplitDetectThreshold uint32 = 100000
var preSplitSizeThreshold uint32 = 32 << 20
// doActionOnMutations groups keys into primary batch and secondary batches, if primary batch exists in the key,
// it does action on primary batch first, then on secondary batches. If action is commit, secondary batches
// is done in background goroutine.
func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCommitAction, mutations CommitterMutations) error {
if mutations.len() == 0 {
return nil
}
groups, err := c.store.regionCache.GroupSortedMutationsByRegion(bo, mutations)
if err != nil {
return errors.Trace(err)
}
// Pre-split regions to avoid too much write workload into a single region.
// In the large transaction case, this operation is important to avoid TiKV 'server is busy' error.
var preSplited bool
preSplitDetectThresholdVal := atomic.LoadUint32(&preSplitDetectThreshold)
for _, group := range groups {
if uint32(group.mutations.len()) >= preSplitDetectThresholdVal {
logutil.BgLogger().Info("2PC detect large amount of mutations on a single region",
zap.Uint64("region", group.region.GetID()),
zap.Int("mutations count", group.mutations.len()))
// Use context.Background, this time should not add up to Backoffer.
if preSplitAndScatterIn2PC(context.Background(), c.store, group) {
preSplited = true
}
}
}
// Reload region cache again.
if preSplited {
groups, err = c.store.regionCache.GroupSortedMutationsByRegion(bo, mutations)
if err != nil {
return errors.Trace(err)
}
}
return c.doActionOnGroupMutations(bo, action, groups)
}
func preSplitAndScatterIn2PC(ctx context.Context, store *tikvStore, group groupedMutations) bool {
splitKeys := make([][]byte, 0, 4)
preSplitSizeThresholdVal := atomic.LoadUint32(&preSplitSizeThreshold)
regionSize := 0
keysLength := group.mutations.len()
valsLength := len(group.mutations.values)
// The value length maybe zero for pessimistic lock keys
for i := 0; i < keysLength; i++ {
regionSize = regionSize + len(group.mutations.keys[i])
if i < valsLength {
regionSize = regionSize + len(group.mutations.values[i])
}
// The second condition is used for testing.
if regionSize >= int(preSplitSizeThresholdVal) {
regionSize = 0
splitKeys = append(splitKeys, group.mutations.keys[i])
}
}
if len(splitKeys) == 0 {
return false
}
regionIDs, err := store.SplitRegions(ctx, splitKeys, true, nil)
if err != nil {
logutil.BgLogger().Warn("2PC split regions failed", zap.Uint64("regionID", group.region.id),
zap.Int("keys count", keysLength), zap.Int("values count", valsLength), zap.Error(err))
return false
}
for _, regionID := range regionIDs {
err := store.WaitScatterRegionFinish(ctx, regionID, 0)
if err != nil {
logutil.BgLogger().Warn("2PC wait scatter region failed", zap.Uint64("regionID", regionID), zap.Error(err))
}
}
// Invalidate the old region cache information.
store.regionCache.InvalidateCachedRegion(group.region)
return true
}
func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPhaseCommitAction, groups []groupedMutations) error {
action.tiKVTxnRegionsNumHistogram().Observe(float64(len(groups)))
var batches []batchMutations
var sizeFunc = c.keySize
switch act := action.(type) {
case actionPrewrite:
// Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest.
if len(bo.errors) == 0 {
for _, group := range groups {
c.regionTxnSize[group.region.id] = group.mutations.len()
}
}
sizeFunc = c.keyValueSize
atomic.AddInt32(&c.getDetail().PrewriteRegionNum, int32(len(groups)))
case actionPessimisticLock:
if act.LockCtx.Stats != nil {
act.LockCtx.Stats.RegionNum = int32(len(groups))
}
}
primaryIdx := -1
for _, group := range groups {
batches = c.appendBatchMutationsBySize(batches, group.region, group.mutations, sizeFunc, txnCommitBatchSize, &primaryIdx)
}
firstIsPrimary := false
// If the batches include the primary key, put it to the first
if primaryIdx >= 0 {
batches[primaryIdx].isPrimary = true
batches[0], batches[primaryIdx] = batches[primaryIdx], batches[0]
firstIsPrimary = true
}
actionCommit, actionIsCommit := action.(actionCommit)
_, actionIsCleanup := action.(actionCleanup)
_, actionIsPessimiticLock := action.(actionPessimisticLock)
var err error
failpoint.Inject("skipKeyReturnOK", func(val failpoint.Value) {
valStr, ok := val.(string)
if ok && c.connID > 0 {
if firstIsPrimary && actionIsPessimiticLock {
logutil.Logger(bo.ctx).Warn("pessimisticLock failpoint", zap.String("valStr", valStr))
switch valStr {
case "pessimisticLockSkipPrimary":
err = c.doActionOnBatches(bo, action, batches)
failpoint.Return(err)
case "pessimisticLockSkipSecondary":
err = c.doActionOnBatches(bo, action, batches[:1])
failpoint.Return(err)
}
}
}
})
failpoint.Inject("pessimisticRollbackDoNth", func() {
_, actionIsPessimisticRollback := action.(actionPessimisticRollback)
if actionIsPessimisticRollback && c.connID > 0 {
logutil.Logger(bo.ctx).Warn("pessimisticRollbackDoNth failpoint")
failpoint.Return(nil)
}
})
if firstIsPrimary && (actionIsCommit || actionIsCleanup || actionIsPessimiticLock) {
// primary should be committed/cleanup/pessimistically locked first
err = c.doActionOnBatches(bo, action, batches[:1])
if err != nil {
return errors.Trace(err)
}
if actionIsCommit && c.testingKnobs.bkAfterCommitPrimary != nil && c.testingKnobs.acAfterCommitPrimary != nil {
c.testingKnobs.acAfterCommitPrimary <- struct{}{}
<-c.testingKnobs.bkAfterCommitPrimary
}
batches = batches[1:]
}
if actionIsCommit && !actionCommit.retry {
// Commit secondary batches in background goroutine to reduce latency.
// The backoffer instance is created outside of the goroutine to avoid
// potential data race in unit test since `CommitMaxBackoff` will be updated
// by test suites.
secondaryBo := NewBackofferWithVars(context.Background(), CommitMaxBackoff, c.txn.vars)
go func() {
failpoint.Inject("beforeCommitSecondaries", func(v failpoint.Value) {
if s, ok := v.(string); !ok {
logutil.Logger(bo.ctx).Info("[failpoint] sleep 2s before commit secondary keys",
zap.Uint64("connID", c.connID), zap.Uint64("startTS", c.startTS))
time.Sleep(2 * time.Second)
} else if s == "skip" {
failpoint.Return()
}
})
e := c.doActionOnBatches(secondaryBo, action, batches)
if e != nil {
logutil.BgLogger().Debug("2PC async doActionOnBatches",
zap.Uint64("conn", c.connID),
zap.Stringer("action type", action),
zap.Error(e))
tikvSecondaryLockCleanupFailureCounterCommit.Inc()
}
}()
} else {
err = c.doActionOnBatches(bo, action, batches)
}
return errors.Trace(err)
}
// doActionOnBatches does action to batches in parallel.
func (c *twoPhaseCommitter) doActionOnBatches(bo *Backoffer, action twoPhaseCommitAction, batches []batchMutations) error {
if len(batches) == 0 {
return nil
}
noNeedFork := len(batches) == 1
if !noNeedFork {
if ac, ok := action.(actionCommit); ok && ac.retry {
noNeedFork = true
}
}
if noNeedFork {
for _, b := range batches {
e := action.handleSingleBatch(c, bo, b)
if e != nil {
logutil.BgLogger().Debug("2PC doActionOnBatches failed",
zap.Uint64("conn", c.connID),
zap.Stringer("action type", action),
zap.Error(e),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(e)
}
}
return nil
}
rateLim := len(batches)
// Set rateLim here for the large transaction.
// If the rate limit is too high, tikv will report service is busy.
// If the rate limit is too low, we can't full utilize the tikv's throughput.
// TODO: Find a self-adaptive way to control the rate limit here.
if rateLim > config.GetGlobalConfig().Performance.CommitterConcurrency {
rateLim = config.GetGlobalConfig().Performance.CommitterConcurrency
}
batchExecutor := newBatchExecutor(rateLim, c, action, bo)
err := batchExecutor.process(batches)
return errors.Trace(err)
}
func (c *twoPhaseCommitter) keyValueSize(key, value []byte) int {
return len(key) + len(value)
}
func (c *twoPhaseCommitter) keySize(key, value []byte) int {
return len(key)
}
func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize uint64) *tikvrpc.Request {
m := &batch.mutations
mutations := make([]*pb.Mutation, m.len())
for i := range m.keys {
mutations[i] = &pb.Mutation{
Op: m.ops[i],
Key: m.keys[i],
Value: m.values[i],
}
}
var minCommitTS uint64
if c.forUpdateTS > 0 {
minCommitTS = c.forUpdateTS + 1
} else {
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
}
})
req := &pb.PrewriteRequest{
Mutations: mutations,
PrimaryLock: c.primary(),
StartVersion: c.startTS,
LockTtl: c.lockTTL,
IsPessimisticLock: m.isPessimisticLock,
ForUpdateTs: c.forUpdateTS,
TxnSize: txnSize,
MinCommitTs: minCommitTS,
}
return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog})
}
func (actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
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 {
resp, err := c.store.SendReq(bo, req, batch.region, readTimeoutShort)
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 large than 32M,
// start the ttlManager. The ttlManager will be closed in tikvTxn.Commit().
if c.txnSize > 32*1024*1024 {
c.run(c, nil)
}
}
return nil
}
var locks []*Lock
for _, keyErr := range keyErrs {
// Check already exists error
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
key := alreadyExist.GetKey()
existErrInfo := c.txn.us.GetKeyExistErrInfo(key)
if existErrInfo == nil {
return errors.Errorf("conn %d, existErr for key:%s should not be nil", c.connID, key)
}
return existErrInfo.Err()
}
// Extract lock from key error
lock, err1 := extractLockFromKeyErr(keyErr)
if err1 != nil {
return errors.Trace(err1)
}
logutil.BgLogger().Info("prewrite encounters lock",
zap.Uint64("conn", c.connID),
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)
}
}
}
}
type ttlManagerState uint32
const (
stateUninitialized ttlManagerState = iota
stateRunning
stateClosed
)
type ttlManager struct {
state ttlManagerState
ch chan struct{}
lockCtx *kv.LockCtx
}
func (tm *ttlManager) run(c *twoPhaseCommitter, lockCtx *kv.LockCtx) {
// Run only once.
if !atomic.CompareAndSwapUint32((*uint32)(&tm.state), uint32(stateUninitialized), uint32(stateRunning)) {
return
}
tm.lockCtx = lockCtx
go tm.keepAlive(c)
}
func (tm *ttlManager) close() {
if !atomic.CompareAndSwapUint32((*uint32)(&tm.state), uint32(stateRunning), uint32(stateClosed)) {
return
}
close(tm.ch)
}
func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) {
// Ticker is set to 1/2 of the ManagedLockTTL.
ticker := time.NewTicker(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond / 2)
defer ticker.Stop()
for {
select {
case <-tm.ch:
return
case <-ticker.C:
// If kill signal is received, the ttlManager should exit.
if tm.lockCtx != nil && tm.lockCtx.Killed != nil && atomic.LoadUint32(tm.lockCtx.Killed) != 0 {
return
}
bo := NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars)
now, err := c.store.GetOracle().GetTimestamp(bo.ctx)
if err != nil {
err1 := bo.Backoff(BoPDRPC, err)
if err1 != nil {
logutil.Logger(bo.ctx).Warn("keepAlive get tso fail",
zap.Error(err))
return
}
continue
}
uptime := uint64(oracle.ExtractPhysical(now) - oracle.ExtractPhysical(c.startTS))
if uptime > config.GetGlobalConfig().Performance.MaxTxnTTL {
// Checks maximum lifetime for the ttlManager, so when something goes wrong
// the key will not be locked forever.
logutil.Logger(bo.ctx).Info("ttlManager live up to its lifetime",
zap.Uint64("txnStartTS", c.startTS),
zap.Uint64("uptime", uptime),
zap.Uint64("maxTxnTTL", config.GetGlobalConfig().Performance.MaxTxnTTL))
metrics.TiKVTTLLifeTimeReachCounter.Inc()
// the pessimistic locks may expire if the ttl manager has timed out, set `LockExpired` flag
// so that this transaction could only commit or rollback with no more statement executions
if c.isPessimistic && tm.lockCtx != nil && tm.lockCtx.LockExpired != nil {
atomic.StoreUint32(tm.lockCtx.LockExpired, 1)
}
return
}
newTTL := uptime + atomic.LoadUint64(&ManagedLockTTL)
logutil.Logger(bo.ctx).Info("send TxnHeartBeat",
zap.Uint64("startTS", c.startTS), zap.Uint64("newTTL", newTTL))
startTime := time.Now()
_, err = sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL)
if err != nil {
tiKVTxnHeartBeatHistogramError.Observe(time.Since(startTime).Seconds())
logutil.Logger(bo.ctx).Warn("send TxnHeartBeat failed",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return
}
tiKVTxnHeartBeatHistogramOK.Observe(time.Since(startTime).Seconds())
}
}
}
func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
m := &batch.mutations
mutations := make([]*pb.Mutation, m.len())
for i := range m.keys {
mut := &pb.Mutation{
Op: pb.Op_PessimisticLock,
Key: m.keys[i],
}
existErr := c.txn.us.GetKeyExistErrInfo(m.keys[i])
if existErr != nil || (c.doingAmend && m.GetOps()[i] == pb.Op_Insert) {
mut.Assertion = pb.Assertion_NotExist
}
mutations[i] = mut
}
elapsed := uint64(time.Since(c.txn.startTime) / time.Millisecond)
req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticLock, &pb.PessimisticLockRequest{
Mutations: mutations,
PrimaryLock: c.primary(),
StartVersion: c.startTS,
ForUpdateTs: c.forUpdateTS,
LockTtl: elapsed + atomic.LoadUint64(&ManagedLockTTL),
IsFirstLock: c.isFirstLock,
WaitTimeout: action.LockWaitTime,
ReturnValues: action.ReturnValues,
MinCommitTs: c.forUpdateTS + 1,
}, pb.Context{Priority: c.priority, SyncLog: c.syncLog})
lockWaitStartTime := action.WaitStartTime
for {
// if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit
if action.LockWaitTime > 0 {
timeLeft := action.LockWaitTime - (time.Since(lockWaitStartTime)).Milliseconds()
if timeLeft <= 0 {
req.PessimisticLock().WaitTimeout = kv.LockNoWait
} else {
req.PessimisticLock().WaitTimeout = timeLeft
}
}
failpoint.Inject("PessimisticLockErrWriteConflict", func() error {
time.Sleep(300 * time.Millisecond)
return kv.ErrWriteConflict
})
startTime := time.Now()
resp, err := c.store.SendReq(bo, req, batch.region, readTimeoutShort)
if action.LockCtx.Stats != nil {
atomic.AddInt64(&action.LockCtx.Stats.LockRPCTime, int64(time.Since(startTime)))
atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1)
}
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.pessimisticLockMutations(bo, action.LockCtx, batch.mutations)
return errors.Trace(err)
}
if resp.Resp == nil {
return errors.Trace(ErrBodyMissing)
}
lockResp := resp.Resp.(*pb.PessimisticLockResponse)
keyErrs := lockResp.GetErrors()
if len(keyErrs) == 0 {
if action.ReturnValues {
action.ValuesLock.Lock()
for i, mutation := range mutations {
action.Values[string(mutation.Key)] = kv.ReturnedValue{Value: lockResp.Values[i]}
}
action.ValuesLock.Unlock()
}
return nil
}
var locks []*Lock
for _, keyErr := range keyErrs {
// Check already exists error
if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil {
key := alreadyExist.GetKey()
existErrInfo := c.txn.us.GetKeyExistErrInfo(key)
if existErrInfo == nil {
return errors.Errorf("conn %d, existErr for key:%s should not be nil", c.connID, key)
}
return existErrInfo.Err()
}
if deadlock := keyErr.Deadlock; deadlock != nil {
return &ErrDeadlock{Deadlock: deadlock}
}
// Extract lock from key error
lock, err1 := extractLockFromKeyErr(keyErr)
if err1 != nil {
return errors.Trace(err1)
}
locks = append(locks, lock)
}
// Because we already waited on tikv, no need to Backoff here.
// tikv default will wait 3s(also the maximum wait value) when lock error occurs
startTime = time.Now()
msBeforeTxnExpired, _, err := c.store.lockResolver.ResolveLocks(bo, 0, locks)
if action.LockCtx.Stats != nil {
atomic.AddInt64(&action.LockCtx.Stats.ResolveLockTime, int64(time.Since(startTime)))
}
if err != nil {
return errors.Trace(err)
}
// If msBeforeTxnExpired is not zero, it means there are still locks blocking us acquiring
// the pessimistic lock. We should return acquire fail with nowait set or timeout error if necessary.
if msBeforeTxnExpired > 0 {
if action.LockWaitTime == kv.LockNoWait {
return ErrLockAcquireFailAndNoWaitSet
} else if action.LockWaitTime == kv.LockAlwaysWait {
// do nothing but keep wait
} else {
// the lockWaitTime is set, we should return wait timeout if we are still blocked by a lock
if time.Since(lockWaitStartTime).Milliseconds() >= action.LockWaitTime {
return errors.Trace(ErrLockWaitTimeout)
}
}
if action.LockCtx.PessimisticLockWaited != nil {
atomic.StoreInt32(action.LockCtx.PessimisticLockWaited, 1)
}
}
// Handle the killed flag when waiting for the pessimistic lock.
// When a txn runs into LockKeys() and backoff here, it has no chance to call
// executor.Next() and check the killed flag.
if action.Killed != nil {
// Do not reset the killed flag here!
// actionPessimisticLock runs on each region parallelly, we have to consider that
// the error may be dropped.
if atomic.LoadUint32(action.Killed) == 1 {
return errors.Trace(ErrQueryInterrupted)
}
}
}
}
func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, &pb.PessimisticRollbackRequest{
StartVersion: c.startTS,
ForUpdateTs: c.forUpdateTS,
Keys: batch.mutations.keys,
})
resp, err := c.store.SendReq(bo, req, batch.region, readTimeoutShort)
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.pessimisticRollbackMutations(bo, batch.mutations)
return errors.Trace(err)
}
return nil
}
func getTxnPriority(txn *tikvTxn) pb.CommandPri {
if pri := txn.us.GetOption(kv.Priority); pri != nil {
return kvPriorityToCommandPri(pri.(int))
}
return pb.CommandPri_Normal
}
func getTxnSyncLog(txn *tikvTxn) bool {
if syncOption := txn.us.GetOption(kv.SyncLog); syncOption != nil {
return syncOption.(bool)
}
return false
}
func kvPriorityToCommandPri(pri int) pb.CommandPri {
switch pri {
case kv.PriorityLow:
return pb.CommandPri_Low
case kv.PriorityHigh:
return pb.CommandPri_High
default:
return pb.CommandPri_Normal
}
}
func (c *twoPhaseCommitter) setDetail(d *execdetails.CommitDetails) {
atomic.StorePointer(&c.detail, unsafe.Pointer(d))
}
func (c *twoPhaseCommitter) getDetail() *execdetails.CommitDetails {
return (*execdetails.CommitDetails)(atomic.LoadPointer(&c.detail))
}
func (c *twoPhaseCommitter) setUndeterminedErr(err error) {
c.mu.Lock()
defer c.mu.Unlock()
c.mu.undeterminedErr = err
}
func (c *twoPhaseCommitter) getUndeterminedErr() error {
c.mu.RLock()
defer c.mu.RUnlock()
return c.mu.undeterminedErr
}
func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &pb.CommitRequest{
StartVersion: c.startTS,
Keys: batch.mutations.keys,
CommitVersion: c.commitTS,
}, pb.Context{Priority: c.priority, SyncLog: c.syncLog})
sender := NewRegionRequestSender(c.store.regionCache, c.store.client)
resp, err := sender.SendReq(bo, req, batch.region, readTimeoutShort)
// If we fail to receive response for the request that commits primary key, it will be undetermined whether this
// transaction has been successfully committed.
// Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw
// an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best
// solution is to populate this error and let upper layer drop the connection to the corresponding mysql client.
if batch.isPrimary && sender.rpcError != nil {
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)
}
// re-split keys and commit again.
err = c.doActionOnMutations(bo, actionCommit{retry: true}, batch.mutations)
return errors.Trace(err)
}
if resp.Resp == nil {
return errors.Trace(ErrBodyMissing)
}
commitResp := resp.Resp.(*pb.CommitResponse)
// Here we can make sure tikv has processed the commit primary key request. So
// we can clean undetermined error.
if batch.isPrimary {
c.setUndeterminedErr(nil)
}
if keyErr := commitResp.GetError(); keyErr != nil {
if rejected := keyErr.GetCommitTsExpired(); rejected != nil {
logutil.Logger(bo.ctx).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS",
zap.Uint64("txnStartTS", c.startTS),
zap.Stringer("info", logutil.Hex(rejected)))
// Do not retry for a txn which has a too large MinCommitTs
// 3600000 << 18 = 943718400000
if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 {
err := errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d",
rejected.MinCommitTs, rejected.AttemptedCommitTs)
return errors.Trace(err)
}
// Update commit ts and retry.
commitTS, err := c.store.getTimestampWithRetry(bo)
if err != nil {
logutil.Logger(bo.ctx).Warn("2PC get commitTS failed",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(err)
}
c.mu.Lock()
c.commitTS = commitTS
c.mu.Unlock()
return c.commitMutations(bo, batch.mutations)
}
c.mu.RLock()
defer c.mu.RUnlock()
err = extractKeyErr(keyErr)
if c.mu.committed {
// No secondary key could be rolled back after it's primary key is committed.
// There must be a serious bug somewhere.
hexBatchKeys := func(keys [][]byte) []string {
var res []string
for _, k := range keys {
res = append(res, hex.EncodeToString(k))
}
return res
}
logutil.Logger(bo.ctx).Error("2PC failed commit key after primary key committed",
zap.Error(err),
zap.Stringer("primaryKey", kv.Key(c.primaryKey)),
zap.Uint64("txnStartTS", c.startTS),
zap.Uint64("commitTS", c.commitTS),
zap.Uint64("forUpdateTS", c.forUpdateTS),
zap.Strings("keys", hexBatchKeys(batch.mutations.keys)))
return errors.Trace(err)
}
// The transaction maybe rolled back by concurrent transactions.
logutil.Logger(bo.ctx).Debug("2PC failed commit primary key",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return err
}
c.mu.Lock()
defer c.mu.Unlock()
// Group that contains primary key is always the first.
// We mark transaction's status committed when we receive the first success response.
c.mu.committed = true
return nil
}
func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error {
req := tikvrpc.NewRequest(tikvrpc.CmdBatchRollback, &pb.BatchRollbackRequest{
Keys: batch.mutations.keys,
StartVersion: c.startTS,
}, pb.Context{Priority: c.priority, SyncLog: c.syncLog})
resp, err := c.store.SendReq(bo, req, batch.region, readTimeoutShort)
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.cleanupMutations(bo, batch.mutations)
return errors.Trace(err)
}
if keyErr := resp.Resp.(*pb.BatchRollbackResponse).GetError(); keyErr != nil {
err = errors.Errorf("conn %d 2PC cleanup failed: %s", c.connID, keyErr)
logutil.BgLogger().Debug("2PC failed cleanup key",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(err)
}
return nil
}
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)
}
return c.doActionOnMutations(bo, actionPrewrite{}, mutations)
}
func (c *twoPhaseCommitter) commitMutations(bo *Backoffer, mutations CommitterMutations) error {
if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("twoPhaseCommitter.commitMutations", opentracing.ChildOf(span.Context()))
defer span1.Finish()
bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1)
}
return c.doActionOnMutations(bo, actionCommit{}, mutations)
}
func (c *twoPhaseCommitter) cleanupMutations(bo *Backoffer, mutations CommitterMutations) error {
return c.doActionOnMutations(bo, actionCleanup{}, mutations)
}
func (c *twoPhaseCommitter) pessimisticLockMutations(bo *Backoffer, lockCtx *kv.LockCtx, mutations CommitterMutations) error {
return c.doActionOnMutations(bo, actionPessimisticLock{lockCtx}, mutations)
}
func (c *twoPhaseCommitter) pessimisticRollbackMutations(bo *Backoffer, mutations CommitterMutations) error {
return c.doActionOnMutations(bo, actionPessimisticRollback{}, mutations)
}
// execute executes the two-phase commit protocol.
func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
var binlogSkipped bool
defer func() {
// Always clean up all written keys if the txn does not commit.
c.mu.RLock()
committed := c.mu.committed
undetermined := c.mu.undeterminedErr != nil
c.mu.RUnlock()
if !committed && !undetermined {
c.cleanWg.Add(1)
go func() {
cleanupKeysCtx := context.WithValue(context.Background(), txnStartKey, ctx.Value(txnStartKey))
err := c.cleanupMutations(NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations)
if err != nil {
tikvSecondaryLockCleanupFailureCounterRollback.Inc()
logutil.Logger(ctx).Info("2PC cleanup failed",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
} else {
logutil.Logger(ctx).Info("2PC clean up done",
zap.Uint64("txnStartTS", c.startTS))
}
c.cleanWg.Done()
}()
}
c.txn.commitTS = c.commitTS
if binlogSkipped {
binloginfo.RemoveOneSkippedCommitter()
} else {
if err != nil {
c.writeFinishBinlog(ctx, binlog.BinlogType_Rollback, 0)
} else {
c.writeFinishBinlog(ctx, binlog.BinlogType_Commit, int64(c.commitTS))
}
}
}()
binlogChan := c.prewriteBinlog(ctx)
prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars)
start := time.Now()
err = c.prewriteMutations(prewriteBo, c.mutations)
commitDetail := c.getDetail()
commitDetail.PrewriteTime = time.Since(start)
if prewriteBo.totalSleep > 0 {
atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.totalSleep)*int64(time.Millisecond))
commitDetail.Mu.Lock()
commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.types...)
commitDetail.Mu.Unlock()
}
if binlogChan != nil {
startWaitBinlog := time.Now()
binlogWriteResult := <-binlogChan
commitDetail.WaitPrewriteBinlogTime = time.Since(startWaitBinlog)
if binlogWriteResult != nil {
binlogSkipped = binlogWriteResult.Skipped()
binlogErr := binlogWriteResult.GetError()
if binlogErr != nil {
return binlogErr
}
}
}
if err != nil {
logutil.Logger(ctx).Debug("2PC failed on prewrite",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(err)
}
// strip check_not_exists keys that no need to commit.
c.stripNoNeedCommitKeys()
start = time.Now()
logutil.Event(ctx, "start get commit ts")
commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars))
if err != nil {
logutil.Logger(ctx).Warn("2PC get commitTS failed",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(err)
}
commitDetail.GetCommitTsTime = time.Since(start)
logutil.Event(ctx, "finish get commit ts")
logutil.SetTag(ctx, "commitTs", commitTS)
tryAmend := c.isPessimistic && c.connID > 0
if !tryAmend {
_, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, false)
if err != nil {
return errors.Trace(err)
}
} else {
relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, true)
if err != nil {
return errors.Trace(err)
}
if memAmended {
// Get new commitTS and check schema valid again.
newCommitTS, err := c.getCommitTS(ctx, commitDetail)
if err != nil {
return errors.Trace(err)
}
// If schema check failed between commitTS and newCommitTs, report schema change error.
_, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false)
if err != nil {
logutil.Logger(ctx).Info("schema check after amend failed, it means the schema version changed again",
zap.Uint64("startTS", c.startTS),
zap.Uint64("amendTS", c.commitTS),
zap.Int64("amendedSchemaVersion", relatedSchemaChange.LatestInfoSchema.SchemaMetaVersion()),
zap.Uint64("newCommitTS", newCommitTS))
return errors.Trace(err)
}
commitTS = newCommitTS
}
}
c.commitTS = commitTS
if c.store.oracle.IsExpired(c.startTS, kv.MaxTxnTimeUse) {
err = errors.Errorf("conn %d txn takes too much time, txnStartTS: %d, comm: %d",
c.connID, c.startTS, c.commitTS)
return err
}
if c.connID > 0 {
failpoint.Inject("beforeCommit", func() {})
}
start = time.Now()
commitBo := NewBackofferWithVars(ctx, CommitMaxBackoff, c.txn.vars)
err = c.commitMutations(commitBo, c.mutations)
commitDetail.CommitTime = time.Since(start)
if commitBo.totalSleep > 0 {
atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.totalSleep)*int64(time.Millisecond))
commitDetail.Mu.Lock()
commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.types...)
commitDetail.Mu.Unlock()
}
if err != nil {
if undeterminedErr := c.getUndeterminedErr(); undeterminedErr != nil {
logutil.Logger(ctx).Error("2PC commit result undetermined",
zap.Error(err),
zap.NamedError("rpcErr", undeterminedErr),
zap.Uint64("txnStartTS", c.startTS))
err = errors.Trace(terror.ErrResultUndetermined)
}
if !c.mu.committed {
logutil.Logger(ctx).Debug("2PC failed on commit",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return errors.Trace(err)
}
logutil.Logger(ctx).Debug("got some exceptions, but 2PC was still successful",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
}
return nil
}
func (c *twoPhaseCommitter) stripNoNeedCommitKeys() {
if len(c.noNeedCommitKeys) == 0 {
return
}
m := &c.mutations
var newIdx int
for oldIdx := range m.keys {
key := m.keys[oldIdx]
if _, ck := c.noNeedCommitKeys[string(key)]; ck {
continue
}
m.keys[newIdx] = key
if m.ops != nil {
m.ops[newIdx] = m.ops[oldIdx]
}
if m.values != nil {
m.values[newIdx] = m.values[oldIdx]
}
if m.isPessimisticLock != nil {
m.isPessimisticLock[newIdx] = m.isPessimisticLock[oldIdx]
}
newIdx++
}
c.mutations = m.subRange(0, newIdx)
}
// SchemaVer is the infoSchema which will return the schema version.
type SchemaVer interface {
// SchemaMetaVersion returns the meta schema version.
SchemaMetaVersion() int64
}
type schemaLeaseChecker interface {
// CheckBySchemaVer checks if the schema has changed for the transaction related tables between the startSchemaVer
// and the schema version at txnTS, all the related schema changes will be returned.
CheckBySchemaVer(txnTS uint64, startSchemaVer SchemaVer) (*RelatedSchemaChange, error)
}
// RelatedSchemaChange contains information about schema diff between two schema versions.
type RelatedSchemaChange struct {
PhyTblIDS []int64
ActionTypes []uint64
LatestInfoSchema SchemaVer
Amendable bool
}
func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema SchemaVer, change *RelatedSchemaChange) (bool, error) {
addMutations, err := c.txn.schemaAmender.AmendTxn(ctx, startInfoSchema, change, c.mutations)
if err != nil {
return false, err
}
// Prewrite new mutations.
if addMutations != nil && len(addMutations.keys) > 0 {
var keysNeedToLock CommitterMutations
for i := 0; i < addMutations.len(); i++ {
if addMutations.isPessimisticLock[i] {
keysNeedToLock.Push(addMutations.ops[i], addMutations.keys[i], addMutations.values[i], addMutations.isPessimisticLock[i])
}
}
// For unique index amend, we need to pessimistic lock the generated new index keys first.
// Set doingAmend to true to force the pessimistic lock do the exist check for these keys.
c.doingAmend = true
defer func() { c.doingAmend = false }()
if keysNeedToLock.len() > 0 {
pessimisticLockBo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars)
lCtx := &kv.LockCtx{
ForUpdateTS: c.forUpdateTS,
LockWaitTime: kv.LockNoWait,
}
err = c.pessimisticLockMutations(pessimisticLockBo, lCtx, keysNeedToLock)
if err != nil {
logutil.Logger(ctx).Warn("amend pessimistic lock has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS))
return false, err
}
logutil.Logger(ctx).Info("amend pessimistic lock", zap.Uint64("startTS", c.startTS), zap.Uint64("forUpdateTS", c.forUpdateTS))
}
prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars)
err = c.prewriteMutations(prewriteBo, *addMutations)
if err != nil {
logutil.Logger(ctx).Warn("amend prewrite has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS))
return false, err
}
// Commit the amended secondary keys in the commit phase.
c.mutations.MergeMutations(*addMutations)
logutil.Logger(ctx).Info("amend prewrite finished", zap.Uint64("txnStartTS", c.startTS))
return true, nil
}
return false, nil
}
func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *execdetails.CommitDetails) (uint64, error) {
start := time.Now()
logutil.Event(ctx, "start get commit ts")
commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars))
if err != nil {
logutil.Logger(ctx).Warn("2PC get commitTS failed",
zap.Error(err),
zap.Uint64("txnStartTS", c.startTS))
return 0, errors.Trace(err)
}
commitDetail.GetCommitTsTime = time.Since(start)
logutil.Event(ctx, "finish get commit ts")
logutil.SetTag(ctx, "commitTS", commitTS)
// Check commitTS.
if commitTS <= c.startTS {
err = errors.Errorf("conn %d invalid transaction tso with txnStartTS=%v while txnCommitTS=%v",
c.connID, c.startTS, commitTS)
logutil.BgLogger().Error("invalid transaction", zap.Error(err))
return 0, errors.Trace(err)
}
return commitTS, nil
}
// checkSchemaValid checks if the schema has changed, if tryAmend is set to true, committer will try to amend
// this transaction using the related schema changes.
func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64, startInfoSchema SchemaVer,
tryAmend bool) (*RelatedSchemaChange, bool, error) {
checker, ok := c.txn.us.GetOption(kv.SchemaChecker).(schemaLeaseChecker)
if !ok {
if c.connID > 0 {
logutil.Logger(ctx).Warn("schemaLeaseChecker is not set for this transaction, schema check skipped",
zap.Uint64("connID", c.connID), zap.Uint64("startTS", c.startTS), zap.Uint64("commitTS", checkTS))
}
return nil, false, nil
}
relatedChanges, err := checker.CheckBySchemaVer(checkTS, startInfoSchema)
if err != nil {
if tryAmend && relatedChanges != nil && relatedChanges.Amendable && c.txn.schemaAmender != nil {
memAmended, amendErr := c.tryAmendTxn(ctx, startInfoSchema, relatedChanges)
if amendErr != nil {
logutil.BgLogger().Info("txn amend has failed", zap.Uint64("connID", c.connID),
zap.Uint64("startTS", c.startTS), zap.Error(amendErr))
return nil, false, err
}
logutil.Logger(ctx).Info("amend txn successfully for pessimistic commit",
zap.Uint64("connID", c.connID), zap.Uint64("txn startTS", c.startTS), zap.Bool("memAmended", memAmended),
zap.Uint64("checkTS", checkTS), zap.Int64("startInfoSchemaVer", startInfoSchema.SchemaMetaVersion()),
zap.Int64s("table ids", relatedChanges.PhyTblIDS), zap.Uint64s("action types", relatedChanges.ActionTypes))
return relatedChanges, memAmended, nil
}
return nil, false, errors.Trace(err)
}
return nil, false, nil
}
func (c *twoPhaseCommitter) prewriteBinlog(ctx context.Context) chan *binloginfo.WriteResult {
if !c.shouldWriteBinlog() {
return nil
}
ch := make(chan *binloginfo.WriteResult, 1)
go func() {
logutil.Eventf(ctx, "start prewrite binlog")
binInfo := c.txn.us.GetOption(kv.BinlogInfo).(*binloginfo.BinlogInfo)
bin := binInfo.Data
bin.StartTs = int64(c.startTS)
if bin.Tp == binlog.BinlogType_Prewrite {
bin.PrewriteKey = c.primary()
}
wr := binInfo.WriteBinlog(c.store.clusterID)
if wr.Skipped() {
binInfo.Data.PrewriteValue = nil
binloginfo.AddOneSkippedCommitter()
}
logutil.Eventf(ctx, "finish prewrite binlog")
ch <- wr
}()
return ch
}
func (c *twoPhaseCommitter) writeFinishBinlog(ctx context.Context, tp binlog.BinlogType, commitTS int64) {
if !c.shouldWriteBinlog() {
return
}
binInfo := c.txn.us.GetOption(kv.BinlogInfo).(*binloginfo.BinlogInfo)
binInfo.Data.Tp = tp
binInfo.Data.CommitTs = commitTS
binInfo.Data.PrewriteValue = nil
wg := sync.WaitGroup{}
mock := false
failpoint.Inject("mockSyncBinlogCommit", func(val failpoint.Value) {
if val.(bool) {
wg.Add(1)
mock = true
}
})
go func() {
logutil.Eventf(ctx, "start write finish binlog")
binlogWriteResult := binInfo.WriteBinlog(c.store.clusterID)
err := binlogWriteResult.GetError()
if err != nil {
logutil.BgLogger().Error("failed to write binlog",
zap.Error(err))
}
logutil.Eventf(ctx, "finish write finish binlog")
if mock {
wg.Done()
}
}()
if mock {
wg.Wait()
}
}
func (c *twoPhaseCommitter) shouldWriteBinlog() bool {
return c.txn.us.GetOption(kv.BinlogInfo) != nil
}
// TiKV recommends each RPC packet should be less than ~1MB. We keep each packet's
// Key+Value size below 16KB.
const txnCommitBatchSize = 16 * 1024
type batchMutations struct {
region RegionVerID
mutations CommitterMutations
isPrimary bool
}
// appendBatchMutationsBySize appends mutations to b. It may split the keys to make
// sure each batch's size does not exceed the limit.
func (c *twoPhaseCommitter) appendBatchMutationsBySize(b []batchMutations, region RegionVerID, mutations CommitterMutations, sizeFn func(k, v []byte) int, limit int, primaryIdx *int) []batchMutations {
var start, end int
for start = 0; start < mutations.len(); start = end {
var size int
for end = start; end < mutations.len() && size < limit; end++ {
var k, v []byte
k = mutations.keys[end]
if end < len(mutations.values) {
v = mutations.values[end]
}
size += sizeFn(k, v)
if *primaryIdx < 0 && bytes.Equal(k, c.primary()) {
*primaryIdx = len(b)
}
}
b = append(b, batchMutations{
region: region,
mutations: mutations.subRange(start, end),
})
}
return b
}
// newBatchExecutor create processor to handle concurrent batch works(prewrite/commit etc)
func newBatchExecutor(rateLimit int, committer *twoPhaseCommitter,
action twoPhaseCommitAction, backoffer *Backoffer) *batchExecutor {
return &batchExecutor{rateLimit, nil, committer,
action, backoffer, time.Duration(1 * time.Millisecond)}
}
// initUtils do initialize batchExecutor related policies like rateLimit util
func (batchExe *batchExecutor) initUtils() error {
// init rateLimiter by injected rate limit number
batchExe.rateLimiter = newRateLimit(batchExe.rateLim)
return nil
}
// startWork concurrently do the work for each batch considering rate limit
func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, batches []batchMutations) {
for idx, batch1 := range batches {
waitStart := time.Now()
if exit := batchExe.rateLimiter.getToken(exitCh); !exit {
batchExe.tokenWaitDuration += time.Since(waitStart)
batch := batch1
go func() {
defer batchExe.rateLimiter.putToken()
var singleBatchBackoffer *Backoffer
if _, ok := batchExe.action.(actionCommit); ok {
// Because the secondary batches of the commit actions are implemented to be
// committed asynchronously in background goroutines, we should not
// fork a child context and call cancel() while the foreground goroutine exits.
// Otherwise the background goroutines will be canceled execeptionally.
// Here we makes a new clone of the original backoffer for this goroutine
// exclusively to avoid the data race when using the same backoffer
// in concurrent goroutines.
singleBatchBackoffer = batchExe.backoffer.Clone()
} else {
var singleBatchCancel context.CancelFunc
singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork()
defer singleBatchCancel()
}
beforeSleep := singleBatchBackoffer.totalSleep
ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch)
commitDetail := batchExe.committer.getDetail()
if commitDetail != nil { // lock operations of pessimistic-txn will let commitDetail be nil
if delta := singleBatchBackoffer.totalSleep - beforeSleep; delta > 0 {
atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.totalSleep-beforeSleep)*int64(time.Millisecond))
commitDetail.Mu.Lock()
commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.types...)
commitDetail.Mu.Unlock()
}
}
}()
} else {
logutil.Logger(batchExe.backoffer.ctx).Info("break startWorker",
zap.Stringer("action", batchExe.action), zap.Int("batch size", len(batches)),
zap.Int("index", idx))
break
}
}
}
// process will start worker routine and collect results
func (batchExe *batchExecutor) process(batches []batchMutations) error {
var err error
err = batchExe.initUtils()
if err != nil {
logutil.Logger(batchExe.backoffer.ctx).Error("batchExecutor initUtils failed", zap.Error(err))
return err
}
// For prewrite, stop sending other requests after receiving first error.
var cancel context.CancelFunc
if _, ok := batchExe.action.(actionPrewrite); ok {
batchExe.backoffer, cancel = batchExe.backoffer.Fork()
defer cancel()
}
// concurrently do the work for each batch.
ch := make(chan error, len(batches))
exitCh := make(chan struct{})
go batchExe.startWorker(exitCh, ch, batches)
// check results
for i := 0; i < len(batches); i++ {
if e := <-ch; e != nil {
logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch failed",
zap.Uint64("conn", batchExe.committer.connID),
zap.Stringer("action type", batchExe.action),
zap.Error(e),
zap.Uint64("txnStartTS", batchExe.committer.startTS))
// Cancel other requests and return the first error.
if cancel != nil {
logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch to cancel other actions",
zap.Uint64("conn", batchExe.committer.connID),
zap.Stringer("action type", batchExe.action),
zap.Uint64("txnStartTS", batchExe.committer.startTS))
cancel()
}
if err == nil {
err = e
}
}
}
close(exitCh)
metrics.TiKVTokenWaitDuration.Observe(batchExe.tokenWaitDuration.Seconds())
return err
}