diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 324d39bc469ec..807f65715202b 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -143,6 +143,7 @@ go_test( "integration_test.go", "main_test.go", "modify_column_test.go", + "multi_schema_change_test.go", "options_test.go", "partition_test.go", "placement_policy_ddl_test.go", diff --git a/sessiontxn/BUILD.bazel b/sessiontxn/BUILD.bazel index 922f80480ac34..bdb84de22657f 100644 --- a/sessiontxn/BUILD.bazel +++ b/sessiontxn/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "//sessionctx", "//sessiontxn/staleread", "//testkit", + "//testkit/testfork", "//testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index e247a14c86f2b..9fec1844606a3 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testfork" "github.com/pingcap/tidb/testkit/testsetup" "github.com/stretchr/testify/require" "go.uber.org/goleak" @@ -725,65 +726,59 @@ 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) { - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - tk2 := testkit.NewSteppedTestKit(t, store) - defer 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") - } - - tk2.SetBreakPoints( - sessiontxn.BreakPointBeforeExecutorFirstRun, - sessiontxn.BreakPointOnStmtRetryAfterLockError, - ) - - var isSelect, isUpdate bool - switch { - case strings.HasPrefix(query, "select"): - isSelect = true - tk2.SteppedMustQuery(query) - case strings.HasPrefix(query, "update"): - isUpdate = true - 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 - 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.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.BreakPointOnStmtRetryAfterLockError) - tk2.Continue().ExpectIdle() - switch { - case isSelect: - tk2.GetQueryResult().Check(testkit.Rows("1 12")) - case isUpdate: - tk2.MustExec("commit") - tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - } - }) - } + testfork.RunTest(t, func(t *testfork.T) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + tk2 := testkit.NewSteppedTestKit(t, store) + defer tk2.MustExec("rollback") + + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", testfork.PickEnum(t, ast.RepeatableRead, ast.ReadCommitted))) + autocommit := testfork.PickEnum(t, 0, 1) + tk2.MustExec(fmt.Sprintf("set autocommit=%d", autocommit)) + if autocommit == 1 { + tk2.MustExec("begin") } - } + + tk2.SetBreakPoints( + sessiontxn.BreakPointBeforeExecutorFirstRun, + sessiontxn.BreakPointOnStmtRetryAfterLockError, + ) + + var isSelect, isUpdate bool + query := testfork.Pick(t, queries) + switch { + case strings.HasPrefix(query, "select"): + isSelect = true + tk2.SteppedMustQuery(query) + case strings.HasPrefix(query, "update"): + isUpdate = true + 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 + 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.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.BreakPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectIdle() + switch { + case isSelect: + 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 index d63a2a1efc7b8..fc6e434ef85b9 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -80,7 +80,7 @@ func (ch steppedTestKitMsgChan) recvMsgWithCheck(tp steppedTestkitMsgType) (*ste } type steppedTestKitCommandContext struct { - t *testing.T + t testing.TB tk *TestKit notifyBreakPointAndWait func(string) } @@ -89,7 +89,7 @@ type steppedTestKitCommand func(ctx *steppedTestKitCommandContext) any // SteppedTestKit is the testkit that can run stepped command type SteppedTestKit struct { - t *testing.T + t testing.TB tk *TestKit // ch1 is used to send msg from foreground to background @@ -105,7 +105,7 @@ type SteppedTestKit struct { } // NewSteppedTestKit creates a new SteppedTestKit -func NewSteppedTestKit(t *testing.T, store kv.Storage) *SteppedTestKit { +func NewSteppedTestKit(t testing.TB, store kv.Storage) *SteppedTestKit { tk := &SteppedTestKit{ t: t, tk: NewTestKit(t, store), diff --git a/testkit/testfork/BUILD.bazel b/testkit/testfork/BUILD.bazel new file mode 100644 index 0000000000000..743bd70da5b0d --- /dev/null +++ b/testkit/testfork/BUILD.bazel @@ -0,0 +1,18 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "testfork", + srcs = ["fork.go"], + importpath = "github.com/pingcap/tidb/testkit/testfork", + visibility = ["//visibility:public"], + deps = [ + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) + +go_test( + name = "testfork_test", + srcs = ["fork_test.go"], + embed = [":testfork"], +) diff --git a/testkit/testfork/fork.go b/testkit/testfork/fork.go new file mode 100644 index 0000000000000..aa70039b2f042 --- /dev/null +++ b/testkit/testfork/fork.go @@ -0,0 +1,135 @@ +// 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 testfork + +import ( + "fmt" + "os" + "strings" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type pickStack struct { + stack [][]any + pos int + valid bool +} + +func newPickStack() *pickStack { + return &pickStack{ + valid: true, + } +} + +func (s *pickStack) NextStack() { + for len(s.stack) > 0 { + lastIndex := len(s.stack) - 1 + s.stack[lastIndex] = s.stack[lastIndex][1:] + if len(s.stack[lastIndex]) > 0 { + break + } + s.stack = s.stack[:lastIndex] + } + + s.pos = 0 + s.valid = len(s.stack) > 0 +} + +func (s *pickStack) PickValue(values []any) (any, error) { + if len(values) == 0 { + return nil, errors.New("values should not be empty") + } + + stackLen := len(s.stack) + if s.pos > stackLen { + return nil, errors.Newf("illegal state %d > %d", s.pos, stackLen) + } + + defer func() { + s.pos++ + }() + + if s.pos == stackLen { + s.stack = append(s.stack, values) + } + return s.stack[s.pos][0], nil +} + +func (s *pickStack) Values() []any { + values := make([]any, 0) + for _, v := range s.stack { + values = append(values, v[0]) + } + return values +} + +func (s *pickStack) ValuesText() string { + values := s.Values() + strValues := make([]string, len(values)) + for i, value := range values { + switch v := value.(type) { + case string: + strValues[i] = fmt.Sprintf(`"%s"`, v) + default: + strValues[i] = fmt.Sprintf("%v", v) + } + } + return "[" + strings.Join(strValues, " ") + "]" +} + +func (s *pickStack) Valid() bool { + return s.valid +} + +// T is used by for test +type T struct { + *testing.T + stack *pickStack +} + +// RunTest runs the test function `f` multiple times util all the values in `Pick` are tested. +func RunTest(t *testing.T, f func(t *T)) { + idx := 0 + for stack := newPickStack(); stack.Valid(); stack.NextStack() { + success := t.Run("", func(t *testing.T) { + f(&T{T: t, stack: stack}) + }) + + if !success { + _, err := fmt.Fprintf(os.Stderr, "SubTest #%v failed, failed values: %s\n", idx, stack.ValuesText()) + require.NoError(t, err) + } + idx++ + } +} + +// Pick returns a value from the values list +func Pick[E any](t *T, values []E) E { + slice := make([]any, len(values)) + for i, item := range values { + slice[i] = item + } + value, err := t.stack.PickValue(slice) + require.NoError(t, err) + return value.(E) +} + +// PickEnum returns a value from the value enums +func PickEnum[E any](t *T, item E, other ...E) E { + return Pick(t, append([]E{item}, other...)) +} diff --git a/testkit/testfork/fork_test.go b/testkit/testfork/fork_test.go new file mode 100644 index 0000000000000..64ed11b7a82e5 --- /dev/null +++ b/testkit/testfork/fork_test.go @@ -0,0 +1,50 @@ +// 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 testfork + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestForkSubTest(t *testing.T) { + var values [][]any + RunTest(t, func(t *T) { + x := Pick(t, []int{1, 2, 3}) + y := PickEnum(t, "a", "b") + var z any + if x == 2 { + z = PickEnum(t, 10, 11) + } else { + z = Pick(t, []string{"g", "h"}) + } + values = append(values, []any{x, y, z}) + }) + require.Equal(t, [][]any{ + {1, "a", "g"}, + {1, "a", "h"}, + {1, "b", "g"}, + {1, "b", "h"}, + {2, "a", 10}, + {2, "a", 11}, + {2, "b", 10}, + {2, "b", 11}, + {3, "a", "g"}, + {3, "a", "h"}, + {3, "b", "g"}, + {3, "b", "h"}, + }, values) +} diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index 95c8ae77a9646..b0e5afb4dc0fd 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//kv", "//meta/autoid", "//parser", + "//parser/ast", "//parser/auth", "//parser/format", "//parser/model",