From cc05adf9dc98ce8b189f8d98ed6a19e6ddc4eb28 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 13 Jun 2022 20:26:41 +0800 Subject: [PATCH 01/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 4 +- executor/point_get.go | 47 +++++++++++++------- sessiontxn/isolation/main_test.go | 21 +++++++++ sessiontxn/isolation/repeatable_read.go | 15 ++++--- sessiontxn/isolation/repeatable_reat_test.go | 30 +++++++++++++ 5 files changed, 95 insertions(+), 22 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index b98597820649b..0654310cd9c60 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -255,7 +255,9 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec } else { // CachedPlan type is already checked in last step pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan) - exec.Init(pointGetPlan, startTs) + if err := exec.Init(pointGetPlan, true); err != nil { + return nil, err + } a.PsStmt.Executor = exec } } diff --git a/executor/point_get.go b/executor/point_get.go index f33ba20b5dd5a..cb094d6c0d5eb 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -17,7 +17,6 @@ package executor import ( "context" "fmt" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" + "math" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -49,23 +50,23 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { return nil } - startTS, err := b.getSnapshotTS() - if err != nil { - b.err = err - return nil - } e := &PointGetExecutor{ baseExecutor: newBaseExecutor(b.ctx, p.Schema(), p.ID()), readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, } - if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(p.TblInfo, startTS) - } e.base().initCap = 1 e.base().maxChunkSize = 1 - e.Init(p, startTS) + err := e.Init(p, false) + if err != nil { + b.err = err + } + + if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { + e.cacheTable = b.getCacheTable(p.TblInfo, e.startTS) + } + if e.lock { b.hasLock = true } @@ -106,13 +107,12 @@ type PointGetExecutor struct { } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field -func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, startTs uint64) { +func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool) error { decoder := NewRowDecoder(e.ctx, p.Schema(), p.TblInfo) e.tblInfo = p.TblInfo e.handle = p.Handle e.idxInfo = p.IndexInfo e.idxVals = p.IndexValues - e.startTS = startTs e.done = false if e.tblInfo.TempTableType == model.TempTableNone { e.lock = p.Lock @@ -122,10 +122,30 @@ func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, startTs uint64) { e.lock = false e.lockWaitTime = 0 } + + if useMaxTs { + e.startTS = uint64(math.MaxUint64) + } else { + var startTs uint64 + var err error + txnManager := sessiontxn.GetTxnManager(e.ctx) + if e.lock { + if startTs, err = txnManager.GetStmtForUpdateTS(); err != nil { + return err + } + } else { + if startTs, err = txnManager.GetStmtReadTS(); err != nil { + return err + } + } + e.startTS = startTs + } + e.rowDecoder = decoder e.partInfo = p.PartitionInfo e.columns = p.Columns e.buildVirtualColumnInfo() + return nil } // buildVirtualColumnInfo saves virtual column indices and sort them in definition order @@ -143,9 +163,6 @@ func (e *PointGetExecutor) buildVirtualColumnInfo() { func (e *PointGetExecutor) Open(context.Context) error { txnCtx := e.ctx.GetSessionVars().TxnCtx snapshotTS := e.startTS - if e.lock { - snapshotTS = txnCtx.GetForUpdateTS() - } var err error e.txn, err = e.ctx.Txn(false) if err != nil { diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index 2a14c31978461..faf117a2cb4b0 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -16,6 +16,7 @@ package isolation_test import ( "context" + "github.com/pingcap/tidb/testkit" "testing" "time" @@ -96,3 +97,23 @@ func (a *txnAssert[T]) CheckAndGetProvider(t *testing.T) T { a.Check(t) return sessiontxn.GetTxnManager(a.sctx).GetContextProvider().(T) } + +func TestGetForUpdateTS(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + + tk2.MustExec("update t set v = v + 10 where id = 1") + + tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + tk.MustExec("commit") +} diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index c7c9a25474352..065a95507a485 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -35,7 +35,8 @@ type PessimisticRRTxnContextProvider struct { baseTxnContextProvider // Used for ForUpdateRead statement - forUpdateTS uint64 + forUpdateTS uint64 + latestForUpdateTS uint64 // It may decide whether to update forUpdateTs when calling provider's getForUpdateTs // See more details in the comments of optimizeWithPlan followingOperatorIsPointGetForUpdate bool @@ -69,7 +70,7 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error } if p.followingOperatorIsPointGetForUpdate { - return p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS(), nil + return p.getTxnStartTS() } var txn kv.Transaction @@ -112,7 +113,8 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { return err } - sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) + //sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) + p.latestForUpdateTS = version.Ver txn.SetOption(kv.SnapshotTS, sctx.GetSessionVars().TxnCtx.GetForUpdateTS()) return nil @@ -136,10 +138,9 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err return err } - txnCtxForUpdateTS := p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() // If TxnCtx.forUpdateTS is updated in OnStmtErrorForNextAction, we assign the value to the provider - if txnCtxForUpdateTS > p.forUpdateTS { - p.forUpdateTS = txnCtxForUpdateTS + if p.latestForUpdateTS > p.forUpdateTS { + p.forUpdateTS = p.latestForUpdateTS } else { p.forUpdateTS = 0 } @@ -191,6 +192,8 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { mayOptimizeForPointGet = true } + } else if _, ok := plan.(*plannercore.PointGetPlan); ok { + mayOptimizeForPointGet = true } p.followingOperatorIsPointGetForUpdate = mayOptimizeForPointGet diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index 54743d5fd51dc..ef740299a8344 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -345,6 +345,8 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { stmt, err := parser.New().ParseOneStmt("delete from t where id = 1", "", "") require.NoError(t, err) compareTs := getOracleTS(t, se) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) compiler := executor.Compiler{Ctx: se} execStmt, err := compiler.Compile(context.TODO(), stmt) require.NoError(t, err) @@ -357,6 +359,8 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { stmt, err = parser.New().ParseOneStmt("update t set v = v + 10 where id = 1", "", "") require.NoError(t, err) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) compiler = executor.Compiler{Ctx: se} execStmt, err = compiler.Compile(context.TODO(), stmt) require.NoError(t, err) @@ -368,6 +372,21 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { stmt, err = parser.New().ParseOneStmt("select * from (select * from t where id = 1 for update) as t1 for update", "", "") require.NoError(t, err) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) + compiler = executor.Compiler{Ctx: se} + execStmt, err = compiler.Compile(context.TODO(), stmt) + require.NoError(t, err) + err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Equal(t, ts, forUpdateTS) + + stmt, err = parser.New().ParseOneStmt("select * from t where id = 1 for update", "", "") + require.NoError(t, err) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) compiler = executor.Compiler{Ctx: se} execStmt, err = compiler.Compile(context.TODO(), stmt) require.NoError(t, err) @@ -376,11 +395,22 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) require.Equal(t, ts, forUpdateTS) + compareTs = getOracleTS(t, se) + // After retry, the ts should be larger than compareTs + action, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + require.NoError(t, err) + require.Equal(t, sessiontxn.StmtActionRetryReady, action) + err = provider.OnStmtRetry(context.TODO()) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.Greater(t, ts, compareTs) // Now, test for one that does not use the optimization stmt, err = parser.New().ParseOneStmt("select * from t for update", "", "") compareTs = getOracleTS(t, se) require.NoError(t, err) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) compiler = executor.Compiler{Ctx: se} execStmt, err = compiler.Compile(context.TODO(), stmt) require.NoError(t, err) From 0f136bd471ccd3ae26193fe5e22273f0957e635b Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 13 Jun 2022 22:48:34 +0800 Subject: [PATCH 02/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 5 ++++- executor/batch_point_get.go | 11 +++++++---- executor/executor.go | 19 ++++++++++++++----- executor/point_get.go | 7 +++++-- sessiontxn/isolation/repeatable_read.go | 2 +- 5 files changed, 31 insertions(+), 13 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 0654310cd9c60..faf3b9a0d59c9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -710,7 +710,10 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { keys = filterTemporaryTableKeys(sctx.GetSessionVars(), keys) seVars := sctx.GetSessionVars() keys = filterLockTableKeys(seVars.StmtCtx, keys) - lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout, len(keys)) + lockCtx, err := newLockCtx(sctx, seVars.LockWaitTimeout, len(keys)) + if err != nil { + return err + } var lockKeyStats *util.LockKeysDetails ctx = context.WithValue(ctx, util.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 0ce745d172e4a..698351d45ec1a 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -540,13 +540,16 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { } // LockKeys locks the keys for pessimistic transaction. -func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { - txnCtx := seCtx.GetSessionVars().TxnCtx - lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime, len(keys)) +func LockKeys(ctx context.Context, sctx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { + txnCtx := sctx.GetSessionVars().TxnCtx + lctx, err := newLockCtx(sctx, lockWaitTime, len(keys)) + if err != nil { + return err + } if txnCtx.IsPessimistic { lctx.InitReturnValues(len(keys)) } - err := doLockKeys(ctx, seCtx, lctx, keys...) + err = doLockKeys(ctx, sctx, lctx, keys...) if err != nil { return err } diff --git a/executor/executor.go b/executor/executor.go index aa71929859898..be3229fbb5cb0 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -17,6 +17,7 @@ package executor import ( "context" "fmt" + "github.com/pingcap/tidb/sessiontxn" "math" "runtime/pprof" "runtime/trace" @@ -1042,12 +1043,20 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { for id := range e.tblID2Handle { e.updateDeltaForTableID(id) } - - return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime, len(e.keys)), e.keys...) + lockCtx, err := newLockCtx(e.ctx, lockWaitTime, len(e.keys)) + if err != nil { + return err + } + return doLockKeys(ctx, e.ctx, lockCtx, e.keys...) } -func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) *tikvstore.LockCtx { - lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) +func newLockCtx(sctx sessionctx.Context, lockWaitTime int64, numKeys int) (*tikvstore.LockCtx, error) { + seVars := sctx.GetSessionVars() + forUpdateTs, err := sessiontxn.GetTxnManager(sctx).GetStmtForUpdateTS() + if err != nil { + return nil, err + } + lockCtx := tikvstore.NewLockCtx(forUpdateTs, lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited lockCtx.LockKeysDuration = &seVars.StmtCtx.LockKeysDuration @@ -1082,7 +1091,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) * if lockCtx.ForUpdateTS > 0 && seVars.AssertionLevel != variable.AssertionLevelOff { lockCtx.InitCheckExistence(numKeys) } - return lockCtx + return lockCtx, nil } // doLockKeys is the main entry for pessimistic lock keys diff --git a/executor/point_get.go b/executor/point_get.go index cb094d6c0d5eb..0ab8a3386ea2d 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -398,9 +398,12 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro } if e.lock { seVars := e.ctx.GetSessionVars() - lockCtx := newLockCtx(seVars, e.lockWaitTime, 1) + lockCtx, err := newLockCtx(e.ctx, e.lockWaitTime, 1) + if err != nil { + return err + } lockCtx.InitReturnValues(1) - err := doLockKeys(ctx, e.ctx, lockCtx, key) + err = doLockKeys(ctx, e.ctx, lockCtx, key) if err != nil { return err } diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 065a95507a485..a36b8202849fd 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -115,7 +115,7 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { //sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) p.latestForUpdateTS = version.Ver - txn.SetOption(kv.SnapshotTS, sctx.GetSessionVars().TxnCtx.GetForUpdateTS()) + txn.SetOption(kv.SnapshotTS, p.latestForUpdateTS) return nil } From 9ad449288c99bb0ea65124607d3c2f14e2bfff07 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Tue, 14 Jun 2022 15:57:02 +0800 Subject: [PATCH 03/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 7 ++ executor/batch_point_get.go | 3 - executor/builder.go | 30 ++++--- sessiontxn/failpoint.go | 15 ++++ sessiontxn/isolation/main_test.go | 24 ++++- sessiontxn/isolation/repeatable_read.go | 18 ++-- sessiontxn/isolation/repeatable_reat_test.go | 92 ++++++++++++++++++++ 7 files changed, 166 insertions(+), 23 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index faf3b9a0d59c9..c0e356d524b8d 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -772,6 +772,13 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if lockErr == nil { return nil, nil } + failpoint.Inject("assertPessimisticLockErr", func() { + if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { + sessiontxn.AddEntrance(a.Ctx, "insertWriteConflict") + } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { + sessiontxn.AddEntrance(a.Ctx, "insertDuplicateKey") + } + }) defer func() { if _, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 698351d45ec1a..1d95a44b517d8 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -101,9 +101,6 @@ func (e *BatchPointGetExec) Open(context.Context) error { sessVars := e.ctx.GetSessionVars() txnCtx := sessVars.TxnCtx stmtCtx := sessVars.StmtCtx - if e.lock { - e.snapshotTS = txnCtx.GetForUpdateTS() - } txn, err := e.ctx.Txn(false) if err != nil { return err diff --git a/executor/builder.go b/executor/builder.go index 8894958ac8879..51e46aa7a6ec0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -657,9 +657,9 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true - if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { - return nil - } + //if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { + // return nil + //} // Build 'select for update' using the 'for update' ts. b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() @@ -4615,18 +4615,12 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan return nil } - startTS, err := b.getSnapshotTS() - if err != nil { - b.err = err - return nil - } decoder := NewRowDecoder(b.ctx, plan.Schema(), plan.TblInfo) e := &BatchPointGetExec{ baseExecutor: newBaseExecutor(b.ctx, plan.Schema(), plan.ID()), tblInfo: plan.TblInfo, idxInfo: plan.IndexInfo, rowDecoder: decoder, - startTS: startTS, readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, keepOrder: plan.KeepOrder, @@ -4639,18 +4633,30 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan partTblID: plan.PartTblID, columns: plan.Columns, } - if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(plan.TblInfo, startTS) - } + if plan.TblInfo.TempTableType != model.TempTableNone { // Temporary table should not do any lock operations e.lock = false e.waitTime = 0 } + txnManager := sessiontxn.GetTxnManager(b.ctx) + var err error if e.lock { b.hasLock = true + e.startTS, err = txnManager.GetStmtForUpdateTS() + } else { + e.startTS, err = txnManager.GetStmtReadTS() } + if err != nil { + b.err = err + return nil + } + + if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { + e.cacheTable = b.getCacheTable(plan.TblInfo, e.startTS) + } + var capacity int if plan.IndexInfo != nil && !isCommonHandleRead(plan.TblInfo, plan.IndexInfo) { e.idxVals = plan.IndexValues diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 241e39b7a8436..d39ed3ac368bf 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -84,3 +84,18 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { panic(fmt.Sprintf("Txn read ts not match, expect:%d, got:%d", expected, actual)) } } + +var AssertInsertErr stringutil.StringerStr = "assertInsertErrors" + +func AddEntrance(sctx sessionctx.Context, name string) { + records, ok := sctx.Value(AssertInsertErr).(map[string]int) + if !ok { + records = make(map[string]int) + sctx.SetValue(AssertInsertErr, records) + } + if v, ok := records[name]; ok { + records[name] = v + 1 + } else { + records[name] = 1 + } +} diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index faf117a2cb4b0..e9e7dc2210729 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -98,7 +98,7 @@ func (a *txnAssert[T]) CheckAndGetProvider(t *testing.T) T { return sessiontxn.GetTxnManager(a.sctx).GetContextProvider().(T) } -func TestGetForUpdateTS(t *testing.T) { +func TestGetForUpdateTS2(t *testing.T) { store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -114,6 +114,26 @@ func TestGetForUpdateTS(t *testing.T) { tk2.MustExec("update t set v = v + 10 where id = 1") - tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 11", "2 2")) + tk.MustExec("commit") +} + +func TestGetForUpdateTS3(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + + tk2.MustExec("update t set v = v + 10 where id = 1") + + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 11", "2 2")) tk.MustExec("commit") } diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index a36b8202849fd..f71b802e0d82d 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -39,7 +39,7 @@ type PessimisticRRTxnContextProvider struct { latestForUpdateTS uint64 // It may decide whether to update forUpdateTs when calling provider's getForUpdateTs // See more details in the comments of optimizeWithPlan - followingOperatorIsPointGetForUpdate bool + followingOperatorDoesNotUpdateForUpdateTS bool } // NewPessimisticRRTxnContextProvider returns a new PessimisticRRTxnContextProvider @@ -69,7 +69,7 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return p.forUpdateTS, nil } - if p.followingOperatorIsPointGetForUpdate { + if p.followingOperatorDoesNotUpdateForUpdateTS { return p.getTxnStartTS() } @@ -115,7 +115,7 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { //sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) p.latestForUpdateTS = version.Ver - txn.SetOption(kv.SnapshotTS, p.latestForUpdateTS) + txn.SetOption(kv.SnapshotTS, version.Ver) return nil } @@ -127,7 +127,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context) error } p.forUpdateTS = 0 - p.followingOperatorIsPointGetForUpdate = false + p.followingOperatorDoesNotUpdateForUpdateTS = false return nil } @@ -145,7 +145,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err p.forUpdateTS = 0 } - p.followingOperatorIsPointGetForUpdate = false + p.followingOperatorDoesNotUpdateForUpdateTS = false return nil } @@ -192,11 +192,17 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { mayOptimizeForPointGet = true } + } else if v, ok := plan.(*plannercore.Insert); ok { + if v.SelectPlan == nil { + mayOptimizeForPointGet = true + } } else if _, ok := plan.(*plannercore.PointGetPlan); ok { mayOptimizeForPointGet = true + } else if _, ok := plan.(*plannercore.BatchPointGetPlan); ok { + mayOptimizeForPointGet = true } - p.followingOperatorIsPointGetForUpdate = mayOptimizeForPointGet + p.followingOperatorDoesNotUpdateForUpdateTS = mayOptimizeForPointGet return nil } diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index ef740299a8344..435a0b9bdefd0 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -17,6 +17,7 @@ package isolation_test import ( "context" "fmt" + "github.com/pingcap/failpoint" "testing" "time" @@ -405,6 +406,32 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { ts, err = provider.GetStmtForUpdateTS() require.Greater(t, ts, compareTs) + provider = initializeRepeatableReadProvider(t, tk) + forUpdateTS = se.GetSessionVars().TxnCtx.GetForUpdateTS() + txnManager = sessiontxn.GetTxnManager(se) + // test for batch point get for update + stmt, err = parser.New().ParseOneStmt("select * from t where id = 1 or id = 2 for update", "", "") + require.NoError(t, err) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) + compiler = executor.Compiler{Ctx: se} + execStmt, err = compiler.Compile(context.TODO(), stmt) + require.NoError(t, err) + err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Equal(t, ts, forUpdateTS) + compareTs = getOracleTS(t, se) + // After retry, the ts should be larger than compareTs + action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + require.NoError(t, err) + require.Equal(t, sessiontxn.StmtActionRetryReady, action) + err = provider.OnStmtRetry(context.TODO()) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.Greater(t, ts, compareTs) + // Now, test for one that does not use the optimization stmt, err = parser.New().ParseOneStmt("select * from t for update", "", "") compareTs = getOracleTS(t, se) @@ -421,6 +448,71 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Greater(t, ts, compareTs) } +var errorsInInsert = []string{ + "insertWriteConflict", + "insertDuplicateKey", +} + +func TestErrorInInsert(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + se.SetValue(sessiontxn.AssertInsertErr, nil) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + + records, ok := se.Value(sessiontxn.AssertInsertErr).(map[string]int) + require.True(t, ok) + + for _, name := range errorsInInsert { + records[name] = 1 + } + + se.SetValue(sessiontxn.AssertInsertErr, nil) + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestErrorInPointGetForUpdate(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + se.SetValue(sessiontxn.AssertInsertErr, nil) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + + records, ok := se.Value(sessiontxn.AssertInsertErr).(map[string]int) + require.True(t, ok) + records["insertWriteConflict"] = 1 + + se.SetValue(sessiontxn.AssertInsertErr, nil) + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, inTxn bool) *txnAssert[*isolation.PessimisticRRTxnContextProvider] { return &txnAssert[*isolation.PessimisticRRTxnContextProvider]{ From 9a68f751e263cdad1ab5d200177f91f825e93305 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Tue, 14 Jun 2022 16:48:40 +0800 Subject: [PATCH 04/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 26 ++--- sessiontxn/failpoint.go | 6 +- sessiontxn/isolation/repeatable_reat_test.go | 109 +++++++++++++++++-- 3 files changed, 117 insertions(+), 24 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 51e46aa7a6ec0..63a4c3dc10a8c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -865,13 +865,13 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - if v.SelectPlan != nil { - // Try to update the forUpdateTS for insert/replace into select statements. - // Set the selectPlan parameter to nil to make it always update the forUpdateTS. - if b.err = b.updateForUpdateTSIfNeeded(nil); b.err != nil { - return nil - } - } + //if v.SelectPlan != nil { + // // Try to update the forUpdateTS for insert/replace into select statements. + // // Set the selectPlan parameter to nil to make it always update the forUpdateTS. + // if b.err = b.updateForUpdateTSIfNeeded(nil); b.err != nil { + // return nil + // } + //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selectExec := b.build(v.SelectPlan) if b.err != nil { @@ -2114,9 +2114,9 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { - return nil - } + //if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + // return nil + //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { @@ -2171,13 +2171,13 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { for _, info := range v.TblColPosInfos { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + //if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { return nil - } + //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { - return nil + // return nil } base := newBaseExecutor(b.ctx, v.Schema(), v.ID(), selExec) base.initCap = chunk.ZeroCapacity diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index d39ed3ac368bf..aae2ffb0c1d5f 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -85,13 +85,13 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } -var AssertInsertErr stringutil.StringerStr = "assertInsertErrors" +var AssertLockErr stringutil.StringerStr = "assertLockError" func AddEntrance(sctx sessionctx.Context, name string) { - records, ok := sctx.Value(AssertInsertErr).(map[string]int) + records, ok := sctx.Value(AssertLockErr).(map[string]int) if !ok { records = make(map[string]int) - sctx.SetValue(AssertInsertErr, records) + sctx.SetValue(AssertLockErr, records) } if v, ok := records[name]; ok { records[name] = v + 1 diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index 435a0b9bdefd0..105528f11d7bc 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -460,7 +460,6 @@ func TestErrorInInsert(t *testing.T) { tk := testkit.NewTestKit(t, store) se := tk.Session() - se.SetValue(sessiontxn.AssertInsertErr, nil) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -469,17 +468,18 @@ func TestErrorInInsert(t *testing.T) { tk.MustExec("begin pessimistic") tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) _, err := tk.Exec("insert into t values (1, 1), (2, 2)") require.Error(t, err) - records, ok := se.Value(sessiontxn.AssertInsertErr).(map[string]int) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) for _, name := range errorsInInsert { - records[name] = 1 + require.Equal(t, records[name], 1) } - se.SetValue(sessiontxn.AssertInsertErr, nil) + se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("rollback") require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } @@ -492,7 +492,6 @@ func TestErrorInPointGetForUpdate(t *testing.T) { tk := testkit.NewTestKit(t, store) se := tk.Session() - se.SetValue(sessiontxn.AssertInsertErr, nil) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -502,13 +501,107 @@ func TestErrorInPointGetForUpdate(t *testing.T) { tk.MustExec("begin pessimistic") tk2.MustExec("update t set v = v + 10 where id = 1") + + se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["insertWriteConflict"], 1) + tk.MustExec("commit") + + // batch point get + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id = 1 or id = 2 for update").Check(testkit.Rows("1 21", "2 2")) + records, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["insertWriteConflict"], 1) + tk.MustExec("commit") + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +// Delete should get the latest ts and thus does not incur write conflict +func TestErrorInDelete(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if sub select in delete is point get, we will incur one write conflict + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["insertWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestErrorInUpdate(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if the sub select plan is point get, we should incur one write conflict + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") - records, ok := se.Value(sessiontxn.AssertInsertErr).(map[string]int) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - records["insertWriteConflict"] = 1 + require.Equal(t, records["insertWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) - se.SetValue(sessiontxn.AssertInsertErr, nil) tk.MustExec("rollback") require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } From ddd7ed4f106b35c9b430550c7a583853fd4fb40e Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Tue, 14 Jun 2022 16:57:58 +0800 Subject: [PATCH 05/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/builder.go b/executor/builder.go index 63a4c3dc10a8c..5a85694379316 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2172,7 +2172,7 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } //if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { - return nil + //return nil //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) From 142f55eeb8f91efa1d4cb90c945d81d9628e4693 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Tue, 14 Jun 2022 19:40:11 +0800 Subject: [PATCH 06/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 50 +-------- executor/executor.go | 2 +- executor/point_get.go | 3 +- sessiontxn/failpoint.go | 3 + sessiontxn/isolation/main_test.go | 41 ------- sessiontxn/isolation/readcommitted.go | 45 +++++++- sessiontxn/isolation/readcommitted_test.go | 106 +++++++++++++++++++ sessiontxn/isolation/repeatable_read.go | 37 +++---- sessiontxn/isolation/repeatable_reat_test.go | 21 ++-- 9 files changed, 180 insertions(+), 128 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 5a85694379316..33d78c9cc0bf5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -657,9 +657,6 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true - //if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { - // return nil - //} // Build 'select for update' using the 'for update' ts. b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() @@ -865,13 +862,6 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - //if v.SelectPlan != nil { - // // Try to update the forUpdateTS for insert/replace into select statements. - // // Set the selectPlan parameter to nil to make it always update the forUpdateTS. - // if b.err = b.updateForUpdateTSIfNeeded(nil); b.err != nil { - // return nil - // } - //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selectExec := b.build(v.SelectPlan) if b.err != nil { @@ -2114,9 +2104,6 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - //if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { - // return nil - //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { @@ -2171,13 +2158,10 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { for _, info := range v.TblColPosInfos { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - //if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { - //return nil - //} b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { - // return nil + return nil } base := newBaseExecutor(b.ctx, v.Schema(), v.ID(), selExec) base.initCap = chunk.ZeroCapacity @@ -2190,38 +2174,6 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } -// updateForUpdateTSIfNeeded updates the ForUpdateTS for a pessimistic transaction if needed. -// PointGet executor will get conflict error if the ForUpdateTS is older than the latest commitTS, -// so we don't need to update now for better latency. -func (b *executorBuilder) updateForUpdateTSIfNeeded(selectPlan plannercore.PhysicalPlan) error { - txnCtx := b.ctx.GetSessionVars().TxnCtx - if !txnCtx.IsPessimistic { - return nil - } - if _, ok := selectPlan.(*plannercore.PointGetPlan); ok { - return nil - } - // Activate the invalid txn, use the txn startTS as newForUpdateTS - txn, err := b.ctx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - _, err := b.ctx.Txn(true) - if err != nil { - return err - } - return nil - } - // The Repeatable Read transaction use Read Committed level to read data for writing (insert, update, delete, select for update), - // We should always update/refresh the for-update-ts no matter the isolation level is RR or RC. - if b.ctx.GetSessionVars().IsPessimisticReadConsistency() { - _, err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() - return err - } - return UpdateForUpdateTS(b.ctx, 0) -} - func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O} _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) diff --git a/executor/executor.go b/executor/executor.go index be3229fbb5cb0..f6b6cbc44c45b 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -17,7 +17,6 @@ package executor import ( "context" "fmt" - "github.com/pingcap/tidb/sessiontxn" "math" "runtime/pprof" "runtime/trace" @@ -50,6 +49,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" diff --git a/executor/point_get.go b/executor/point_get.go index 0ab8a3386ea2d..ddb61bde8f284 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -17,6 +17,8 @@ package executor import ( "context" "fmt" + "math" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" @@ -41,7 +43,6 @@ import ( "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" - "math" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index aae2ffb0c1d5f..beea9b0e44dc0 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -85,8 +85,11 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } +// AssertLockErr is used to record the lock errors we encountered +// Only for test var AssertLockErr stringutil.StringerStr = "assertLockError" +// AddEntrance is used only for test func AddEntrance(sctx sessionctx.Context, name string) { records, ok := sctx.Value(AssertLockErr).(map[string]int) if !ok { diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index e9e7dc2210729..2a14c31978461 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -16,7 +16,6 @@ package isolation_test import ( "context" - "github.com/pingcap/tidb/testkit" "testing" "time" @@ -97,43 +96,3 @@ func (a *txnAssert[T]) CheckAndGetProvider(t *testing.T) T { a.Check(t) return sessiontxn.GetTxnManager(a.sctx).GetContextProvider().(T) } - -func TestGetForUpdateTS2(t *testing.T) { - store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk2.MustExec("use test") - tk.MustExec("create table t (id int primary key, v int)") - tk.MustExec("insert into t values (1, 1), (2, 2)") - - tk.MustExec("begin pessimistic") - - tk2.MustExec("update t set v = v + 10 where id = 1") - - tk.MustQuery("select * from t for update").Check(testkit.Rows("1 11", "2 2")) - tk.MustExec("commit") -} - -func TestGetForUpdateTS3(t *testing.T) { - store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk2.MustExec("use test") - tk.MustExec("create table t (id int primary key, v int)") - tk.MustExec("insert into t values (1, 1), (2, 2)") - - tk.MustExec("begin pessimistic") - - tk2.MustExec("update t set v = v + 10 where id = 1") - - tk.MustQuery("select * from t for update").Check(testkit.Rows("1 11", "2 2")) - tk.MustExec("commit") -} diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 36c02e782d162..a563c46b1cfa0 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -31,10 +32,11 @@ import ( ) type stmtState struct { - stmtTS uint64 - stmtTSFuture oracle.Future - stmtUseStartTS bool - onNextRetryOrStmt func() error + stmtTS uint64 + stmtTSFuture oracle.Future + stmtUseStartTS bool + optimizeForNotFetchingLatestTS bool + onNextRetryOrStmt func() error } func (s *stmtState) prepareStmt(useStartTS bool) error { @@ -81,6 +83,7 @@ func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context) error if err := p.baseTxnContextProvider.OnStmtStart(ctx); err != nil { return err } + p.optimizeForNotFetchingLatestTS = false return p.prepareStmt(!p.isTxnPrepared) } @@ -104,6 +107,7 @@ func (p *PessimisticRCTxnContextProvider) OnStmtRetry(ctx context.Context) error if err := p.baseTxnContextProvider.OnStmtRetry(ctx); err != nil { return err } + p.optimizeForNotFetchingLatestTS = false return p.prepareStmt(false) } @@ -131,6 +135,10 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { return p.stmtTS, nil } + if p.optimizeForNotFetchingLatestTS { + return p.getTxnStartTS() + } + var txn kv.Transaction if txn, err = p.activeTxn(); err != nil { return 0, err @@ -207,3 +215,32 @@ func (p *PessimisticRCTxnContextProvider) AdviseWarmup() error { p.prepareStmtTS() return nil } + +// AdviseOptimizeWithPlan in RC covers much fewer cases compared with pessimistic repeatable read. +// We only optimize with insert operator with no selection in that we do not fetch latest ts immediately. +// We only update ts if write conflict is incurred. +func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { + if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { + return nil + } + + plan, ok := val.(plannercore.Plan) + if !ok { + return nil + } + + if execute, ok := plan.(*plannercore.Execute); ok { + plan = execute.Plan + } + + optimizeForNotFetchingLatestTS := false + if v, ok := plan.(*plannercore.Insert); ok { + if v.SelectPlan == nil { + optimizeForNotFetchingLatestTS = true + } + } + + p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS + + return nil +} diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index 3700fa4e63568..9f81d94897c3e 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -376,6 +377,111 @@ func TestTidbSnapshotVarInRC(t *testing.T) { tk.MustExec("rollback") } +func TestErrorInInsertInRC(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("set tx_isolation='READ-COMMITTED'") + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + for _, name := range errorsInInsert { + require.Equal(t, records[name], 1) + } + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestErrorInDeleteInRC(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("set tx_isolation='READ-COMMITTED'") + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestErrorInUpdateInRC(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("set tx_isolation='READ-COMMITTED'") + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + func activeRCTxnAssert(t *testing.T, sctx sessionctx.Context, inTxn bool) *txnAssert[*isolation.PessimisticRCTxnContextProvider] { return &txnAssert[*isolation.PessimisticRCTxnContextProvider]{ sctx: sctx, diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index f71b802e0d82d..9280a2f6c8b7a 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -39,7 +39,7 @@ type PessimisticRRTxnContextProvider struct { latestForUpdateTS uint64 // It may decide whether to update forUpdateTs when calling provider's getForUpdateTs // See more details in the comments of optimizeWithPlan - followingOperatorDoesNotUpdateForUpdateTS bool + optimizeForNotFetchingLatestTS bool } // NewPessimisticRRTxnContextProvider returns a new PessimisticRRTxnContextProvider @@ -69,7 +69,7 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return p.forUpdateTS, nil } - if p.followingOperatorDoesNotUpdateForUpdateTS { + if p.optimizeForNotFetchingLatestTS { return p.getTxnStartTS() } @@ -127,7 +127,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context) error } p.forUpdateTS = 0 - p.followingOperatorDoesNotUpdateForUpdateTS = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -145,7 +145,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err p.forUpdateTS = 0 } - p.followingOperatorDoesNotUpdateForUpdateTS = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -165,6 +165,8 @@ func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point session // We expect that the data that the point get acquires has not been changed. // Benefit: Save the cost of acquiring ts from PD. // Drawbacks: If the data has been changed since the ts we used, we need to retry. +// One exception is insert operation, when it does not fetch data, we do not fetch the latest ts immediately. We only update ts +// if write conflict is incurred. func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { return nil @@ -179,30 +181,29 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - mayOptimizeForPointGet := false - if v, ok := plan.(*plannercore.PhysicalLock); ok { + optimizeForNotFetchingLatestTS := false + switch v := plan.(type) { + case *plannercore.PhysicalLock: if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + optimizeForNotFetchingLatestTS = true } - } else if v, ok := plan.(*plannercore.Update); ok { + case *plannercore.Update: if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + optimizeForNotFetchingLatestTS = true } - } else if v, ok := plan.(*plannercore.Delete); ok { + case *plannercore.Delete: if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + optimizeForNotFetchingLatestTS = true } - } else if v, ok := plan.(*plannercore.Insert); ok { + case *plannercore.Insert: if v.SelectPlan == nil { - mayOptimizeForPointGet = true + optimizeForNotFetchingLatestTS = true } - } else if _, ok := plan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true - } else if _, ok := plan.(*plannercore.BatchPointGetPlan); ok { - mayOptimizeForPointGet = true + case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: + optimizeForNotFetchingLatestTS = true } - p.followingOperatorDoesNotUpdateForUpdateTS = mayOptimizeForPointGet + p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS return nil } diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index 105528f11d7bc..d111e9c195e7e 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -17,11 +17,11 @@ package isolation_test import ( "context" "fmt" - "github.com/pingcap/failpoint" "testing" "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -404,6 +404,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { err = provider.OnStmtRetry(context.TODO()) require.NoError(t, err) ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) require.Greater(t, ts, compareTs) provider = initializeRepeatableReadProvider(t, tk) @@ -430,6 +431,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { err = provider.OnStmtRetry(context.TODO()) require.NoError(t, err) ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) require.Greater(t, ts, compareTs) // Now, test for one that does not use the optimization @@ -453,7 +455,7 @@ var errorsInInsert = []string{ "insertDuplicateKey", } -func TestErrorInInsert(t *testing.T) { +func TestErrorInInsertInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -471,10 +473,8 @@ func TestErrorInInsert(t *testing.T) { se.SetValue(sessiontxn.AssertLockErr, nil) _, err := tk.Exec("insert into t values (1, 1), (2, 2)") require.Error(t, err) - records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - for _, name := range errorsInInsert { require.Equal(t, records[name], 1) } @@ -484,10 +484,9 @@ func TestErrorInInsert(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } -func TestErrorInPointGetForUpdate(t *testing.T) { +func TestErrorInPointGetForUpdateInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() tk := testkit.NewTestKit(t, store) @@ -524,10 +523,9 @@ func TestErrorInPointGetForUpdate(t *testing.T) { } // Delete should get the latest ts and thus does not incur write conflict -func TestErrorInDelete(t *testing.T) { +func TestErrorInDeleteInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() tk := testkit.NewTestKit(t, store) @@ -544,10 +542,8 @@ func TestErrorInDelete(t *testing.T) { se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("delete from t where v = 1") - _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.False(t, ok) - tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) tk.MustExec("commit") @@ -566,10 +562,9 @@ func TestErrorInDelete(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } -func TestErrorInUpdate(t *testing.T) { +func TestErrorInUpdateInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() tk := testkit.NewTestKit(t, store) @@ -586,7 +581,6 @@ func TestErrorInUpdate(t *testing.T) { se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("update t set v = v + 10") - _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.False(t, ok) tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) @@ -596,7 +590,6 @@ func TestErrorInUpdate(t *testing.T) { // However, if the sub select plan is point get, we should incur one write conflict tk2.MustExec("update t set v = v + 10 where id = 1") tk.MustExec("update t set v = v + 10 where id = 1") - records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) require.Equal(t, records["insertWriteConflict"], 1) From ebb581fedfdf9a2a53ff3e14fdfc761a374d40f1 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 11:19:32 +0800 Subject: [PATCH 07/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 2 +- executor/point_get.go | 10 +++++++--- sessiontxn/isolation/repeatable_read.go | 2 +- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index c0e356d524b8d..d2fecbe6e8f4a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -255,7 +255,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec } else { // CachedPlan type is already checked in last step pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan) - if err := exec.Init(pointGetPlan, true); err != nil { + if err := exec.Init(pointGetPlan, true, false); err != nil { return nil, err } a.PsStmt.Executor = exec diff --git a/executor/point_get.go b/executor/point_get.go index ddb61bde8f284..59b1fd5803480 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -59,7 +59,10 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { e.base().initCap = 1 e.base().maxChunkSize = 1 - err := e.Init(p, false) + + upperLock := b.inInsertStmt || b.inUpdateStmt || b.inDeleteStmt || b.inSelectLockStmt + + err := e.Init(p, false, upperLock) if err != nil { b.err = err } @@ -71,6 +74,7 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { if e.lock { b.hasLock = true } + return e } @@ -108,7 +112,7 @@ type PointGetExecutor struct { } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field -func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool) error { +func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool, upperLock bool) error { decoder := NewRowDecoder(e.ctx, p.Schema(), p.TblInfo) e.tblInfo = p.TblInfo e.handle = p.Handle @@ -130,7 +134,7 @@ func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool) erro var startTs uint64 var err error txnManager := sessiontxn.GetTxnManager(e.ctx) - if e.lock { + if e.lock || upperLock { if startTs, err = txnManager.GetStmtForUpdateTS(); err != nil { return err } diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 9280a2f6c8b7a..fc769f7c4b35a 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -113,7 +113,7 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { return err } - //sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) + sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) p.latestForUpdateTS = version.Ver txn.SetOption(kv.SnapshotTS, version.Ver) From b595d3d4885c6cf24c5122d598c007dba2443977 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 12:04:20 +0800 Subject: [PATCH 08/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/legacy/provider.go | 10 ---------- sessiontxn/txn_context_test.go | 2 +- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/sessiontxn/legacy/provider.go b/sessiontxn/legacy/provider.go index 856cdc467aebc..7986bbbfab5dc 100644 --- a/sessiontxn/legacy/provider.go +++ b/sessiontxn/legacy/provider.go @@ -189,16 +189,6 @@ func (p *SimpleTxnContextProvider) OnStmtRetry(_ context.Context) error { return nil } -//// todo: remove -//// Advise is used to give advice to provider -//func (p *SimpleTxnContextProvider) Advise(tp sessiontxn.AdviceType, _ []any) error { -// switch tp { -// case sessiontxn.AdviceWarmUp: -// p.Sctx.PrepareTSFuture(p.Ctx) -// } -// return nil -//} - // activeTxn actives the txn func (p *SimpleTxnContextProvider) activeTxn() (kv.Transaction, error) { if p.isTxnActive { diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index c7c03d67e8b5e..86f213310fdaf 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -441,7 +441,7 @@ func TestTxnContextForHistoricalRead(t *testing.T) { }) doWithCheckPath(t, se, normalPathRecords, func() { - tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) }) tk.MustExec("rollback") From 673d389fbbbab09eebcd48ac537cf175e6c634db Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 19:23:16 +0800 Subject: [PATCH 09/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 10 ++-- executor/batch_point_get.go | 2 +- executor/builder.go | 31 +++++++++-- executor/executor.go | 4 +- executor/point_get.go | 58 +++++++------------- executor/trace_test.go | 2 +- sessiontxn/isolation/readcommitted_test.go | 43 ++++----------- sessiontxn/isolation/repeatable_reat_test.go | 12 ++-- 8 files changed, 72 insertions(+), 90 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index d2fecbe6e8f4a..f3a8afd10d3c8 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -255,9 +255,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec } else { // CachedPlan type is already checked in last step pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan) - if err := exec.Init(pointGetPlan, true, false); err != nil { - return nil, err - } + exec.Init(pointGetPlan, startTs) a.PsStmt.Executor = exec } } @@ -453,7 +451,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { curTxnStartTS = sctx.GetSessionVars().SnapshotTS } logutil.BgLogger().Info("Enable mockDelayInnerSessionExecute when execute statement", - zap.Uint64("startTS", curTxnStartTS)) + zap.Uint64("snapshotTS", curTxnStartTS)) time.Sleep(200 * time.Millisecond) }) @@ -774,9 +772,9 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } failpoint.Inject("assertPessimisticLockErr", func() { if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { - sessiontxn.AddEntrance(a.Ctx, "insertWriteConflict") + sessiontxn.AddEntrance(a.Ctx, "errWriteConflict") } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { - sessiontxn.AddEntrance(a.Ctx, "insertDuplicateKey") + sessiontxn.AddEntrance(a.Ctx, "errDuplicateKey") } }) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 1d95a44b517d8..33d655bc5d271 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -108,7 +108,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.txn = txn var snapshot kv.Snapshot if txn.Valid() && txnCtx.StartTS == txnCtx.GetForUpdateTS() && txnCtx.StartTS == e.snapshotTS { - // We can safely reuse the transaction snapshot if startTS is equal to forUpdateTS. + // We can safely reuse the transaction snapshot if snapshotTS is equal to forUpdateTS. // The snapshot may contains cache that can reduce RPC call. snapshot = txn.GetSnapshot() } else { diff --git a/executor/builder.go b/executor/builder.go index 33d78c9cc0bf5..d4b24282cea5e 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -657,8 +657,13 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true + // Build 'select for update' using the 'for update' ts. - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + var forUpdateTS uint64 + if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + return nil + } + b.forUpdateTS = forUpdateTS src := b.build(v.Children()[0]) if b.err != nil { @@ -862,7 +867,13 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + + var forUpdateTS uint64 + if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + return nil + } + b.forUpdateTS = forUpdateTS + selectExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2104,7 +2115,13 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + + var forUpdateTS uint64 + if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + return nil + } + b.forUpdateTS = forUpdateTS + selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2158,7 +2175,13 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { for _, info := range v.TblColPosInfos { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + + var forUpdateTS uint64 + if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + return nil + } + b.forUpdateTS = forUpdateTS + selExec := b.build(v.SelectPlan) if b.err != nil { return nil diff --git a/executor/executor.go b/executor/executor.go index f6b6cbc44c45b..d2b726f24adbb 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1052,11 +1052,11 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { func newLockCtx(sctx sessionctx.Context, lockWaitTime int64, numKeys int) (*tikvstore.LockCtx, error) { seVars := sctx.GetSessionVars() - forUpdateTs, err := sessiontxn.GetTxnManager(sctx).GetStmtForUpdateTS() + forUpdateTS, err := sessiontxn.GetTxnManager(sctx).GetStmtForUpdateTS() if err != nil { return nil, err } - lockCtx := tikvstore.NewLockCtx(forUpdateTs, lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) + lockCtx := tikvstore.NewLockCtx(forUpdateTS, lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited lockCtx.LockKeysDuration = &seVars.StmtCtx.LockKeysDuration diff --git a/executor/point_get.go b/executor/point_get.go index 59b1fd5803480..900677ef30602 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -17,8 +17,6 @@ package executor import ( "context" "fmt" - "math" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" @@ -31,7 +29,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -51,26 +48,32 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { return nil } + if p.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() + if err != nil { + return nil + } + e := &PointGetExecutor{ baseExecutor: newBaseExecutor(b.ctx, p.Schema(), p.ID()), readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, } - e.base().initCap = 1 - e.base().maxChunkSize = 1 - - upperLock := b.inInsertStmt || b.inUpdateStmt || b.inDeleteStmt || b.inSelectLockStmt - - err := e.Init(p, false, upperLock) - if err != nil { - b.err = err - } - if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(p.TblInfo, e.startTS) + e.cacheTable = b.getCacheTable(p.TblInfo, snapshotTS) } + e.base().initCap = 1 + e.base().maxChunkSize = 1 + e.Init(p, snapshotTS) + if e.lock { b.hasLock = true } @@ -89,7 +92,7 @@ type PointGetExecutor struct { idxKey kv.Key handleVal []byte idxVals []types.Datum - startTS uint64 + snapshotTS uint64 readReplicaScope string isStaleness bool txn kv.Transaction @@ -112,12 +115,13 @@ type PointGetExecutor struct { } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field -func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool, upperLock bool) error { +func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, snapshotTS uint64) { decoder := NewRowDecoder(e.ctx, p.Schema(), p.TblInfo) e.tblInfo = p.TblInfo e.handle = p.Handle e.idxInfo = p.IndexInfo e.idxVals = p.IndexValues + e.snapshotTS = snapshotTS e.done = false if e.tblInfo.TempTableType == model.TempTableNone { e.lock = p.Lock @@ -127,30 +131,10 @@ func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, useMaxTs bool, uppe e.lock = false e.lockWaitTime = 0 } - - if useMaxTs { - e.startTS = uint64(math.MaxUint64) - } else { - var startTs uint64 - var err error - txnManager := sessiontxn.GetTxnManager(e.ctx) - if e.lock || upperLock { - if startTs, err = txnManager.GetStmtForUpdateTS(); err != nil { - return err - } - } else { - if startTs, err = txnManager.GetStmtReadTS(); err != nil { - return err - } - } - e.startTS = startTs - } - e.rowDecoder = decoder e.partInfo = p.PartitionInfo e.columns = p.Columns e.buildVirtualColumnInfo() - return nil } // buildVirtualColumnInfo saves virtual column indices and sort them in definition order @@ -167,7 +151,7 @@ func (e *PointGetExecutor) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *PointGetExecutor) Open(context.Context) error { txnCtx := e.ctx.GetSessionVars().TxnCtx - snapshotTS := e.startTS + snapshotTS := e.snapshotTS var err error e.txn, err = e.ctx.Txn(false) if err != nil { diff --git a/executor/trace_test.go b/executor/trace_test.go index f8e8e91ddebd7..396c160d18000 100644 --- a/executor/trace_test.go +++ b/executor/trace_test.go @@ -33,7 +33,7 @@ func TestTraceExec(t *testing.T) { require.GreaterOrEqual(t, len(rows), 1) // +---------------------------+-----------------+------------+ - // | operation | startTS | duration | + // | operation | snapshotTS | duration | // +---------------------------+-----------------+------------+ // | session.getTxnFuture | 22:08:38.247834 | 78.909µs | // | ├─session.Execute | 22:08:38.247829 | 1.478487ms | diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index 9f81d94897c3e..f9d0d6f1edf4c 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -377,7 +377,7 @@ func TestTidbSnapshotVarInRC(t *testing.T) { tk.MustExec("rollback") } -func TestErrorInInsertInRC(t *testing.T) { +func TestConflictErrorsInRC(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -391,6 +391,8 @@ func TestErrorInInsertInRC(t *testing.T) { tk.MustExec("create table t (id int primary key, v int)") tk.MustExec("set tx_isolation='READ-COMMITTED'") + + // Test for insert tk.MustExec("begin pessimistic") tk2.MustExec("insert into t values (1, 2)") se.SetValue(sessiontxn.AssertLockErr, nil) @@ -404,30 +406,17 @@ func TestErrorInInsertInRC(t *testing.T) { se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("rollback") - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) -} - -func TestErrorInDeleteInRC(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) - store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - se := tk.Session() - tk2 := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk2.MustExec("use test") - tk.MustExec("create table t (id int primary key, v int)") + // Test for delete + tk.MustExec("truncate t") tk.MustExec("insert into t values (1, 1), (2, 2)") - tk.MustExec("set tx_isolation='READ-COMMITTED'") tk.MustExec("begin pessimistic") tk2.MustExec("insert into t values (3, 1)") se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("delete from t where v = 1") - _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) require.False(t, ok) tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) tk.MustExec("commit") @@ -442,30 +431,17 @@ func TestErrorInDeleteInRC(t *testing.T) { tk.MustQuery("select * from t for update").Check(testkit.Rows()) tk.MustExec("rollback") - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) -} - -func TestErrorInUpdateInRC(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) - store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - se := tk.Session() - tk2 := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk2.MustExec("use test") - tk.MustExec("create table t (id int primary key, v int)") + // Test for update + tk.MustExec("truncate t") tk.MustExec("insert into t values (1, 1), (2, 2)") - tk.MustExec("set tx_isolation='READ-COMMITTED'") tk.MustExec("begin pessimistic") tk2.MustExec("update t set v = v + 10") se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("update t set v = v + 10") - _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) require.False(t, ok) tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) tk.MustExec("commit") @@ -479,6 +455,7 @@ func TestErrorInUpdateInRC(t *testing.T) { tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index d111e9c195e7e..abc2b3c4e2250 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -451,8 +451,8 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { } var errorsInInsert = []string{ - "insertWriteConflict", - "insertDuplicateKey", + "errWriteConflict", + "errDuplicateKey", } func TestErrorInInsertInRR(t *testing.T) { @@ -505,7 +505,7 @@ func TestErrorInPointGetForUpdateInRR(t *testing.T) { tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - require.Equal(t, records["insertWriteConflict"], 1) + require.Equal(t, records["errWriteConflict"], 1) tk.MustExec("commit") // batch point get @@ -515,7 +515,7 @@ func TestErrorInPointGetForUpdateInRR(t *testing.T) { tk.MustQuery("select * from t where id = 1 or id = 2 for update").Check(testkit.Rows("1 21", "2 2")) records, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - require.Equal(t, records["insertWriteConflict"], 1) + require.Equal(t, records["errWriteConflict"], 1) tk.MustExec("commit") tk.MustExec("rollback") @@ -555,7 +555,7 @@ func TestErrorInDeleteInRR(t *testing.T) { records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - require.Equal(t, records["insertWriteConflict"], 1) + require.Equal(t, records["errWriteConflict"], 1) tk.MustQuery("select * from t for update").Check(testkit.Rows()) tk.MustExec("rollback") @@ -592,7 +592,7 @@ func TestErrorInUpdateInRR(t *testing.T) { tk.MustExec("update t set v = v + 10 where id = 1") records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) require.True(t, ok) - require.Equal(t, records["insertWriteConflict"], 1) + require.Equal(t, records["errWriteConflict"], 1) tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) tk.MustExec("rollback") From 0b431c257b8e5864d19c07ef8be57be23bf2865d Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 19:39:42 +0800 Subject: [PATCH 10/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 16 ++++------------ sessiontxn/isolation/readcommitted.go | 9 ++++++++- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index d4b24282cea5e..6fda7df1ae558 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -659,11 +659,9 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor b.hasLock = true // Build 'select for update' using the 'for update' ts. - var forUpdateTS uint64 - if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = forUpdateTS src := b.build(v.Children()[0]) if b.err != nil { @@ -868,11 +866,9 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - var forUpdateTS uint64 - if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = forUpdateTS selectExec := b.build(v.SelectPlan) if b.err != nil { @@ -2116,11 +2112,9 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } - var forUpdateTS uint64 - if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = forUpdateTS selExec := b.build(v.SelectPlan) if b.err != nil { @@ -2176,11 +2170,9 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - var forUpdateTS uint64 - if forUpdateTS, b.err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS(); b.err != nil { + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = forUpdateTS selExec := b.build(v.SelectPlan) if b.err != nil { diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index a563c46b1cfa0..ca25c6243c92e 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -54,6 +54,8 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState + // latestStmtTS records the latest stmtTS we fetched + latestStmtTS uint64 availableRCCheckTS uint64 } @@ -136,7 +138,11 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { } if p.optimizeForNotFetchingLatestTS { - return p.getTxnStartTS() + if p.latestStmtTS != 0 { + return p.latestStmtTS, nil + } else { + return p.getTxnStartTS() + } } var txn kv.Transaction @@ -155,6 +161,7 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { txn.SetOption(kv.SnapshotTS, ts) p.stmtTS = ts + p.latestStmtTS = ts p.availableRCCheckTS = ts return } From 516530b9e423b83108ba573a4ff8e11ebf71f02e Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 19:54:35 +0800 Subject: [PATCH 11/36] update Signed-off-by: SpadeA-Tang --- executor/point_get.go | 1 + sessiontxn/isolation/readcommitted.go | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/point_get.go b/executor/point_get.go index 900677ef30602..51c58785b94da 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -17,6 +17,7 @@ package executor import ( "context" "fmt" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index ca25c6243c92e..cf12811ac448b 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -140,9 +140,8 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { if p.optimizeForNotFetchingLatestTS { if p.latestStmtTS != 0 { return p.latestStmtTS, nil - } else { - return p.getTxnStartTS() } + return p.getTxnStartTS() } var txn kv.Transaction From 4c832a2629c1d180b506a2592db7ec1e05ee6ac4 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 15 Jun 2022 20:40:59 +0800 Subject: [PATCH 12/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 5 +++ sessiontxn/isolation/repeatable_reat_test.go | 37 ++++++++++++++++---- 2 files changed, 35 insertions(+), 7 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index fc769f7c4b35a..f33ddc6925c54 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -187,6 +187,10 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { optimizeForNotFetchingLatestTS = true } + case *plannercore.PhysicalSelection: + if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { + optimizeForNotFetchingLatestTS = true + } case *plannercore.Update: if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { optimizeForNotFetchingLatestTS = true @@ -201,6 +205,7 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} } case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: optimizeForNotFetchingLatestTS = true + } p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_reat_test.go index abc2b3c4e2250..6e2c72c2b81e8 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_reat_test.go @@ -455,7 +455,7 @@ var errorsInInsert = []string{ "errDuplicateKey", } -func TestErrorInInsertInRR(t *testing.T) { +func TestConflictErrorInInsertInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -484,7 +484,7 @@ func TestErrorInInsertInRR(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } -func TestErrorInPointGetForUpdateInRR(t *testing.T) { +func TestConflictErrorInPointGetForUpdateInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -500,7 +500,6 @@ func TestErrorInPointGetForUpdateInRR(t *testing.T) { tk.MustExec("begin pessimistic") tk2.MustExec("update t set v = v + 10 where id = 1") - se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) @@ -523,7 +522,7 @@ func TestErrorInPointGetForUpdateInRR(t *testing.T) { } // Delete should get the latest ts and thus does not incur write conflict -func TestErrorInDeleteInRR(t *testing.T) { +func TestConflictErrorInDeleteInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -539,7 +538,6 @@ func TestErrorInDeleteInRR(t *testing.T) { tk.MustExec("begin pessimistic") tk2.MustExec("insert into t values (3, 1)") - se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("delete from t where v = 1") _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) @@ -562,7 +560,7 @@ func TestErrorInDeleteInRR(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } -func TestErrorInUpdateInRR(t *testing.T) { +func TestConflictErrorInUpdateInRR(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -578,7 +576,6 @@ func TestErrorInUpdateInRR(t *testing.T) { tk.MustExec("begin pessimistic") tk2.MustExec("update t set v = v + 10") - se.SetValue(sessiontxn.AssertLockErr, nil) tk.MustExec("update t set v = v + 10") _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) @@ -599,6 +596,32 @@ func TestErrorInUpdateInRR(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } +func TestConflictErrorInOtherQueryContainingPointGet(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id=1 and v > 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, inTxn bool) *txnAssert[*isolation.PessimisticRRTxnContextProvider] { return &txnAssert[*isolation.PessimisticRRTxnContextProvider]{ From 5e7fc566cf2d0db5e80f5e66aa6f83aa7ab37b3a Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 16 Jun 2022 11:31:01 +0800 Subject: [PATCH 13/36] checkpoint Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/readcommitted.go | 78 +++++++++------------- sessiontxn/isolation/readcommitted_test.go | 5 +- 2 files changed, 32 insertions(+), 51 deletions(-) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index cf12811ac448b..154bea0993378 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -32,21 +32,16 @@ import ( ) type stmtState struct { - stmtTS uint64 - stmtTSFuture oracle.Future - stmtUseStartTS bool - optimizeForNotFetchingLatestTS bool - onNextRetryOrStmt func() error + stmtTS uint64 + stmtTSFuture oracle.Future + stmtUseStartTS bool + stmtReuseStmtTS bool } func (s *stmtState) prepareStmt(useStartTS bool) error { - onNextStmt := s.onNextRetryOrStmt *s = stmtState{ stmtUseStartTS: useStartTS, } - if onNextStmt != nil { - return onNextStmt() - } return nil } @@ -54,9 +49,9 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState - // latestStmtTS records the latest stmtTS we fetched - latestStmtTS uint64 - availableRCCheckTS uint64 + // latestTS records the latest stmtTS we fetched + latestTS uint64 + latestTSValid bool } // NewPessimisticRCTxnContextProvider returns a new PessimisticRCTxnContextProvider @@ -69,12 +64,15 @@ func NewPessimisticRCTxnContextProvider(sctx sessionctx.Context, causalConsisten txnCtx.IsPessimistic = true txnCtx.Isolation = ast.ReadCommitted }, - onTxnActive: func(txn kv.Transaction) { - txn.SetOption(kv.Pessimistic, true) - }, }, } + provider.onTxnActive = func(txn kv.Transaction) { + txn.SetOption(kv.Pessimistic, true) + provider.latestTS = txn.StartTS() + provider.latestTSValid = true + } + provider.getStmtReadTSFunc = provider.getStmtTS provider.getStmtForUpdateTSFunc = provider.getStmtTS return provider @@ -85,14 +83,12 @@ func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context) error if err := p.baseTxnContextProvider.OnStmtStart(ctx); err != nil { return err } - p.optimizeForNotFetchingLatestTS = false return p.prepareStmt(!p.isTxnPrepared) } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { // Invalid rc check for next statement or retry when error occurs - p.availableRCCheckTS = 0 switch point { case sessiontxn.StmtErrAfterQuery: @@ -109,7 +105,6 @@ func (p *PessimisticRCTxnContextProvider) OnStmtRetry(ctx context.Context) error if err := p.baseTxnContextProvider.OnStmtRetry(ctx); err != nil { return err } - p.optimizeForNotFetchingLatestTS = false return p.prepareStmt(false) } @@ -123,8 +118,8 @@ func (p *PessimisticRCTxnContextProvider) prepareStmtTS() { switch { case p.stmtUseStartTS: stmtTSFuture = sessiontxn.FuncFuture(p.getTxnStartTS) - case p.availableRCCheckTS != 0 && sessVars.StmtCtx.RCCheckTS: - stmtTSFuture = sessiontxn.ConstantFuture(p.availableRCCheckTS) + case p.latestTSValid && (sessVars.StmtCtx.RCCheckTS || p.stmtReuseStmtTS): + stmtTSFuture = sessiontxn.ConstantFuture(p.latestTS) default: stmtTSFuture = sessiontxn.NewOracleFuture(p.ctx, p.sctx, sessVars.TxnCtx.TxnScope) } @@ -137,13 +132,6 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { return p.stmtTS, nil } - if p.optimizeForNotFetchingLatestTS { - if p.latestStmtTS != 0 { - return p.latestStmtTS, nil - } - return p.getTxnStartTS() - } - var txn kv.Transaction if txn, err = p.activeTxn(); err != nil { return 0, err @@ -157,11 +145,12 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { // forUpdateTS should exactly equal to the read ts txnCtx := p.sctx.GetSessionVars().TxnCtx txnCtx.SetForUpdateTS(ts) + ts = txnCtx.GetForUpdateTS() txn.SetOption(kv.SnapshotTS, ts) p.stmtTS = ts - p.latestStmtTS = ts - p.availableRCCheckTS = ts + p.latestTS = ts + p.latestTSValid = true return } @@ -169,16 +158,18 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { // At this point the query will be retried from the beginning. func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() - if sessVars.StmtCtx.RCCheckTS && errors.ErrorEqual(queryErr, kv.ErrWriteConflict) { - logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", - zap.String("sql", sessVars.StmtCtx.OriginalSQL)) - return sessiontxn.RetryReady() + if !errors.ErrorEqual(queryErr, kv.ErrWriteConflict) || !sessVars.StmtCtx.RCCheckTS { + return sessiontxn.NoIdea() } - return sessiontxn.NoIdea() + p.latestTSValid = false + logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", + zap.String("sql", sessVars.StmtCtx.OriginalSQL)) + return sessiontxn.RetryReady() } func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { + p.latestTSValid = false txnCtx := p.sctx.GetSessionVars().TxnCtx retryable := false if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok && deadlock.IsRetryable { @@ -196,12 +187,6 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr retryable = true } - // force refresh ts in next retry or statement when lock error occurs - p.onNextRetryOrStmt = func() error { - _, err := p.getStmtTS() - return err - } - if retryable { return sessiontxn.RetryReady() } @@ -230,6 +215,10 @@ func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{} return nil } + if p.stmtUseStartTS || p.stmtReuseStmtTS || !p.latestTSValid { + return nil + } + plan, ok := val.(plannercore.Plan) if !ok { return nil @@ -239,14 +228,9 @@ func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - optimizeForNotFetchingLatestTS := false - if v, ok := plan.(*plannercore.Insert); ok { - if v.SelectPlan == nil { - optimizeForNotFetchingLatestTS = true - } + if v, ok := plan.(*plannercore.Insert); ok && v.SelectPlan == nil { + p.stmtReuseStmtTS = true } - p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS - return nil } diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index f9d0d6f1edf4c..49284d557cb45 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -102,14 +102,11 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, errors.New("err")) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) - compareTS = getOracleTS(t, se) - require.Greater(t, compareTS, rcCheckTS) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO())) ts, err = provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) - rcCheckTS = ts + require.Equal(t, rcCheckTS, ts) // `StmtErrAfterPessimisticLock` will still disable rc check require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) From 781ce8ff3110c2efcf9fcf75f596657b21408bd5 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 16 Jun 2022 14:01:23 +0800 Subject: [PATCH 14/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/readcommitted.go | 56 ++++++++++++++------------- 1 file changed, 29 insertions(+), 27 deletions(-) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 154bea0993378..36dfa19304c77 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -32,10 +32,9 @@ import ( ) type stmtState struct { - stmtTS uint64 - stmtTSFuture oracle.Future - stmtUseStartTS bool - stmtReuseStmtTS bool + stmtTS uint64 + stmtTSFuture oracle.Future + stmtUseStartTS bool } func (s *stmtState) prepareStmt(useStartTS bool) error { @@ -49,9 +48,8 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState - // latestTS records the latest stmtTS we fetched - latestTS uint64 - latestTSValid bool + latestOracleTS uint64 + latestOracleTSValid bool } // NewPessimisticRCTxnContextProvider returns a new PessimisticRCTxnContextProvider @@ -69,10 +67,9 @@ func NewPessimisticRCTxnContextProvider(sctx sessionctx.Context, causalConsisten provider.onTxnActive = func(txn kv.Transaction) { txn.SetOption(kv.Pessimistic, true) - provider.latestTS = txn.StartTS() - provider.latestTSValid = true + provider.latestOracleTS = txn.StartTS() + provider.latestOracleTSValid = true } - provider.getStmtReadTSFunc = provider.getStmtTS provider.getStmtForUpdateTSFunc = provider.getStmtTS return provider @@ -88,8 +85,6 @@ func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { - // Invalid rc check for next statement or retry when error occurs - switch point { case sessiontxn.StmtErrAfterQuery: return p.handleAfterQueryError(err) @@ -118,15 +113,30 @@ func (p *PessimisticRCTxnContextProvider) prepareStmtTS() { switch { case p.stmtUseStartTS: stmtTSFuture = sessiontxn.FuncFuture(p.getTxnStartTS) - case p.latestTSValid && (sessVars.StmtCtx.RCCheckTS || p.stmtReuseStmtTS): - stmtTSFuture = sessiontxn.ConstantFuture(p.latestTS) + case p.latestOracleTSValid && sessVars.StmtCtx.RCCheckTS: + stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) default: - stmtTSFuture = sessiontxn.NewOracleFuture(p.ctx, p.sctx, sessVars.TxnCtx.TxnScope) + stmtTSFuture = p.getOracleFuture() } p.stmtTSFuture = stmtTSFuture } +func (p *PessimisticRCTxnContextProvider) getOracleFuture() sessiontxn.FuncFuture { + txnCtx := p.sctx.GetSessionVars().TxnCtx + future := sessiontxn.NewOracleFuture(p.ctx, p.sctx, txnCtx.TxnScope) + return func() (ts uint64, err error) { + if ts, err = future.Wait(); err != nil { + return + } + txnCtx.SetForUpdateTS(ts) + ts = txnCtx.GetForUpdateTS() + p.latestOracleTS = ts + p.latestOracleTSValid = true + return + } +} + func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { if p.stmtTS != 0 { return p.stmtTS, nil @@ -142,15 +152,8 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { return 0, err } - // forUpdateTS should exactly equal to the read ts - txnCtx := p.sctx.GetSessionVars().TxnCtx - txnCtx.SetForUpdateTS(ts) - ts = txnCtx.GetForUpdateTS() txn.SetOption(kv.SnapshotTS, ts) - p.stmtTS = ts - p.latestTS = ts - p.latestTSValid = true return } @@ -162,14 +165,14 @@ func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) return sessiontxn.NoIdea() } - p.latestTSValid = false + p.latestOracleTSValid = false logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", zap.String("sql", sessVars.StmtCtx.OriginalSQL)) return sessiontxn.RetryReady() } func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { - p.latestTSValid = false + p.latestOracleTSValid = false txnCtx := p.sctx.GetSessionVars().TxnCtx retryable := false if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok && deadlock.IsRetryable { @@ -190,7 +193,6 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr if retryable { return sessiontxn.RetryReady() } - return sessiontxn.ErrorAction(lockErr) } @@ -215,7 +217,7 @@ func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{} return nil } - if p.stmtUseStartTS || p.stmtReuseStmtTS || !p.latestTSValid { + if p.stmtUseStartTS || !p.latestOracleTSValid { return nil } @@ -229,7 +231,7 @@ func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{} } if v, ok := plan.(*plannercore.Insert); ok && v.SelectPlan == nil { - p.stmtReuseStmtTS = true + p.stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) } return nil From b54c3683199479cd3f81d2c137edc6f3640e66cf Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 16 Jun 2022 16:54:37 +0800 Subject: [PATCH 15/36] save Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/main_test.go | 27 +++++++++ sessiontxn/isolation/repeatable_read.go | 73 +++++++++++++++++-------- 2 files changed, 77 insertions(+), 23 deletions(-) diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index 2a14c31978461..48d5ef8718930 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -16,6 +16,8 @@ package isolation_test import ( "context" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/testkit" "testing" "time" @@ -96,3 +98,28 @@ func (a *txnAssert[T]) CheckAndGetProvider(t *testing.T) T { a.Check(t) return sessiontxn.GetTxnManager(a.sctx).GetContextProvider().(T) } + +func TestConflictErrorInOtherQueryContainingPointGet111(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 1), (2,2),(3,3),(4,4)") + tk.MustQuery("select * from t where id=1 for update union all select * from t where id = 2 for update").Check(testkit.Rows("1 1", "2 2")) + //tk.MustExec("select * from t where id = 1 and v > 1 for update") + //tk.MustExec("select * from t where id in (1, 2, 3) order by id for update") + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + + tk.MustExec("rollback") +} diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index f33ddc6925c54..79c62972d8f75 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -181,36 +181,63 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - optimizeForNotFetchingLatestTS := false + //optimizeForNotFetchingLatestTS := false + //switch v := plan.(type) { + //case *plannercore.PhysicalLock: + // if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { + // optimizeForNotFetchingLatestTS = true + // } + //case *plannercore.PhysicalSelection: + // if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { + // optimizeForNotFetchingLatestTS = true + // } + //case *plannercore.Update: + // if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { + // optimizeForNotFetchingLatestTS = true + // } + //case *plannercore.Delete: + // if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { + // optimizeForNotFetchingLatestTS = true + // } + //case *plannercore.Insert: + // if v.SelectPlan == nil { + // optimizeForNotFetchingLatestTS = true + // } + //case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: + // optimizeForNotFetchingLatestTS = true + // + //} + + p.optimizeForNotFetchingLatestTS = unname(plan) + + return nil +} + +func unname(plan plannercore.Plan) bool { switch v := plan.(type) { - case *plannercore.PhysicalLock: - if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - optimizeForNotFetchingLatestTS = true + case *plannercore.PointGetPlan: + return true + case *plannercore.BatchPointGetPlan: + return true + case plannercore.PhysicalPlan: + if v.Children() == nil { + return false } - case *plannercore.PhysicalSelection: - if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - optimizeForNotFetchingLatestTS = true + allChildrenArePointGet := true + for _, p := range v.Children() { + allChildrenArePointGet = allChildrenArePointGet && unname(p) } + return allChildrenArePointGet case *plannercore.Update: - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - optimizeForNotFetchingLatestTS = true - } + _, ok := v.SelectPlan.(*plannercore.PointGetPlan) + return ok case *plannercore.Delete: - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - optimizeForNotFetchingLatestTS = true - } + _, ok := v.SelectPlan.(*plannercore.PointGetPlan) + return ok case *plannercore.Insert: - if v.SelectPlan == nil { - optimizeForNotFetchingLatestTS = true - } - case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: - optimizeForNotFetchingLatestTS = true - + return v.SelectPlan == nil } - - p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS - - return nil + return false } func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { From 139e506e2a16876a360319d01c1209592678165b Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Sat, 18 Jun 2022 11:26:38 +0800 Subject: [PATCH 16/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 43 ++++++------------------- 1 file changed, 9 insertions(+), 34 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 79c62972d8f75..1d7968cddf3d8 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -69,15 +69,16 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return p.forUpdateTS, nil } - if p.optimizeForNotFetchingLatestTS { - return p.getTxnStartTS() - } - var txn kv.Transaction if txn, err = p.activeTxn(); err != nil { return 0, err } + if p.optimizeForNotFetchingLatestTS { + p.forUpdateTS = p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() + return p.forUpdateTS, nil + } + txnCtx := p.sctx.GetSessionVars().TxnCtx futureTS := sessiontxn.NewOracleFuture(p.ctx, p.sctx, txnCtx.TxnScope) @@ -181,39 +182,13 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - //optimizeForNotFetchingLatestTS := false - //switch v := plan.(type) { - //case *plannercore.PhysicalLock: - // if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - // optimizeForNotFetchingLatestTS = true - // } - //case *plannercore.PhysicalSelection: - // if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - // optimizeForNotFetchingLatestTS = true - // } - //case *plannercore.Update: - // if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - // optimizeForNotFetchingLatestTS = true - // } - //case *plannercore.Delete: - // if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - // optimizeForNotFetchingLatestTS = true - // } - //case *plannercore.Insert: - // if v.SelectPlan == nil { - // optimizeForNotFetchingLatestTS = true - // } - //case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: - // optimizeForNotFetchingLatestTS = true - // - //} - - p.optimizeForNotFetchingLatestTS = unname(plan) + p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan) return nil } -func unname(plan plannercore.Plan) bool { +// optimizeForNotFetchingLatestTS searches for optimization condition recursively +func optimizeForNotFetchingLatestTS(plan plannercore.Plan) bool { switch v := plan.(type) { case *plannercore.PointGetPlan: return true @@ -225,7 +200,7 @@ func unname(plan plannercore.Plan) bool { } allChildrenArePointGet := true for _, p := range v.Children() { - allChildrenArePointGet = allChildrenArePointGet && unname(p) + allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p) } return allChildrenArePointGet case *plannercore.Update: From b3dd7eef92b53d386db55222e7aea367b347aada Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Sat, 18 Jun 2022 12:04:27 +0800 Subject: [PATCH 17/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/main_test.go | 4 ++-- sessiontxn/isolation/repeatable_read_test.go | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index 48d5ef8718930..d2f3600fa9a37 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -16,15 +16,15 @@ package isolation_test import ( "context" - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/testkit" "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testsetup" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index fa2dc3a6bff14..c6a72e283f7e4 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -410,7 +410,6 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.NoError(t, err) require.Greater(t, ts, compareTs) - provider = initializeRepeatableReadProvider(t, tk) forUpdateTS = se.GetSessionVars().TxnCtx.GetForUpdateTS() txnManager = sessiontxn.GetTxnManager(se) // test for batch point get for update From 3c7d1d31765a01505434b78e854eb7294065f863 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Sat, 18 Jun 2022 12:49:13 +0800 Subject: [PATCH 18/36] delete unrelated test Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/main_test.go | 27 ---------------------- sessiontxn/isolation/readcommitted_test.go | 4 ++-- 2 files changed, 2 insertions(+), 29 deletions(-) diff --git a/sessiontxn/isolation/main_test.go b/sessiontxn/isolation/main_test.go index d2f3600fa9a37..2a14c31978461 100644 --- a/sessiontxn/isolation/main_test.go +++ b/sessiontxn/isolation/main_test.go @@ -19,12 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testsetup" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -98,28 +96,3 @@ func (a *txnAssert[T]) CheckAndGetProvider(t *testing.T) T { a.Check(t) return sessiontxn.GetTxnManager(a.sctx).GetContextProvider().(T) } - -func TestConflictErrorInOtherQueryContainingPointGet111(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) - store, _, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - se := tk.Session() - tk2 := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk2.MustExec("use test") - tk.MustExec("create table t (id int primary key, v int)") - - tk.MustExec("begin pessimistic") - tk2.MustExec("insert into t values (1, 1), (2,2),(3,3),(4,4)") - tk.MustQuery("select * from t where id=1 for update union all select * from t where id = 2 for update").Check(testkit.Rows("1 1", "2 2")) - //tk.MustExec("select * from t where id = 1 and v > 1 for update") - //tk.MustExec("select * from t where id in (1, 2, 3) order by id for update") - records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) - require.True(t, ok) - require.Equal(t, records["errWriteConflict"], 1) - - tk.MustExec("rollback") -} diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index 49284d557cb45..3cde637b71ef7 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -54,13 +54,13 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.NoError(t, err) forUpdateStmt := stmts[0] - compareTS := getOracleTS(t, se) + compareTS := se.GetSessionVars().TxnCtx.StartTS // first ts should request from tso require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO())) ts, err := provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) + require.Equal(t, ts, compareTS) rcCheckTS := ts // second ts should reuse first ts From 0b70f0ee3828df338e9b57df1a04694f8327cc97 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Sun, 19 Jun 2022 14:00:21 +0800 Subject: [PATCH 19/36] modify some comments Signed-off-by: SpadeA-Tang --- executor/adapter.go | 2 +- executor/batch_point_get.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 5b0dc365c0b82..50ff0bc8f86ce 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -352,7 +352,7 @@ func IsFastPlan(p plannercore.Plan) bool { } // Exec builds an Executor from a plan. If the Executor doesn't return result, -// like the INSERT, UPDATE statements, it executes in this function, if the Executor returns +// like the INSERT, UPDATE statements, it executes in this function. If the Executor returns // result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { defer func() { diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 33d655bc5d271..ac0abb9f75a1d 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -109,7 +109,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { var snapshot kv.Snapshot if txn.Valid() && txnCtx.StartTS == txnCtx.GetForUpdateTS() && txnCtx.StartTS == e.snapshotTS { // We can safely reuse the transaction snapshot if snapshotTS is equal to forUpdateTS. - // The snapshot may contains cache that can reduce RPC call. + // The snapshot may contain cache that can reduce RPC call. snapshot = txn.GetSnapshot() } else { snapshot = e.ctx.GetSnapshotWithTS(e.snapshotTS) From 9a500e9b321cbedc2c05f3864fa8469c831794d5 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 09:47:28 +0800 Subject: [PATCH 20/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 4 ++-- sessiontxn/failpoint.go | 4 ++-- sessiontxn/isolation/readcommitted.go | 3 ++- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 50ff0bc8f86ce..74299abb672af 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -776,9 +776,9 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } failpoint.Inject("assertPessimisticLockErr", func() { if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { - sessiontxn.AddEntrance(a.Ctx, "errWriteConflict") + sessiontxn.AddEntranceForLockError(a.Ctx, "errWriteConflict") } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { - sessiontxn.AddEntrance(a.Ctx, "errDuplicateKey") + sessiontxn.AddEntranceForLockError(a.Ctx, "errDuplicateKey") } }) diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 563022eca858c..26bc71bbb4079 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -98,8 +98,8 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } -// AddEntrance is used only for test -func AddEntrance(sctx sessionctx.Context, name string) { +// AddEntranceForLockError is used only for test +func AddEntranceForLockError(sctx sessionctx.Context, name string) { records, ok := sctx.Value(AssertLockErr).(map[string]int) if !ok { records = make(map[string]int) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 36dfa19304c77..b400a3b181228 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -48,7 +48,8 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState - latestOracleTS uint64 + latestOracleTS uint64 + // latestOracleTSValid shows whether we have already fetched a ts from pd and whether the ts we fetched is still valid. latestOracleTSValid bool } From b88d5ab1b0be3af3f1af7f421e6860e9695c61d1 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 11:10:06 +0800 Subject: [PATCH 21/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 4 ++-- sessiontxn/failpoint.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 74299abb672af..503eb1e7ccc00 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -776,9 +776,9 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } failpoint.Inject("assertPessimisticLockErr", func() { if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { - sessiontxn.AddEntranceForLockError(a.Ctx, "errWriteConflict") + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errWriteConflict") } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { - sessiontxn.AddEntranceForLockError(a.Ctx, "errDuplicateKey") + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errDuplicateKey") } }) diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 26bc71bbb4079..1f57f53a5374d 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -98,8 +98,8 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } -// AddEntranceForLockError is used only for test -func AddEntranceForLockError(sctx sessionctx.Context, name string) { +// AddAssertEntranceForLockError is used only for test +func AddAssertEntranceForLockError(sctx sessionctx.Context, name string) { records, ok := sctx.Value(AssertLockErr).(map[string]int) if !ok { records = make(map[string]int) From 7b40adc2f11671adf292760f7732b8a1c87601ea Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 14:29:05 +0800 Subject: [PATCH 22/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 40 +++- sessiontxn/isolation/repeatable_read_test.go | 184 ++++++++----------- 2 files changed, 108 insertions(+), 116 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 1d7968cddf3d8..1d404f69328bb 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -182,17 +182,21 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan) + p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan, nil) return nil } // optimizeForNotFetchingLatestTS searches for optimization condition recursively -func optimizeForNotFetchingLatestTS(plan plannercore.Plan) bool { +// Note: For point get and batch point get (name it plan), if the parent node is update/delete/physicalLock, we should check +// whether the plan.Lock. If the plan.Lock is false which means the lock has not pushed down to it successfully due to some reasons, +// we should not optimize for it. +func optimizeForNotFetchingLatestTS(plan plannercore.Plan, parentPlan plannercore.Plan) bool { switch v := plan.(type) { - case *plannercore.PointGetPlan: - return true - case *plannercore.BatchPointGetPlan: + case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: + if needNotToBeOptimized(plan, parentPlan) { + return false + } return true case plannercore.PhysicalPlan: if v.Children() == nil { @@ -200,21 +204,37 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan) bool { } allChildrenArePointGet := true for _, p := range v.Children() { - allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p) + allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, v) } return allChildrenArePointGet case *plannercore.Update: - _, ok := v.SelectPlan.(*plannercore.PointGetPlan) - return ok + return optimizeForNotFetchingLatestTS(v.SelectPlan, v) case *plannercore.Delete: - _, ok := v.SelectPlan.(*plannercore.PointGetPlan) - return ok + return optimizeForNotFetchingLatestTS(v.SelectPlan, v) case *plannercore.Insert: return v.SelectPlan == nil } return false } +func needNotToBeOptimized(plan plannercore.Plan, parentPlan plannercore.Plan) bool { + var flag bool + switch parentPlan.(type) { + case *plannercore.Update, *plannercore.Delete, *plannercore.PhysicalLock: + flag = true + } + + locked := true + switch v := plan.(type) { + case *plannercore.PointGetPlan: + locked = v.Lock + case *plannercore.BatchPointGetPlan: + locked = v.Lock + } + + return flag && !locked +} + func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() txnCtx := sessVars.TxnCtx diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index c6a72e283f7e4..5eaec72ce98d1 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -340,117 +340,89 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { tk.MustExec("insert into t values (1,1), (2,2)") se := tk.Session() provider := initializeRepeatableReadProvider(t, tk, true) - forUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() + lastFetchedForUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() txnManager := sessiontxn.GetTxnManager(se) - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err := parser.New().ParseOneStmt("delete from t where id = 1", "", "") - require.NoError(t, err) - compareTs := getOracleTS(t, se) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler := executor.Compiler{Ctx: se} - execStmt, err := compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err := provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, compareTs, ts) - require.Equal(t, ts, forUpdateTS) + type testStruct struct { + sql string + shouldOptimize bool + } - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("update t set v = v + 10 where id = 1", "", "") - require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + cases := []testStruct{ + { + "delete from t where id = 1", + true, + }, + { + "update t set v = v + 10 where id = 1", + true, + }, + { + "select * from (select * from t where id = 1 for update) as t1 for update", + true, + }, + { + "select * from t where id = 1 for update", + true, + }, + { + "select * from t where id = 1 or id = 2 for update", + true, + }, + { + "select * from t for update", + false, + }, + } - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("select * from (select * from t where id = 1 for update) as t1 for update", "", "") - require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + var stmt ast.StmtNode + var err error + var execStmt *executor.ExecStmt + var compiler executor.Compiler + var ts, compareTS uint64 + var action sessiontxn.StmtErrorAction - stmt, err = parser.New().ParseOneStmt("select * from t where id = 1 for update", "", "") - require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) - compareTs = getOracleTS(t, se) - // After retry, the ts should be larger than compareTs - action, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) - require.NoError(t, err) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - err = provider.OnStmtRetry(context.TODO()) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, ts, compareTs) + for _, c := range cases { + compareTS = getOracleTS(t, se) - forUpdateTS = se.GetSessionVars().TxnCtx.GetForUpdateTS() - txnManager = sessiontxn.GetTxnManager(se) - // test for batch point get for update - stmt, err = parser.New().ParseOneStmt("select * from t where id = 1 or id = 2 for update", "", "") - require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) - compareTs = getOracleTS(t, se) - // After retry, the ts should be larger than compareTs - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) - require.NoError(t, err) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - err = provider.OnStmtRetry(context.TODO()) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, ts, compareTs) + require.NoError(t, txnManager.OnStmtStart(context.TODO())) + stmt, err = parser.New().ParseOneStmt(c.sql, "", "") + require.NoError(t, err) - // Now, test for one that does not use the optimization - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("select * from t for update", "", "") - compareTs = getOracleTS(t, se) - require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, ts, compareTs) + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) + + compiler = executor.Compiler{Ctx: se} + execStmt, err = compiler.Compile(context.TODO(), stmt) + require.NoError(t, err) + + err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) + require.NoError(t, err) + + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + + if c.shouldOptimize { + require.Greater(t, compareTS, ts) + require.Equal(t, ts, lastFetchedForUpdateTS) + } else { + require.Greater(t, ts, compareTS) + } + + // retry + if c.shouldOptimize { + action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + require.NoError(t, err) + require.Equal(t, sessiontxn.StmtActionRetryReady, action) + err = provider.OnStmtRetry(context.TODO()) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Greater(t, ts, compareTS) + + lastFetchedForUpdateTS = ts + } + } // Test use startTS after optimize when autocommit=0 activeAssert := activePessimisticRRAssert(t, tk.Session(), true) @@ -468,7 +440,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Equal(t, tk.Session().GetSessionVars().TxnCtx.StartTS, ts) // Test still fetch for update ts after optimize when autocommit=0 - compareTs = getOracleTS(t, se) + compareTS = getOracleTS(t, se) activeAssert = activePessimisticRRAssert(t, tk.Session(), true) provider = initializeRepeatableReadProvider(t, tk, false) require.NoError(t, txnManager.OnStmtStart(context.TODO())) @@ -480,7 +452,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.NoError(t, err) ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) - require.Greater(t, ts, compareTs) + require.Greater(t, ts, compareTS) } var errorsInInsert = []string{ From 4906089c768f6f85f422a5168f4c8d6e53230d3a Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 14:35:36 +0800 Subject: [PATCH 23/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index b353b12c464c8..56c876069acdd 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4612,6 +4612,19 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan return nil } + if plan.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() + if err != nil { + b.err = err + return nil + } + decoder := NewRowDecoder(b.ctx, plan.Schema(), plan.TblInfo) e := &BatchPointGetExec{ baseExecutor: newBaseExecutor(b.ctx, plan.Schema(), plan.ID()), @@ -4631,27 +4644,18 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan columns: plan.Columns, } + if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { + e.cacheTable = b.getCacheTable(plan.TblInfo, snapshotTS) + } + if plan.TblInfo.TempTableType != model.TempTableNone { // Temporary table should not do any lock operations e.lock = false e.waitTime = 0 } - txnManager := sessiontxn.GetTxnManager(b.ctx) - var err error if e.lock { b.hasLock = true - e.startTS, err = txnManager.GetStmtForUpdateTS() - } else { - e.startTS, err = txnManager.GetStmtReadTS() - } - if err != nil { - b.err = err - return nil - } - - if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(plan.TblInfo, e.startTS) } var capacity int From bd12553d1d6268124f83ac938c03b9e34f0e6893 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 14:53:03 +0800 Subject: [PATCH 24/36] update Signed-off-by: SpadeA-Tang --- executor/builder.go | 1 + 1 file changed, 1 insertion(+) diff --git a/executor/builder.go b/executor/builder.go index 56c876069acdd..6f3077302bf78 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4631,6 +4631,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan tblInfo: plan.TblInfo, idxInfo: plan.IndexInfo, rowDecoder: decoder, + startTS: snapshotTS, readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, keepOrder: plan.KeepOrder, From 350efbb51ece098438d553aa42e3a8482bfa68b2 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 17:45:30 +0800 Subject: [PATCH 25/36] update Signed-off-by: SpadeA-Tang --- executor/batch_point_get.go | 1 - executor/builder.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index ac0abb9f75a1d..d75474618839c 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -97,7 +97,6 @@ func (e *BatchPointGetExec) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *BatchPointGetExec) Open(context.Context) error { - e.snapshotTS = e.startTS sessVars := e.ctx.GetSessionVars() txnCtx := sessVars.TxnCtx stmtCtx := sessVars.StmtCtx diff --git a/executor/builder.go b/executor/builder.go index 6f3077302bf78..34c067f791dc5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4631,7 +4631,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan tblInfo: plan.TblInfo, idxInfo: plan.IndexInfo, rowDecoder: decoder, - startTS: snapshotTS, + snapshotTS: snapshotTS, readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, keepOrder: plan.KeepOrder, From 2f28146f1a011b9a0fa2dff2315ff731673ca253 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 18:16:15 +0800 Subject: [PATCH 26/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 33 +++++++++++++------------ 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 1d404f69328bb..4be88bc0f1487 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -182,19 +182,19 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan, nil) + p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan, false) return nil } // optimizeForNotFetchingLatestTS searches for optimization condition recursively -// Note: For point get and batch point get (name it plan), if the parent node is update/delete/physicalLock, we should check -// whether the plan.Lock. If the plan.Lock is false which means the lock has not pushed down to it successfully due to some reasons, -// we should not optimize for it. -func optimizeForNotFetchingLatestTS(plan plannercore.Plan, parentPlan plannercore.Plan) bool { +// Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, +// we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. +// flag = true means one of the ancestor node is update/delete/physicalLock. +func optimizeForNotFetchingLatestTS(plan plannercore.Plan, flag bool) bool { switch v := plan.(type) { case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: - if needNotToBeOptimized(plan, parentPlan) { + if needNotToBeOptimized(plan, flag) { return false } return true @@ -203,27 +203,28 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan, parentPlan plannercor return false } allChildrenArePointGet := true + _, isPhysicalLock := v.(*plannercore.PhysicalLock) for _, p := range v.Children() { - allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, v) + allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, isPhysicalLock || flag) } return allChildrenArePointGet case *plannercore.Update: - return optimizeForNotFetchingLatestTS(v.SelectPlan, v) + return optimizeForNotFetchingLatestTS(v.SelectPlan, true) case *plannercore.Delete: - return optimizeForNotFetchingLatestTS(v.SelectPlan, v) + return optimizeForNotFetchingLatestTS(v.SelectPlan, true) case *plannercore.Insert: return v.SelectPlan == nil } return false } -func needNotToBeOptimized(plan plannercore.Plan, parentPlan plannercore.Plan) bool { - var flag bool - switch parentPlan.(type) { - case *plannercore.Update, *plannercore.Delete, *plannercore.PhysicalLock: - flag = true - } - +// needNotToBeOptimized +// The argument `flag` means whether one of the ancestor of the plan is update/delete/physicalLock. +// We do not optimize the point get/ batch point get if plan.lock = false and flag = true. +// Theoretically, the plan.lock should be true if the flag is true. But due to the bug describing in Issue35524, +// the plan.lock can be false in the case of flag being true. In this case, optimization here can lead to different results +// which cannot be accepted as AdviseOptimizeWithPlan cannot change results. +func needNotToBeOptimized(plan plannercore.Plan, flag bool) bool { locked := true switch v := plan.(type) { case *plannercore.PointGetPlan: From 4e52ee6f5a32b2dda66ec4b5896b0b9d1352cbb7 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 20 Jun 2022 18:23:51 +0800 Subject: [PATCH 27/36] fmt Signed-off-by: SpadeA-Tang --- executor/batch_point_get.go | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index d75474618839c..b5eb68a8b12de 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -56,7 +56,6 @@ type BatchPointGetExec struct { singlePart bool partTblID int64 idxVals [][]types.Datum - startTS uint64 readReplicaScope string isStaleness bool snapshotTS uint64 From 1c5a1e9bccc4eedb1b1aae2982efc5c958c7d28e Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Tue, 21 Jun 2022 09:59:11 +0800 Subject: [PATCH 28/36] update optimizeForNotFetchingLatestTS Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 39 ++++++++----------------- 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 4be88bc0f1487..56f4d05a2513c 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -166,7 +166,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point session // We expect that the data that the point get acquires has not been changed. // Benefit: Save the cost of acquiring ts from PD. // Drawbacks: If the data has been changed since the ts we used, we need to retry. -// One exception is insert operation, when it does not fetch data, we do not fetch the latest ts immediately. We only update ts +// One exception is insert operation, when it has no select plan, we do not fetch the latest ts immediately. We only update ts // if write conflict is incurred. func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { @@ -190,14 +190,17 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} // optimizeForNotFetchingLatestTS searches for optimization condition recursively // Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, // we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. -// flag = true means one of the ancestor node is update/delete/physicalLock. -func optimizeForNotFetchingLatestTS(plan plannercore.Plan, flag bool) bool { +// inLockOrWriteStmt = true means one of the ancestor node is update/delete/physicalLock. +func optimizeForNotFetchingLatestTS(plan plannercore.Plan, inLockOrWriteStmt bool) bool { switch v := plan.(type) { - case *plannercore.PointGetPlan, *plannercore.BatchPointGetPlan: - if needNotToBeOptimized(plan, flag) { - return false - } - return true + case *plannercore.PointGetPlan: + // We do not optimize the point get/ batch point get if plan.lock = false and inLockOrWriteStmt = true. + // Theoretically, the plan.lock should be true if the flag is true. But due to the bug describing in Issue35524, + // the plan.lock can be false in the case of inLockOrWriteStmt being true. In this case, optimization here can lead to different results + // which cannot be accepted as AdviseOptimizeWithPlan cannot change results. + return !inLockOrWriteStmt || v.Lock + case *plannercore.BatchPointGetPlan: + return !inLockOrWriteStmt || v.Lock case plannercore.PhysicalPlan: if v.Children() == nil { return false @@ -205,7 +208,7 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan, flag bool) bool { allChildrenArePointGet := true _, isPhysicalLock := v.(*plannercore.PhysicalLock) for _, p := range v.Children() { - allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, isPhysicalLock || flag) + allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, isPhysicalLock || inLockOrWriteStmt) } return allChildrenArePointGet case *plannercore.Update: @@ -218,24 +221,6 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan, flag bool) bool { return false } -// needNotToBeOptimized -// The argument `flag` means whether one of the ancestor of the plan is update/delete/physicalLock. -// We do not optimize the point get/ batch point get if plan.lock = false and flag = true. -// Theoretically, the plan.lock should be true if the flag is true. But due to the bug describing in Issue35524, -// the plan.lock can be false in the case of flag being true. In this case, optimization here can lead to different results -// which cannot be accepted as AdviseOptimizeWithPlan cannot change results. -func needNotToBeOptimized(plan plannercore.Plan, flag bool) bool { - locked := true - switch v := plan.(type) { - case *plannercore.PointGetPlan: - locked = v.Lock - case *plannercore.BatchPointGetPlan: - locked = v.Lock - } - - return flag && !locked -} - func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() txnCtx := sessVars.TxnCtx From be1c61196751d7486dd4a22672046028013d7ebb Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 22 Jun 2022 09:27:02 +0800 Subject: [PATCH 29/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/adapter.go b/executor/adapter.go index 972f62d9cdb1c..f414844069172 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -453,7 +453,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { curTxnStartTS = sctx.GetSessionVars().SnapshotTS } logutil.BgLogger().Info("Enable mockDelayInnerSessionExecute when execute statement", - zap.Uint64("snapshotTS", curTxnStartTS)) + zap.Uint64("startTS", curTxnStartTS)) time.Sleep(200 * time.Millisecond) }) From 83c5fa923f6ab8a47ee5ff8d6ff56cb51423b6c1 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Wed, 22 Jun 2022 12:07:00 +0800 Subject: [PATCH 30/36] remove unused code Signed-off-by: SpadeA-Tang --- executor/adapter.go | 32 -------------------------------- 1 file changed, 32 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index f414844069172..7785afbfe0f91 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -735,38 +735,6 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } } -// UpdateForUpdateTS updates the ForUpdateTS, if newForUpdateTS is 0, it obtain a new TS from PD. -func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { - txn, err := seCtx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - return errors.Trace(kv.ErrInvalidTxn) - } - - // The Oracle serializable isolation is actually SI in pessimistic mode. - // Do not update ForUpdateTS when the user is using the Serializable isolation level. - // It can be used temporarily on the few occasions when an Oracle-like isolation level is needed. - // Support for this does not mean that TiDB supports serializable isolation of MySQL. - // tidb_skip_isolation_level_check should still be disabled by default. - if seCtx.GetSessionVars().IsIsolation(ast.Serializable) { - return nil - } - if newForUpdateTS == 0 { - // Because the ForUpdateTS is used for the snapshot for reading data in DML. - // We can avoid allocating a global TSO here to speed it up by using the local TSO. - version, err := seCtx.GetStore().CurrentVersion(seCtx.GetSessionVars().TxnCtx.TxnScope) - if err != nil { - return err - } - newForUpdateTS = version.Ver - } - seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS) - txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) - return nil -} - // handlePessimisticLockError updates TS and rebuild executor if the err is write conflict. func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error) (_ Executor, err error) { if lockErr == nil { From 1da2557b7c3b967c3e0360dcb59ba6784a19adae Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 23 Jun 2022 13:25:18 +0800 Subject: [PATCH 31/36] update Signed-off-by: SpadeA-Tang --- executor/point_get.go | 1 + 1 file changed, 1 insertion(+) diff --git a/executor/point_get.go b/executor/point_get.go index 51c58785b94da..1b4d6666663b5 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -58,6 +58,7 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { snapshotTS, err := b.getSnapshotTS() if err != nil { + b.err = err return nil } From 251a87562dde484a0474d66931455bd060f200af Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 23 Jun 2022 14:48:38 +0800 Subject: [PATCH 32/36] update Signed-off-by: SpadeA-Tang --- executor/trace_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/trace_test.go b/executor/trace_test.go index 396c160d18000..9b448670cc39a 100644 --- a/executor/trace_test.go +++ b/executor/trace_test.go @@ -33,7 +33,7 @@ func TestTraceExec(t *testing.T) { require.GreaterOrEqual(t, len(rows), 1) // +---------------------------+-----------------+------------+ - // | operation | snapshotTS | duration | + // | operation | snapshotTS | duration | // +---------------------------+-----------------+------------+ // | session.getTxnFuture | 22:08:38.247834 | 78.909µs | // | ├─session.Execute | 22:08:38.247829 | 1.478487ms | From fe66d11294d23a1be55d407738b6a802c76c10b6 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Thu, 23 Jun 2022 15:42:15 +0800 Subject: [PATCH 33/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 0901302aa4fb7..c67d0fb55c67e 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -205,12 +205,13 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan, inLockOrWriteStmt boo if v.Children() == nil { return false } - allChildrenArePointGet := true _, isPhysicalLock := v.(*plannercore.PhysicalLock) for _, p := range v.Children() { - allChildrenArePointGet = allChildrenArePointGet && optimizeForNotFetchingLatestTS(p, isPhysicalLock || inLockOrWriteStmt) + if !optimizeForNotFetchingLatestTS(p, isPhysicalLock || inLockOrWriteStmt) { + return false + } } - return allChildrenArePointGet + return true case *plannercore.Update: return optimizeForNotFetchingLatestTS(v.SelectPlan, true) case *plannercore.Delete: From 3cfb2fa45ea785d71223cf09239f0463108d610e Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Fri, 24 Jun 2022 13:49:32 +0800 Subject: [PATCH 34/36] udpate Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index c67d0fb55c67e..1c931a127a6fd 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -182,16 +182,16 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - p.optimizeForNotFetchingLatestTS = optimizeForNotFetchingLatestTS(plan, false) + p.optimizeForNotFetchingLatestTS = notNeedGetLatestTSFromPD(plan, false) return nil } -// optimizeForNotFetchingLatestTS searches for optimization condition recursively +// notNeedGetLatestTSFromPD searches for optimization condition recursively // Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, // we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. // inLockOrWriteStmt = true means one of the ancestor node is update/delete/physicalLock. -func optimizeForNotFetchingLatestTS(plan plannercore.Plan, inLockOrWriteStmt bool) bool { +func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) bool { switch v := plan.(type) { case *plannercore.PointGetPlan: // We do not optimize the point get/ batch point get if plan.lock = false and inLockOrWriteStmt = true. @@ -207,15 +207,15 @@ func optimizeForNotFetchingLatestTS(plan plannercore.Plan, inLockOrWriteStmt boo } _, isPhysicalLock := v.(*plannercore.PhysicalLock) for _, p := range v.Children() { - if !optimizeForNotFetchingLatestTS(p, isPhysicalLock || inLockOrWriteStmt) { + if !notNeedGetLatestTSFromPD(p, isPhysicalLock || inLockOrWriteStmt) { return false } } return true case *plannercore.Update: - return optimizeForNotFetchingLatestTS(v.SelectPlan, true) + return notNeedGetLatestTSFromPD(v.SelectPlan, true) case *plannercore.Delete: - return optimizeForNotFetchingLatestTS(v.SelectPlan, true) + return notNeedGetLatestTSFromPD(v.SelectPlan, true) case *plannercore.Insert: return v.SelectPlan == nil } From 0d3e267dc2a4e1a60677c890d38ed26a4f8662eb Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Fri, 24 Jun 2022 17:35:08 +0800 Subject: [PATCH 35/36] update Signed-off-by: SpadeA-Tang --- sessiontxn/isolation/repeatable_read.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 1c931a127a6fd..571d2754be9a3 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -202,7 +202,7 @@ func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) boo case *plannercore.BatchPointGetPlan: return !inLockOrWriteStmt || v.Lock case plannercore.PhysicalPlan: - if v.Children() == nil { + if len(v.Children()) == 0 { return false } _, isPhysicalLock := v.(*plannercore.PhysicalLock) From 2d4caf4e30b872e7c9f6f95ef0858753244bfb24 Mon Sep 17 00:00:00 2001 From: SpadeA-Tang Date: Mon, 27 Jun 2022 11:24:22 +0800 Subject: [PATCH 36/36] update Signed-off-by: SpadeA-Tang --- executor/adapter.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 79d573ea72920..31a8c8d20150f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -752,7 +752,8 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } }() - action, err := sessiontxn.GetTxnManager(a.Ctx).OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + txnManager := sessiontxn.GetTxnManager(a.Ctx) + action, err := txnManager.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) if err != nil { return nil, err } @@ -767,10 +768,17 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error a.retryCount++ a.retryStartTime = time.Now() - err = sessiontxn.GetTxnManager(a.Ctx).OnStmtRetry(ctx) + err = txnManager.OnStmtRetry(ctx) if err != nil { return nil, err } + + // Without this line of code, the result will still be correct. But it can ensure that the update time of for update read + // is determined which is beneficial for testing. + if _, err = txnManager.GetStmtForUpdateTS(); err != nil { + return nil, err + } + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointOnStmtRetryAfterLockError) e, err := a.buildExecutor()