diff --git a/ddl/callback.go b/ddl/callback.go index b6160d150e717..7fc82467acd1a 100644 --- a/ddl/callback.go +++ b/ddl/callback.go @@ -18,6 +18,7 @@ import ( "context" "fmt" "strings" + "sync/atomic" "time" "github.com/pingcap/errors" @@ -142,6 +143,119 @@ func newDefaultCallBack(do DomainReloader) Callback { // ****************************** End of Default DDL Callback Instance ********************************************* +// ****************************** Start of Test DDL Callback Instance *********************************************** + +// TestDDLCallback is used to customize user callback themselves. +type TestDDLCallback struct { + *BaseCallback + // We recommended to pass the domain parameter to the test ddl callback, it will ensure + // domain to reload schema before your ddl stepping into the next state change. + Do DomainReloader + + onJobRunBefore func(*model.Job) + OnJobRunBeforeExported func(*model.Job) + onJobUpdated func(*model.Job) + OnJobUpdatedExported atomic.Pointer[func(*model.Job)] + onWatched func(ctx context.Context) + OnGetJobBeforeExported func(string) + OnGetJobAfterExported func(string, *model.Job) + OnJobSchemaStateChanged func(int64) +} + +// OnChanged mock the same behavior with the main DDL hook. +func (tc *TestDDLCallback) OnChanged(err error) error { + if err != nil { + return err + } + logutil.BgLogger().Info("performing DDL change, must reload") + if tc.Do != nil { + err = tc.Do.Reload() + if err != nil { + logutil.BgLogger().Error("performing DDL change failed", zap.Error(err)) + } + } + return nil +} + +// OnSchemaStateChanged mock the same behavior with the main ddl hook. +func (tc *TestDDLCallback) OnSchemaStateChanged(schemaVer int64) { + if tc.Do != nil { + if err := tc.Do.Reload(); err != nil { + logutil.BgLogger().Warn("reload failed on schema state changed", zap.Error(err)) + } + } + + if tc.OnJobSchemaStateChanged != nil { + tc.OnJobSchemaStateChanged(schemaVer) + return + } +} + +// OnJobRunBefore is used to run the user customized logic of `onJobRunBefore` first. +func (tc *TestDDLCallback) OnJobRunBefore(job *model.Job) { + logutil.BgLogger().Info("on job run before", zap.String("job", job.String())) + if tc.OnJobRunBeforeExported != nil { + tc.OnJobRunBeforeExported(job) + return + } + if tc.onJobRunBefore != nil { + tc.onJobRunBefore(job) + return + } + + tc.BaseCallback.OnJobRunBefore(job) +} + +// OnJobUpdated is used to run the user customized logic of `OnJobUpdated` first. +func (tc *TestDDLCallback) OnJobUpdated(job *model.Job) { + logutil.BgLogger().Info("on job updated", zap.String("job", job.String())) + if onJobUpdatedExportedFunc := tc.OnJobUpdatedExported.Load(); onJobUpdatedExportedFunc != nil { + (*onJobUpdatedExportedFunc)(job) + return + } + if tc.onJobUpdated != nil { + tc.onJobUpdated(job) + return + } + + tc.BaseCallback.OnJobUpdated(job) +} + +// OnWatched is used to run the user customized logic of `OnWatched` first. +func (tc *TestDDLCallback) OnWatched(ctx context.Context) { + if tc.onWatched != nil { + tc.onWatched(ctx) + return + } + + tc.BaseCallback.OnWatched(ctx) +} + +// OnGetJobBefore implements Callback.OnGetJobBefore interface. +func (tc *TestDDLCallback) OnGetJobBefore(jobType string) { + if tc.OnGetJobBeforeExported != nil { + tc.OnGetJobBeforeExported(jobType) + return + } + tc.BaseCallback.OnGetJobBefore(jobType) +} + +// OnGetJobAfter implements Callback.OnGetJobAfter interface. +func (tc *TestDDLCallback) OnGetJobAfter(jobType string, job *model.Job) { + if tc.OnGetJobAfterExported != nil { + tc.OnGetJobAfterExported(jobType, job) + return + } + tc.BaseCallback.OnGetJobAfter(jobType, job) +} + +// Clone copies the callback and take its reference +func (tc *TestDDLCallback) Clone() *TestDDLCallback { + return &*tc +} + +// ****************************** End of Test DDL Callback Instance *********************************************** + // ****************************** Start of CTC DDL Callback Instance *********************************************** // ctcCallback is the customized callback that TiDB will use. diff --git a/ddl/callback_test.go b/ddl/callback_test.go index 5a97e8212689e..21985a924ba2b 100644 --- a/ddl/callback_test.go +++ b/ddl/callback_test.go @@ -16,15 +16,11 @@ package ddl import ( "context" - "sync/atomic" "testing" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/util/logutil" "github.com/stretchr/testify/require" - "go.uber.org/zap" ) type TestInterceptor struct { @@ -41,115 +37,6 @@ func (ti *TestInterceptor) OnGetInfoSchema(ctx sessionctx.Context, is infoschema return ti.BaseInterceptor.OnGetInfoSchema(ctx, is) } -// TestDDLCallback is used to customize user callback themselves. -type TestDDLCallback struct { - *BaseCallback - // We recommended to pass the domain parameter to the test ddl callback, it will ensure - // domain to reload schema before your ddl stepping into the next state change. - Do DomainReloader - - onJobRunBefore func(*model.Job) - OnJobRunBeforeExported func(*model.Job) - onJobUpdated func(*model.Job) - OnJobUpdatedExported atomic.Pointer[func(*model.Job)] - onWatched func(ctx context.Context) - OnGetJobBeforeExported func(string) - OnGetJobAfterExported func(string, *model.Job) - OnJobSchemaStateChanged func(int64) -} - -// OnChanged mock the same behavior with the main DDL hook. -func (tc *TestDDLCallback) OnChanged(err error) error { - if err != nil { - return err - } - logutil.BgLogger().Info("performing DDL change, must reload") - if tc.Do != nil { - err = tc.Do.Reload() - if err != nil { - logutil.BgLogger().Error("performing DDL change failed", zap.Error(err)) - } - } - return nil -} - -// OnSchemaStateChanged mock the same behavior with the main ddl hook. -func (tc *TestDDLCallback) OnSchemaStateChanged(schemaVer int64) { - if tc.Do != nil { - if err := tc.Do.Reload(); err != nil { - logutil.BgLogger().Warn("reload failed on schema state changed", zap.Error(err)) - } - } - - if tc.OnJobSchemaStateChanged != nil { - tc.OnJobSchemaStateChanged(schemaVer) - return - } -} - -// OnJobRunBefore is used to run the user customized logic of `onJobRunBefore` first. -func (tc *TestDDLCallback) OnJobRunBefore(job *model.Job) { - logutil.BgLogger().Info("on job run before", zap.String("job", job.String())) - if tc.OnJobRunBeforeExported != nil { - tc.OnJobRunBeforeExported(job) - return - } - if tc.onJobRunBefore != nil { - tc.onJobRunBefore(job) - return - } - - tc.BaseCallback.OnJobRunBefore(job) -} - -// OnJobUpdated is used to run the user customized logic of `OnJobUpdated` first. -func (tc *TestDDLCallback) OnJobUpdated(job *model.Job) { - logutil.BgLogger().Info("on job updated", zap.String("job", job.String())) - if onJobUpdatedExportedFunc := tc.OnJobUpdatedExported.Load(); onJobUpdatedExportedFunc != nil { - (*onJobUpdatedExportedFunc)(job) - return - } - if tc.onJobUpdated != nil { - tc.onJobUpdated(job) - return - } - - tc.BaseCallback.OnJobUpdated(job) -} - -// OnWatched is used to run the user customized logic of `OnWatched` first. -func (tc *TestDDLCallback) OnWatched(ctx context.Context) { - if tc.onWatched != nil { - tc.onWatched(ctx) - return - } - - tc.BaseCallback.OnWatched(ctx) -} - -// OnGetJobBefore implements Callback.OnGetJobBefore interface. -func (tc *TestDDLCallback) OnGetJobBefore(jobType string) { - if tc.OnGetJobBeforeExported != nil { - tc.OnGetJobBeforeExported(jobType) - return - } - tc.BaseCallback.OnGetJobBefore(jobType) -} - -// OnGetJobAfter implements Callback.OnGetJobAfter interface. -func (tc *TestDDLCallback) OnGetJobAfter(jobType string, job *model.Job) { - if tc.OnGetJobAfterExported != nil { - tc.OnGetJobAfterExported(jobType, job) - return - } - tc.BaseCallback.OnGetJobAfter(jobType, job) -} - -// Clone copies the callback and take its reference -func (tc *TestDDLCallback) Clone() *TestDDLCallback { - return &*tc -} - func TestCallback(t *testing.T) { cb := &BaseCallback{} require.Nil(t, cb.OnChanged(nil)) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index e7bf4d09078c2..64a583ca0ab8d 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -79,7 +79,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral // step 3: add metadata lock and check each table's schema version schemaNotMatch := false for i := 0; i < len(stmt.dbName); i++ { - _, ok := is.TableByID(stmt.tbls[i].Meta().ID) + tbl, ok := is.TableByID(stmt.tbls[i].Meta().ID) if !ok { tblByName, err := is.TableByName(stmt.dbName[i], stmt.tbls[i].Meta().Name) if err != nil { @@ -94,7 +94,12 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral schemaNotMatch = true continue } - if stmt.tbls[i].Meta().Revision != newTbl.Meta().Revision { + // The revision of tbl and newTbl may not be the same. + // Example: + // The version of stmt.tbls[i] is taken from the prepare statement and is revision v1. + // When stmt.tbls[i] is locked in MDL, the revision of newTbl is also v1. + // The revision of tbl is v2. The reason may have other statements trigger "tryLockMDLAndUpdateSchemaIfNecessary" before, leading to tbl revision update. + if stmt.tbls[i].Meta().Revision != newTbl.Meta().Revision || (tbl != nil && tbl.Meta().Revision != newTbl.Meta().Revision) { schemaNotMatch = true } stmt.tbls[i] = newTbl diff --git a/server/server_test.go b/server/server_test.go index 202b6fe0609c8..98555313784c5 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -16,6 +16,7 @@ package server import ( "bytes" + "context" "database/sql" "encoding/json" "fmt" @@ -36,9 +37,13 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" tmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/versioninfo" "github.com/stretchr/testify/require" @@ -2660,3 +2665,197 @@ func (cli *testServerClient) RunTestStmtCountLimit(t *testing.T) { require.Equal(t, 5, count) }) } + +func (cli *tidbTestSuite) getNewDB(t *testing.T, overrider configOverrider) *testkit.DBTestKit { + db, err := sql.Open("mysql", cli.getDSN(overrider)) + require.NoError(t, err) + + return testkit.NewDBTestKit(t, db) +} + +func MustExec(ctx context.Context, t *testing.T, conn *sql.Conn, sql string) { + _, err := conn.QueryContext(ctx, sql) + require.NoError(t, err) +} + +func MustQueryWithRetry(ctx context.Context, t *testing.T, cli *tidbTestSuite, conn *sql.Conn, stmt string) { + var rs *sql.Rows + var err error + retryCnt := 1 + for i := 0; i < 20; i++ { + rs, err = conn.QueryContext(ctx, stmt) + if err == nil { + break + } + if !strings.Contains(err.Error(), "Information schema is changed") { + break + } + retryCnt++ + time.Sleep(100 * time.Millisecond) + } + t.Logf("running test case retry count:%v, stmt:%v", retryCnt, stmt) + require.NoError(t, err) + if rs != nil { + cli.Rows(t, rs) + rs.Close() + } +} + +type sqlWithErr struct { + stmt *sql.Stmt + sql string +} + +type expectQuery struct { + sql string + rows []string +} + +func (cli *testServerClient) runTestIssue53634(t *testing.T, ts *tidbTestSuite, dom *domain.Domain) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.MaxAllowedPacket = 1024 + }, "MDL", func(dbt *testkit.DBTestKit) { + ctx := context.Background() + + conn, err := dbt.GetDB().Conn(ctx) + require.NoError(t, err) + MustExec(ctx, t, conn, "set global tidb_enable_metadata_lock=1") + t.Logf("enable mdl:%v", variable.EnableMDL.Load()) + MustExec(ctx, t, conn, "create database test_db_state default charset utf8 default collate utf8_bin") + MustExec(ctx, t, conn, "use test_db_state") + MustExec(ctx, t, conn, `CREATE TABLE stock ( + a int NOT NULL, + b char(30) NOT NULL, + c int, + d char(64), + PRIMARY KEY(a,b) +) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin COMMENT='…comment'; +`) + MustExec(ctx, t, conn, "insert into stock values(1, 'a', 11, 'x'), (2, 'b', 22, 'y')") + MustExec(ctx, t, conn, "alter table stock add column cct_1 int default 10") + MustExec(ctx, t, conn, "alter table stock modify cct_1 json") + MustExec(ctx, t, conn, "alter table stock add column adc_1 smallint") + defer MustExec(ctx, t, conn, "drop database test_db_state") + + sqls := make([]sqlWithErr, 5) + sqls[0] = sqlWithErr{nil, "begin"} + sqls[1] = sqlWithErr{nil, "SELECT a, c, d from stock where (a, b) IN ((?, ?),(?, ?)) FOR UPDATE"} + sqls[2] = sqlWithErr{nil, "UPDATE stock SET c = ? WHERE a= ? AND b = 'a'"} + sqls[3] = sqlWithErr{nil, "UPDATE stock SET c = ?, d = 'z' WHERE a= ? AND b = 'b'"} + sqls[4] = sqlWithErr{nil, "commit"} + dropColumnSQL := "alter table stock drop column cct_1" + query := &expectQuery{sql: "select * from stock;", rows: []string{"1 a 101 x \n2 b 102 z "}} + runTestInSchemaState(t, conn, ts, dom, model.StateWriteReorganization, true, dropColumnSQL, sqls, query) + }) +} + +func runTestInSchemaState( + t *testing.T, + conn *sql.Conn, + cli *tidbTestSuite, + dom *domain.Domain, + state model.SchemaState, + isOnJobUpdated bool, + dropColumnSQL string, + sqlWithErrs []sqlWithErr, + expectQuery *expectQuery, +) { + ctx := context.Background() + MustExec(ctx, t, conn, "use test_db_state") + + callback := &ddl.TestDDLCallback{Do: dom} + prevState := model.StateNone + var checkErr error + dbt := cli.getNewDB(t, func(config *mysql.Config) { + config.MaxAllowedPacket = 1024 + }) + conn1, err := dbt.GetDB().Conn(ctx) + require.NoError(t, err) + defer func() { + err := dbt.GetDB().Close() + require.NoError(t, err) + }() + MustExec(ctx, t, conn1, "use test_db_state") + + for i, sqlWithErr := range sqlWithErrs { + // Start the test txn. + // Step 1: begin(when i = 0). + if i == 0 || i == len(sqlWithErrs)-1 { + sqlWithErr := sqlWithErrs[i] + MustExec(ctx, t, conn1, sqlWithErr.sql) + } else { + // Step 2: prepare stmts. + // SELECT a, c, d from stock where (a, b) IN ((?, ?),(?, ?)) FOR UPDATE + // UPDATE stock SET c = ? WHERE a= ? AND b = 'a' + // UPDATE stock SET c = ?, d = 'z' WHERE a= ? AND b = 'b' + stmt, err := conn1.PrepareContext(ctx, sqlWithErr.sql) + require.NoError(t, err) + sqlWithErr.stmt = stmt + sqlWithErrs[i] = sqlWithErr + } + } + + // Step 3: begin. + sqlWithErr := sqlWithErrs[0] + MustExec(ctx, t, conn1, sqlWithErr.sql) + + prevState = model.StateNone + state = model.StateWriteOnly + cbFunc1 := func(job *model.Job) { + if jobStateOrLastSubJobState(job) == prevState || checkErr != nil { + return + } + prevState = jobStateOrLastSubJobState(job) + if prevState != state { + return + } + // Step 4: exec stmts in write-only state (dropping a colum). + // SELECT a, c, d from stock where (a, b) IN ((?, ?),(?, ?)) FOR UPDATE, args:(1,"a"),(2,"b") + // UPDATE stock SET c = ? WHERE a= ? AND b = 'a', args:(100+1, 1) + // UPDATE stock SET c = ?, d = 'z' WHERE a= ? AND b = 'b', args:(100+2, 2) + // commit. + sqls := sqlWithErrs[1:] + for i, sqlWithErr := range sqls { + if i == 0 { + _, err = sqlWithErr.stmt.ExecContext(ctx, 1, "a", 2, "b") + require.NoError(t, err) + } else if i == 1 || i == 2 { + _, err = sqlWithErr.stmt.ExecContext(ctx, 100+i, i) + require.NoError(t, err) + } else { + MustQueryWithRetry(ctx, t, cli, conn1, sqlWithErr.sql) + } + } + } + if isOnJobUpdated { + callback.OnJobUpdatedExported.Store(&cbFunc1) + } else { + callback.OnJobRunBeforeExported = cbFunc1 + } + d := dom.DDL() + originalCallback := d.GetHook() + d.SetHook(callback) + MustExec(ctx, t, conn, dropColumnSQL) + require.NoError(t, checkErr) + + // Check the result. + // select * from stock + if expectQuery != nil { + rs, err := conn.QueryContext(ctx, expectQuery.sql) + require.NoError(t, err) + if expectQuery.rows == nil { + require.Nil(t, rs) + } else { + cli.checkRows(t, rs, expectQuery.rows[0]) + } + } + d.SetHook(originalCallback) +} + +func jobStateOrLastSubJobState(job *model.Job) model.SchemaState { + if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { + subs := job.MultiSchemaInfo.SubJobs + return subs[len(subs)-1].SchemaState + } + return job.SchemaState +} diff --git a/server/tidb_test.go b/server/tidb_test.go index 259942d929a52..18c5b47b8ce50 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -83,6 +83,30 @@ type tidbTestSuite struct { } func createTidbTestSuite(t *testing.T) *tidbTestSuite { +<<<<<<< HEAD +======= + cfg := newTestConfig() + cfg.Port = 0 + cfg.Status.ReportStatus = true + cfg.Status.StatusPort = 0 + cfg.Performance.TCPKeepAlive = true + return createTidbTestSuiteWithCfg(t, cfg) +} + +// parseDuration parses lease argument string. +func parseDuration(lease string) (time.Duration, error) { + dur, err := time.ParseDuration(lease) + if err != nil { + dur, err = time.ParseDuration(lease + "s") + } + if err != nil || dur < 0 { + return 0, errors.Errorf("invalid lease duration: %v", lease) + } + return dur, nil +} + +func createTidbTestSuiteWithCfg(t *testing.T, cfg *config.Config) *tidbTestSuite { +>>>>>>> 996098ec4ae (*: fix a bug that update statement uses point get and update plan with different tblInfo (#54183) (#54259)) ts := &tidbTestSuite{testServerClient: newTestServerClient()} // setup tidbTestSuite @@ -90,6 +114,9 @@ func createTidbTestSuite(t *testing.T) *tidbTestSuite { ts.store, err = mockstore.NewMockStore() session.DisableStats4Test() require.NoError(t, err) + ddlLeaseDuration, err := parseDuration(cfg.Lease) + require.NoError(t, err) + session.SetSchemaLease(ddlLeaseDuration) ts.domain, err = session.BootstrapSession(ts.store) require.NoError(t, err) ts.tidbdrv = NewTiDBDriver(ts.store) @@ -3188,7 +3215,7 @@ func TestProxyProtocolWithIpFallbackable(t *testing.T) { func TestProxyProtocolWithIpNoFallbackable(t *testing.T) { cfg := newTestConfig() - cfg.Port = 4000 + cfg.Port = 4005 cfg.Status.ReportStatus = false // Setup proxy protocol config cfg.ProxyProtocol.Networks = "*" @@ -3229,3 +3256,95 @@ func TestLoadData(t *testing.T) { ts.runTestLoadDataReplace(t) ts.runTestLoadDataReplaceNonclusteredPK(t) } +<<<<<<< HEAD +======= + +func TestAuthSocket(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/server/MockOSUserForAuthSocket", "return(true)")) + defer func() { + mockOSUserForAuthSocketTest.Store(nil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/server/MockOSUserForAuthSocket")) + }() + + cfg := newTestConfig() + cfg.Socket = filepath.Join(t.TempDir(), "authsock.sock") + cfg.Port = 0 + cfg.Status.StatusPort = 0 + ts := createTidbTestSuiteWithCfg(t, cfg) + ts.waitUntilServerCanConnect() + + ts.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec("CREATE USER 'u1'@'%' IDENTIFIED WITH auth_socket;") + dbt.MustExec("CREATE USER 'u2'@'%' IDENTIFIED WITH auth_socket AS 'sockuser'") + dbt.MustExec("CREATE USER 'sockuser'@'%' IDENTIFIED WITH auth_socket;") + }) + + // network login should be denied + for _, uname := range []string{"u1", "u2", "u3"} { + mockOSUserForAuthSocketTest.Store(&uname) + db, err := sql.Open("mysql", ts.getDSN(func(config *mysql.Config) { + config.User = uname + })) + require.NoError(t, err) + _, err = db.Conn(context.TODO()) + require.EqualError(t, + err, + fmt.Sprintf("Error 1045: Access denied for user '%s'@'127.0.0.1' (using password: NO)", uname), + ) + require.NoError(t, db.Close()) + } + + socketAuthConf := func(user string) func(*mysql.Config) { + return func(config *mysql.Config) { + config.User = user + config.Net = "unix" + config.Addr = cfg.Socket + config.DBName = "" + } + } + + mockOSUser := "sockuser" + mockOSUserForAuthSocketTest.Store(&mockOSUser) + + // mysql username that is different with the OS user should be rejected. + db, err := sql.Open("mysql", ts.getDSN(socketAuthConf("u1"))) + require.NoError(t, err) + _, err = db.Conn(context.TODO()) + require.EqualError(t, err, "Error 1045: Access denied for user 'u1'@'localhost' (using password: YES)") + require.NoError(t, db.Close()) + + // mysql username that is the same with the OS user should be accepted. + ts.runTests(t, socketAuthConf("sockuser"), func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select current_user();") + ts.checkRows(t, rows, "sockuser@%") + }) + + // When a user is created with `IDENTIFIED WITH auth_socket AS ...`. + // It should be accepted when username or as string is the same with OS user. + ts.runTests(t, socketAuthConf("u2"), func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select current_user();") + ts.checkRows(t, rows, "u2@%") + }) + + mockOSUser = "u2" + mockOSUserForAuthSocketTest.Store(&mockOSUser) + ts.runTests(t, socketAuthConf("u2"), func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select current_user();") + ts.checkRows(t, rows, "u2@%") + }) +} + +func TestIssue53634(t *testing.T) { + if !variable.DefTiDBEnableConcurrentDDL { + t.Skip("skip this mdl test when DefTiDBEnableConcurrentDDL is false") + } + + cfg := newTestConfig() + cfg.Lease = "20s" + cfg.Port = 4123 + cfg.Status.StatusPort = 10088 + ts := createTidbTestSuiteWithCfg(t, cfg) + + ts.runTestIssue53634(t, ts, ts.domain) +} +>>>>>>> 996098ec4ae (*: fix a bug that update statement uses point get and update plan with different tblInfo (#54183) (#54259))