From 322468eef53778bab57e2bb4f4e4c4f2384b3707 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Fri, 17 Jun 2022 15:08:28 +0800 Subject: [PATCH 01/21] txn: fix bug that some times lock ts and read ts are not the same --- executor/adapter.go | 9 ++ executor/builder.go | 10 +- session/txnmanager.go | 17 ++- sessiontxn/failpoint.go | 22 +++ sessiontxn/isolation/repeatable_read.go | 9 +- ...e_reat_test.go => repeatable_read_test.go} | 66 +++++++-- sessiontxn/txn_context_test.go | 126 ++++++++++++++++++ 7 files changed, 236 insertions(+), 23 deletions(-) rename sessiontxn/isolation/{repeatable_reat_test.go => repeatable_read_test.go} (86%) diff --git a/executor/adapter.go b/executor/adapter.go index b98597820649b..5fab6678d9c81 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -417,6 +417,10 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) + failpoint.Inject("hookBeforeFirstRunExecutor", func() { + sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookBeforeFirstRunExecutorKey) + }) + if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -793,6 +797,11 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if err != nil { return nil, err } + + failpoint.Inject("hookAfterOnStmtRetryWithLockError", func() { + sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookAfterOnStmtRetryWithLockErrorKey) + }) + e, err := a.buildExecutor() if err != nil { return nil, err diff --git a/executor/builder.go b/executor/builder.go index 9445565e1c26c..d86200b573058 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2213,13 +2213,9 @@ func (b *executorBuilder) updateForUpdateTSIfNeeded(selectPlan plannercore.Physi } 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) + // GetStmtForUpdateTS will auto update the for update ts if necessary + _, err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() + return err } func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { diff --git a/session/txnmanager.go b/session/txnmanager.go index 2e04286bfb51f..9f254b6dd24ac 100644 --- a/session/txnmanager.go +++ b/session/txnmanager.go @@ -16,8 +16,10 @@ package session import ( "context" + "fmt" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/ast" @@ -71,7 +73,20 @@ func (m *txnManager) GetStmtForUpdateTS() (uint64, error) { if m.ctxProvider == nil { return 0, errors.New("context provider not set") } - return m.ctxProvider.GetStmtForUpdateTS() + + ts, err := m.ctxProvider.GetStmtForUpdateTS() + if err != nil { + return 0, err + } + + failpoint.Inject("assertTxnManagerForUpdateTSEqual", func() { + sessVars := m.sctx.GetSessionVars() + if txnCtxForUpdateTS := sessVars.TxnCtx.GetForUpdateTS(); sessVars.SnapshotTS == 0 && ts != txnCtxForUpdateTS { + panic(fmt.Sprintf("forUpdateTS not equal %d != %d", ts, txnCtxForUpdateTS)) + } + }) + + return ts, nil } func (m *txnManager) GetContextProvider() sessiontxn.TxnContextProvider { diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 241e39b7a8436..76ff4ea60e52a 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -16,6 +16,7 @@ package sessiontxn import ( "fmt" + "time" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" @@ -34,6 +35,14 @@ var AssertTxnInfoSchemaKey stringutil.StringerStr = "assertTxnInfoSchemaKey" // Only for test var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSchemaAfterRetryKey" +// HookBeforeFirstRunExecutorKey is the hook key for the executor first run +// Only for test +var HookBeforeFirstRunExecutorKey stringutil.StringerStr = "testHookKeyBeforeFirstRunExecutor" + +// HookAfterOnStmtRetryWithLockErrorKey is the hook key for after OnStmtRetry with lock error +// Only for test +var HookAfterOnStmtRetryWithLockErrorKey stringutil.StringerStr = "testHookKeyAfterOnStmtRetryWithLockError" + // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { records, ok := sctx.Value(AssertRecordsKey).(map[string]interface{}) @@ -84,3 +93,16 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { panic(fmt.Sprintf("Txn read ts not match, expect:%d, got:%d", expected, actual)) } } + +// ExecTestHook is used only for test. It consumes hookKey in session wait do what it gets from it. +func ExecTestHook(sctx sessionctx.Context, hookKey fmt.Stringer) { + c := sctx.Value(hookKey) + if ch, ok := c.(chan func()); ok { + select { + case fn := <-ch: + fn() + case <-time.After(time.Second * 10): + panic("timeout waiting for chan") + } + } +} diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index c7c9a25474352..861686e414bb7 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -68,15 +68,16 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return p.forUpdateTS, nil } - if p.followingOperatorIsPointGetForUpdate { - return p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS(), nil - } - var txn kv.Transaction if txn, err = p.activeTxn(); err != nil { return 0, err } + if p.followingOperatorIsPointGetForUpdate { + 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) diff --git a/sessiontxn/isolation/repeatable_reat_test.go b/sessiontxn/isolation/repeatable_read_test.go similarity index 86% rename from sessiontxn/isolation/repeatable_reat_test.go rename to sessiontxn/isolation/repeatable_read_test.go index 54743d5fd51dc..1efd2f459a37b 100644 --- a/sessiontxn/isolation/repeatable_reat_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -48,7 +48,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) se := tk.Session() - provider := initializeRepeatableReadProvider(t, tk) + provider := initializeRepeatableReadProvider(t, tk, true) var lockErr error @@ -138,7 +138,7 @@ func TestRepeatableReadProviderTS(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) se := tk.Session() - provider := initializeRepeatableReadProvider(t, tk) + provider := initializeRepeatableReadProvider(t, tk, true) stmts, _, err := parser.New().Parse("select * from t", "", "") require.NoError(t, err) @@ -223,7 +223,7 @@ func TestRepeatableReadProviderInitialize(t *testing.T) { // non-active txn and then active it tk.MustExec("rollback") tk.MustExec("set @@autocommit=0") - assert = inActivePessimisticRRAssert(se) + assert = inactivePessimisticRRAssert(se) assertAfterActive := activePessimisticRRAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider := assert.CheckAndGetProvider(t) @@ -236,7 +236,7 @@ func TestRepeatableReadProviderInitialize(t *testing.T) { // Case Pessimistic Autocommit config.GetGlobalConfig().PessimisticTxn.PessimisticAutoCommit.Store(true) - assert = inActivePessimisticRRAssert(se) + assert = inactivePessimisticRRAssert(se) assertAfterActive = activePessimisticRRAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) @@ -314,7 +314,7 @@ func TestTidbSnapshotVarInPessimisticRepeatableRead(t *testing.T) { tk.MustExec("rollback") tk.MustExec("set @@tidb_txn_mode='pessimistic'") tk.MustExec("set @@autocommit=0") - assert = inActivePessimisticRRAssert(se) + assert = inactivePessimisticRRAssert(se) assertAfterActive := activePessimisticRRAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) @@ -338,10 +338,11 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { tk.MustExec("create table t (id int primary key, v int)") tk.MustExec("insert into t values (1,1), (2,2)") se := tk.Session() - provider := initializeRepeatableReadProvider(t, tk) + provider := initializeRepeatableReadProvider(t, tk, true) forUpdateTS := 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) @@ -355,6 +356,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Greater(t, compareTs, ts) require.Equal(t, ts, forUpdateTS) + 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) compiler = executor.Compiler{Ctx: se} @@ -366,6 +368,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.NoError(t, err) require.Equal(t, ts, forUpdateTS) + 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) compiler = executor.Compiler{Ctx: se} @@ -378,6 +381,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Equal(t, ts, forUpdateTS) // 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) @@ -389,6 +393,36 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) require.Greater(t, ts, compareTs) + + // Test use startTS after optimize when autocommit=0 + activeAssert := activePessimisticRRAssert(t, tk.Session(), true) + provider = initializeRepeatableReadProvider(t, tk, false) + 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) + 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.Same(t, provider, activeAssert.CheckAndGetProvider(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) + activeAssert = activePessimisticRRAssert(t, tk.Session(), true) + provider = initializeRepeatableReadProvider(t, tk, false) + require.NoError(t, txnManager.OnStmtStart(context.TODO())) + stmt, err = parser.New().ParseOneStmt("select * from t", "", "") + require.NoError(t, err) + 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) } func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, @@ -403,7 +437,7 @@ func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, } } -func inActivePessimisticRRAssert(sctx sessionctx.Context) *txnAssert[*isolation.PessimisticRRTxnContextProvider] { +func inactivePessimisticRRAssert(sctx sessionctx.Context) *txnAssert[*isolation.PessimisticRRTxnContextProvider] { return &txnAssert[*isolation.PessimisticRRTxnContextProvider]{ sctx: sctx, isolation: "REPEATABLE-READ", @@ -412,9 +446,19 @@ func inActivePessimisticRRAssert(sctx sessionctx.Context) *txnAssert[*isolation. } } -func initializeRepeatableReadProvider(t *testing.T, tk *testkit.TestKit) *isolation.PessimisticRRTxnContextProvider { +func initializeRepeatableReadProvider(t *testing.T, tk *testkit.TestKit, active bool) *isolation.PessimisticRRTxnContextProvider { + tk.MustExec("commit") tk.MustExec("set @@tx_isolation = 'REPEATABLE-READ'") - assert := activePessimisticRRAssert(t, tk.Session(), true) - tk.MustExec("begin pessimistic") - return assert.CheckAndGetProvider(t) + tk.MustExec("set @@tidb_txn_mode= 'pessimistic'") + + if active { + assert := activePessimisticRRAssert(t, tk.Session(), true) + tk.MustExec("begin pessimistic") + return assert.CheckAndGetProvider(t) + } else { + tk.MustExec("set @@autocommit=0") + assert := inactivePessimisticRRAssert(tk.Session()) + require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) + return assert.CheckAndGetProvider(t) + } } diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index c7c03d67e8b5e..8ac6f0aaa7322 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -17,12 +17,14 @@ package sessiontxn_test import ( "context" "fmt" + "strings" "testing" "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" @@ -50,9 +52,12 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan", "return")) store, do, clean := testkit.CreateMockStoreAndDomain(t) @@ -80,9 +85,12 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan")) tk.Session().SetValue(sessiontxn.AssertRecordsKey, nil) @@ -702,3 +710,121 @@ func TestTxnContextPreparedStmtWithForUpdate(t *testing.T) { se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) tk.MustExec("rollback") } + +// See issue: https://github.com/pingcap/tidb/issues/35459 +func TestStillWriteConflictAfterRetry(t *testing.T) { + store, _, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + queries := []string{ + "select * from t1 for update", + "select * from t1 where id=1 for update", + "select * from t1 where id in (1, 2, 3) for update", + "select * from t1 where id=1 and v>0 for update", + "select * from t1 where id=1 for update union select * from t1 where id=1 for update", + "update t1 set v=v+1", + "update t1 set v=v+1 where id=1", + "update t1 set v=v+1 where id=1 and v>0", + "update t1 set v=v+1 where id in (1, 2, 3)", + "update t1 set v=v+1 where id in (1, 2, 3) and v>0", + } + + for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { + for _, query := range queries { + for _, autocommit := range []bool{true, false} { + t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { + testStillWriteConflictAfterRetry(t, store, isolation, query, autocommit) + }) + } + } + } +} + +func testStillWriteConflictAfterRetry(t *testing.T, store kv.Storage, isolation string, query string, autocommit bool) { + tk := testkit.NewTestKit(t, store) + defer tk.MustExec("rollback") + + tk.MustExec("use test") + tk.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) + tk.MustExec("set autocommit=1") + tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + + se := tk.Session() + chanBeforeRunStmt := make(chan func(), 1) + chanAfterOnStmtRetry := make(chan func(), 1) + c2 := make(chan string, 1) + c3 := make(chan string, 1) + wait := func(ch chan string, expect string) { + select { + case got := <-ch: + if got != expect { + panic(fmt.Sprintf("expect '%s', got '%s'", expect, got)) + } + case <-time.After(time.Second * 10): + panic("wait2 timeout") + } + } + + if autocommit { + tk.MustExec("begin") + } else { + tk.MustExec("set @@autocommit=0") + } + + se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, chanBeforeRunStmt) + se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, chanAfterOnStmtRetry) + defer func() { + se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) + se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) + }() + + chanBeforeRunStmt <- func() { + c2 <- "now before session1 runStmt" + wait(c3, "session2 updated v=v+1 done") + } + + chanAfterOnStmtRetry <- func() { + c2 <- "now after OnStmtRetry before rebuild executor" + wait(c3, "session2 updated v=v+1 again done") + } + + go func() { + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + // first conflict + wait(c2, "now before session1 runStmt") + tk2.MustExec("update t1 set v=v+1 where id=1") + c3 <- "session2 updated v=v+1 done" + + // second conflict + wait(c2, "now after OnStmtRetry before rebuild executor") + tk2.MustExec("update t1 set v=v+1 where id=1") + c3 <- "session2 updated v=v+1 again done" + chanAfterOnStmtRetry <- func() {} + c3 <- "done" + }() + + isSelect := false + if strings.HasPrefix(query, "update ") { + tk.MustExec(query) + } else if strings.HasPrefix(query, "select ") { + isSelect = true + tk.MustQuery(query).Check(testkit.Rows("1 12")) + } else { + require.FailNowf(t, "invalid query: %s", query) + } + + wait(c3, "done") + + se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) + se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) + if isSelect { + tk.MustExec("update t1 set v=v+1") + } + tk.MustExec("commit") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 13")) + tk.MustExec("rollback") +} From 240140ba2055831de403b5f256d812e426496a59 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Fri, 17 Jun 2022 15:28:06 +0800 Subject: [PATCH 02/21] update --- sessiontxn/isolation/repeatable_read_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index 1efd2f459a37b..39fc031b71adf 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -455,10 +455,10 @@ func initializeRepeatableReadProvider(t *testing.T, tk *testkit.TestKit, active assert := activePessimisticRRAssert(t, tk.Session(), true) tk.MustExec("begin pessimistic") return assert.CheckAndGetProvider(t) - } else { - tk.MustExec("set @@autocommit=0") - assert := inactivePessimisticRRAssert(tk.Session()) - require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) - return assert.CheckAndGetProvider(t) } + + tk.MustExec("set @@autocommit=0") + assert := inactivePessimisticRRAssert(tk.Session()) + require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) + return assert.CheckAndGetProvider(t) } From 92c87e4af56487b66c1eb18dec0c53df5ee62775 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Fri, 17 Jun 2022 22:11:58 +0800 Subject: [PATCH 03/21] Add session stop --- executor/adapter.go | 10 +- sessiontxn/failpoint.go | 4 + sessiontxn/txn_context_test.go | 33 +++- testkit/testkit.go | 89 ++++++++++ util/multithreadtest/runner.go | 261 ++++++++++++++++++++++++++++ util/multithreadtest/runner_test.go | 49 ++++++ 6 files changed, 438 insertions(+), 8 deletions(-) create mode 100644 util/multithreadtest/runner.go create mode 100644 util/multithreadtest/runner_test.go diff --git a/executor/adapter.go b/executor/adapter.go index 5fab6678d9c81..4b9ef59722bb1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -24,6 +24,8 @@ import ( "sync/atomic" "time" + "github.com/pingcap/tidb/util/multithreadtest" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -417,8 +419,8 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) - failpoint.Inject("hookBeforeFirstRunExecutor", func() { - sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookBeforeFirstRunExecutorKey) + failpoint.Inject("sessionStop", func() { + multithreadtest.SessionStop(a.Ctx, sessiontxn.TestSessionStopBeforeExecutorFirstRun) }) if err = e.Open(ctx); err != nil { @@ -798,8 +800,8 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error return nil, err } - failpoint.Inject("hookAfterOnStmtRetryWithLockError", func() { - sessiontxn.ExecTestHook(a.Ctx, sessiontxn.HookAfterOnStmtRetryWithLockErrorKey) + failpoint.Inject("sessionStop", func() { + multithreadtest.SessionStop(a.Ctx, sessiontxn.TestSessionOnStmtRetryAfterLockError) }) e, err := a.buildExecutor() diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 76ff4ea60e52a..d34a77ccc131e 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -43,6 +43,10 @@ var HookBeforeFirstRunExecutorKey stringutil.StringerStr = "testHookKeyBeforeFir // Only for test var HookAfterOnStmtRetryWithLockErrorKey stringutil.StringerStr = "testHookKeyAfterOnStmtRetryWithLockError" +var TestSessionStopBeforeExecutorFirstRun = "before executor first run" + +var TestSessionOnStmtRetryAfterLockError = "lock error, OnStmtRetry called" + // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { records, ok := sctx.Value(AssertRecordsKey).(map[string]interface{}) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 8ac6f0aaa7322..dc53f73088d6b 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -52,8 +52,7 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/sessionStop", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) @@ -85,8 +84,7 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookBeforeFirstRunExecutor")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/hookAfterOnStmtRetryWithLockError")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/sessionStop")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) @@ -711,6 +709,33 @@ func TestTxnContextPreparedStmtWithForUpdate(t *testing.T) { tk.MustExec("rollback") } +func TestAA(t *testing.T) { + store, _, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + stops := []string{sessiontxn.TestSessionStopBeforeExecutorFirstRun, sessiontxn.TestSessionOnStmtRetryAfterLockError} + runner := testkit.NewMultiSessionsRunner(t, store) + s1 := runner.Session("s1").StopWhen(stops).Start(func(t *testing.T, tk *testkit.SessionThreadTestKit) { + tk.MustExec("use test") + tk.MustExec("begin pessimistic") + tk.EnableSessionStopPoint() + tk.MustQuery("select * from t1 for update").Check(testkit.Rows("1 12")) + tk.DisableSessionStopPoint() + tk.MustExec("rollback") + }) + + s1.Step().CheckCurrentStop(sessiontxn.TestSessionStopBeforeExecutorFirstRun) + tk2.MustExec("update t1 set v=v+1") + s1.Step().CheckCurrentStop(sessiontxn.TestSessionOnStmtRetryAfterLockError) + tk2.MustExec("update t1 set v=v+1") + s1.Step().CheckCurrentStop(sessiontxn.TestSessionOnStmtRetryAfterLockError) + s1.Step().CheckDone() + time.Sleep(time.Second) +} + // See issue: https://github.com/pingcap/tidb/issues/35459 func TestStillWriteConflictAfterRetry(t *testing.T) { store, _, deferFunc := setupTxnContextTest(t) diff --git a/testkit/testkit.go b/testkit/testkit.go index 912b5292dad10..69c960bfc486b 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/multithreadtest" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -434,3 +435,91 @@ func (c *RegionProperityClient) SendRequest(ctx context.Context, addr string, re } return c.Client.SendRequest(ctx, addr, req, timeout) } + +type SessionThreadTestKit struct { + *TestKit + ch *multithreadtest.Chan +} + +func (tk *SessionThreadTestKit) EnableSessionStopPoint() { + multithreadtest.EnableSessionStopPoint(tk.Session(), tk.ch) +} + +func (tk *SessionThreadTestKit) DisableSessionStopPoint() { + multithreadtest.DisableSessionStopPoint(tk.Session()) +} + +type SessionThread struct { + *multithreadtest.Thread + t *testing.T + store kv.Storage +} + +func (s *SessionThread) StopWhen(stops []string) *SessionThread { + s.Thread.StopWhen(stops) + return s +} + +func (s *SessionThread) StopWhenNot(stops []string) *SessionThread { + s.Thread.StopWhenNot(stops) + return s +} + +func (s *SessionThread) StopForAllPoints() *SessionThread { + s.Thread.StopForAllPoints() + return s +} + +func (s *SessionThread) Start(run func(t *testing.T, tk *SessionThreadTestKit)) *SessionThread { + s.Thread.Start(func(ch *multithreadtest.Chan, _ []any) []any { + tk := NewTestKit(s.t, s.store) + run(s.t, &SessionThreadTestKit{ + TestKit: tk, + ch: ch, + }) + return nil + }) + return s +} + +func (s *SessionThread) Step() *SessionThread { + s.Thread.Step() + return s +} + +func (s *SessionThread) StepUntilDone() *SessionThread { + s.Thread.StepUntilDone() + return s +} + +func (s *SessionThread) CheckCurrentStop(name string) *SessionThread { + s.Thread.CheckCurrentStop(name) + return s +} + +func (s *SessionThread) CheckDone() *SessionThread { + s.Thread.CheckDone() + return s +} + +type MultiSessionsRunner struct { + *multithreadtest.ThreadsRunner + t *testing.T + store kv.Storage +} + +func NewMultiSessionsRunner(t *testing.T, store kv.Storage) *MultiSessionsRunner { + return &MultiSessionsRunner{ + ThreadsRunner: multithreadtest.NewThreadsRunner(), + t: t, + store: store, + } +} + +func (r *MultiSessionsRunner) Session(name string) *SessionThread { + return &SessionThread{ + Thread: r.Thread(name), + t: r.t, + store: r.store, + } +} diff --git a/util/multithreadtest/runner.go b/util/multithreadtest/runner.go new file mode 100644 index 0000000000000..c2fc38dce3e62 --- /dev/null +++ b/util/multithreadtest/runner.go @@ -0,0 +1,261 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package multithreadtest + +import ( + "fmt" + "time" + + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" +) + +type Chan struct { + ch1 chan *StopPoint + ch2 chan any +} + +func (c *Chan) Stop(name string, value ...any) { + c.StopAtPoint(NewStopPoint(name, value)) +} + +func (c *Chan) StopAtPoint(point *StopPoint) { + c.ch1 <- point + <-c.ch2 +} + +func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan) { + sctx.SetValue(stringutil.StringerStr("multiThreadTestChain"), c) +} + +func DisableSessionStopPoint(sctx sessionctx.Context) { + sctx.SetValue(stringutil.StringerStr("multiThreadTestChain"), nil) +} + +func SessionStop(sctx sessionctx.Context, stopName string, value ...any) { + if ch, ok := sctx.Value(stringutil.StringerStr("multiThreadTestChain")).(*Chan); ok { + ch.Stop(stopName, value) + } +} + +type ThreadFunc func(ch *Chan, args []any) []any + +type StopPoint struct { + name string + value []any +} + +func NewStopPoint(name string, value ...any) *StopPoint { + return &StopPoint{ + name: name, + value: value, + } +} + +func (p *StopPoint) Name() string { + return p.name +} + +func (p *StopPoint) Value() []any { + return p.value +} + +func StartPoint(value []any) *StopPoint { + return &StopPoint{name: "START", value: value} +} + +func DonePoint(value []any) *StopPoint { + return &StopPoint{name: "DONE", value: value} +} + +type Thread struct { + name string + runner *ThreadsRunner + ch *Chan + run ThreadFunc + + currentStop *StopPoint + path []*StopPoint + whiteList bool + stopList map[string]struct{} +} + +func newThread(name string, runner *ThreadsRunner) *Thread { + return &Thread{ + name: name, + runner: runner, + } +} + +func (t *Thread) StopWhen(stops []string) *Thread { + t.setStopList(true, stops) + return t +} + +func (t *Thread) StopWhenNot(stops []string) *Thread { + t.setStopList(false, stops) + return t +} + +func (t *Thread) StopForAllPoints() *Thread { + t.setStopList(false, nil) + return t +} + +func (t *Thread) setStopList(whiteList bool, stops []string) { + list := make(map[string]struct{}) + for _, stop := range stops { + list[stop] = struct{}{} + } + + t.stopList = list + t.whiteList = whiteList +} + +func (t *Thread) Start(run ThreadFunc, args ...any) *Thread { + t.ch = &Chan{ + ch1: make(chan *StopPoint), + ch2: make(chan any), + } + t.run = run + + go func() { + t.ch.StopAtPoint(StartPoint(args)) + var value []any + defer func() { + t.ch.ch1 <- DonePoint(value) + }() + value = t.run(t.ch, args) + }() + t.runner.addThread(t) + t.waitNextStop() + t.record(t.currentStop) + return t +} + +func (t *Thread) CurrentStop() *StopPoint { + return t.currentStop +} + +func (t *Thread) Step() *Thread { + t.step() + t.record(t.currentStop) + return t +} + +func (t *Thread) StepUntilDone() *Thread { + if t.IsDone() { + return t + } + + for !t.IsDone() { + t.step() + } + t.record(t.currentStop) + return t +} + +func (t *Thread) CheckCurrentStop(name string) *Thread { + if t.currentStop.name != name { + panic(fmt.Sprintf("Expected current stop '%s', actual '%s'", name, t.currentStop.name)) + } + return t +} + +func (t *Thread) CheckDone() *Thread { + if !t.IsDone() { + panic(fmt.Sprintf("Expected current stop is done, actual '%s'", t.currentStop.name)) + } + return t +} + +func (t *Thread) IsDone() bool { + return t.currentStop.name == "DONE" +} + +func (t *Thread) step() { + for { + if t.stepOne(); t.IsDone() { + break + } + + if _, ok := t.stopList[t.currentStop.Name()]; t.whiteList == ok { + break + } + } +} + +func (t *Thread) record(stop *StopPoint) { + t.path = append(t.path, stop) + t.runner.recordPath(t.name, stop) +} + +func (t *Thread) stepOne() { + if t.IsDone() { + panic("cannot step a done thread") + } + t.ch.ch2 <- struct{}{} + t.waitNextStop() +} + +func (t *Thread) waitNextStop() { + select { + case stop := <-t.ch.ch1: + t.currentStop = stop + case <-time.After(time.Second * 10): + panic("timeout") + } +} + +type ThreadsRunner struct { + threads map[string]*Thread + path []struct { + thread *Thread + stop *StopPoint + } +} + +func NewThreadsRunner() *ThreadsRunner { + return &ThreadsRunner{ + threads: make(map[string]*Thread), + } +} + +func (r *ThreadsRunner) Thread(name string) *Thread { + if thread, ok := r.threads[name]; ok { + return thread + } + + return newThread(name, r) +} + +func (r *ThreadsRunner) addThread(thread *Thread) { + if _, ok := r.threads[thread.name]; ok { + panic(fmt.Sprintf("thread '%s' already exists", thread.name)) + } + r.threads[thread.name] = thread +} + +func (r *ThreadsRunner) recordPath(threadName string, stop *StopPoint) { + thread, ok := r.threads[threadName] + if !ok { + panic(fmt.Sprintf("thread '%s' not exists", threadName)) + } + + r.path = append(r.path, struct { + thread *Thread + stop *StopPoint + }{thread: thread, stop: stop}) +} diff --git a/util/multithreadtest/runner_test.go b/util/multithreadtest/runner_test.go new file mode 100644 index 0000000000000..30a43675c780f --- /dev/null +++ b/util/multithreadtest/runner_test.go @@ -0,0 +1,49 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package multithreadtest + +import ( + "fmt" + "testing" +) + +func TestRunner(t *testing.T) { + runner := NewThreadsRunner() + t1 := runner.Thread("s1").Start(func(ch *Chan, args []any) []any { + fmt.Println("s1.1") + ch.Stop("after s1.1") + fmt.Println("s1.2") + ch.Stop("after s1.2") + fmt.Println("s1.3") + return nil + }) + + t2 := runner.Thread("s2").Start(func(ch *Chan, args []any) []any { + fmt.Println("s2.1") + ch.Stop("after s2.1") + fmt.Println("s2.2") + ch.Stop("after s2.2") + fmt.Println("s2.3") + return nil + }) + + t1.Step() + t2.Step() + t1.Step() + t2.Step() + t1.Step() + t2.Step() + fmt.Println(runner.path) +} From 9ecee1646b21752479c668c11462a30f376ab555 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 10:50:51 +0800 Subject: [PATCH 04/21] update --- sessiontxn/txn_context_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index dc53f73088d6b..51cd26b367f7b 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -733,7 +733,6 @@ func TestAA(t *testing.T) { tk2.MustExec("update t1 set v=v+1") s1.Step().CheckCurrentStop(sessiontxn.TestSessionOnStmtRetryAfterLockError) s1.Step().CheckDone() - time.Sleep(time.Second) } // See issue: https://github.com/pingcap/tidb/issues/35459 From 7f042b0606c1540d0b7c9c37b50c9f677449404c Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 17:09:25 +0800 Subject: [PATCH 05/21] update --- executor/adapter.go | 13 +- sessiontxn/failpoint.go | 12 +- sessiontxn/txn_context_test.go | 161 +++++------------ testkit/taskstop.go | 203 ++++++++++++++++++++++ testkit/testkit.go | 89 ---------- util/multithreadtest/runner.go | 261 ---------------------------- util/multithreadtest/runner_test.go | 49 ------ util/taskstop/stop.go | 113 ++++++++++++ 8 files changed, 366 insertions(+), 535 deletions(-) create mode 100644 testkit/taskstop.go delete mode 100644 util/multithreadtest/runner.go delete mode 100644 util/multithreadtest/runner_test.go create mode 100644 util/taskstop/stop.go diff --git a/executor/adapter.go b/executor/adapter.go index 6333efe864a54..f69dfd4c94170 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -24,6 +24,8 @@ import ( "sync/atomic" "time" + "github.com/pingcap/tidb/util/taskstop" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -55,7 +57,6 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tidb/util/multithreadtest" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" @@ -418,10 +419,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) - failpoint.Inject("sessionStop", func() { - multithreadtest.SessionStop(a.Ctx, sessiontxn.TestSessionStopBeforeExecutorFirstRun) - }) - + taskstop.InjectSessionStopPoint(a.Ctx, sessiontxn.StopPointBeforeExecutorFirstRun) if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -798,10 +796,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if err != nil { return nil, err } - - failpoint.Inject("sessionStop", func() { - multithreadtest.SessionStop(a.Ctx, sessiontxn.TestSessionOnStmtRetryAfterLockError) - }) + taskstop.InjectSessionStopPoint(a.Ctx, sessiontxn.StopPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() if err != nil { diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index d34a77ccc131e..fa74bea426118 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -35,17 +35,13 @@ var AssertTxnInfoSchemaKey stringutil.StringerStr = "assertTxnInfoSchemaKey" // Only for test var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSchemaAfterRetryKey" -// HookBeforeFirstRunExecutorKey is the hook key for the executor first run +// StopPointBeforeExecutorFirstRun is the key for the stop point where session stops before executor's first run // Only for test -var HookBeforeFirstRunExecutorKey stringutil.StringerStr = "testHookKeyBeforeFirstRunExecutor" +var StopPointBeforeExecutorFirstRun = "before executor first run" -// HookAfterOnStmtRetryWithLockErrorKey is the hook key for after OnStmtRetry with lock error +// StopPointOnStmtRetryAfterLockError s the key for the stop point where session stops after OnStmtRetry when lock error happens // Only for test -var HookAfterOnStmtRetryWithLockErrorKey stringutil.StringerStr = "testHookKeyAfterOnStmtRetryWithLockError" - -var TestSessionStopBeforeExecutorFirstRun = "before executor first run" - -var TestSessionOnStmtRetryAfterLockError = "lock error, OnStmtRetry called" +var StopPointOnStmtRetryAfterLockError = "lock error, OnStmtRetry called" // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 51cd26b367f7b..5fa4b4be670cd 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -17,7 +17,6 @@ package sessiontxn_test import ( "context" "fmt" - "strings" "testing" "time" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testsetup" + "github.com/pingcap/tidb/util/taskstop" "github.com/stretchr/testify/require" "go.uber.org/goleak" ) @@ -52,12 +52,12 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/sessionStop", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan", "return")) + require.NoError(t, taskstop.EnableGlobalSessionStopFailPoint()) store, do, clean := testkit.CreateMockStoreAndDomain(t) @@ -84,12 +84,12 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/sessionStop")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan")) + require.NoError(t, taskstop.DisableGlobalSessionStopFailPoint()) tk.Session().SetValue(sessiontxn.AssertRecordsKey, nil) tk.Session().SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) @@ -709,37 +709,12 @@ func TestTxnContextPreparedStmtWithForUpdate(t *testing.T) { tk.MustExec("rollback") } -func TestAA(t *testing.T) { - store, _, deferFunc := setupTxnContextTest(t) - defer deferFunc() - - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - - stops := []string{sessiontxn.TestSessionStopBeforeExecutorFirstRun, sessiontxn.TestSessionOnStmtRetryAfterLockError} - runner := testkit.NewMultiSessionsRunner(t, store) - s1 := runner.Session("s1").StopWhen(stops).Start(func(t *testing.T, tk *testkit.SessionThreadTestKit) { - tk.MustExec("use test") - tk.MustExec("begin pessimistic") - tk.EnableSessionStopPoint() - tk.MustQuery("select * from t1 for update").Check(testkit.Rows("1 12")) - tk.DisableSessionStopPoint() - tk.MustExec("rollback") - }) - - s1.Step().CheckCurrentStop(sessiontxn.TestSessionStopBeforeExecutorFirstRun) - tk2.MustExec("update t1 set v=v+1") - s1.Step().CheckCurrentStop(sessiontxn.TestSessionOnStmtRetryAfterLockError) - tk2.MustExec("update t1 set v=v+1") - s1.Step().CheckCurrentStop(sessiontxn.TestSessionOnStmtRetryAfterLockError) - s1.Step().CheckDone() -} - // See issue: https://github.com/pingcap/tidb/issues/35459 func TestStillWriteConflictAfterRetry(t *testing.T) { store, _, deferFunc := setupTxnContextTest(t) defer deferFunc() + require.True(t, taskstop.IsGlobalSessionStopFailPointEnabled(), "should enable global session stop failPoint") queries := []string{ "select * from t1 for update", "select * from t1 where id=1 for update", @@ -753,102 +728,50 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { "update t1 set v=v+1 where id in (1, 2, 3) and v>0", } + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { for _, query := range queries { for _, autocommit := range []bool{true, false} { t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { - testStillWriteConflictAfterRetry(t, store, isolation, query, autocommit) + tk.MustExec("rollback") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + + stops := []string{sessiontxn.StopPointBeforeExecutorFirstRun, sessiontxn.StopPointOnStmtRetryAfterLockError} + runner := testkit.NewStoppableTasksRunner(t) + s2 := runner.Task("s2").StopWhen(stops).Start(func(ch *taskstop.Chan) { + tk2 := testkit.NewTestKit(t, store) + defer func() { + taskstop.DisableSessionStopPoint(tk2.Session()) + tk2.MustExec("rollback") + }() + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) + if autocommit { + tk2.MustExec("set autocommit=1") + tk2.MustExec("begin") + } else { + tk2.MustExec("set autocommit=0") + } + + taskstop.EnableSessionStopPoint(tk2.Session(), ch) + tk2.MustQuery("select * from t1 for update").Check(testkit.Rows("1 12")) + taskstop.DisableSessionStopPoint(tk2.Session()) + }) + + // Stop the s2 before the executor first run and then update the record in another session + s2.Step().CheckStopAt(sessiontxn.StopPointBeforeExecutorFirstRun) + tk.MustExec("update t1 set v=v+1") + // Then the s2 will get a lock error and retry + s2.Step().CheckStopAt(sessiontxn.StopPointOnStmtRetryAfterLockError) + // At this time if another session updates the record again, the s2 should also lock failed + tk.MustExec("update t1 set v=v+1") + s2.Step().CheckStopAt(sessiontxn.StopPointOnStmtRetryAfterLockError) + s2.Step().CheckDone() }) } } } } - -func testStillWriteConflictAfterRetry(t *testing.T, store kv.Storage, isolation string, query string, autocommit bool) { - tk := testkit.NewTestKit(t, store) - defer tk.MustExec("rollback") - - tk.MustExec("use test") - tk.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) - tk.MustExec("set autocommit=1") - tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - - se := tk.Session() - chanBeforeRunStmt := make(chan func(), 1) - chanAfterOnStmtRetry := make(chan func(), 1) - c2 := make(chan string, 1) - c3 := make(chan string, 1) - wait := func(ch chan string, expect string) { - select { - case got := <-ch: - if got != expect { - panic(fmt.Sprintf("expect '%s', got '%s'", expect, got)) - } - case <-time.After(time.Second * 10): - panic("wait2 timeout") - } - } - - if autocommit { - tk.MustExec("begin") - } else { - tk.MustExec("set @@autocommit=0") - } - - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, chanBeforeRunStmt) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, chanAfterOnStmtRetry) - defer func() { - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) - }() - - chanBeforeRunStmt <- func() { - c2 <- "now before session1 runStmt" - wait(c3, "session2 updated v=v+1 done") - } - - chanAfterOnStmtRetry <- func() { - c2 <- "now after OnStmtRetry before rebuild executor" - wait(c3, "session2 updated v=v+1 again done") - } - - go func() { - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - - // first conflict - wait(c2, "now before session1 runStmt") - tk2.MustExec("update t1 set v=v+1 where id=1") - c3 <- "session2 updated v=v+1 done" - - // second conflict - wait(c2, "now after OnStmtRetry before rebuild executor") - tk2.MustExec("update t1 set v=v+1 where id=1") - c3 <- "session2 updated v=v+1 again done" - chanAfterOnStmtRetry <- func() {} - c3 <- "done" - }() - - isSelect := false - if strings.HasPrefix(query, "update ") { - tk.MustExec(query) - } else if strings.HasPrefix(query, "select ") { - isSelect = true - tk.MustQuery(query).Check(testkit.Rows("1 12")) - } else { - require.FailNowf(t, "invalid query: %s", query) - } - - wait(c3, "done") - - se.SetValue(sessiontxn.HookBeforeFirstRunExecutorKey, nil) - se.SetValue(sessiontxn.HookAfterOnStmtRetryWithLockErrorKey, nil) - if isSelect { - tk.MustExec("update t1 set v=v+1") - } - tk.MustExec("commit") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - tk.MustExec("rollback") -} diff --git a/testkit/taskstop.go b/testkit/taskstop.go new file mode 100644 index 0000000000000..5c1760fc69a49 --- /dev/null +++ b/testkit/taskstop.go @@ -0,0 +1,203 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testkit + +import ( + "testing" + "time" + + "github.com/pingcap/tidb/util/taskstop" + "github.com/stretchr/testify/require" +) + +type TaskFunc func(ch *taskstop.Chan) +type SessionTaskFunc func(ch *taskstop.Chan, tk *TestKit) + +type StoppableTask struct { + t *testing.T + name string + runner *StoppableTasksRunner + ch *taskstop.Chan + runFunc TaskFunc + + currentStop *taskstop.StopPoint + path []*taskstop.StopPoint + whiteList bool + stopList map[string]struct{} +} + +func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask { + return &StoppableTask{ + t: runner.t, + name: name, + runner: runner, + } +} + +func (t *StoppableTask) StopWhen(stops []string) *StoppableTask { + t.setStopList(true, stops) + return t +} + +func (t *StoppableTask) StopWhenNot(stops []string) *StoppableTask { + t.setStopList(false, stops) + return t +} + +func (t *StoppableTask) StopEveryPoint() *StoppableTask { + t.setStopList(false, nil) + return t +} + +func (t *StoppableTask) setStopList(whiteList bool, stops []string) { + list := make(map[string]struct{}) + for _, stopName := range stops { + list[stopName] = struct{}{} + } + + t.stopList = list + t.whiteList = whiteList +} + +func (t *StoppableTask) Start(fn TaskFunc) *StoppableTask { + t.ch = taskstop.NewChan() + t.runFunc = fn + + go func() { + require.NoError(t.t, t.ch.SignalOnStopAt("START")) + <-t.ch.WaitStepSignal() + defer func() { + require.NoError(t.t, t.ch.SignalOnStopAt("DONE")) + }() + t.runFunc(t.ch) + }() + t.runner.addTask(t) + t.waitNextStop() + t.record(t.currentStop) + return t +} + +func (t *StoppableTask) CurrentStop() *taskstop.StopPoint { + return t.currentStop +} + +func (t *StoppableTask) Step() *StoppableTask { + t.step() + t.record(t.currentStop) + return t +} + +func (t *StoppableTask) StepUntilDone() *StoppableTask { + if t.IsDone() { + return t + } + + for !t.IsDone() { + t.step() + } + t.record(t.currentStop) + return t +} + +func (t *StoppableTask) CheckStopAt(name string) *StoppableTask { + require.Equal(t.t, name, t.currentStop.Name()) + return t +} + +func (t *StoppableTask) CheckDone() *StoppableTask { + require.Truef(t.t, t.IsDone(), "current stop: '%s'", t.currentStop.Name()) + return t +} + +func (t *StoppableTask) IsDone() bool { + return t.currentStop.Name() == "DONE" +} + +func (t *StoppableTask) step() { + for { + if t.stepOne(); t.IsDone() { + break + } + + if _, ok := t.stopList[t.currentStop.Name()]; t.whiteList == ok { + break + } + } +} + +func (t *StoppableTask) record(stop *taskstop.StopPoint) { + t.path = append(t.path, stop) + t.runner.recordPath(t.name, stop) +} + +func (t *StoppableTask) stepOne() { + if t.IsDone() { + panic("cannot step a done thread") + } + require.NoError(t.t, t.ch.SignalStep()) + t.waitNextStop() +} + +func (t *StoppableTask) waitNextStop() { + select { + case stopPoint := <-t.ch.WaitOnStop(): + t.currentStop = stopPoint + case <-time.After(time.Second * 10): + t.t.Fatal("timeout") + } +} + +type StoppableTasksRunner struct { + t *testing.T + tasks map[string]*StoppableTask + path []struct { + task *StoppableTask + stop *taskstop.StopPoint + } +} + +func NewStoppableTasksRunner(t *testing.T) *StoppableTasksRunner { + return &StoppableTasksRunner{ + t: t, + tasks: make(map[string]*StoppableTask), + } +} + +func (r *StoppableTasksRunner) Task(name string) *StoppableTask { + if thread, ok := r.tasks[name]; ok { + return thread + } + + return newStoppableTask(name, r) +} + +func (r *StoppableTasksRunner) addTask(thread *StoppableTask) { + if _, ok := r.tasks[thread.name]; ok { + r.t.Fatalf("thread '%s' already exists", thread.name) + } + r.tasks[thread.name] = thread +} + +func (r *StoppableTasksRunner) recordPath(threadName string, stop *taskstop.StopPoint) { + task, ok := r.tasks[threadName] + if !ok { + r.t.Fatalf("thread '%s' not exist", task.name) + } + + r.path = append(r.path, struct { + task *StoppableTask + stop *taskstop.StopPoint + }{task: task, stop: stop}) +} diff --git a/testkit/testkit.go b/testkit/testkit.go index 69c960bfc486b..912b5292dad10 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/multithreadtest" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -435,91 +434,3 @@ func (c *RegionProperityClient) SendRequest(ctx context.Context, addr string, re } return c.Client.SendRequest(ctx, addr, req, timeout) } - -type SessionThreadTestKit struct { - *TestKit - ch *multithreadtest.Chan -} - -func (tk *SessionThreadTestKit) EnableSessionStopPoint() { - multithreadtest.EnableSessionStopPoint(tk.Session(), tk.ch) -} - -func (tk *SessionThreadTestKit) DisableSessionStopPoint() { - multithreadtest.DisableSessionStopPoint(tk.Session()) -} - -type SessionThread struct { - *multithreadtest.Thread - t *testing.T - store kv.Storage -} - -func (s *SessionThread) StopWhen(stops []string) *SessionThread { - s.Thread.StopWhen(stops) - return s -} - -func (s *SessionThread) StopWhenNot(stops []string) *SessionThread { - s.Thread.StopWhenNot(stops) - return s -} - -func (s *SessionThread) StopForAllPoints() *SessionThread { - s.Thread.StopForAllPoints() - return s -} - -func (s *SessionThread) Start(run func(t *testing.T, tk *SessionThreadTestKit)) *SessionThread { - s.Thread.Start(func(ch *multithreadtest.Chan, _ []any) []any { - tk := NewTestKit(s.t, s.store) - run(s.t, &SessionThreadTestKit{ - TestKit: tk, - ch: ch, - }) - return nil - }) - return s -} - -func (s *SessionThread) Step() *SessionThread { - s.Thread.Step() - return s -} - -func (s *SessionThread) StepUntilDone() *SessionThread { - s.Thread.StepUntilDone() - return s -} - -func (s *SessionThread) CheckCurrentStop(name string) *SessionThread { - s.Thread.CheckCurrentStop(name) - return s -} - -func (s *SessionThread) CheckDone() *SessionThread { - s.Thread.CheckDone() - return s -} - -type MultiSessionsRunner struct { - *multithreadtest.ThreadsRunner - t *testing.T - store kv.Storage -} - -func NewMultiSessionsRunner(t *testing.T, store kv.Storage) *MultiSessionsRunner { - return &MultiSessionsRunner{ - ThreadsRunner: multithreadtest.NewThreadsRunner(), - t: t, - store: store, - } -} - -func (r *MultiSessionsRunner) Session(name string) *SessionThread { - return &SessionThread{ - Thread: r.Thread(name), - t: r.t, - store: r.store, - } -} diff --git a/util/multithreadtest/runner.go b/util/multithreadtest/runner.go deleted file mode 100644 index c2fc38dce3e62..0000000000000 --- a/util/multithreadtest/runner.go +++ /dev/null @@ -1,261 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package multithreadtest - -import ( - "fmt" - "time" - - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/util/stringutil" -) - -type Chan struct { - ch1 chan *StopPoint - ch2 chan any -} - -func (c *Chan) Stop(name string, value ...any) { - c.StopAtPoint(NewStopPoint(name, value)) -} - -func (c *Chan) StopAtPoint(point *StopPoint) { - c.ch1 <- point - <-c.ch2 -} - -func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan) { - sctx.SetValue(stringutil.StringerStr("multiThreadTestChain"), c) -} - -func DisableSessionStopPoint(sctx sessionctx.Context) { - sctx.SetValue(stringutil.StringerStr("multiThreadTestChain"), nil) -} - -func SessionStop(sctx sessionctx.Context, stopName string, value ...any) { - if ch, ok := sctx.Value(stringutil.StringerStr("multiThreadTestChain")).(*Chan); ok { - ch.Stop(stopName, value) - } -} - -type ThreadFunc func(ch *Chan, args []any) []any - -type StopPoint struct { - name string - value []any -} - -func NewStopPoint(name string, value ...any) *StopPoint { - return &StopPoint{ - name: name, - value: value, - } -} - -func (p *StopPoint) Name() string { - return p.name -} - -func (p *StopPoint) Value() []any { - return p.value -} - -func StartPoint(value []any) *StopPoint { - return &StopPoint{name: "START", value: value} -} - -func DonePoint(value []any) *StopPoint { - return &StopPoint{name: "DONE", value: value} -} - -type Thread struct { - name string - runner *ThreadsRunner - ch *Chan - run ThreadFunc - - currentStop *StopPoint - path []*StopPoint - whiteList bool - stopList map[string]struct{} -} - -func newThread(name string, runner *ThreadsRunner) *Thread { - return &Thread{ - name: name, - runner: runner, - } -} - -func (t *Thread) StopWhen(stops []string) *Thread { - t.setStopList(true, stops) - return t -} - -func (t *Thread) StopWhenNot(stops []string) *Thread { - t.setStopList(false, stops) - return t -} - -func (t *Thread) StopForAllPoints() *Thread { - t.setStopList(false, nil) - return t -} - -func (t *Thread) setStopList(whiteList bool, stops []string) { - list := make(map[string]struct{}) - for _, stop := range stops { - list[stop] = struct{}{} - } - - t.stopList = list - t.whiteList = whiteList -} - -func (t *Thread) Start(run ThreadFunc, args ...any) *Thread { - t.ch = &Chan{ - ch1: make(chan *StopPoint), - ch2: make(chan any), - } - t.run = run - - go func() { - t.ch.StopAtPoint(StartPoint(args)) - var value []any - defer func() { - t.ch.ch1 <- DonePoint(value) - }() - value = t.run(t.ch, args) - }() - t.runner.addThread(t) - t.waitNextStop() - t.record(t.currentStop) - return t -} - -func (t *Thread) CurrentStop() *StopPoint { - return t.currentStop -} - -func (t *Thread) Step() *Thread { - t.step() - t.record(t.currentStop) - return t -} - -func (t *Thread) StepUntilDone() *Thread { - if t.IsDone() { - return t - } - - for !t.IsDone() { - t.step() - } - t.record(t.currentStop) - return t -} - -func (t *Thread) CheckCurrentStop(name string) *Thread { - if t.currentStop.name != name { - panic(fmt.Sprintf("Expected current stop '%s', actual '%s'", name, t.currentStop.name)) - } - return t -} - -func (t *Thread) CheckDone() *Thread { - if !t.IsDone() { - panic(fmt.Sprintf("Expected current stop is done, actual '%s'", t.currentStop.name)) - } - return t -} - -func (t *Thread) IsDone() bool { - return t.currentStop.name == "DONE" -} - -func (t *Thread) step() { - for { - if t.stepOne(); t.IsDone() { - break - } - - if _, ok := t.stopList[t.currentStop.Name()]; t.whiteList == ok { - break - } - } -} - -func (t *Thread) record(stop *StopPoint) { - t.path = append(t.path, stop) - t.runner.recordPath(t.name, stop) -} - -func (t *Thread) stepOne() { - if t.IsDone() { - panic("cannot step a done thread") - } - t.ch.ch2 <- struct{}{} - t.waitNextStop() -} - -func (t *Thread) waitNextStop() { - select { - case stop := <-t.ch.ch1: - t.currentStop = stop - case <-time.After(time.Second * 10): - panic("timeout") - } -} - -type ThreadsRunner struct { - threads map[string]*Thread - path []struct { - thread *Thread - stop *StopPoint - } -} - -func NewThreadsRunner() *ThreadsRunner { - return &ThreadsRunner{ - threads: make(map[string]*Thread), - } -} - -func (r *ThreadsRunner) Thread(name string) *Thread { - if thread, ok := r.threads[name]; ok { - return thread - } - - return newThread(name, r) -} - -func (r *ThreadsRunner) addThread(thread *Thread) { - if _, ok := r.threads[thread.name]; ok { - panic(fmt.Sprintf("thread '%s' already exists", thread.name)) - } - r.threads[thread.name] = thread -} - -func (r *ThreadsRunner) recordPath(threadName string, stop *StopPoint) { - thread, ok := r.threads[threadName] - if !ok { - panic(fmt.Sprintf("thread '%s' not exists", threadName)) - } - - r.path = append(r.path, struct { - thread *Thread - stop *StopPoint - }{thread: thread, stop: stop}) -} diff --git a/util/multithreadtest/runner_test.go b/util/multithreadtest/runner_test.go deleted file mode 100644 index 30a43675c780f..0000000000000 --- a/util/multithreadtest/runner_test.go +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package multithreadtest - -import ( - "fmt" - "testing" -) - -func TestRunner(t *testing.T) { - runner := NewThreadsRunner() - t1 := runner.Thread("s1").Start(func(ch *Chan, args []any) []any { - fmt.Println("s1.1") - ch.Stop("after s1.1") - fmt.Println("s1.2") - ch.Stop("after s1.2") - fmt.Println("s1.3") - return nil - }) - - t2 := runner.Thread("s2").Start(func(ch *Chan, args []any) []any { - fmt.Println("s2.1") - ch.Stop("after s2.1") - fmt.Println("s2.2") - ch.Stop("after s2.2") - fmt.Println("s2.3") - return nil - }) - - t1.Step() - t2.Step() - t1.Step() - t2.Step() - t1.Step() - t2.Step() - fmt.Println(runner.path) -} diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go new file mode 100644 index 0000000000000..b78060785e3b3 --- /dev/null +++ b/util/taskstop/stop.go @@ -0,0 +1,113 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package taskstop + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" +) + +type StopPoint struct { + name string + value []any +} + +func NewStopPoint(name string, value ...any) *StopPoint { + return &StopPoint{ + name: name, + value: value, + } +} + +func (p *StopPoint) Name() string { + return p.name +} + +func (p *StopPoint) Value() []any { + return p.value +} + +type Chan struct { + ch1 chan *StopPoint + ch2 chan any +} + +func NewChan() *Chan { + return &Chan{ + ch1: make(chan *StopPoint), + ch2: make(chan any), + } +} + +func (ch *Chan) SignalOnStopAt(stopName string) error { + select { + case ch.ch1 <- NewStopPoint(stopName): + return nil + default: + return errors.New("Cannot signal stop at") + } + +} + +func (ch *Chan) WaitOnStop() chan *StopPoint { + return ch.ch1 +} + +func (ch *Chan) SignalStep() error { + select { + case ch.ch2 <- struct{}{}: + return nil + default: + return errors.New("Cannot signal step") + } +} + +func (ch *Chan) WaitStepSignal() chan any { + return ch.ch2 +} + +func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan) { + sctx.SetValue(stringutil.StringerStr("sessionStopChain"), c) +} + +func DisableSessionStopPoint(sctx sessionctx.Context) { + sctx.SetValue(stringutil.StringerStr("sessionStopChain"), nil) +} + +func InjectSessionStopPoint(sctx sessionctx.Context, stopName string) { + failpoint.Inject("sessionStop", func() { + if ch, ok := sctx.Value(stringutil.StringerStr("sessionStopChain")).(*Chan); ok { + if err := ch.SignalOnStopAt(stopName); err != nil { + panic(err) + } + <-ch.WaitStepSignal() + } + }) +} + +func EnableGlobalSessionStopFailPoint() error { + return failpoint.Enable("github.com/pingcap/tidb/util/taskstop/sessionStop", "return") +} + +func DisableGlobalSessionStopFailPoint() error { + return failpoint.Disable("github.com/pingcap/tidb/util/taskstop/sessionStop") +} + +func IsGlobalSessionStopFailPointEnabled() bool { + status, err := failpoint.Status("github.com/pingcap/tidb/util/taskstop/sessionStop") + return err == nil && status == "return" +} From bac48a19bb94ffd01c3ad005cbd80f548b2941f3 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 18:10:26 +0800 Subject: [PATCH 06/21] update --- executor/adapter.go | 3 +- sessiontxn/txn_context_test.go | 99 ++++++++++++++++++---------------- testkit/taskstop.go | 33 ++++++------ 3 files changed, 72 insertions(+), 63 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index f69dfd4c94170..5b601c1d2d5c2 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -24,8 +24,6 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/util/taskstop" - "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -61,6 +59,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/taskstop" "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" tikverr "github.com/tikv/client-go/v2/error" diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 5fa4b4be670cd..7076e33e74256 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -717,59 +717,68 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { require.True(t, taskstop.IsGlobalSessionStopFailPointEnabled(), "should enable global session stop failPoint") queries := []string{ "select * from t1 for update", - "select * from t1 where id=1 for update", - "select * from t1 where id in (1, 2, 3) for update", - "select * from t1 where id=1 and v>0 for update", - "select * from t1 where id=1 for update union select * from t1 where id=1 for update", - "update t1 set v=v+1", - "update t1 set v=v+1 where id=1", - "update t1 set v=v+1 where id=1 and v>0", - "update t1 set v=v+1 where id in (1, 2, 3)", - "update t1 set v=v+1 where id in (1, 2, 3) and v>0", + //"select * from t1 where id=1 for update", + //"select * from t1 where id in (1, 2, 3) for update", + //"select * from t1 where id=1 and v>0 for update", + //"select * from t1 where id=1 for update union select * from t1 where id=1 for update", + //"update t1 set v=v+1", + //"update t1 set v=v+1 where id=1", + //"update t1 set v=v+1 where id=1 and v>0", + //"update t1 set v=v+1 where id in (1, 2, 3)", + //"update t1 set v=v+1 where id in (1, 2, 3) and v>0", } tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { + prepareTask := func(t *testing.T, isolation string, autocommit bool, sql string) *testkit.StoppableTask { + tk.MustExec("rollback") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + task := testkit.NewStoppableTasksRunner(t).Task("s2").StopWhen( + sessiontxn.StopPointBeforeExecutorFirstRun, + sessiontxn.StopPointOnStmtRetryAfterLockError, + ) + + return task.Create(func(ch *taskstop.Chan) { + tk2 := testkit.NewTestKit(t, store) + defer func() { + taskstop.DisableSessionStopPoint(tk2.Session()) + tk2.MustExec("rollback") + }() + + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) + if autocommit { + tk2.MustExec("set autocommit=1") + tk2.MustExec("begin") + } else { + tk2.MustExec("set autocommit=0") + } + taskstop.EnableSessionStopPoint(tk2.Session(), ch) + tk2.MustQuery(sql).Check(testkit.Rows("1 12")) + taskstop.DisableSessionStopPoint(tk2.Session()) + }) + } + + for _, isolation := range []string{ast.RepeatableRead} { for _, query := range queries { - for _, autocommit := range []bool{true, false} { + for _, autocommit := range []bool{true} { t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { - tk.MustExec("rollback") - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - - stops := []string{sessiontxn.StopPointBeforeExecutorFirstRun, sessiontxn.StopPointOnStmtRetryAfterLockError} - runner := testkit.NewStoppableTasksRunner(t) - s2 := runner.Task("s2").StopWhen(stops).Start(func(ch *taskstop.Chan) { - tk2 := testkit.NewTestKit(t, store) - defer func() { - taskstop.DisableSessionStopPoint(tk2.Session()) - tk2.MustExec("rollback") - }() - tk2.MustExec("use test") - tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") - tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) - if autocommit { - tk2.MustExec("set autocommit=1") - tk2.MustExec("begin") - } else { - tk2.MustExec("set autocommit=0") - } - - taskstop.EnableSessionStopPoint(tk2.Session(), ch) - tk2.MustQuery("select * from t1 for update").Check(testkit.Rows("1 12")) - taskstop.DisableSessionStopPoint(tk2.Session()) - }) - - // Stop the s2 before the executor first run and then update the record in another session - s2.Step().CheckStopAt(sessiontxn.StopPointBeforeExecutorFirstRun) + task := prepareTask(t, isolation, autocommit, query) + + // Pause the session before the executor first run and then update the record in another session + task.Start().ExpectStoppedAt(sessiontxn.StopPointBeforeExecutorFirstRun) tk.MustExec("update t1 set v=v+1") - // Then the s2 will get a lock error and retry - s2.Step().CheckStopAt(sessiontxn.StopPointOnStmtRetryAfterLockError) - // At this time if another session updates the record again, the s2 should also lock failed + + // Session continues, it should get a lock error and retry, we pause the session before the executor's next run + // and then update the record in another session again. + task.Continue().ExpectStoppedAt(sessiontxn.StopPointOnStmtRetryAfterLockError) tk.MustExec("update t1 set v=v+1") - s2.Step().CheckStopAt(sessiontxn.StopPointOnStmtRetryAfterLockError) - s2.Step().CheckDone() + + // Because the record is updated by another session again, when this session continues, it will get a lock error again. + task.Continue().ExpectStoppedAt(sessiontxn.StopPointOnStmtRetryAfterLockError) + task.Continue().ExpectDone() }) } } diff --git a/testkit/taskstop.go b/testkit/taskstop.go index 5c1760fc69a49..420c6c6b84d52 100644 --- a/testkit/taskstop.go +++ b/testkit/taskstop.go @@ -46,12 +46,12 @@ func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask } } -func (t *StoppableTask) StopWhen(stops []string) *StoppableTask { +func (t *StoppableTask) StopWhen(stops ...string) *StoppableTask { t.setStopList(true, stops) return t } -func (t *StoppableTask) StopWhenNot(stops []string) *StoppableTask { +func (t *StoppableTask) StopWhenNot(stops ...string) *StoppableTask { t.setStopList(false, stops) return t } @@ -71,7 +71,7 @@ func (t *StoppableTask) setStopList(whiteList bool, stops []string) { t.whiteList = whiteList } -func (t *StoppableTask) Start(fn TaskFunc) *StoppableTask { +func (t *StoppableTask) Create(fn TaskFunc) *StoppableTask { t.ch = taskstop.NewChan() t.runFunc = fn @@ -93,34 +93,35 @@ func (t *StoppableTask) CurrentStop() *taskstop.StopPoint { return t.currentStop } -func (t *StoppableTask) Step() *StoppableTask { +func (t *StoppableTask) Start() *StoppableTask { + return t.ExpectWaitingStart().Continue() +} + +func (t *StoppableTask) Continue() *StoppableTask { t.step() t.record(t.currentStop) return t } -func (t *StoppableTask) StepUntilDone() *StoppableTask { - if t.IsDone() { - return t - } - - for !t.IsDone() { - t.step() - } - t.record(t.currentStop) +func (t *StoppableTask) ExpectStoppedAt(name string) *StoppableTask { + require.Equal(t.t, name, t.currentStop.Name()) return t } -func (t *StoppableTask) CheckStopAt(name string) *StoppableTask { - require.Equal(t.t, name, t.currentStop.Name()) +func (t *StoppableTask) ExpectWaitingStart() *StoppableTask { + require.Truef(t.t, t.IsWaitingStart(), "current stop: '%s'", t.currentStop.Name()) return t } -func (t *StoppableTask) CheckDone() *StoppableTask { +func (t *StoppableTask) ExpectDone() *StoppableTask { require.Truef(t.t, t.IsDone(), "current stop: '%s'", t.currentStop.Name()) return t } +func (t *StoppableTask) IsWaitingStart() bool { + return t.currentStop.Name() == "START" +} + func (t *StoppableTask) IsDone() bool { return t.currentStop.Name() == "DONE" } From 7544019cf9615d4662072fe6905c4f138c99efa2 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 20:17:54 +0800 Subject: [PATCH 07/21] update --- sessiontxn/txn_context_test.go | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 7076e33e74256..e325e7611446c 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -17,6 +17,7 @@ package sessiontxn_test import ( "context" "fmt" + "strings" "testing" "time" @@ -717,15 +718,15 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { require.True(t, taskstop.IsGlobalSessionStopFailPointEnabled(), "should enable global session stop failPoint") queries := []string{ "select * from t1 for update", - //"select * from t1 where id=1 for update", - //"select * from t1 where id in (1, 2, 3) for update", - //"select * from t1 where id=1 and v>0 for update", - //"select * from t1 where id=1 for update union select * from t1 where id=1 for update", - //"update t1 set v=v+1", - //"update t1 set v=v+1 where id=1", - //"update t1 set v=v+1 where id=1 and v>0", - //"update t1 set v=v+1 where id in (1, 2, 3)", - //"update t1 set v=v+1 where id in (1, 2, 3) and v>0", + "select * from t1 where id=1 for update", + "select * from t1 where id in (1, 2, 3) for update", + "select * from t1 where id=1 and v>0 for update", + "select * from t1 where id=1 for update union select * from t1 where id=1 for update", + "update t1 set v=v+1", + "update t1 set v=v+1 where id=1", + "update t1 set v=v+1 where id=1 and v>0", + "update t1 set v=v+1 where id in (1, 2, 3)", + "update t1 set v=v+1 where id in (1, 2, 3) and v>0", } tk := testkit.NewTestKit(t, store) @@ -756,14 +757,21 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk2.MustExec("set autocommit=0") } taskstop.EnableSessionStopPoint(tk2.Session(), ch) - tk2.MustQuery(sql).Check(testkit.Rows("1 12")) - taskstop.DisableSessionStopPoint(tk2.Session()) + if strings.HasPrefix(sql, "update") { + tk2.MustExec(sql) + taskstop.DisableSessionStopPoint(tk2.Session()) + tk2.MustExec("commit") + tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) + } else { + tk2.MustQuery(sql).Check(testkit.Rows("1 12")) + taskstop.DisableSessionStopPoint(tk2.Session()) + } }) } - for _, isolation := range []string{ast.RepeatableRead} { + for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { for _, query := range queries { - for _, autocommit := range []bool{true} { + for _, autocommit := range []bool{true, false} { t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { task := prepareTask(t, isolation, autocommit, query) From 1bbef30144e91405ba981db51328d465c7b21432 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 20:45:02 +0800 Subject: [PATCH 08/21] update --- sessiontxn/txn_context_test.go | 14 +++++------ testkit/taskstop.go | 43 ++-------------------------------- util/taskstop/stop.go | 35 ++++++++++++++++++++++----- 3 files changed, 38 insertions(+), 54 deletions(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index e325e7611446c..7c49e0262d93d 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -735,12 +735,7 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk.MustExec("rollback") tk.MustExec("truncate table t1") tk.MustExec("insert into t1 values(1, 10)") - task := testkit.NewStoppableTasksRunner(t).Task("s2").StopWhen( - sessiontxn.StopPointBeforeExecutorFirstRun, - sessiontxn.StopPointOnStmtRetryAfterLockError, - ) - - return task.Create(func(ch *taskstop.Chan) { + return testkit.NewStoppableTasksRunner(t).Task("s2").Create(func(ch *taskstop.Chan) { tk2 := testkit.NewTestKit(t, store) defer func() { taskstop.DisableSessionStopPoint(tk2.Session()) @@ -756,7 +751,12 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { } else { tk2.MustExec("set autocommit=0") } - taskstop.EnableSessionStopPoint(tk2.Session(), ch) + taskstop.EnableSessionStopPoint( + tk2.Session(), + ch, + sessiontxn.StopPointBeforeExecutorFirstRun, + sessiontxn.StopPointOnStmtRetryAfterLockError, + ) if strings.HasPrefix(sql, "update") { tk2.MustExec(sql) taskstop.DisableSessionStopPoint(tk2.Session()) diff --git a/testkit/taskstop.go b/testkit/taskstop.go index 420c6c6b84d52..5fcbfb1515e7f 100644 --- a/testkit/taskstop.go +++ b/testkit/taskstop.go @@ -34,8 +34,6 @@ type StoppableTask struct { currentStop *taskstop.StopPoint path []*taskstop.StopPoint - whiteList bool - stopList map[string]struct{} } func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask { @@ -46,31 +44,6 @@ func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask } } -func (t *StoppableTask) StopWhen(stops ...string) *StoppableTask { - t.setStopList(true, stops) - return t -} - -func (t *StoppableTask) StopWhenNot(stops ...string) *StoppableTask { - t.setStopList(false, stops) - return t -} - -func (t *StoppableTask) StopEveryPoint() *StoppableTask { - t.setStopList(false, nil) - return t -} - -func (t *StoppableTask) setStopList(whiteList bool, stops []string) { - list := make(map[string]struct{}) - for _, stopName := range stops { - list[stopName] = struct{}{} - } - - t.stopList = list - t.whiteList = whiteList -} - func (t *StoppableTask) Create(fn TaskFunc) *StoppableTask { t.ch = taskstop.NewChan() t.runFunc = fn @@ -126,26 +99,14 @@ func (t *StoppableTask) IsDone() bool { return t.currentStop.Name() == "DONE" } -func (t *StoppableTask) step() { - for { - if t.stepOne(); t.IsDone() { - break - } - - if _, ok := t.stopList[t.currentStop.Name()]; t.whiteList == ok { - break - } - } -} - func (t *StoppableTask) record(stop *taskstop.StopPoint) { t.path = append(t.path, stop) t.runner.recordPath(t.name, stop) } -func (t *StoppableTask) stepOne() { +func (t *StoppableTask) step() { if t.IsDone() { - panic("cannot step a done thread") + t.t.Fatal("cannot step a done task") } require.NoError(t.t, t.ch.SignalStep()) t.waitNextStop() diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go index b78060785e3b3..aaf2efb379c1e 100644 --- a/util/taskstop/stop.go +++ b/util/taskstop/stop.go @@ -80,21 +80,44 @@ func (ch *Chan) WaitStepSignal() chan any { return ch.ch2 } -func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan) { - sctx.SetValue(stringutil.StringerStr("sessionStopChain"), c) +type sessionStopInjection struct { + ch *Chan + stopEveryPoint bool + stopList []string +} + +func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan, stopList ...string) { + sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), &sessionStopInjection{ + ch: c, + stopEveryPoint: len(stopList) == 0, + stopList: stopList, + }) } func DisableSessionStopPoint(sctx sessionctx.Context) { - sctx.SetValue(stringutil.StringerStr("sessionStopChain"), nil) + sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), nil) } func InjectSessionStopPoint(sctx sessionctx.Context, stopName string) { failpoint.Inject("sessionStop", func() { - if ch, ok := sctx.Value(stringutil.StringerStr("sessionStopChain")).(*Chan); ok { - if err := ch.SignalOnStopAt(stopName); err != nil { + if inject, ok := sctx.Value(stringutil.StringerStr("sessionStopInjection")).(*sessionStopInjection); ok { + if !inject.stopEveryPoint { + shouldStop := false + for _, stop := range inject.stopList { + if stop == stopName { + shouldStop = true + } + } + + if !shouldStop { + return + } + } + + if err := inject.ch.SignalOnStopAt(stopName); err != nil { panic(err) } - <-ch.WaitStepSignal() + <-inject.ch.WaitStepSignal() } }) } From 0258494fe651940c1cb2d745bf34450125ca57e2 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 20:53:40 +0800 Subject: [PATCH 09/21] update --- sessiontxn/txn_context_test.go | 2 +- testkit/taskstop.go | 28 +++++++++++----------------- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 7c49e0262d93d..2bbe5a38852cd 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -735,7 +735,7 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk.MustExec("rollback") tk.MustExec("truncate table t1") tk.MustExec("insert into t1 values(1, 10)") - return testkit.NewStoppableTasksRunner(t).Task("s2").Create(func(ch *taskstop.Chan) { + return testkit.NewStoppableTasksRunner(t).CreateTask("s2", func(ch *taskstop.Chan) { tk2 := testkit.NewTestKit(t, store) defer func() { taskstop.DisableSessionStopPoint(tk2.Session()) diff --git a/testkit/taskstop.go b/testkit/taskstop.go index 5fcbfb1515e7f..1db110c3d2ddf 100644 --- a/testkit/taskstop.go +++ b/testkit/taskstop.go @@ -44,7 +44,7 @@ func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask } } -func (t *StoppableTask) Create(fn TaskFunc) *StoppableTask { +func (t *StoppableTask) setup(fn TaskFunc) { t.ch = taskstop.NewChan() t.runFunc = fn @@ -56,10 +56,8 @@ func (t *StoppableTask) Create(fn TaskFunc) *StoppableTask { }() t.runFunc(t.ch) }() - t.runner.addTask(t) t.waitNextStop() t.record(t.currentStop) - return t } func (t *StoppableTask) CurrentStop() *taskstop.StopPoint { @@ -137,25 +135,21 @@ func NewStoppableTasksRunner(t *testing.T) *StoppableTasksRunner { } } -func (r *StoppableTasksRunner) Task(name string) *StoppableTask { - if thread, ok := r.tasks[name]; ok { - return thread +func (r *StoppableTasksRunner) CreateTask(name string, fn TaskFunc) *StoppableTask { + if _, ok := r.tasks[name]; ok { + r.t.Fatalf("task '%s' already exists", name) } - return newStoppableTask(name, r) -} - -func (r *StoppableTasksRunner) addTask(thread *StoppableTask) { - if _, ok := r.tasks[thread.name]; ok { - r.t.Fatalf("thread '%s' already exists", thread.name) - } - r.tasks[thread.name] = thread + task := newStoppableTask(name, r) + r.tasks[name] = task + task.setup(fn) + return task } -func (r *StoppableTasksRunner) recordPath(threadName string, stop *taskstop.StopPoint) { - task, ok := r.tasks[threadName] +func (r *StoppableTasksRunner) recordPath(taskName string, stop *taskstop.StopPoint) { + task, ok := r.tasks[taskName] if !ok { - r.t.Fatalf("thread '%s' not exist", task.name) + r.t.Fatalf("task '%s' not exist", task.name) } r.path = append(r.path, struct { From 3a0bc58c618f1526fb968beeb916dd4588ee4741 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Mon, 20 Jun 2022 21:20:44 +0800 Subject: [PATCH 10/21] comments --- testkit/taskstop.go | 14 +++++++++++++- util/taskstop/stop.go | 27 ++++++++++++++++++--------- 2 files changed, 31 insertions(+), 10 deletions(-) diff --git a/testkit/taskstop.go b/testkit/taskstop.go index 1db110c3d2ddf..ebcf2702a0c80 100644 --- a/testkit/taskstop.go +++ b/testkit/taskstop.go @@ -22,9 +22,10 @@ import ( "github.com/stretchr/testify/require" ) +// TaskFunc is the func for the task type TaskFunc func(ch *taskstop.Chan) -type SessionTaskFunc func(ch *taskstop.Chan, tk *TestKit) +// StoppableTask is a task object that can be paused type StoppableTask struct { t *testing.T name string @@ -60,39 +61,47 @@ func (t *StoppableTask) setup(fn TaskFunc) { t.record(t.currentStop) } +// CurrentStop returns the task's current stop func (t *StoppableTask) CurrentStop() *taskstop.StopPoint { return t.currentStop } +// Start starts the task func (t *StoppableTask) Start() *StoppableTask { return t.ExpectWaitingStart().Continue() } +// Continue resumes the task func (t *StoppableTask) Continue() *StoppableTask { t.step() t.record(t.currentStop) return t } +// ExpectStoppedAt will check the task stops at the specified stop point func (t *StoppableTask) ExpectStoppedAt(name string) *StoppableTask { require.Equal(t.t, name, t.currentStop.Name()) return t } +// ExpectWaitingStart will check the task is waiting for start func (t *StoppableTask) ExpectWaitingStart() *StoppableTask { require.Truef(t.t, t.IsWaitingStart(), "current stop: '%s'", t.currentStop.Name()) return t } +// ExpectDone will check the task is done func (t *StoppableTask) ExpectDone() *StoppableTask { require.Truef(t.t, t.IsDone(), "current stop: '%s'", t.currentStop.Name()) return t } +// IsWaitingStart returns whether the current stop is waiting for start func (t *StoppableTask) IsWaitingStart() bool { return t.currentStop.Name() == "START" } +// IsDone returns whether the current stop is done func (t *StoppableTask) IsDone() bool { return t.currentStop.Name() == "DONE" } @@ -119,6 +128,7 @@ func (t *StoppableTask) waitNextStop() { } } +// StoppableTasksRunner is used to manage all StoppableTasks type StoppableTasksRunner struct { t *testing.T tasks map[string]*StoppableTask @@ -128,6 +138,7 @@ type StoppableTasksRunner struct { } } +// NewStoppableTasksRunner creates a new StoppableTasksRunner func NewStoppableTasksRunner(t *testing.T) *StoppableTasksRunner { return &StoppableTasksRunner{ t: t, @@ -135,6 +146,7 @@ func NewStoppableTasksRunner(t *testing.T) *StoppableTasksRunner { } } +// CreateTask creates a new task func (r *StoppableTasksRunner) CreateTask(name string, fn TaskFunc) *StoppableTask { if _, ok := r.tasks[name]; ok { r.t.Fatalf("task '%s' already exists", name) diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go index aaf2efb379c1e..f178080d9d712 100644 --- a/util/taskstop/stop.go +++ b/util/taskstop/stop.go @@ -21,31 +21,30 @@ import ( "github.com/pingcap/tidb/util/stringutil" ) +// StopPoint is stop point object type StopPoint struct { - name string - value []any + name string } -func NewStopPoint(name string, value ...any) *StopPoint { +// NewStopPoint creates a new stop point +func NewStopPoint(name string) *StopPoint { return &StopPoint{ - name: name, - value: value, + name: name, } } +// Name returns the name of the stop point func (p *StopPoint) Name() string { return p.name } -func (p *StopPoint) Value() []any { - return p.value -} - +// Chan is used to communicate with between stoppable task and other thread type Chan struct { ch1 chan *StopPoint ch2 chan any } +// NewChan creates a new Chan func NewChan() *Chan { return &Chan{ ch1: make(chan *StopPoint), @@ -53,6 +52,7 @@ func NewChan() *Chan { } } +// SignalOnStopAt writes the chan to indicate that task now stopped at a point func (ch *Chan) SignalOnStopAt(stopName string) error { select { case ch.ch1 <- NewStopPoint(stopName): @@ -63,10 +63,12 @@ func (ch *Chan) SignalOnStopAt(stopName string) error { } +// WaitOnStop returns a chan to wait on stop signal func (ch *Chan) WaitOnStop() chan *StopPoint { return ch.ch1 } +// SignalStep writes the chan to tell the task to continue func (ch *Chan) SignalStep() error { select { case ch.ch2 <- struct{}{}: @@ -76,6 +78,7 @@ func (ch *Chan) SignalStep() error { } } +// WaitStepSignal returns a chan to wait step signal func (ch *Chan) WaitStepSignal() chan any { return ch.ch2 } @@ -86,6 +89,7 @@ type sessionStopInjection struct { stopList []string } +// EnableSessionStopPoint enables the stop points for a session func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan, stopList ...string) { sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), &sessionStopInjection{ ch: c, @@ -94,10 +98,12 @@ func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan, stopList ...string }) } +// DisableSessionStopPoint disables the stop points for a session func DisableSessionStopPoint(sctx sessionctx.Context) { sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), nil) } +// InjectSessionStopPoint injects a stop point func InjectSessionStopPoint(sctx sessionctx.Context, stopName string) { failpoint.Inject("sessionStop", func() { if inject, ok := sctx.Value(stringutil.StringerStr("sessionStopInjection")).(*sessionStopInjection); ok { @@ -122,14 +128,17 @@ func InjectSessionStopPoint(sctx sessionctx.Context, stopName string) { }) } +// EnableGlobalSessionStopFailPoint enables the global session stop fail point func EnableGlobalSessionStopFailPoint() error { return failpoint.Enable("github.com/pingcap/tidb/util/taskstop/sessionStop", "return") } +// DisableGlobalSessionStopFailPoint disables the global session stop fail point func DisableGlobalSessionStopFailPoint() error { return failpoint.Disable("github.com/pingcap/tidb/util/taskstop/sessionStop") } +// IsGlobalSessionStopFailPointEnabled returns whether the global session stop fail point is enabled func IsGlobalSessionStopFailPointEnabled() bool { status, err := failpoint.Status("github.com/pingcap/tidb/util/taskstop/sessionStop") return err == nil && status == "return" From 72ad391956c26f3c53e120cfe90845cb7827f7a7 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Tue, 21 Jun 2022 14:16:01 +0800 Subject: [PATCH 11/21] add SteppedCommandTask --- executor/BUILD.bazel | 1 + sessiontxn/BUILD.bazel | 1 + sessiontxn/txn_context_test.go | 80 +++---- testkit/BUILD.bazel | 2 + testkit/steppedtask.go | 379 +++++++++++++++++++++++++++++++++ testkit/taskstop.go | 171 --------------- util/taskstop/BUILD.bazel | 14 ++ util/taskstop/stop.go | 18 +- 8 files changed, 450 insertions(+), 216 deletions(-) create mode 100644 testkit/steppedtask.go delete mode 100644 testkit/taskstop.go create mode 100644 util/taskstop/BUILD.bazel diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 295b9e07c1f3d..9731542e22c76 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -185,6 +185,7 @@ go_library( "//util/stmtsummary", "//util/stringutil", "//util/table-filter", + "//util/taskstop", "//util/timeutil", "//util/tls", "//util/topsql", diff --git a/sessiontxn/BUILD.bazel b/sessiontxn/BUILD.bazel index 922f80480ac34..587982dadee7f 100644 --- a/sessiontxn/BUILD.bazel +++ b/sessiontxn/BUILD.bazel @@ -36,6 +36,7 @@ go_test( "//sessiontxn/staleread", "//testkit", "//testkit/testsetup", + "//util/taskstop", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 2bbe5a38852cd..d4393d1e88b62 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -731,52 +731,45 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - prepareTask := func(t *testing.T, isolation string, autocommit bool, sql string) *testkit.StoppableTask { - tk.MustExec("rollback") - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - return testkit.NewStoppableTasksRunner(t).CreateTask("s2", func(ch *taskstop.Chan) { - tk2 := testkit.NewTestKit(t, store) - defer func() { - taskstop.DisableSessionStopPoint(tk2.Session()) - tk2.MustExec("rollback") - }() - - tk2.MustExec("use test") - tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") - tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) - if autocommit { - tk2.MustExec("set autocommit=1") - tk2.MustExec("begin") - } else { - tk2.MustExec("set autocommit=0") - } - taskstop.EnableSessionStopPoint( - tk2.Session(), - ch, - sessiontxn.StopPointBeforeExecutorFirstRun, - sessiontxn.StopPointOnStmtRetryAfterLockError, - ) - if strings.HasPrefix(sql, "update") { - tk2.MustExec(sql) - taskstop.DisableSessionStopPoint(tk2.Session()) - tk2.MustExec("commit") - tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - } else { - tk2.MustQuery(sql).Check(testkit.Rows("1 12")) - taskstop.DisableSessionStopPoint(tk2.Session()) - } - }) - } - for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { for _, query := range queries { for _, autocommit := range []bool{true, false} { t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { - task := prepareTask(t, isolation, autocommit, query) + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + tk2 := testkit.NewSteppedTasksRunner(t).CreateSteppedTestKit("s2", store) + defer tk2.Close() + + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) + if autocommit { + tk2.MustExec("set autocommit=1") + tk2.MustExec("begin") + } else { + tk2.MustExec("set autocommit=0") + } + + tk2.EnableSessionStopPoint( + sessiontxn.StopPointBeforeExecutorFirstRun, + sessiontxn.StopPointOnStmtRetryAfterLockError, + ) + + var task *testkit.SteppedCommandTask + var isSelect, isUpdate bool + switch { + case strings.HasPrefix(query, "select"): + isSelect = true + task = tk2.SteppedMustQuery(query) + case strings.HasPrefix(query, "update"): + isUpdate = true + task = tk2.SteppedMustExec(query) + default: + require.FailNowf(t, "invalid query: ", query) + } // Pause the session before the executor first run and then update the record in another session - task.Start().ExpectStoppedAt(sessiontxn.StopPointBeforeExecutorFirstRun) + task.ExpectStoppedAt(sessiontxn.StopPointBeforeExecutorFirstRun) tk.MustExec("update t1 set v=v+1") // Session continues, it should get a lock error and retry, we pause the session before the executor's next run @@ -787,6 +780,13 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { // Because the record is updated by another session again, when this session continues, it will get a lock error again. task.Continue().ExpectStoppedAt(sessiontxn.StopPointOnStmtRetryAfterLockError) task.Continue().ExpectDone() + switch { + case isSelect: + task.GetQueryResult().Check(testkit.Rows("1 12")) + case isUpdate: + tk2.MustExec("commit") + tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) + } }) } } diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index c4c43cf90cfdc..4d1ff06187ccf 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "mocksessionmanager.go", "mockstore.go", "result.go", + "steppedtask.go", "testkit.go", ], importpath = "github.com/pingcap/tidb/testkit", @@ -24,6 +25,7 @@ go_library( "//types", "//util", "//util/sqlexec", + "//util/taskstop", "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/testkit/steppedtask.go b/testkit/steppedtask.go new file mode 100644 index 0000000000000..f90762650656f --- /dev/null +++ b/testkit/steppedtask.go @@ -0,0 +1,379 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testkit + +import ( + "testing" + "time" + + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/taskstop" + "github.com/stretchr/testify/require" +) + +// SteppedTaskFunc is the func for the task +type SteppedTaskFunc func(ch *taskstop.Chan) + +// SteppedTask is a task object that can be paused +type SteppedTask struct { + t *testing.T + name string + runner *SteppedTasksRunner + ch *taskstop.Chan + run SteppedTaskFunc + + currentStop *taskstop.StopPoint + path []*taskstop.StopPoint +} + +func newSteppedTask(name string, runner *SteppedTasksRunner) *SteppedTask { + return &SteppedTask{ + t: runner.t, + name: name, + runner: runner, + } +} + +func (t *SteppedTask) setup(fn SteppedTaskFunc) { + t.ch = taskstop.NewChan() + t.run = fn + + go func() { + defer func() { + require.NoError(t.t, t.ch.SignalOnStopAt(taskstop.DonePointName)) + }() + + require.NoError(t.t, t.ch.SignalOnStopAt(taskstop.StartPointName)) + <-t.ch.WaitStepSignal() + t.run(t.ch) + }() + t.waitNextStop() + t.record(t.currentStop) +} + +// CurrentStop returns the task's current stop +func (t *SteppedTask) CurrentStop() *taskstop.StopPoint { + return t.currentStop +} + +// Start starts the task +func (t *SteppedTask) Start() *SteppedTask { + return t.ExpectWaitingStart().Continue() +} + +// Continue resumes the task +func (t *SteppedTask) Continue() *SteppedTask { + return t.ContinueWithValue(nil) +} + +// ContinueWithValue resumes the task with the value +func (t *SteppedTask) ContinueWithValue(val any) *SteppedTask { + t.step(val) + t.record(t.currentStop) + return t +} + +// ExpectStoppedAt will check the task stops at the specified stop point +func (t *SteppedTask) ExpectStoppedAt(name string) *SteppedTask { + require.Equal(t.t, name, t.currentStop.Name()) + return t +} + +// ExpectWaitingStart will check the task is waiting for start +func (t *SteppedTask) ExpectWaitingStart() *SteppedTask { + return t.ExpectStoppedAt(taskstop.StartPointName) +} + +// ExpectDone will check the task is done +func (t *SteppedTask) ExpectDone() *SteppedTask { + return t.ExpectStoppedAt(taskstop.DonePointName) +} + +// IsWaitingStart returns whether the current stop is waiting for start +func (t *SteppedTask) IsWaitingStart() bool { + return t.currentStop.Name() == taskstop.StartPointName +} + +// IsDone returns whether the current stop is done +func (t *SteppedTask) IsDone() bool { + return t.currentStop.Name() == taskstop.DonePointName +} + +func (t *SteppedTask) record(stop *taskstop.StopPoint) { + t.path = append(t.path, stop) + t.runner.recordPath(t.name, stop) +} + +func (t *SteppedTask) step(val any) { + if t.IsDone() { + t.t.Fatal("cannot step a done task") + } + require.NoError(t.t, t.ch.SignalStep(val)) + t.waitNextStop() +} + +func (t *SteppedTask) waitNextStop() { + select { + case stopPoint := <-t.ch.WaitOnStop(): + t.currentStop = stopPoint + case <-time.After(time.Second * 10): + t.t.Fatal("timeout") + } +} + +// SteppedTasksRunner is used to manage all SteppedTasks +type SteppedTasksRunner struct { + t *testing.T + tasks map[string]*SteppedTask + path []struct { + task *SteppedTask + stop *taskstop.StopPoint + } +} + +// NewSteppedTasksRunner creates a new SteppedTasksRunner +func NewSteppedTasksRunner(t *testing.T) *SteppedTasksRunner { + return &SteppedTasksRunner{ + t: t, + tasks: make(map[string]*SteppedTask), + } +} + +// CreateTask creates a new task +func (r *SteppedTasksRunner) CreateTask(name string, fn SteppedTaskFunc) *SteppedTask { + if _, ok := r.tasks[name]; ok { + r.t.Fatalf("task '%s' already exists", name) + } + + task := newSteppedTask(name, r) + r.tasks[name] = task + task.setup(fn) + return task +} + +// CreateSteppedTestKit creates a new SteppedTestKit +func (r *SteppedTasksRunner) CreateSteppedTestKit(name string, store kv.Storage) *SteppedTestKit { + return newSteppedTestKit(r.t, name, store, r) +} + +func (r *SteppedTasksRunner) recordPath(taskName string, stop *taskstop.StopPoint) { + task, ok := r.tasks[taskName] + if !ok { + r.t.Fatalf("task '%s' not exist", taskName) + } + + r.path = append(r.path, struct { + task *SteppedTask + stop *taskstop.StopPoint + }{task: task, stop: stop}) +} + +// SteppedCommandTask is a stepped task for sql command +type SteppedCommandTask struct { + t *testing.T + task *SteppedTask + done bool + resultChan chan any + result any +} + +func (t *SteppedCommandTask) updateState() *SteppedCommandTask { + switch t.task.currentStop.Name() { + case waitingCommandStopPointName: + t.done = true + select { + case result := <-t.resultChan: + t.result = result + case <-time.After(time.Second * 10): + require.FailNow(t.t, "timeout") + } + case taskstop.DonePointName: + t.done = true + } + return t +} + +// CurrentStop returns the task's current stop +func (t *SteppedCommandTask) CurrentStop() *taskstop.StopPoint { + if t.done { + return taskstop.NewStopPoint(taskstop.DonePointName) + } + return t.task.CurrentStop() +} + +// Continue resumes the task +func (t *SteppedCommandTask) Continue() *SteppedCommandTask { + t.ExpectNotDone() + t.task.Continue() + return t.updateState() +} + +// IsDone returns whether the current stop is done +func (t *SteppedCommandTask) IsDone() bool { + return t.CurrentStop().Name() == taskstop.DonePointName +} + +// ExpectStoppedAt will check the task stops at the specified stop poin +func (t *SteppedCommandTask) ExpectStoppedAt(name string) *SteppedCommandTask { + require.Equal(t.t, name, t.CurrentStop().Name()) + return t +} + +// ExpectDone will check the task is done +func (t *SteppedCommandTask) ExpectDone() *SteppedCommandTask { + require.Equal(t.t, taskstop.DonePointName, t.CurrentStop().Name()) + return t +} + +// ExpectNotDone will check the task not done yet +func (t *SteppedCommandTask) ExpectNotDone() *SteppedCommandTask { + require.NotEqual(t.t, taskstop.DonePointName, t.CurrentStop().Name()) + return t +} + +// GetResult returns the result of command +func (t *SteppedCommandTask) GetResult() any { + return t.ExpectDone().result +} + +// GetQueryResult returns the result of query +func (t *SteppedCommandTask) GetQueryResult() *Result { + return t.GetResult().(*Result) +} + +const waitingCommandStopPointName = "steppedTestKitWaitingCommand" + +// SteppedTestKitCommand is what we want to run for the stepped task +type SteppedTestKitCommand func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any + +// SteppedTestKit is the testkit that can be paused +type SteppedTestKit struct { + t *testing.T + name string + store kv.Storage + resultChan chan any + + task *SteppedTask +} + +func newSteppedTestKit(t *testing.T, name string, store kv.Storage, runner *SteppedTasksRunner) *SteppedTestKit { + tk := &SteppedTestKit{ + t: t, + name: name, + store: store, + resultChan: make(chan any, 1), + } + tk.start(runner) + return tk +} + +func (tk *SteppedTestKit) start(runner *SteppedTasksRunner) { + tk.task = runner.CreateTask(tk.name, tk.run) + tk.task.Start() +} + +func (tk *SteppedTestKit) run(ch *taskstop.Chan) { + rawTestKit := NewTestKit(tk.t, tk.store) + defer func() { + taskstop.DisableSessionStopPoint(rawTestKit.Session()) + rawTestKit.MustExec("rollback") + }() + + for { + require.NoError(tk.t, ch.SignalOnStopAt(waitingCommandStopPointName)) + switch fn := (<-ch.WaitStepSignal()).(type) { + case SteppedTestKitCommand: + select { + case tk.resultChan <- fn(tk.t, rawTestKit, ch): + default: + require.FailNow(tk.t, "the previous value not consumed") + } + default: + return + } + } +} + +// Close closes the current test kit +func (tk *SteppedTestKit) Close() { + tk.task.ContinueWithValue(struct{}{}) +} + +// SteppedCommand create a new stepped task for the command +func (tk *SteppedTestKit) SteppedCommand(fn SteppedTestKitCommand) *SteppedCommandTask { + tk.task.ExpectStoppedAt(waitingCommandStopPointName) + tk.task.ContinueWithValue(fn) + cmd := &SteppedCommandTask{ + t: tk.t, + task: tk.task, + resultChan: tk.resultChan, + } + return cmd.updateState() +} + +// Command executes a command +func (tk *SteppedTestKit) Command(fn SteppedTestKitCommand) any { + cmd := tk.SteppedCommand(fn) + for !cmd.IsDone() { + cmd.Continue() + } + return cmd.GetResult() +} + +// EnableSessionStopPoint enables the session's stop point +func (tk *SteppedTestKit) EnableSessionStopPoint(point ...string) { + tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + taskstop.EnableSessionStopPoint(tk.Session(), ch, point...) + return nil + }) +} + +// DisableSessionStopPoint disables the session's stop point +func (tk *SteppedTestKit) DisableSessionStopPoint() { + tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + taskstop.DisableSessionStopPoint(tk.Session()) + return nil + }) +} + +// MustExec executes MustExec +func (tk *SteppedTestKit) MustExec(sql string, args ...interface{}) { + tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + tk.MustExec(sql, args...) + return nil + }) +} + +// MustQuery executes MustQuery +func (tk *SteppedTestKit) MustQuery(sql string, args ...interface{}) *Result { + return tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + return tk.MustQuery(sql, args...) + }).(*Result) +} + +// SteppedMustExec creates a new stepped task for MustExec +func (tk *SteppedTestKit) SteppedMustExec(sql string, args ...interface{}) *SteppedCommandTask { + return tk.SteppedCommand(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + tk.MustExec(sql, args...) + return nil + }) +} + +// SteppedMustQuery creates a new stepped task for MustQuery +func (tk *SteppedTestKit) SteppedMustQuery(sql string, args ...interface{}) *SteppedCommandTask { + return tk.SteppedCommand(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { + return tk.MustQuery(sql, args...) + }) +} diff --git a/testkit/taskstop.go b/testkit/taskstop.go deleted file mode 100644 index ebcf2702a0c80..0000000000000 --- a/testkit/taskstop.go +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package testkit - -import ( - "testing" - "time" - - "github.com/pingcap/tidb/util/taskstop" - "github.com/stretchr/testify/require" -) - -// TaskFunc is the func for the task -type TaskFunc func(ch *taskstop.Chan) - -// StoppableTask is a task object that can be paused -type StoppableTask struct { - t *testing.T - name string - runner *StoppableTasksRunner - ch *taskstop.Chan - runFunc TaskFunc - - currentStop *taskstop.StopPoint - path []*taskstop.StopPoint -} - -func newStoppableTask(name string, runner *StoppableTasksRunner) *StoppableTask { - return &StoppableTask{ - t: runner.t, - name: name, - runner: runner, - } -} - -func (t *StoppableTask) setup(fn TaskFunc) { - t.ch = taskstop.NewChan() - t.runFunc = fn - - go func() { - require.NoError(t.t, t.ch.SignalOnStopAt("START")) - <-t.ch.WaitStepSignal() - defer func() { - require.NoError(t.t, t.ch.SignalOnStopAt("DONE")) - }() - t.runFunc(t.ch) - }() - t.waitNextStop() - t.record(t.currentStop) -} - -// CurrentStop returns the task's current stop -func (t *StoppableTask) CurrentStop() *taskstop.StopPoint { - return t.currentStop -} - -// Start starts the task -func (t *StoppableTask) Start() *StoppableTask { - return t.ExpectWaitingStart().Continue() -} - -// Continue resumes the task -func (t *StoppableTask) Continue() *StoppableTask { - t.step() - t.record(t.currentStop) - return t -} - -// ExpectStoppedAt will check the task stops at the specified stop point -func (t *StoppableTask) ExpectStoppedAt(name string) *StoppableTask { - require.Equal(t.t, name, t.currentStop.Name()) - return t -} - -// ExpectWaitingStart will check the task is waiting for start -func (t *StoppableTask) ExpectWaitingStart() *StoppableTask { - require.Truef(t.t, t.IsWaitingStart(), "current stop: '%s'", t.currentStop.Name()) - return t -} - -// ExpectDone will check the task is done -func (t *StoppableTask) ExpectDone() *StoppableTask { - require.Truef(t.t, t.IsDone(), "current stop: '%s'", t.currentStop.Name()) - return t -} - -// IsWaitingStart returns whether the current stop is waiting for start -func (t *StoppableTask) IsWaitingStart() bool { - return t.currentStop.Name() == "START" -} - -// IsDone returns whether the current stop is done -func (t *StoppableTask) IsDone() bool { - return t.currentStop.Name() == "DONE" -} - -func (t *StoppableTask) record(stop *taskstop.StopPoint) { - t.path = append(t.path, stop) - t.runner.recordPath(t.name, stop) -} - -func (t *StoppableTask) step() { - if t.IsDone() { - t.t.Fatal("cannot step a done task") - } - require.NoError(t.t, t.ch.SignalStep()) - t.waitNextStop() -} - -func (t *StoppableTask) waitNextStop() { - select { - case stopPoint := <-t.ch.WaitOnStop(): - t.currentStop = stopPoint - case <-time.After(time.Second * 10): - t.t.Fatal("timeout") - } -} - -// StoppableTasksRunner is used to manage all StoppableTasks -type StoppableTasksRunner struct { - t *testing.T - tasks map[string]*StoppableTask - path []struct { - task *StoppableTask - stop *taskstop.StopPoint - } -} - -// NewStoppableTasksRunner creates a new StoppableTasksRunner -func NewStoppableTasksRunner(t *testing.T) *StoppableTasksRunner { - return &StoppableTasksRunner{ - t: t, - tasks: make(map[string]*StoppableTask), - } -} - -// CreateTask creates a new task -func (r *StoppableTasksRunner) CreateTask(name string, fn TaskFunc) *StoppableTask { - if _, ok := r.tasks[name]; ok { - r.t.Fatalf("task '%s' already exists", name) - } - - task := newStoppableTask(name, r) - r.tasks[name] = task - task.setup(fn) - return task -} - -func (r *StoppableTasksRunner) recordPath(taskName string, stop *taskstop.StopPoint) { - task, ok := r.tasks[taskName] - if !ok { - r.t.Fatalf("task '%s' not exist", task.name) - } - - r.path = append(r.path, struct { - task *StoppableTask - stop *taskstop.StopPoint - }{task: task, stop: stop}) -} diff --git a/util/taskstop/BUILD.bazel b/util/taskstop/BUILD.bazel new file mode 100644 index 0000000000000..4f121f5caad3d --- /dev/null +++ b/util/taskstop/BUILD.bazel @@ -0,0 +1,14 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "taskstop", + srcs = ["stop.go"], + importpath = "github.com/pingcap/tidb/util/taskstop", + visibility = ["//visibility:public"], + deps = [ + "//sessionctx", + "//util/stringutil", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", + ], +) diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go index f178080d9d712..af9161931ba4f 100644 --- a/util/taskstop/stop.go +++ b/util/taskstop/stop.go @@ -15,12 +15,20 @@ package taskstop import ( + "time" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/stringutil" ) +// StartPointName is the stop point for start +var StartPointName = "START" + +// DonePointName is the stop point for done +var DonePointName = "DONE" + // StopPoint is stop point object type StopPoint struct { name string @@ -38,7 +46,7 @@ func (p *StopPoint) Name() string { return p.name } -// Chan is used to communicate with between stoppable task and other thread +// Chan is used to communicate with between stepped task and other thread type Chan struct { ch1 chan *StopPoint ch2 chan any @@ -57,7 +65,7 @@ func (ch *Chan) SignalOnStopAt(stopName string) error { select { case ch.ch1 <- NewStopPoint(stopName): return nil - default: + case <-time.After(time.Second * 10): return errors.New("Cannot signal stop at") } @@ -69,11 +77,11 @@ func (ch *Chan) WaitOnStop() chan *StopPoint { } // SignalStep writes the chan to tell the task to continue -func (ch *Chan) SignalStep() error { +func (ch *Chan) SignalStep(val any) error { select { - case ch.ch2 <- struct{}{}: + case ch.ch2 <- val: return nil - default: + case <-time.After(time.Second * 10): return errors.New("Cannot signal step") } } From d4b9ed7b0ad37c9bac45fb00d99ec4680753e83d Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 22 Jun 2022 18:41:36 +0800 Subject: [PATCH 12/21] refactor --- sessiontxn/txn_context_test.go | 24 +-- testkit/stepped.go | 239 +++++++++++++++++++++ testkit/steppedtask.go | 379 --------------------------------- util/taskstop/stop.go | 104 +-------- 4 files changed, 260 insertions(+), 486 deletions(-) create mode 100644 testkit/stepped.go delete mode 100644 testkit/steppedtask.go diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index d4393d1e88b62..9d848d5b39ab7 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -715,7 +715,6 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { store, _, deferFunc := setupTxnContextTest(t) defer deferFunc() - require.True(t, taskstop.IsGlobalSessionStopFailPointEnabled(), "should enable global session stop failPoint") queries := []string{ "select * from t1 for update", "select * from t1 where id=1 for update", @@ -737,8 +736,8 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { tk.MustExec("truncate table t1") tk.MustExec("insert into t1 values(1, 10)") - tk2 := testkit.NewSteppedTasksRunner(t).CreateSteppedTestKit("s2", store) - defer tk2.Close() + tk2 := testkit.NewSteppedTestKit(t, store) + defer tk2.MustExec("rollback") tk2.MustExec("use test") tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") @@ -750,39 +749,38 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk2.MustExec("set autocommit=0") } - tk2.EnableSessionStopPoint( + tk2.SetBreakPoints([]string{ sessiontxn.StopPointBeforeExecutorFirstRun, sessiontxn.StopPointOnStmtRetryAfterLockError, - ) + }) - var task *testkit.SteppedCommandTask var isSelect, isUpdate bool switch { case strings.HasPrefix(query, "select"): isSelect = true - task = tk2.SteppedMustQuery(query) + tk2.SteppedMustQuery(query) case strings.HasPrefix(query, "update"): isUpdate = true - task = tk2.SteppedMustExec(query) + tk2.SteppedMustExec(query) default: require.FailNowf(t, "invalid query: ", query) } // Pause the session before the executor first run and then update the record in another session - task.ExpectStoppedAt(sessiontxn.StopPointBeforeExecutorFirstRun) + tk2.ExpectStopOnBreakPoint(sessiontxn.StopPointBeforeExecutorFirstRun) tk.MustExec("update t1 set v=v+1") // Session continues, it should get a lock error and retry, we pause the session before the executor's next run // and then update the record in another session again. - task.Continue().ExpectStoppedAt(sessiontxn.StopPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.StopPointOnStmtRetryAfterLockError) tk.MustExec("update t1 set v=v+1") // Because the record is updated by another session again, when this session continues, it will get a lock error again. - task.Continue().ExpectStoppedAt(sessiontxn.StopPointOnStmtRetryAfterLockError) - task.Continue().ExpectDone() + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.StopPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectIdle() switch { case isSelect: - task.GetQueryResult().Check(testkit.Rows("1 12")) + tk2.GetQueryResult().Check(testkit.Rows("1 12")) case isUpdate: tk2.MustExec("commit") tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) diff --git a/testkit/stepped.go b/testkit/stepped.go new file mode 100644 index 0000000000000..75b61c9a53ba6 --- /dev/null +++ b/testkit/stepped.go @@ -0,0 +1,239 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testkit + +import ( + "testing" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/taskstop" + "github.com/stretchr/testify/require" +) + +var commandInternalError = errors.New("command error") + +type steppedTestkitMsgType int + +const ( + msgTpCmdContinue steppedTestkitMsgType = iota + msgTpCmdStopOnBreakPoint + msgTpCmdDone +) + +type steppedTestKitMsg struct { + tp steppedTestkitMsgType + val any +} + +type steppedTestKitMsgChan chan *steppedTestKitMsg + +func (ch steppedTestKitMsgChan) sendMsg(tp steppedTestkitMsgType, val any) error { + select { + case ch <- &steppedTestKitMsg{tp: tp, val: val}: + return nil + case <-time.After(time.Second * 10): + return errors.New("send msg timeout") + } +} + +func (ch steppedTestKitMsgChan) sendMsgWithType(tp steppedTestkitMsgType) error { + return ch.sendMsg(tp, nil) +} + +func (ch steppedTestKitMsgChan) recvMsg() (*steppedTestKitMsg, error) { + select { + case msg := <-ch: + return msg, nil + case <-time.After(time.Second * 10): + return nil, errors.New("send msg timeout") + } +} + +func (ch steppedTestKitMsgChan) recvMsgWithCheck(tp steppedTestkitMsgType) (*steppedTestKitMsg, error) { + msg, err := ch.recvMsg() + if err != nil { + return nil, err + } + + if msg.tp != tp { + return nil, errors.Errorf("unexpected msg type: %v, expect: %v", msg.tp, tp) + } + + return msg, nil +} + +type steppedTestKitCommandContext struct { + t *testing.T + tk *TestKit + notifyBreakPointAndWait func(string) +} + +type steppedTestKitCommand func(ctx *steppedTestKitCommandContext) any + +// SteppedTestKit is the testkit that can run stepped command +type SteppedTestKit struct { + t *testing.T + tk *TestKit + + // ch1 is used to send msg from foreground to background + ch1 steppedTestKitMsgChan + // ch2 is used to send msg from background to foreground + ch2 steppedTestKitMsgChan + // breakPoints is the break points we want to stop at + breakPoints []string + // cmdStopAt is the current break point it stopped at + cmdStopAt string + // the result of the current command + cmdResult any +} + +func NewSteppedTestKit(t *testing.T, store kv.Storage) *SteppedTestKit { + tk := &SteppedTestKit{ + t: t, + tk: NewTestKit(t, store), + ch1: make(steppedTestKitMsgChan), + ch2: make(steppedTestKitMsgChan), + } + return tk +} + +// ExpectIdle checks no command is running +func (tk *SteppedTestKit) ExpectIdle() { + require.Equal(tk.t, "", tk.cmdStopAt) +} + +// ExpectStopOnBreakPoint checks stopped on the specified break point +func (tk *SteppedTestKit) ExpectStopOnBreakPoint(breakPoint string) { + require.Equal(tk.t, breakPoint, tk.cmdStopAt) +} + +// ExpectStopOnAnyBreakPoint checks stopped on any break point +func (tk *SteppedTestKit) ExpectStopOnAnyBreakPoint() { + require.NotEqual(tk.t, "", tk.cmdStopAt) +} + +func (tk *SteppedTestKit) SetBreakPoints(breakPoints []string) { + tk.breakPoints = breakPoints +} + +func (tk *SteppedTestKit) handleCommandMsg() { + msg, err := tk.ch2.recvMsg() + require.NoError(tk.t, err) + switch msg.tp { + case msgTpCmdDone: + tk.cmdStopAt = "" + if msg.val == commandInternalError { + require.FailNow(tk.t, "internal command failed") + } else { + tk.cmdResult = msg.val + } + case msgTpCmdStopOnBreakPoint: + require.IsType(tk.t, "", msg.val) + require.NotEqual(tk.t, "", msg.val) + tk.cmdStopAt = msg.val.(string) + default: + require.FailNow(tk.t, "invalid msg type", "tp %v", msg.tp) + } +} + +func (tk *SteppedTestKit) beforeCommand() { + tk.ExpectIdle() + tk.cmdResult = nil +} + +func (tk *SteppedTestKit) steppedCommand(cmd steppedTestKitCommand) *SteppedTestKit { + tk.beforeCommand() + go func() { + var success bool + var result any + defer func() { + taskstop.DisableSessionStopPoint(tk.tk.Session()) + if !success { + result = commandInternalError + } + require.NoError(tk.t, tk.ch2.sendMsg(msgTpCmdDone, result)) + }() + + ctx := &steppedTestKitCommandContext{ + t: tk.t, + tk: tk.tk, + notifyBreakPointAndWait: func(breakPoint string) { + require.NoError(tk.t, tk.ch2.sendMsg(msgTpCmdStopOnBreakPoint, breakPoint)) + _, err := tk.ch1.recvMsgWithCheck(msgTpCmdContinue) + require.NoError(tk.t, err) + }, + } + + taskstop.EnableSessionStopPoint(tk.tk.Session(), ctx.notifyBreakPointAndWait, tk.breakPoints...) + result = cmd(ctx) + success = true + }() + + tk.handleCommandMsg() + return tk +} + +// Continue continues current command +func (tk *SteppedTestKit) Continue() *SteppedTestKit { + tk.ExpectStopOnAnyBreakPoint() + require.NoError(tk.t, tk.ch1.sendMsgWithType(msgTpCmdContinue)) + tk.handleCommandMsg() + return tk +} + +// SteppedMustExec creates a new stepped task for MustExec +func (tk *SteppedTestKit) SteppedMustExec(sql string, args ...interface{}) *SteppedTestKit { + return tk.steppedCommand(func(_ *steppedTestKitCommandContext) any { + tk.MustExec(sql, args...) + return nil + }) +} + +// SteppedMustQuery creates a new stepped task for MustQuery +func (tk *SteppedTestKit) SteppedMustQuery(sql string, args ...interface{}) *SteppedTestKit { + return tk.steppedCommand(func(_ *steppedTestKitCommandContext) any { + return tk.MustQuery(sql, args...) + }) +} + +// MustExec executes a sql statement and asserts nil error. +func (tk *SteppedTestKit) MustExec(sql string, args ...interface{}) { + tk.beforeCommand() + tk.tk.MustExec(sql, args...) +} + +// MustQuery query the statements and returns result rows. +// If expected result is set it asserts the query result equals expected result. +func (tk *SteppedTestKit) MustQuery(sql string, args ...interface{}) *Result { + tk.beforeCommand() + result := tk.tk.MustQuery(sql, args...) + tk.cmdResult = result + return result +} + +// GetResult returns the result of the latest command +func (tk *SteppedTestKit) GetResult() any { + tk.ExpectIdle() + return tk.cmdResult +} + +// GetQueryResult returns the query result of the latest command +func (tk *SteppedTestKit) GetQueryResult() *Result { + tk.ExpectIdle() + require.IsType(tk.t, &Result{}, tk.cmdResult) + return tk.cmdResult.(*Result) +} diff --git a/testkit/steppedtask.go b/testkit/steppedtask.go deleted file mode 100644 index f90762650656f..0000000000000 --- a/testkit/steppedtask.go +++ /dev/null @@ -1,379 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package testkit - -import ( - "testing" - "time" - - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/util/taskstop" - "github.com/stretchr/testify/require" -) - -// SteppedTaskFunc is the func for the task -type SteppedTaskFunc func(ch *taskstop.Chan) - -// SteppedTask is a task object that can be paused -type SteppedTask struct { - t *testing.T - name string - runner *SteppedTasksRunner - ch *taskstop.Chan - run SteppedTaskFunc - - currentStop *taskstop.StopPoint - path []*taskstop.StopPoint -} - -func newSteppedTask(name string, runner *SteppedTasksRunner) *SteppedTask { - return &SteppedTask{ - t: runner.t, - name: name, - runner: runner, - } -} - -func (t *SteppedTask) setup(fn SteppedTaskFunc) { - t.ch = taskstop.NewChan() - t.run = fn - - go func() { - defer func() { - require.NoError(t.t, t.ch.SignalOnStopAt(taskstop.DonePointName)) - }() - - require.NoError(t.t, t.ch.SignalOnStopAt(taskstop.StartPointName)) - <-t.ch.WaitStepSignal() - t.run(t.ch) - }() - t.waitNextStop() - t.record(t.currentStop) -} - -// CurrentStop returns the task's current stop -func (t *SteppedTask) CurrentStop() *taskstop.StopPoint { - return t.currentStop -} - -// Start starts the task -func (t *SteppedTask) Start() *SteppedTask { - return t.ExpectWaitingStart().Continue() -} - -// Continue resumes the task -func (t *SteppedTask) Continue() *SteppedTask { - return t.ContinueWithValue(nil) -} - -// ContinueWithValue resumes the task with the value -func (t *SteppedTask) ContinueWithValue(val any) *SteppedTask { - t.step(val) - t.record(t.currentStop) - return t -} - -// ExpectStoppedAt will check the task stops at the specified stop point -func (t *SteppedTask) ExpectStoppedAt(name string) *SteppedTask { - require.Equal(t.t, name, t.currentStop.Name()) - return t -} - -// ExpectWaitingStart will check the task is waiting for start -func (t *SteppedTask) ExpectWaitingStart() *SteppedTask { - return t.ExpectStoppedAt(taskstop.StartPointName) -} - -// ExpectDone will check the task is done -func (t *SteppedTask) ExpectDone() *SteppedTask { - return t.ExpectStoppedAt(taskstop.DonePointName) -} - -// IsWaitingStart returns whether the current stop is waiting for start -func (t *SteppedTask) IsWaitingStart() bool { - return t.currentStop.Name() == taskstop.StartPointName -} - -// IsDone returns whether the current stop is done -func (t *SteppedTask) IsDone() bool { - return t.currentStop.Name() == taskstop.DonePointName -} - -func (t *SteppedTask) record(stop *taskstop.StopPoint) { - t.path = append(t.path, stop) - t.runner.recordPath(t.name, stop) -} - -func (t *SteppedTask) step(val any) { - if t.IsDone() { - t.t.Fatal("cannot step a done task") - } - require.NoError(t.t, t.ch.SignalStep(val)) - t.waitNextStop() -} - -func (t *SteppedTask) waitNextStop() { - select { - case stopPoint := <-t.ch.WaitOnStop(): - t.currentStop = stopPoint - case <-time.After(time.Second * 10): - t.t.Fatal("timeout") - } -} - -// SteppedTasksRunner is used to manage all SteppedTasks -type SteppedTasksRunner struct { - t *testing.T - tasks map[string]*SteppedTask - path []struct { - task *SteppedTask - stop *taskstop.StopPoint - } -} - -// NewSteppedTasksRunner creates a new SteppedTasksRunner -func NewSteppedTasksRunner(t *testing.T) *SteppedTasksRunner { - return &SteppedTasksRunner{ - t: t, - tasks: make(map[string]*SteppedTask), - } -} - -// CreateTask creates a new task -func (r *SteppedTasksRunner) CreateTask(name string, fn SteppedTaskFunc) *SteppedTask { - if _, ok := r.tasks[name]; ok { - r.t.Fatalf("task '%s' already exists", name) - } - - task := newSteppedTask(name, r) - r.tasks[name] = task - task.setup(fn) - return task -} - -// CreateSteppedTestKit creates a new SteppedTestKit -func (r *SteppedTasksRunner) CreateSteppedTestKit(name string, store kv.Storage) *SteppedTestKit { - return newSteppedTestKit(r.t, name, store, r) -} - -func (r *SteppedTasksRunner) recordPath(taskName string, stop *taskstop.StopPoint) { - task, ok := r.tasks[taskName] - if !ok { - r.t.Fatalf("task '%s' not exist", taskName) - } - - r.path = append(r.path, struct { - task *SteppedTask - stop *taskstop.StopPoint - }{task: task, stop: stop}) -} - -// SteppedCommandTask is a stepped task for sql command -type SteppedCommandTask struct { - t *testing.T - task *SteppedTask - done bool - resultChan chan any - result any -} - -func (t *SteppedCommandTask) updateState() *SteppedCommandTask { - switch t.task.currentStop.Name() { - case waitingCommandStopPointName: - t.done = true - select { - case result := <-t.resultChan: - t.result = result - case <-time.After(time.Second * 10): - require.FailNow(t.t, "timeout") - } - case taskstop.DonePointName: - t.done = true - } - return t -} - -// CurrentStop returns the task's current stop -func (t *SteppedCommandTask) CurrentStop() *taskstop.StopPoint { - if t.done { - return taskstop.NewStopPoint(taskstop.DonePointName) - } - return t.task.CurrentStop() -} - -// Continue resumes the task -func (t *SteppedCommandTask) Continue() *SteppedCommandTask { - t.ExpectNotDone() - t.task.Continue() - return t.updateState() -} - -// IsDone returns whether the current stop is done -func (t *SteppedCommandTask) IsDone() bool { - return t.CurrentStop().Name() == taskstop.DonePointName -} - -// ExpectStoppedAt will check the task stops at the specified stop poin -func (t *SteppedCommandTask) ExpectStoppedAt(name string) *SteppedCommandTask { - require.Equal(t.t, name, t.CurrentStop().Name()) - return t -} - -// ExpectDone will check the task is done -func (t *SteppedCommandTask) ExpectDone() *SteppedCommandTask { - require.Equal(t.t, taskstop.DonePointName, t.CurrentStop().Name()) - return t -} - -// ExpectNotDone will check the task not done yet -func (t *SteppedCommandTask) ExpectNotDone() *SteppedCommandTask { - require.NotEqual(t.t, taskstop.DonePointName, t.CurrentStop().Name()) - return t -} - -// GetResult returns the result of command -func (t *SteppedCommandTask) GetResult() any { - return t.ExpectDone().result -} - -// GetQueryResult returns the result of query -func (t *SteppedCommandTask) GetQueryResult() *Result { - return t.GetResult().(*Result) -} - -const waitingCommandStopPointName = "steppedTestKitWaitingCommand" - -// SteppedTestKitCommand is what we want to run for the stepped task -type SteppedTestKitCommand func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any - -// SteppedTestKit is the testkit that can be paused -type SteppedTestKit struct { - t *testing.T - name string - store kv.Storage - resultChan chan any - - task *SteppedTask -} - -func newSteppedTestKit(t *testing.T, name string, store kv.Storage, runner *SteppedTasksRunner) *SteppedTestKit { - tk := &SteppedTestKit{ - t: t, - name: name, - store: store, - resultChan: make(chan any, 1), - } - tk.start(runner) - return tk -} - -func (tk *SteppedTestKit) start(runner *SteppedTasksRunner) { - tk.task = runner.CreateTask(tk.name, tk.run) - tk.task.Start() -} - -func (tk *SteppedTestKit) run(ch *taskstop.Chan) { - rawTestKit := NewTestKit(tk.t, tk.store) - defer func() { - taskstop.DisableSessionStopPoint(rawTestKit.Session()) - rawTestKit.MustExec("rollback") - }() - - for { - require.NoError(tk.t, ch.SignalOnStopAt(waitingCommandStopPointName)) - switch fn := (<-ch.WaitStepSignal()).(type) { - case SteppedTestKitCommand: - select { - case tk.resultChan <- fn(tk.t, rawTestKit, ch): - default: - require.FailNow(tk.t, "the previous value not consumed") - } - default: - return - } - } -} - -// Close closes the current test kit -func (tk *SteppedTestKit) Close() { - tk.task.ContinueWithValue(struct{}{}) -} - -// SteppedCommand create a new stepped task for the command -func (tk *SteppedTestKit) SteppedCommand(fn SteppedTestKitCommand) *SteppedCommandTask { - tk.task.ExpectStoppedAt(waitingCommandStopPointName) - tk.task.ContinueWithValue(fn) - cmd := &SteppedCommandTask{ - t: tk.t, - task: tk.task, - resultChan: tk.resultChan, - } - return cmd.updateState() -} - -// Command executes a command -func (tk *SteppedTestKit) Command(fn SteppedTestKitCommand) any { - cmd := tk.SteppedCommand(fn) - for !cmd.IsDone() { - cmd.Continue() - } - return cmd.GetResult() -} - -// EnableSessionStopPoint enables the session's stop point -func (tk *SteppedTestKit) EnableSessionStopPoint(point ...string) { - tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - taskstop.EnableSessionStopPoint(tk.Session(), ch, point...) - return nil - }) -} - -// DisableSessionStopPoint disables the session's stop point -func (tk *SteppedTestKit) DisableSessionStopPoint() { - tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - taskstop.DisableSessionStopPoint(tk.Session()) - return nil - }) -} - -// MustExec executes MustExec -func (tk *SteppedTestKit) MustExec(sql string, args ...interface{}) { - tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - tk.MustExec(sql, args...) - return nil - }) -} - -// MustQuery executes MustQuery -func (tk *SteppedTestKit) MustQuery(sql string, args ...interface{}) *Result { - return tk.Command(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - return tk.MustQuery(sql, args...) - }).(*Result) -} - -// SteppedMustExec creates a new stepped task for MustExec -func (tk *SteppedTestKit) SteppedMustExec(sql string, args ...interface{}) *SteppedCommandTask { - return tk.SteppedCommand(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - tk.MustExec(sql, args...) - return nil - }) -} - -// SteppedMustQuery creates a new stepped task for MustQuery -func (tk *SteppedTestKit) SteppedMustQuery(sql string, args ...interface{}) *SteppedCommandTask { - return tk.SteppedCommand(func(t *testing.T, tk *TestKit, ch *taskstop.Chan) any { - return tk.MustQuery(sql, args...) - }) -} diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go index af9161931ba4f..3028cb7d66455 100644 --- a/util/taskstop/stop.go +++ b/util/taskstop/stop.go @@ -15,94 +15,21 @@ package taskstop import ( - "time" - - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/stringutil" ) -// StartPointName is the stop point for start -var StartPointName = "START" - -// DonePointName is the stop point for done -var DonePointName = "DONE" - -// StopPoint is stop point object -type StopPoint struct { - name string -} - -// NewStopPoint creates a new stop point -func NewStopPoint(name string) *StopPoint { - return &StopPoint{ - name: name, - } -} - -// Name returns the name of the stop point -func (p *StopPoint) Name() string { - return p.name -} - -// Chan is used to communicate with between stepped task and other thread -type Chan struct { - ch1 chan *StopPoint - ch2 chan any -} - -// NewChan creates a new Chan -func NewChan() *Chan { - return &Chan{ - ch1: make(chan *StopPoint), - ch2: make(chan any), - } -} - -// SignalOnStopAt writes the chan to indicate that task now stopped at a point -func (ch *Chan) SignalOnStopAt(stopName string) error { - select { - case ch.ch1 <- NewStopPoint(stopName): - return nil - case <-time.After(time.Second * 10): - return errors.New("Cannot signal stop at") - } - -} - -// WaitOnStop returns a chan to wait on stop signal -func (ch *Chan) WaitOnStop() chan *StopPoint { - return ch.ch1 -} - -// SignalStep writes the chan to tell the task to continue -func (ch *Chan) SignalStep(val any) error { - select { - case ch.ch2 <- val: - return nil - case <-time.After(time.Second * 10): - return errors.New("Cannot signal step") - } -} - -// WaitStepSignal returns a chan to wait step signal -func (ch *Chan) WaitStepSignal() chan any { - return ch.ch2 -} - type sessionStopInjection struct { - ch *Chan - stopEveryPoint bool - stopList []string + fn func(string) + breakPoints []string } // EnableSessionStopPoint enables the stop points for a session -func EnableSessionStopPoint(sctx sessionctx.Context, c *Chan, stopList ...string) { +func EnableSessionStopPoint(sctx sessionctx.Context, fn func(string), breakPoints ...string) { sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), &sessionStopInjection{ - ch: c, - stopEveryPoint: len(stopList) == 0, - stopList: stopList, + fn: fn, + breakPoints: breakPoints, }) } @@ -112,26 +39,15 @@ func DisableSessionStopPoint(sctx sessionctx.Context) { } // InjectSessionStopPoint injects a stop point -func InjectSessionStopPoint(sctx sessionctx.Context, stopName string) { +func InjectSessionStopPoint(sctx sessionctx.Context, breakPoint string) { failpoint.Inject("sessionStop", func() { if inject, ok := sctx.Value(stringutil.StringerStr("sessionStopInjection")).(*sessionStopInjection); ok { - if !inject.stopEveryPoint { - shouldStop := false - for _, stop := range inject.stopList { - if stop == stopName { - shouldStop = true - } - } - - if !shouldStop { - return + for _, p := range inject.breakPoints { + if p == breakPoint { + inject.fn(breakPoint) + break } } - - if err := inject.ch.SignalOnStopAt(stopName); err != nil { - panic(err) - } - <-inject.ch.WaitStepSignal() } }) } From 9146d0c9eb0c2c3b4f6404d4c187304c0ffba9d2 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 22 Jun 2022 21:08:29 +0800 Subject: [PATCH 13/21] update --- util/taskstop/stop.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go index 3028cb7d66455..e1807d2951a0c 100644 --- a/util/taskstop/stop.go +++ b/util/taskstop/stop.go @@ -61,9 +61,3 @@ func EnableGlobalSessionStopFailPoint() error { func DisableGlobalSessionStopFailPoint() error { return failpoint.Disable("github.com/pingcap/tidb/util/taskstop/sessionStop") } - -// IsGlobalSessionStopFailPointEnabled returns whether the global session stop fail point is enabled -func IsGlobalSessionStopFailPointEnabled() bool { - status, err := failpoint.Status("github.com/pingcap/tidb/util/taskstop/sessionStop") - return err == nil && status == "return" -} From 1814743c78e2a8ff891980201c5684f579edd138 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 22 Jun 2022 22:41:34 +0800 Subject: [PATCH 14/21] update --- executor/BUILD.bazel | 2 +- executor/adapter.go | 6 +-- sessiontxn/BUILD.bazel | 1 - sessiontxn/failpoint.go | 4 +- sessiontxn/txn_context_test.go | 3 -- testkit/BUILD.bazel | 5 +- testkit/stepped.go | 19 +++++-- util/{taskstop => breakpoint}/BUILD.bazel | 12 +++++ util/breakpoint/stop.go | 32 ++++++++++++ util/taskstop/stop.go | 63 ----------------------- 10 files changed, 69 insertions(+), 78 deletions(-) rename util/{taskstop => breakpoint}/BUILD.bazel (57%) create mode 100644 util/breakpoint/stop.go delete mode 100644 util/taskstop/stop.go diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 9731542e22c76..e9b7c23298645 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -152,6 +152,7 @@ go_library( "//util", "//util/admin", "//util/bitmap", + "//util/breakpoint", "//util/chunk", "//util/codec", "//util/collate", @@ -185,7 +186,6 @@ go_library( "//util/stmtsummary", "//util/stringutil", "//util/table-filter", - "//util/taskstop", "//util/timeutil", "//util/tls", "//util/topsql", diff --git a/executor/adapter.go b/executor/adapter.go index 9fda41c2068e2..099d746e8be64 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/breakpoint" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hint" @@ -58,7 +59,6 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" - "github.com/pingcap/tidb/util/taskstop" "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" tikverr "github.com/tikv/client-go/v2/error" @@ -416,7 +416,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) - taskstop.InjectSessionStopPoint(a.Ctx, sessiontxn.StopPointBeforeExecutorFirstRun) + breakpoint.Inject(a.Ctx, sessiontxn.StopPointBeforeExecutorFirstRun) if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -793,7 +793,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if err != nil { return nil, err } - taskstop.InjectSessionStopPoint(a.Ctx, sessiontxn.StopPointOnStmtRetryAfterLockError) + breakpoint.Inject(a.Ctx, sessiontxn.StopPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() if err != nil { diff --git a/sessiontxn/BUILD.bazel b/sessiontxn/BUILD.bazel index 587982dadee7f..922f80480ac34 100644 --- a/sessiontxn/BUILD.bazel +++ b/sessiontxn/BUILD.bazel @@ -36,7 +36,6 @@ go_test( "//sessiontxn/staleread", "//testkit", "//testkit/testsetup", - "//util/taskstop", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index fa74bea426118..39287c2c248ce 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -37,11 +37,11 @@ var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSche // StopPointBeforeExecutorFirstRun is the key for the stop point where session stops before executor's first run // Only for test -var StopPointBeforeExecutorFirstRun = "before executor first run" +var StopPointBeforeExecutorFirstRun = "beforeExecutorFirstRun" // StopPointOnStmtRetryAfterLockError s the key for the stop point where session stops after OnStmtRetry when lock error happens // Only for test -var StopPointOnStmtRetryAfterLockError = "lock error, OnStmtRetry called" +var StopPointOnStmtRetryAfterLockError = "lockErrorAndThenOnStmtRetryCalled" // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 9d848d5b39ab7..29d8157a2cfad 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testsetup" - "github.com/pingcap/tidb/util/taskstop" "github.com/stretchr/testify/require" "go.uber.org/goleak" ) @@ -58,7 +57,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan", "return")) - require.NoError(t, taskstop.EnableGlobalSessionStopFailPoint()) store, do, clean := testkit.CreateMockStoreAndDomain(t) @@ -90,7 +88,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerForUpdateTSEqual")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan")) - require.NoError(t, taskstop.DisableGlobalSessionStopFailPoint()) tk.Session().SetValue(sessiontxn.AssertRecordsKey, nil) tk.Session().SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index 4d1ff06187ccf..17034f2e778f5 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -8,7 +8,7 @@ go_library( "mocksessionmanager.go", "mockstore.go", "result.go", - "steppedtask.go", + "stepped.go", "testkit.go", ], importpath = "github.com/pingcap/tidb/testkit", @@ -24,9 +24,10 @@ go_library( "//store/mockstore", "//types", "//util", + "//util/breakpoint", "//util/sqlexec", - "//util/taskstop", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", diff --git a/testkit/stepped.go b/testkit/stepped.go index 75b61c9a53ba6..cd4d2497cd198 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -19,8 +19,9 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/util/taskstop" + "github.com/pingcap/tidb/util/breakpoint" "github.com/stretchr/testify/require" ) @@ -160,11 +161,17 @@ func (tk *SteppedTestKit) steppedCommand(cmd steppedTestKitCommand) *SteppedTest go func() { var success bool var result any + var breakPointPaths []string defer func() { - taskstop.DisableSessionStopPoint(tk.tk.Session()) if !success { result = commandInternalError } + + tk.tk.Session().SetValue(breakpoint.NotifyBreakPointFuncKey, nil) + for _, path := range breakPointPaths { + require.NoError(tk.t, failpoint.Disable(path)) + } + require.NoError(tk.t, tk.ch2.sendMsg(msgTpCmdDone, result)) }() @@ -178,7 +185,13 @@ func (tk *SteppedTestKit) steppedCommand(cmd steppedTestKitCommand) *SteppedTest }, } - taskstop.EnableSessionStopPoint(tk.tk.Session(), ctx.notifyBreakPointAndWait, tk.breakPoints...) + tk.tk.Session().SetValue(breakpoint.NotifyBreakPointFuncKey, ctx.notifyBreakPointAndWait) + for _, breakPoint := range tk.breakPoints { + path := "github.com/pingcap/tidb/util/breakpoint/" + breakPoint + require.NoError(tk.t, failpoint.Enable(path, "return")) + breakPointPaths = append(breakPointPaths, path) + } + result = cmd(ctx) success = true }() diff --git a/util/taskstop/BUILD.bazel b/util/breakpoint/BUILD.bazel similarity index 57% rename from util/taskstop/BUILD.bazel rename to util/breakpoint/BUILD.bazel index 4f121f5caad3d..dac0844cad437 100644 --- a/util/taskstop/BUILD.bazel +++ b/util/breakpoint/BUILD.bazel @@ -12,3 +12,15 @@ go_library( "@com_github_pingcap_failpoint//:failpoint", ], ) + +go_library( + name = "breakpoint", + srcs = ["stop.go"], + importpath = "github.com/pingcap/tidb/util/breakpoint", + visibility = ["//visibility:public"], + deps = [ + "//sessionctx", + "//util/stringutil", + "@com_github_pingcap_failpoint//:failpoint", + ], +) diff --git a/util/breakpoint/stop.go b/util/breakpoint/stop.go new file mode 100644 index 0000000000000..6bbfdc17ac44e --- /dev/null +++ b/util/breakpoint/stop.go @@ -0,0 +1,32 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package breakpoint + +import ( + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" +) + +const NotifyBreakPointFuncKey = stringutil.StringerStr("breakPointNotifyFunc") + +func Inject(sctx sessionctx.Context, name string) { + failpoint.Inject(name, func(_ failpoint.Value) { + val := sctx.Value(NotifyBreakPointFuncKey) + if breakPointNotifyAndWaitContinue, ok := val.(func(string)); ok { + breakPointNotifyAndWaitContinue(name) + } + }) +} diff --git a/util/taskstop/stop.go b/util/taskstop/stop.go deleted file mode 100644 index e1807d2951a0c..0000000000000 --- a/util/taskstop/stop.go +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package taskstop - -import ( - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/util/stringutil" -) - -type sessionStopInjection struct { - fn func(string) - breakPoints []string -} - -// EnableSessionStopPoint enables the stop points for a session -func EnableSessionStopPoint(sctx sessionctx.Context, fn func(string), breakPoints ...string) { - sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), &sessionStopInjection{ - fn: fn, - breakPoints: breakPoints, - }) -} - -// DisableSessionStopPoint disables the stop points for a session -func DisableSessionStopPoint(sctx sessionctx.Context) { - sctx.SetValue(stringutil.StringerStr("sessionStopInjection"), nil) -} - -// InjectSessionStopPoint injects a stop point -func InjectSessionStopPoint(sctx sessionctx.Context, breakPoint string) { - failpoint.Inject("sessionStop", func() { - if inject, ok := sctx.Value(stringutil.StringerStr("sessionStopInjection")).(*sessionStopInjection); ok { - for _, p := range inject.breakPoints { - if p == breakPoint { - inject.fn(breakPoint) - break - } - } - } - }) -} - -// EnableGlobalSessionStopFailPoint enables the global session stop fail point -func EnableGlobalSessionStopFailPoint() error { - return failpoint.Enable("github.com/pingcap/tidb/util/taskstop/sessionStop", "return") -} - -// DisableGlobalSessionStopFailPoint disables the global session stop fail point -func DisableGlobalSessionStopFailPoint() error { - return failpoint.Disable("github.com/pingcap/tidb/util/taskstop/sessionStop") -} From 5f9942e2116e217afe1c9cd190739a04896c1b73 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 22 Jun 2022 22:55:13 +0800 Subject: [PATCH 15/21] rename --- util/breakpoint/{stop.go => breakpoint.go} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename util/breakpoint/{stop.go => breakpoint.go} (100%) diff --git a/util/breakpoint/stop.go b/util/breakpoint/breakpoint.go similarity index 100% rename from util/breakpoint/stop.go rename to util/breakpoint/breakpoint.go From 25c1db35fb938a615f435fd7541c49ca5b5d311e Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 22 Jun 2022 23:01:05 +0800 Subject: [PATCH 16/21] rename --- executor/adapter.go | 4 ++-- sessiontxn/failpoint.go | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 099d746e8be64..faf894816c702 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -416,7 +416,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.observeStmtBeginForTopSQL(ctx) - breakpoint.Inject(a.Ctx, sessiontxn.StopPointBeforeExecutorFirstRun) + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointBeforeExecutorFirstRun) if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -793,7 +793,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error if err != nil { return nil, err } - breakpoint.Inject(a.Ctx, sessiontxn.StopPointOnStmtRetryAfterLockError) + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() if err != nil { diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index 39287c2c248ce..d33984649b371 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -35,13 +35,13 @@ var AssertTxnInfoSchemaKey stringutil.StringerStr = "assertTxnInfoSchemaKey" // Only for test var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSchemaAfterRetryKey" -// StopPointBeforeExecutorFirstRun is the key for the stop point where session stops before executor's first run +// BreakPointBeforeExecutorFirstRun is the key for the stop point where session stops before executor's first run // Only for test -var StopPointBeforeExecutorFirstRun = "beforeExecutorFirstRun" +var BreakPointBeforeExecutorFirstRun = "beforeExecutorFirstRun" -// StopPointOnStmtRetryAfterLockError s the key for the stop point where session stops after OnStmtRetry when lock error happens +// BreakPointOnStmtRetryAfterLockError s the key for the stop point where session stops after OnStmtRetry when lock error happens // Only for test -var StopPointOnStmtRetryAfterLockError = "lockErrorAndThenOnStmtRetryCalled" +var BreakPointOnStmtRetryAfterLockError = "lockErrorAndThenOnStmtRetryCalled" // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { From c6af7342e153d6636da08e88c620a74e9c5a1b3d Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 23 Jun 2022 00:02:16 +0800 Subject: [PATCH 17/21] lint --- testkit/stepped.go | 8 +++++--- util/breakpoint/breakpoint.go | 2 ++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/testkit/stepped.go b/testkit/stepped.go index cd4d2497cd198..a0364368b07ef 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -25,7 +25,7 @@ import ( "github.com/stretchr/testify/require" ) -var commandInternalError = errors.New("command error") +var errCommandRunFailed = errors.New("command run failed") type steppedTestkitMsgType int @@ -102,6 +102,7 @@ type SteppedTestKit struct { cmdResult any } +// NewSteppedTestKit creates a new SteppedTestKit func NewSteppedTestKit(t *testing.T, store kv.Storage) *SteppedTestKit { tk := &SteppedTestKit{ t: t, @@ -127,6 +128,7 @@ func (tk *SteppedTestKit) ExpectStopOnAnyBreakPoint() { require.NotEqual(tk.t, "", tk.cmdStopAt) } +// SetBreakPoints sets the break points we want to stop at func (tk *SteppedTestKit) SetBreakPoints(breakPoints []string) { tk.breakPoints = breakPoints } @@ -137,7 +139,7 @@ func (tk *SteppedTestKit) handleCommandMsg() { switch msg.tp { case msgTpCmdDone: tk.cmdStopAt = "" - if msg.val == commandInternalError { + if msg.val == errCommandRunFailed { require.FailNow(tk.t, "internal command failed") } else { tk.cmdResult = msg.val @@ -164,7 +166,7 @@ func (tk *SteppedTestKit) steppedCommand(cmd steppedTestKitCommand) *SteppedTest var breakPointPaths []string defer func() { if !success { - result = commandInternalError + result = errCommandRunFailed } tk.tk.Session().SetValue(breakpoint.NotifyBreakPointFuncKey, nil) diff --git a/util/breakpoint/breakpoint.go b/util/breakpoint/breakpoint.go index 6bbfdc17ac44e..b94a90be02954 100644 --- a/util/breakpoint/breakpoint.go +++ b/util/breakpoint/breakpoint.go @@ -20,8 +20,10 @@ import ( "github.com/pingcap/tidb/util/stringutil" ) +// NotifyBreakPointFuncKey is the key where break point notify function located const NotifyBreakPointFuncKey = stringutil.StringerStr("breakPointNotifyFunc") +// Inject injects a break point to a session func Inject(sctx sessionctx.Context, name string) { failpoint.Inject(name, func(_ failpoint.Value) { val := sctx.Value(NotifyBreakPointFuncKey) From 4304e97520c8fb0e607a2e83f15cbe6c1afd43a6 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 23 Jun 2022 10:39:50 +0800 Subject: [PATCH 18/21] update --- sessiontxn/txn_context_test.go | 10 +++++----- util/breakpoint/BUILD.bazel | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 29d8157a2cfad..3315694ef8e04 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -747,8 +747,8 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { } tk2.SetBreakPoints([]string{ - sessiontxn.StopPointBeforeExecutorFirstRun, - sessiontxn.StopPointOnStmtRetryAfterLockError, + sessiontxn.BreakPointBeforeExecutorFirstRun, + sessiontxn.BreakPointOnStmtRetryAfterLockError, }) var isSelect, isUpdate bool @@ -764,16 +764,16 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { } // Pause the session before the executor first run and then update the record in another session - tk2.ExpectStopOnBreakPoint(sessiontxn.StopPointBeforeExecutorFirstRun) + tk2.ExpectStopOnBreakPoint(sessiontxn.BreakPointBeforeExecutorFirstRun) tk.MustExec("update t1 set v=v+1") // Session continues, it should get a lock error and retry, we pause the session before the executor's next run // and then update the record in another session again. - tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.StopPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) tk.MustExec("update t1 set v=v+1") // Because the record is updated by another session again, when this session continues, it will get a lock error again. - tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.StopPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) tk2.Continue().ExpectIdle() switch { case isSelect: diff --git a/util/breakpoint/BUILD.bazel b/util/breakpoint/BUILD.bazel index dac0844cad437..8575643c2f183 100644 --- a/util/breakpoint/BUILD.bazel +++ b/util/breakpoint/BUILD.bazel @@ -15,7 +15,7 @@ go_library( go_library( name = "breakpoint", - srcs = ["stop.go"], + srcs = ["breakpoint.go"], importpath = "github.com/pingcap/tidb/util/breakpoint", visibility = ["//visibility:public"], deps = [ From 44911243cc66a40c68edd82a14bde677b72c1365 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 23 Jun 2022 11:10:42 +0800 Subject: [PATCH 19/21] update bazel --- util/breakpoint/BUILD.bazel | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/util/breakpoint/BUILD.bazel b/util/breakpoint/BUILD.bazel index 8575643c2f183..ecc1f6e7f6e03 100644 --- a/util/breakpoint/BUILD.bazel +++ b/util/breakpoint/BUILD.bazel @@ -1,18 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -go_library( - name = "taskstop", - srcs = ["stop.go"], - importpath = "github.com/pingcap/tidb/util/taskstop", - visibility = ["//visibility:public"], - deps = [ - "//sessionctx", - "//util/stringutil", - "@com_github_pingcap_errors//:errors", - "@com_github_pingcap_failpoint//:failpoint", - ], -) - go_library( name = "breakpoint", srcs = ["breakpoint.go"], From d6b41509120ed18ba8e13f77c1814fbb3ebb844a Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 23 Jun 2022 16:58:00 +0800 Subject: [PATCH 20/21] update SetBreakPoints --- sessiontxn/txn_context_test.go | 4 ++-- testkit/stepped.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 3315694ef8e04..d3e1f32124575 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -746,10 +746,10 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { tk2.MustExec("set autocommit=0") } - tk2.SetBreakPoints([]string{ + tk2.SetBreakPoints( sessiontxn.BreakPointBeforeExecutorFirstRun, sessiontxn.BreakPointOnStmtRetryAfterLockError, - }) + ) var isSelect, isUpdate bool switch { diff --git a/testkit/stepped.go b/testkit/stepped.go index a0364368b07ef..36f488eb57351 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -129,7 +129,7 @@ func (tk *SteppedTestKit) ExpectStopOnAnyBreakPoint() { } // SetBreakPoints sets the break points we want to stop at -func (tk *SteppedTestKit) SetBreakPoints(breakPoints []string) { +func (tk *SteppedTestKit) SetBreakPoints(breakPoints ...string) { tk.breakPoints = breakPoints } From 4dd804e45857237c3cb350ae651854575738dc92 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 23 Jun 2022 17:23:12 +0800 Subject: [PATCH 21/21] consts --- testkit/stepped.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/testkit/stepped.go b/testkit/stepped.go index 36f488eb57351..d63a2a1efc7b8 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -27,6 +27,8 @@ import ( var errCommandRunFailed = errors.New("command run failed") +var defaultChanTimeout = time.Second * 10 + type steppedTestkitMsgType int const ( @@ -46,7 +48,7 @@ func (ch steppedTestKitMsgChan) sendMsg(tp steppedTestkitMsgType, val any) error select { case ch <- &steppedTestKitMsg{tp: tp, val: val}: return nil - case <-time.After(time.Second * 10): + case <-time.After(defaultChanTimeout): return errors.New("send msg timeout") } } @@ -59,7 +61,7 @@ func (ch steppedTestKitMsgChan) recvMsg() (*steppedTestKitMsg, error) { select { case msg := <-ch: return msg, nil - case <-time.After(time.Second * 10): + case <-time.After(defaultChanTimeout): return nil, errors.New("send msg timeout") } }