mirror of https://github.com/tikv/client-go.git
txnkv/transaction: remove use of errors.Trace (#351)
Signed-off-by: disksing <i@disksing.com>
This commit is contained in:
parent
a5250aa7c4
commit
2d145163d0
|
|
@ -1051,7 +1051,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary() {
|
||||||
s.Nil(failpoint.Disable("tikvclient/txnNotFoundRetTTL"))
|
s.Nil(failpoint.Disable("tikvclient/txnNotFoundRetTTL"))
|
||||||
s.Nil(err)
|
s.Nil(err)
|
||||||
waitErr := <-doneCh
|
waitErr := <-doneCh
|
||||||
s.Equal(tikverr.ErrLockWaitTimeout, waitErr)
|
s.Equal(tikverr.ErrLockWaitTimeout, errors.Unwrap(waitErr))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *testCommitterSuite) TestResolvePessimisticLock() {
|
func (s *testCommitterSuite) TestResolvePessimisticLock() {
|
||||||
|
|
|
||||||
|
|
@ -26,7 +26,7 @@ func TestMain(m *testing.M) {
|
||||||
|
|
||||||
opts := []goleak.Option{
|
opts := []goleak.Option{
|
||||||
goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain"),
|
goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain"),
|
||||||
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/tikv.(*ttlManager).keepAlive"), // TODO: fix ttlManager goroutine leak
|
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), // TODO: fix ttlManager goroutine leak
|
||||||
}
|
}
|
||||||
|
|
||||||
goleak.VerifyTestMain(m, opts...)
|
goleak.VerifyTestMain(m, opts...)
|
||||||
|
|
|
||||||
|
|
@ -374,7 +374,7 @@ func (c *twoPhaseCommitter) extractKeyExistsErr(err *tikverr.ErrKeyExist) error
|
||||||
if !c.txn.us.HasPresumeKeyNotExists(err.GetKey()) {
|
if !c.txn.us.HasPresumeKeyNotExists(err.GetKey()) {
|
||||||
return errors.Errorf("session %d, existErr for key:%s should not be nil", c.sessionID, err.GetKey())
|
return errors.Errorf("session %d, existErr for key:%s should not be nil", c.sessionID, err.GetKey())
|
||||||
}
|
}
|
||||||
return errors.Trace(err)
|
return errors.WithStack(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// KVFilter is a filter that filters out unnecessary KV pairs.
|
// KVFilter is a filter that filters out unnecessary KV pairs.
|
||||||
|
|
@ -484,7 +484,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error {
|
||||||
logutil.BgLogger().Error("commit failed",
|
logutil.BgLogger().Error("commit failed",
|
||||||
zap.Uint64("session", c.sessionID),
|
zap.Uint64("session", c.sessionID),
|
||||||
zap.Error(err))
|
zap.Error(err))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
commitDetail := &util.CommitDetails{WriteSize: size, WriteKeys: c.mutations.Len()}
|
commitDetail := &util.CommitDetails{WriteSize: size, WriteKeys: c.mutations.Len()}
|
||||||
|
|
@ -565,7 +565,7 @@ func (c *twoPhaseCommitter) doActionOnMutations(bo *retry.Backoffer, action twoP
|
||||||
}
|
}
|
||||||
groups, err := c.groupMutations(bo, mutations)
|
groups, err := c.groupMutations(bo, mutations)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// This is redundant since `doActionOnGroupMutations` will still split groups into batches and
|
// This is redundant since `doActionOnGroupMutations` will still split groups into batches and
|
||||||
|
|
@ -599,7 +599,7 @@ func groupSortedMutationsByRegion(c *locate.RegionCache, bo *retry.Backoffer, m
|
||||||
var err error
|
var err error
|
||||||
lastLoc, err = c.LocateKey(bo, m.GetKey(i))
|
lastLoc, err = c.LocateKey(bo, m.GetKey(i))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -616,7 +616,7 @@ func groupSortedMutationsByRegion(c *locate.RegionCache, bo *retry.Backoffer, m
|
||||||
func (c *twoPhaseCommitter) groupMutations(bo *retry.Backoffer, mutations CommitterMutations) ([]groupedMutations, error) {
|
func (c *twoPhaseCommitter) groupMutations(bo *retry.Backoffer, mutations CommitterMutations) ([]groupedMutations, error) {
|
||||||
groups, err := groupSortedMutationsByRegion(c.store.GetRegionCache(), bo, mutations)
|
groups, err := groupSortedMutationsByRegion(c.store.GetRegionCache(), bo, mutations)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Pre-split regions to avoid too much write workload into a single region.
|
// Pre-split regions to avoid too much write workload into a single region.
|
||||||
|
|
@ -637,7 +637,7 @@ func (c *twoPhaseCommitter) groupMutations(bo *retry.Backoffer, mutations Commit
|
||||||
if didPreSplit {
|
if didPreSplit {
|
||||||
groups, err = groupSortedMutationsByRegion(c.store.GetRegionCache(), bo, mutations)
|
groups, err = groupSortedMutationsByRegion(c.store.GetRegionCache(), bo, mutations)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -749,7 +749,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *retry.Backoffer, action
|
||||||
// primary should be committed(not async commit)/cleanup/pessimistically locked first
|
// primary should be committed(not async commit)/cleanup/pessimistically locked first
|
||||||
err = c.doActionOnBatches(bo, action, batchBuilder.primaryBatch())
|
err = c.doActionOnBatches(bo, action, batchBuilder.primaryBatch())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if actionIsCommit && c.testingKnobs.bkAfterCommitPrimary != nil && c.testingKnobs.acAfterCommitPrimary != nil {
|
if actionIsCommit && c.testingKnobs.bkAfterCommitPrimary != nil && c.testingKnobs.acAfterCommitPrimary != nil {
|
||||||
c.testingKnobs.acAfterCommitPrimary <- struct{}{}
|
c.testingKnobs.acAfterCommitPrimary <- struct{}{}
|
||||||
|
|
@ -791,7 +791,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *retry.Backoffer, action
|
||||||
} else {
|
} else {
|
||||||
err = c.doActionOnBatches(bo, action, batchBuilder.allBatches())
|
err = c.doActionOnBatches(bo, action, batchBuilder.allBatches())
|
||||||
}
|
}
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// doActionOnBatches does action to batches in parallel.
|
// doActionOnBatches does action to batches in parallel.
|
||||||
|
|
@ -815,7 +815,7 @@ func (c *twoPhaseCommitter) doActionOnBatches(bo *retry.Backoffer, action twoPha
|
||||||
zap.Stringer("action type", action),
|
zap.Stringer("action type", action),
|
||||||
zap.Error(e),
|
zap.Error(e),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(e)
|
return e
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
|
|
@ -829,8 +829,7 @@ func (c *twoPhaseCommitter) doActionOnBatches(bo *retry.Backoffer, action twoPha
|
||||||
rateLim = config.GetGlobalConfig().CommitterConcurrency
|
rateLim = config.GetGlobalConfig().CommitterConcurrency
|
||||||
}
|
}
|
||||||
batchExecutor := newBatchExecutor(rateLim, c, action, bo)
|
batchExecutor := newBatchExecutor(rateLim, c, action, bo)
|
||||||
err := batchExecutor.process(batches)
|
return batchExecutor.process(batches)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *twoPhaseCommitter) keyValueSize(key, value []byte) int {
|
func (c *twoPhaseCommitter) keyValueSize(key, value []byte) int {
|
||||||
|
|
@ -966,16 +965,16 @@ func sendTxnHeartBeat(bo *retry.Backoffer, store kvstore, primary []byte, startT
|
||||||
for {
|
for {
|
||||||
loc, err := store.GetRegionCache().LocateKey(bo, primary)
|
loc, err := store.GetRegionCache().LocateKey(bo, primary)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, false, errors.Trace(err)
|
return 0, false, err
|
||||||
}
|
}
|
||||||
req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds())
|
req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds())
|
||||||
resp, err := store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort)
|
resp, err := store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, false, errors.Trace(err)
|
return 0, false, err
|
||||||
}
|
}
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, false, errors.Trace(err)
|
return 0, false, err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
// For other region error and the fake region error, backoff because
|
// For other region error and the fake region error, backoff because
|
||||||
|
|
@ -984,13 +983,13 @@ func sendTxnHeartBeat(bo *retry.Backoffer, store kvstore, primary []byte, startT
|
||||||
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, false, errors.Trace(err)
|
return 0, false, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if resp.Resp == nil {
|
if resp.Resp == nil {
|
||||||
return 0, false, errors.Trace(tikverr.ErrBodyMissing)
|
return 0, false, errors.WithStack(tikverr.ErrBodyMissing)
|
||||||
}
|
}
|
||||||
cmdResp := resp.Resp.(*kvrpcpb.TxnHeartBeatResponse)
|
cmdResp := resp.Resp.(*kvrpcpb.TxnHeartBeatResponse)
|
||||||
if keyErr := cmdResp.GetError(); keyErr != nil {
|
if keyErr := cmdResp.GetError(); keyErr != nil {
|
||||||
|
|
@ -1195,7 +1194,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
// instead of falling back to the normal 2PC because a normal 2PC will
|
// instead of falling back to the normal 2PC because a normal 2PC will
|
||||||
// also be likely to fail due to the same timestamp issue.
|
// also be likely to fail due to the same timestamp issue.
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
// Plus 1 to avoid producing the same commit TS with previously committed transactions
|
// Plus 1 to avoid producing the same commit TS with previously committed transactions
|
||||||
c.minCommitTS = latestTS + 1
|
c.minCommitTS = latestTS + 1
|
||||||
|
|
@ -1203,7 +1202,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
// Calculate maxCommitTS if necessary
|
// Calculate maxCommitTS if necessary
|
||||||
if commitTSMayBeCalculated {
|
if commitTSMayBeCalculated {
|
||||||
if err = c.calculateMaxCommitTS(ctx); err != nil {
|
if err = c.calculateMaxCommitTS(ctx); err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1229,7 +1228,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.NamedError("rpcErr", undeterminedErr),
|
zap.NamedError("rpcErr", undeterminedErr),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(tikverr.ErrResultUndetermined)
|
return errors.WithStack(tikverr.ErrResultUndetermined)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1261,7 +1260,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
logutil.Logger(ctx).Debug("2PC failed on prewrite",
|
logutil.Logger(ctx).Debug("2PC failed on prewrite",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// strip check_not_exists keys that no need to commit.
|
// strip check_not_exists keys that no need to commit.
|
||||||
|
|
@ -1271,8 +1270,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
|
|
||||||
if c.isOnePC() {
|
if c.isOnePC() {
|
||||||
if c.onePCCommitTS == 0 {
|
if c.onePCCommitTS == 0 {
|
||||||
err = errors.Errorf("session %d invalid onePCCommitTS for 1PC protocol after prewrite, startTS=%v", c.sessionID, c.startTS)
|
return errors.Errorf("session %d invalid onePCCommitTS for 1PC protocol after prewrite, startTS=%v", c.sessionID, c.startTS)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
c.commitTS = c.onePCCommitTS
|
c.commitTS = c.onePCCommitTS
|
||||||
c.txn.commitTS = c.commitTS
|
c.txn.commitTS = c.commitTS
|
||||||
|
|
@ -1289,8 +1287,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
|
|
||||||
if c.isAsyncCommit() {
|
if c.isAsyncCommit() {
|
||||||
if c.minCommitTS == 0 {
|
if c.minCommitTS == 0 {
|
||||||
err = errors.Errorf("session %d invalid minCommitTS for async commit protocol after prewrite, startTS=%v", c.sessionID, c.startTS)
|
return errors.Errorf("session %d invalid minCommitTS for async commit protocol after prewrite, startTS=%v", c.sessionID, c.startTS)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
commitTS = c.minCommitTS
|
commitTS = c.minCommitTS
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -1301,7 +1298,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
logutil.Logger(ctx).Warn("2PC get commitTS failed",
|
logutil.Logger(ctx).Warn("2PC get commitTS failed",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
commitDetail.GetCommitTsTime = time.Since(start)
|
commitDetail.GetCommitTsTime = time.Since(start)
|
||||||
logutil.Event(ctx, "finish get commit ts")
|
logutil.Event(ctx, "finish get commit ts")
|
||||||
|
|
@ -1313,18 +1310,18 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
if !tryAmend {
|
if !tryAmend {
|
||||||
_, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, false)
|
_, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, true)
|
relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if memAmended {
|
if memAmended {
|
||||||
// Get new commitTS and check schema valid again.
|
// Get new commitTS and check schema valid again.
|
||||||
newCommitTS, err := c.getCommitTS(ctx, commitDetail)
|
newCommitTS, err := c.getCommitTS(ctx, commitDetail)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
// If schema check failed between commitTS and newCommitTs, report schema change error.
|
// If schema check failed between commitTS and newCommitTs, report schema change error.
|
||||||
_, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false)
|
_, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false)
|
||||||
|
|
@ -1334,7 +1331,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
|
||||||
zap.Uint64("amendTS", commitTS),
|
zap.Uint64("amendTS", commitTS),
|
||||||
zap.Int64("amendedSchemaVersion", relatedSchemaChange.LatestInfoSchema.SchemaMetaVersion()),
|
zap.Int64("amendedSchemaVersion", relatedSchemaChange.LatestInfoSchema.SchemaMetaVersion()),
|
||||||
zap.Uint64("newCommitTS", newCommitTS))
|
zap.Uint64("newCommitTS", newCommitTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
commitTS = newCommitTS
|
commitTS = newCommitTS
|
||||||
}
|
}
|
||||||
|
|
@ -1413,13 +1410,13 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.NamedError("rpcErr", undeterminedErr),
|
zap.NamedError("rpcErr", undeterminedErr),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
err = errors.Trace(tikverr.ErrResultUndetermined)
|
err = errors.WithStack(tikverr.ErrResultUndetermined)
|
||||||
}
|
}
|
||||||
if !c.mu.committed {
|
if !c.mu.committed {
|
||||||
logutil.Logger(ctx).Debug("2PC failed on commit",
|
logutil.Logger(ctx).Debug("2PC failed on commit",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
logutil.Logger(ctx).Debug("got some exceptions, but 2PC was still successful",
|
logutil.Logger(ctx).Debug("got some exceptions, but 2PC was still successful",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
|
|
@ -1492,7 +1489,7 @@ func (c *twoPhaseCommitter) amendPessimisticLock(ctx context.Context, addMutatio
|
||||||
if _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict); ok {
|
if _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict); ok {
|
||||||
newForUpdateTSVer, err := c.store.CurrentTimestamp(oracle.GlobalTxnScope)
|
newForUpdateTSVer, err := c.store.CurrentTimestamp(oracle.GlobalTxnScope)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
lCtx.ForUpdateTS = newForUpdateTSVer
|
lCtx.ForUpdateTS = newForUpdateTSVer
|
||||||
c.forUpdateTS = newForUpdateTSVer
|
c.forUpdateTS = newForUpdateTSVer
|
||||||
|
|
@ -1569,7 +1566,7 @@ func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.
|
||||||
logutil.Logger(ctx).Warn("2PC get commitTS failed",
|
logutil.Logger(ctx).Warn("2PC get commitTS failed",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return 0, errors.Trace(err)
|
return 0, err
|
||||||
}
|
}
|
||||||
commitDetail.GetCommitTsTime = time.Since(start)
|
commitDetail.GetCommitTsTime = time.Since(start)
|
||||||
logutil.Event(ctx, "finish get commit ts")
|
logutil.Event(ctx, "finish get commit ts")
|
||||||
|
|
@ -1580,7 +1577,7 @@ func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.
|
||||||
err = errors.Errorf("session %d invalid transaction tso with txnStartTS=%v while txnCommitTS=%v",
|
err = errors.Errorf("session %d invalid transaction tso with txnStartTS=%v while txnCommitTS=%v",
|
||||||
c.sessionID, c.startTS, commitTS)
|
c.sessionID, c.startTS, commitTS)
|
||||||
logutil.BgLogger().Error("invalid transaction", zap.Error(err))
|
logutil.BgLogger().Error("invalid transaction", zap.Error(err))
|
||||||
return 0, errors.Trace(err)
|
return 0, err
|
||||||
}
|
}
|
||||||
return commitTS, nil
|
return commitTS, nil
|
||||||
}
|
}
|
||||||
|
|
@ -1592,8 +1589,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64
|
||||||
if _, err := util.EvalFailpoint("failCheckSchemaValid"); err == nil {
|
if _, err := util.EvalFailpoint("failCheckSchemaValid"); err == nil {
|
||||||
logutil.Logger(ctx).Info("[failpoint] injected fail schema check",
|
logutil.Logger(ctx).Info("[failpoint] injected fail schema check",
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
err := errors.Errorf("mock check schema valid failure")
|
return nil, false, errors.Errorf("mock check schema valid failure")
|
||||||
return nil, false, err
|
|
||||||
}
|
}
|
||||||
if c.txn.schemaLeaseChecker == nil {
|
if c.txn.schemaLeaseChecker == nil {
|
||||||
if c.sessionID > 0 {
|
if c.sessionID > 0 {
|
||||||
|
|
@ -1611,7 +1607,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64
|
||||||
if amendErr != nil {
|
if amendErr != nil {
|
||||||
logutil.BgLogger().Info("txn amend has failed", zap.Uint64("sessionID", c.sessionID),
|
logutil.BgLogger().Info("txn amend has failed", zap.Uint64("sessionID", c.sessionID),
|
||||||
zap.Uint64("startTS", c.startTS), zap.Error(amendErr))
|
zap.Uint64("startTS", c.startTS), zap.Error(amendErr))
|
||||||
return nil, false, err
|
return nil, false, errors.WithStack(err)
|
||||||
}
|
}
|
||||||
logutil.Logger(ctx).Info("amend txn successfully",
|
logutil.Logger(ctx).Info("amend txn successfully",
|
||||||
zap.Uint64("sessionID", c.sessionID), zap.Uint64("txn startTS", c.startTS), zap.Bool("memAmended", memAmended),
|
zap.Uint64("sessionID", c.sessionID), zap.Uint64("txn startTS", c.startTS), zap.Bool("memAmended", memAmended),
|
||||||
|
|
@ -1619,7 +1615,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64
|
||||||
zap.Int64s("table ids", relatedChanges.PhyTblIDS), zap.Uint64s("action types", relatedChanges.ActionTypes))
|
zap.Int64s("table ids", relatedChanges.PhyTblIDS), zap.Uint64s("action types", relatedChanges.ActionTypes))
|
||||||
return relatedChanges, memAmended, nil
|
return relatedChanges, memAmended, nil
|
||||||
}
|
}
|
||||||
return nil, false, errors.Trace(err)
|
return nil, false, errors.WithStack(err)
|
||||||
}
|
}
|
||||||
return nil, false, nil
|
return nil, false, nil
|
||||||
}
|
}
|
||||||
|
|
@ -1632,7 +1628,7 @@ func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error {
|
||||||
logutil.Logger(ctx).Info("Schema changed for async commit txn",
|
logutil.Logger(ctx).Info("Schema changed for async commit txn",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("startTS", c.startTS))
|
zap.Uint64("startTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
safeWindow := config.GetGlobalConfig().TiKVClient.AsyncCommit.SafeWindow
|
safeWindow := config.GetGlobalConfig().TiKVClient.AsyncCommit.SafeWindow
|
||||||
|
|
@ -1665,7 +1661,7 @@ func (b *batchMutations) relocate(bo *retry.Backoffer, c *locate.RegionCache) (b
|
||||||
begin, end := b.mutations.GetKey(0), b.mutations.GetKey(b.mutations.Len()-1)
|
begin, end := b.mutations.GetKey(0), b.mutations.GetKey(b.mutations.Len()-1)
|
||||||
loc, err := c.LocateKey(bo, begin)
|
loc, err := c.LocateKey(bo, begin)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, errors.Trace(err)
|
return false, err
|
||||||
}
|
}
|
||||||
if !loc.Contains(end) {
|
if !loc.Contains(end) {
|
||||||
return false, nil
|
return false, nil
|
||||||
|
|
|
||||||
|
|
@ -37,7 +37,6 @@ package transaction
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/pingcap/errors"
|
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
)
|
)
|
||||||
|
|
@ -79,13 +78,13 @@ func (b *BufferBatchGetter) BatchGet(ctx context.Context, keys [][]byte) (map[st
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if !tikverr.IsErrNotFound(err) {
|
if !tikverr.IsErrNotFound(err) {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
shrinkKeys = append(shrinkKeys, key)
|
shrinkKeys = append(shrinkKeys, key)
|
||||||
}
|
}
|
||||||
storageValues, err := b.snapshot.BatchGet(ctx, shrinkKeys)
|
storageValues, err := b.snapshot.BatchGet(ctx, shrinkKeys)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
for i, key := range keys {
|
for i, key := range keys {
|
||||||
if len(bufferValues[i]) == 0 {
|
if len(bufferValues[i]) == 0 {
|
||||||
|
|
|
||||||
|
|
@ -66,26 +66,25 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer
|
||||||
MaxExecutionDurationMs: uint64(client.MaxWriteExecutionTime.Milliseconds())})
|
MaxExecutionDurationMs: uint64(client.MaxWriteExecutionTime.Milliseconds())})
|
||||||
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
err = c.cleanupMutations(bo, batch.mutations)
|
return c.cleanupMutations(bo, batch.mutations)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
if keyErr := resp.Resp.(*kvrpcpb.BatchRollbackResponse).GetError(); keyErr != nil {
|
if keyErr := resp.Resp.(*kvrpcpb.BatchRollbackResponse).GetError(); keyErr != nil {
|
||||||
err = errors.Errorf("session %d 2PC cleanup failed: %s", c.sessionID, keyErr)
|
err = errors.Errorf("session %d 2PC cleanup failed: %s", c.sessionID, keyErr)
|
||||||
logutil.BgLogger().Debug("2PC failed cleanup key",
|
logutil.BgLogger().Debug("2PC failed cleanup key",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -92,17 +92,17 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer,
|
||||||
// an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best
|
// 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.
|
// solution is to populate this error and let upper layer drop the connection to the corresponding mysql client.
|
||||||
if batch.isPrimary && sender.GetRPCError() != nil && !c.isAsyncCommit() {
|
if batch.isPrimary && sender.GetRPCError() != nil && !c.isAsyncCommit() {
|
||||||
c.setUndeterminedErr(errors.Trace(sender.GetRPCError()))
|
c.setUndeterminedErr(errors.WithStack(sender.GetRPCError()))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unexpected error occurs, return it.
|
// Unexpected error occurs, return it.
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
// For other region error and the fake region error, backoff because
|
// For other region error and the fake region error, backoff because
|
||||||
|
|
@ -111,22 +111,21 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer,
|
||||||
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if same {
|
if same {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
err = c.doActionOnMutations(bo, actionCommit{true}, batch.mutations)
|
return c.doActionOnMutations(bo, actionCommit{true}, batch.mutations)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if resp.Resp == nil {
|
if resp.Resp == nil {
|
||||||
return errors.Trace(tikverr.ErrBodyMissing)
|
return errors.WithStack(tikverr.ErrBodyMissing)
|
||||||
}
|
}
|
||||||
commitResp := resp.Resp.(*kvrpcpb.CommitResponse)
|
commitResp := resp.Resp.(*kvrpcpb.CommitResponse)
|
||||||
// Here we can make sure tikv has processed the commit primary key request. So
|
// Here we can make sure tikv has processed the commit primary key request. So
|
||||||
|
|
@ -143,9 +142,8 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer,
|
||||||
// Do not retry for a txn which has a too large MinCommitTs
|
// Do not retry for a txn which has a too large MinCommitTs
|
||||||
// 3600000 << 18 = 943718400000
|
// 3600000 << 18 = 943718400000
|
||||||
if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 {
|
if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 {
|
||||||
err := errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d",
|
return errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d",
|
||||||
rejected.MinCommitTs, rejected.AttemptedCommitTs)
|
rejected.MinCommitTs, rejected.AttemptedCommitTs)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Update commit ts and retry.
|
// Update commit ts and retry.
|
||||||
|
|
@ -154,7 +152,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer,
|
||||||
logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed",
|
logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed",
|
||||||
zap.Error(err),
|
zap.Error(err),
|
||||||
zap.Uint64("txnStartTS", c.startTS))
|
zap.Uint64("txnStartTS", c.startTS))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
c.mu.Lock()
|
c.mu.Lock()
|
||||||
|
|
@ -183,7 +181,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *retry.Backoffer,
|
||||||
zap.Uint64("txnStartTS", c.startTS),
|
zap.Uint64("txnStartTS", c.startTS),
|
||||||
zap.Uint64("commitTS", c.commitTS),
|
zap.Uint64("commitTS", c.commitTS),
|
||||||
zap.Strings("keys", hexBatchKeys(keys)))
|
zap.Strings("keys", hexBatchKeys(keys)))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
// The transaction maybe rolled back by concurrent transactions.
|
// The transaction maybe rolled back by concurrent transactions.
|
||||||
logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key",
|
logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key",
|
||||||
|
|
|
||||||
|
|
@ -132,7 +132,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
}
|
}
|
||||||
if _, err := util.EvalFailpoint("PessimisticLockErrWriteConflict"); err == nil {
|
if _, err := util.EvalFailpoint("PessimisticLockErrWriteConflict"); err == nil {
|
||||||
time.Sleep(300 * time.Millisecond)
|
time.Sleep(300 * time.Millisecond)
|
||||||
return &tikverr.ErrWriteConflict{WriteConflict: nil}
|
return errors.WithStack(&tikverr.ErrWriteConflict{WriteConflict: nil})
|
||||||
}
|
}
|
||||||
startTime := time.Now()
|
startTime := time.Now()
|
||||||
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
||||||
|
|
@ -141,11 +141,11 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1)
|
atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
// For other region error and the fake region error, backoff because
|
// For other region error and the fake region error, backoff because
|
||||||
|
|
@ -154,21 +154,21 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if same {
|
if same {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
err = c.pessimisticLockMutations(bo, action.LockCtx, batch.mutations)
|
err = c.pessimisticLockMutations(bo, action.LockCtx, batch.mutations)
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if resp.Resp == nil {
|
if resp.Resp == nil {
|
||||||
return errors.Trace(tikverr.ErrBodyMissing)
|
return errors.WithStack(tikverr.ErrBodyMissing)
|
||||||
}
|
}
|
||||||
lockResp := resp.Resp.(*kvrpcpb.PessimisticLockResponse)
|
lockResp := resp.Resp.(*kvrpcpb.PessimisticLockResponse)
|
||||||
keyErrs := lockResp.GetErrors()
|
keyErrs := lockResp.GetErrors()
|
||||||
|
|
@ -196,13 +196,13 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
return c.extractKeyExistsErr(e)
|
return c.extractKeyExistsErr(e)
|
||||||
}
|
}
|
||||||
if deadlock := keyErr.Deadlock; deadlock != nil {
|
if deadlock := keyErr.Deadlock; deadlock != nil {
|
||||||
return &tikverr.ErrDeadlock{Deadlock: deadlock}
|
return errors.WithStack(&tikverr.ErrDeadlock{Deadlock: deadlock})
|
||||||
}
|
}
|
||||||
|
|
||||||
// Extract lock from key error
|
// Extract lock from key error
|
||||||
lock, err1 := txnlock.ExtractLockFromKeyErr(keyErr)
|
lock, err1 := txnlock.ExtractLockFromKeyErr(keyErr)
|
||||||
if err1 != nil {
|
if err1 != nil {
|
||||||
return errors.Trace(err1)
|
return err1
|
||||||
}
|
}
|
||||||
locks = append(locks, lock)
|
locks = append(locks, lock)
|
||||||
}
|
}
|
||||||
|
|
@ -211,7 +211,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
startTime = time.Now()
|
startTime = time.Now()
|
||||||
msBeforeTxnExpired, _, err := c.store.GetLockResolver().ResolveLocks(bo, 0, locks)
|
msBeforeTxnExpired, _, err := c.store.GetLockResolver().ResolveLocks(bo, 0, locks)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if action.LockCtx.Stats != nil {
|
if action.LockCtx.Stats != nil {
|
||||||
atomic.AddInt64(&action.LockCtx.Stats.ResolveLockTime, int64(time.Since(startTime)))
|
atomic.AddInt64(&action.LockCtx.Stats.ResolveLockTime, int64(time.Since(startTime)))
|
||||||
|
|
@ -221,13 +221,13 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
// the pessimistic lock. We should return acquire fail with nowait set or timeout error if necessary.
|
// the pessimistic lock. We should return acquire fail with nowait set or timeout error if necessary.
|
||||||
if msBeforeTxnExpired > 0 {
|
if msBeforeTxnExpired > 0 {
|
||||||
if action.LockWaitTime() == kv.LockNoWait {
|
if action.LockWaitTime() == kv.LockNoWait {
|
||||||
return tikverr.ErrLockAcquireFailAndNoWaitSet
|
return errors.WithStack(tikverr.ErrLockAcquireFailAndNoWaitSet)
|
||||||
} else if action.LockWaitTime() == kv.LockAlwaysWait {
|
} else if action.LockWaitTime() == kv.LockAlwaysWait {
|
||||||
// do nothing but keep wait
|
// do nothing but keep wait
|
||||||
} else {
|
} else {
|
||||||
// the lockWaitTime is set, we should return wait timeout if we are still blocked by a lock
|
// the lockWaitTime is set, we should return wait timeout if we are still blocked by a lock
|
||||||
if time.Since(lockWaitStartTime).Milliseconds() >= action.LockWaitTime() {
|
if time.Since(lockWaitStartTime).Milliseconds() >= action.LockWaitTime() {
|
||||||
return errors.Trace(tikverr.ErrLockWaitTimeout)
|
return errors.WithStack(tikverr.ErrLockWaitTimeout)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if action.LockCtx.PessimisticLockWaited != nil {
|
if action.LockCtx.PessimisticLockWaited != nil {
|
||||||
|
|
@ -243,7 +243,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *
|
||||||
// actionPessimisticLock runs on each region parallelly, we have to consider that
|
// actionPessimisticLock runs on each region parallelly, we have to consider that
|
||||||
// the error may be dropped.
|
// the error may be dropped.
|
||||||
if atomic.LoadUint32(action.Killed) == 1 {
|
if atomic.LoadUint32(action.Killed) == 1 {
|
||||||
return errors.Trace(tikverr.ErrQueryInterrupted)
|
return errors.WithStack(tikverr.ErrQueryInterrupted)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -258,19 +258,18 @@ func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *ret
|
||||||
req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds())
|
req.MaxExecutionDurationMs = uint64(client.MaxWriteExecutionTime.Milliseconds())
|
||||||
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
err = c.pessimisticRollbackMutations(bo, batch.mutations)
|
return c.pessimisticRollbackMutations(bo, batch.mutations)
|
||||||
return errors.Trace(err)
|
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -194,7 +194,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
// If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined
|
// If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined
|
||||||
// errors.
|
// errors.
|
||||||
if (c.isAsyncCommit() || c.isOnePC()) && sender.GetRPCError() != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 {
|
if (c.isAsyncCommit() || c.isOnePC()) && sender.GetRPCError() != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 {
|
||||||
c.setUndeterminedErr(errors.Trace(sender.GetRPCError()))
|
c.setUndeterminedErr(sender.GetRPCError())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
@ -208,12 +208,12 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort)
|
||||||
// Unexpected error occurs, return it
|
// Unexpected error occurs, return it
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
regionErr, err := resp.GetRegionError()
|
regionErr, err := resp.GetRegionError()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if regionErr != nil {
|
if regionErr != nil {
|
||||||
// For other region error and the fake region error, backoff because
|
// For other region error and the fake region error, backoff because
|
||||||
|
|
@ -222,7 +222,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) {
|
||||||
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String()))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if regionErr.GetDiskFull() != nil {
|
if regionErr.GetDiskFull() != nil {
|
||||||
|
|
@ -236,21 +236,21 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
zap.String("store_id", desc),
|
zap.String("store_id", desc),
|
||||||
zap.String("reason", regionErr.GetDiskFull().GetReason()))
|
zap.String("reason", regionErr.GetDiskFull().GetReason()))
|
||||||
|
|
||||||
return errors.Trace(errors.New(regionErr.String()))
|
return errors.New(regionErr.String())
|
||||||
}
|
}
|
||||||
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
same, err := batch.relocate(bo, c.store.GetRegionCache())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if same {
|
if same {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
err = c.doActionOnMutations(bo, actionPrewrite{true}, batch.mutations)
|
err = c.doActionOnMutations(bo, actionPrewrite{true}, batch.mutations)
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
if resp.Resp == nil {
|
if resp.Resp == nil {
|
||||||
return errors.Trace(tikverr.ErrBodyMissing)
|
return errors.WithStack(tikverr.ErrBodyMissing)
|
||||||
}
|
}
|
||||||
prewriteResp := resp.Resp.(*kvrpcpb.PrewriteResponse)
|
prewriteResp := resp.Resp.(*kvrpcpb.PrewriteResponse)
|
||||||
keyErrs := prewriteResp.GetErrors()
|
keyErrs := prewriteResp.GetErrors()
|
||||||
|
|
@ -271,7 +271,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
if c.isOnePC() {
|
if c.isOnePC() {
|
||||||
if prewriteResp.OnePcCommitTs == 0 {
|
if prewriteResp.OnePcCommitTs == 0 {
|
||||||
if prewriteResp.MinCommitTs != 0 {
|
if prewriteResp.MinCommitTs != 0 {
|
||||||
return errors.Trace(errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc"))
|
return errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc")
|
||||||
}
|
}
|
||||||
logutil.Logger(bo.GetCtx()).Warn("1pc failed and fallbacks to normal commit procedure",
|
logutil.Logger(bo.GetCtx()).Warn("1pc failed and fallbacks to normal commit procedure",
|
||||||
zap.Uint64("startTS", c.startTS))
|
zap.Uint64("startTS", c.startTS))
|
||||||
|
|
@ -324,7 +324,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
// Extract lock from key error
|
// Extract lock from key error
|
||||||
lock, err1 := txnlock.ExtractLockFromKeyErr(keyErr)
|
lock, err1 := txnlock.ExtractLockFromKeyErr(keyErr)
|
||||||
if err1 != nil {
|
if err1 != nil {
|
||||||
return errors.Trace(err1)
|
return err1
|
||||||
}
|
}
|
||||||
logutil.BgLogger().Info("prewrite encounters lock",
|
logutil.BgLogger().Info("prewrite encounters lock",
|
||||||
zap.Uint64("session", c.sessionID),
|
zap.Uint64("session", c.sessionID),
|
||||||
|
|
@ -334,13 +334,13 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *retry.B
|
||||||
start := time.Now()
|
start := time.Now()
|
||||||
msBeforeExpired, err := c.store.GetLockResolver().ResolveLocksForWrite(bo, c.startTS, c.forUpdateTS, locks)
|
msBeforeExpired, err := c.store.GetLockResolver().ResolveLocksForWrite(bo, c.startTS, c.forUpdateTS, locks)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start)))
|
atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start)))
|
||||||
if msBeforeExpired > 0 {
|
if msBeforeExpired > 0 {
|
||||||
err = bo.BackoffWithCfgAndMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks)))
|
err = bo.BackoffWithCfgAndMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks)))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,7 +20,6 @@ import (
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/pingcap/errors"
|
|
||||||
"github.com/tikv/client-go/v2/internal/locate"
|
"github.com/tikv/client-go/v2/internal/locate"
|
||||||
"github.com/tikv/client-go/v2/internal/retry"
|
"github.com/tikv/client-go/v2/internal/retry"
|
||||||
"github.com/tikv/client-go/v2/internal/unionstore"
|
"github.com/tikv/client-go/v2/internal/unionstore"
|
||||||
|
|
@ -95,10 +94,10 @@ func (txn TxnProbe) GetStartTime() time.Time {
|
||||||
func newTwoPhaseCommitterWithInit(txn *KVTxn, sessionID uint64) (*twoPhaseCommitter, error) {
|
func newTwoPhaseCommitterWithInit(txn *KVTxn, sessionID uint64) (*twoPhaseCommitter, error) {
|
||||||
c, err := newTwoPhaseCommitter(txn, sessionID)
|
c, err := newTwoPhaseCommitter(txn, sessionID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
if err = c.initKeysAndMutations(); err != nil {
|
if err = c.initKeysAndMutations(); err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
return c, nil
|
return c, nil
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -157,7 +157,7 @@ func (txn *KVTxn) Get(ctx context.Context, k []byte) ([]byte, error) {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Trace(err)
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return ret, nil
|
return ret, nil
|
||||||
|
|
@ -339,7 +339,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error {
|
||||||
if committer == nil {
|
if committer == nil {
|
||||||
committer, err = newTwoPhaseCommitter(txn, sessionID)
|
committer, err = newTwoPhaseCommitter(txn, sessionID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
txn.committer = committer
|
txn.committer = committer
|
||||||
}
|
}
|
||||||
|
|
@ -352,7 +352,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error {
|
||||||
err = committer.initKeysAndMutations()
|
err = committer.initKeysAndMutations()
|
||||||
initRegion.End()
|
initRegion.End()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
if committer.mutations.Len() == 0 {
|
if committer.mutations.Len() == 0 {
|
||||||
return nil
|
return nil
|
||||||
|
|
@ -383,7 +383,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error {
|
||||||
txn.onCommitted(err)
|
txn.onCommitted(err)
|
||||||
}
|
}
|
||||||
logutil.Logger(ctx).Debug("[kv] txnLatches disabled, 2pc directly", zap.Error(err))
|
logutil.Logger(ctx).Debug("[kv] txnLatches disabled, 2pc directly", zap.Error(err))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// latches enabled
|
// latches enabled
|
||||||
|
|
@ -407,7 +407,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error {
|
||||||
lock.SetCommitTS(committer.commitTS)
|
lock.SetCommitTS(committer.commitTS)
|
||||||
}
|
}
|
||||||
logutil.Logger(ctx).Debug("[kv] txnLatches enabled while txn retryable", zap.Error(err))
|
logutil.Logger(ctx).Debug("[kv] txnLatches enabled while txn retryable", zap.Error(err))
|
||||||
return errors.Trace(err)
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
func (txn *KVTxn) close() {
|
func (txn *KVTxn) close() {
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue