diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 3ccb33749d004..d691eb00e52f3 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -137,7 +137,7 @@ go_library( go_test( name = "ddl_test", - timeout = "moderate", + timeout = "long", srcs = [ "attributes_sql_test.go", "callback_test.go", diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index 926ed6b13cdb3..7780104d9768d 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -1041,6 +1041,7 @@ func TestWriteReorgForColumnTypeChangeOnAmendTxn(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, columnModifyLease) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("set global tidb_enable_amend_pessimistic_txn = ON") defer tk.MustExec("set global tidb_enable_amend_pessimistic_txn = OFF") diff --git a/ddl/concurrentddltest/BUILD.bazel b/ddl/concurrentddltest/BUILD.bazel new file mode 100644 index 0000000000000..c06a49479286b --- /dev/null +++ b/ddl/concurrentddltest/BUILD.bazel @@ -0,0 +1,23 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "concurrentddltest_test", + timeout = "short", + srcs = [ + "main_test.go", + "switch_test.go", + ], + flaky = True, + deps = [ + "//config", + "//ddl", + "//kv", + "//meta", + "//testkit", + "//testkit/testsetup", + "//util", + "@com_github_stretchr_testify//require", + "@org_uber_go_atomic//:atomic", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/ddl/concurrentddltest/main_test.go b/ddl/concurrentddltest/main_test.go new file mode 100644 index 0000000000000..d6b52492ddb07 --- /dev/null +++ b/ddl/concurrentddltest/main_test.go @@ -0,0 +1,44 @@ +// 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 concurrentddltest + +import ( + "testing" + "time" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + }) + + ddl.SetWaitTimeWhenErrorOccurred(time.Microsecond) + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + goleak.VerifyTestMain(m, opts...) +} diff --git a/ddl/concurrentddltest/switch_test.go b/ddl/concurrentddltest/switch_test.go new file mode 100644 index 0000000000000..892f73531da89 --- /dev/null +++ b/ddl/concurrentddltest/switch_test.go @@ -0,0 +1,139 @@ +// 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 concurrentddltest + +import ( + "context" + "fmt" + "math/rand" + "testing" + "time" + + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +func TestConcurrentDDLSwitch(t *testing.T) { + store := testkit.CreateMockStore(t) + + type table struct { + columnIdx int + indexIdx int + } + + var tables []*table + tblCount := 20 + for i := 0; i < tblCount; i++ { + tables = append(tables, &table{1, 0}) + } + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt=1") + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size=32") + + for i := range tables { + tk.MustExec(fmt.Sprintf("create table t%d (col0 int) partition by range columns (col0) ("+ + "partition p1 values less than (100), "+ + "partition p2 values less than (300), "+ + "partition p3 values less than (500), "+ + "partition p4 values less than (700), "+ + "partition p5 values less than (1000), "+ + "partition p6 values less than maxvalue);", + i)) + for j := 0; j < 1000; j++ { + tk.MustExec(fmt.Sprintf("insert into t%d values (%d)", i, j)) + } + } + + ddls := make([]string, 0, tblCount) + ddlCount := 100 + for i := 0; i < ddlCount; i++ { + tblIdx := rand.Intn(tblCount) + if rand.Intn(2) == 0 { + ddls = append(ddls, fmt.Sprintf("alter table t%d add index idx%d (col0)", tblIdx, tables[tblIdx].indexIdx)) + tables[tblIdx].indexIdx++ + } else { + ddls = append(ddls, fmt.Sprintf("alter table t%d add column col%d int", tblIdx, tables[tblIdx].columnIdx)) + tables[tblIdx].columnIdx++ + } + } + + c := atomic.NewInt32(0) + ch := make(chan struct{}) + go func() { + var wg util.WaitGroupWrapper + for i := range ddls { + wg.Add(1) + go func(idx int) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(ddls[idx]) + c.Add(1) + wg.Done() + }(i) + } + wg.Wait() + ch <- struct{}{} + }() + + ticker := time.NewTicker(time.Second) + count := 0 + done := false + for !done { + select { + case <-ch: + done = true + case <-ticker.C: + var b bool + var err error + err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, false, func(ctx context.Context, txn kv.Transaction) error { + b, err = meta.NewMeta(txn).IsConcurrentDDL() + return err + }) + require.NoError(t, err) + rs, err := testkit.NewTestKit(t, store).Exec(fmt.Sprintf("set @@global.tidb_enable_concurrent_ddl=%t", !b)) + if rs != nil { + require.NoError(t, rs.Close()) + } + if err == nil { + count++ + if b { + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select count(*) from mysql.tidb_ddl_job").Check(testkit.Rows("0")) + tk.MustQuery("select count(*) from mysql.tidb_ddl_reorg").Check(testkit.Rows("0")) + } + } + } + } + + require.Equal(t, int32(ddlCount), c.Load()) + require.Greater(t, count, 0) + + tk = testkit.NewTestKit(t, store) + tk.MustExec("use test") + for i, tbl := range tables { + tk.MustQuery(fmt.Sprintf("select count(*) from information_schema.columns where TABLE_SCHEMA = 'test' and TABLE_NAME = 't%d'", i)).Check(testkit.Rows(fmt.Sprintf("%d", tbl.columnIdx))) + tk.MustExec(fmt.Sprintf("admin check table t%d", i)) + for j := 0; j < tbl.indexIdx; j++ { + tk.MustExec(fmt.Sprintf("admin check index t%d idx%d", i, j)) + } + } +} diff --git a/ddl/constant.go b/ddl/constant.go index dd5db4a851a36..bf4d69fb8fd33 100644 --- a/ddl/constant.go +++ b/ddl/constant.go @@ -32,6 +32,8 @@ const ( ReorgTableID = meta.MaxInt48 - 2 // HistoryTableID is the table ID of `tidb_ddl_history`. HistoryTableID = meta.MaxInt48 - 3 + // MDLTableID is the table ID of `tidb_mdl_info`. + MDLTableID = meta.MaxInt48 - 4 // JobTableSQL is the CREATE TABLE SQL of `tidb_ddl_job`. JobTableSQL = "create table " + JobTable + "(job_id bigint not null, reorg int, schema_ids text(65535), table_ids text(65535), job_meta longblob, type int, processing int, primary key(job_id))" diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go index fb3cabd224c93..6506f24e4ef0d 100644 --- a/ddl/db_cache_test.go +++ b/ddl/db_cache_test.go @@ -22,8 +22,6 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/util/dbterror" @@ -129,20 +127,10 @@ func TestIndexOnCacheTable(t *testing.T) { } func TestAlterTableCache(t *testing.T) { - store, err := mockstore.NewMockStore() - require.NoError(t, err) - session.SetSchemaLease(600 * time.Millisecond) - session.DisableStats4Test() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) + store, dom := testkit.CreateMockStoreAndDomain(t) dom.SetStatsUpdating(true) - t.Cleanup(func() { - dom.Close() - err := store.Close() - require.NoError(t, err) - }) tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) @@ -157,6 +145,7 @@ func TestAlterTableCache(t *testing.T) { checkTableCacheStatus(t, tk, "test", "t1", model.TableCacheStatusEnable) tk.MustExec("alter table t1 nocache") tk.MustExec("drop table if exists t1") + tk.MustExec("set global tidb_enable_metadata_lock=0") /*Test can't skip schema checker*/ tk.MustExec("drop table if exists t1,t2") tk.MustExec("CREATE TABLE t1 (a int)") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 116a508bba732..99ccabdfa337e 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -3324,6 +3324,7 @@ func TestPartitionErrorCode(t *testing.T) { // Reduce the impact on DML when executing partition DDL tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk1.MustExec("drop table if exists t;") tk1.MustExec(`create table t(id int primary key) partition by hash(id) partitions 4;`) @@ -3485,6 +3486,7 @@ func TestCommitWhenSchemaChange(t *testing.T) { }) store := testkit.CreateMockStoreWithSchemaLease(t, time.Second) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("set @@global.tidb_max_delta_schema_count= 4096") tk.MustExec("use test") tk.MustExec(`create table schema_change (a int, b timestamp) diff --git a/ddl/db_table_test.go b/ddl/db_table_test.go index a00fb40a53907..730e3bb941848 100644 --- a/ddl/db_table_test.go +++ b/ddl/db_table_test.go @@ -157,7 +157,7 @@ func TestAddNotNullColumnWhileInsertOnDupUpdate(t *testing.T) { } func TestTransactionOnAddDropColumn(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, time.Microsecond*500) + store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("set @@global.tidb_max_delta_schema_count= 4096") tk.MustExec("use test") @@ -381,7 +381,7 @@ func TestAlterTableWithValidation(t *testing.T) { } func TestBatchCreateTable(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, time.Microsecond*500) + store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tables_1") @@ -600,9 +600,10 @@ func TestWriteLocal(t *testing.T) { } func TestLockTables(t *testing.T) { - store := testkit.CreateMockStoreWithSchemaLease(t, time.Microsecond*500) + store := testkit.CreateMockStore(t) setTxnTk := testkit.NewTestKit(t, store) setTxnTk.MustExec("set global tidb_txn_mode=''") + setTxnTk.MustExec("set global tidb_enable_metadata_lock=0") tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") @@ -834,7 +835,7 @@ func TestDDLWithInvalidTableInfo(t *testing.T) { } func TestAddColumn2(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, time.Microsecond*500) + store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") diff --git a/ddl/db_test.go b/ddl/db_test.go index aef54b19d856c..0b369e92a82ad 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -244,6 +244,7 @@ func TestIssue22819(t *testing.T) { store := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk1.MustExec("use test;") tk1.MustExec("create table t1 (v int) partition by hash (v) partitions 2") tk1.MustExec("insert into t1 values (1)") @@ -817,7 +818,11 @@ func TestForbidCacheTableForSystemTable(t *testing.T) { for _, one := range sysTables { err := tk.ExecToErr(fmt.Sprintf("alter table `%s` cache", one)) if db == "MySQL" { - require.EqualError(t, err, "[ddl:8200]ALTER table cache for tables in system database is currently unsupported") + if one == "tidb_mdl_view" { + require.EqualError(t, err, "[ddl:1347]'MySQL.tidb_mdl_view' is not BASE TABLE") + } else { + require.EqualError(t, err, "[ddl:8200]ALTER table cache for tables in system database is currently unsupported") + } } else { require.EqualError(t, err, fmt.Sprintf("[planner:1142]ALTER command denied to user 'root'@'%%' for table '%s'", strings.ToLower(one))) } @@ -980,6 +985,7 @@ func TestCommitTxnWithIndexChange(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease) // Prepare work. tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1;") tk.MustExec("use test") tk.MustExec("create table t1 (c1 int primary key, c2 int, c3 int, index ok2(c2))") @@ -1325,6 +1331,7 @@ func TestTxnSavepointWithDDL(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test;") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test;") prepareFn := func() { @@ -1376,6 +1383,7 @@ func TestAmendTxnSavepointWithDDL(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test;") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test;") tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1;") @@ -1437,7 +1445,7 @@ func TestSnapshotVersion(t *testing.T) { // For updating the self schema version. goCtx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) - err := dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, is.SchemaMetaVersion()) + err := dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, 0, is.SchemaMetaVersion()) cancel() require.NoError(t, err) @@ -1447,7 +1455,7 @@ func TestSnapshotVersion(t *testing.T) { // Make sure that the self schema version doesn't be changed. goCtx, cancel = context.WithTimeout(context.Background(), 100*time.Millisecond) - err = dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, is.SchemaMetaVersion()) + err = dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, 0, is.SchemaMetaVersion()) cancel() require.NoError(t, err) @@ -1498,33 +1506,33 @@ func TestSchemaValidator(t *testing.T) { require.NoError(t, err) ts := ver.Ver - _, res := dom.SchemaValidator.Check(ts, schemaVer, nil) + _, res := dom.SchemaValidator.Check(ts, schemaVer, nil, true) require.Equal(t, domain.ResultSucc, res) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`)) err = dom.Reload() require.Error(t, err) - _, res = dom.SchemaValidator.Check(ts, schemaVer, nil) + _, res = dom.SchemaValidator.Check(ts, schemaVer, nil, true) require.Equal(t, domain.ResultSucc, res) time.Sleep(dbTestLease) ver, err = store.CurrentVersion(kv.GlobalTxnScope) require.NoError(t, err) ts = ver.Ver - _, res = dom.SchemaValidator.Check(ts, schemaVer, nil) + _, res = dom.SchemaValidator.Check(ts, schemaVer, nil, true) require.Equal(t, domain.ResultUnknown, res) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed")) err = dom.Reload() require.NoError(t, err) - _, res = dom.SchemaValidator.Check(ts, schemaVer, nil) + _, res = dom.SchemaValidator.Check(ts, schemaVer, nil, true) require.Equal(t, domain.ResultSucc, res) // For schema check, it tests for getting the result of "ResultUnknown". is := dom.InfoSchema() - schemaChecker := domain.NewSchemaChecker(dom, is.SchemaMetaVersion(), nil) + schemaChecker := domain.NewSchemaChecker(dom, is.SchemaMetaVersion(), nil, true) // Make sure it will retry one time and doesn't take a long time. domain.SchemaOutOfDateRetryTimes.Store(1) domain.SchemaOutOfDateRetryInterval.Store(time.Millisecond * 1) diff --git a/ddl/ddl.go b/ddl/ddl.go index 63275c5bffe5b..0fac4e23ec0b9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -573,6 +573,7 @@ func newDDL(ctx context.Context, options ...Option) *ddl { variable.EnableDDL = d.EnableDDL variable.DisableDDL = d.DisableDDL variable.SwitchConcurrentDDL = d.SwitchConcurrentDDL + variable.SwitchMDL = d.SwitchMDL return d } @@ -1161,6 +1162,37 @@ func (d *ddl) SwitchConcurrentDDL(toConcurrentDDL bool) error { return err } +// SwitchMDL enables MDL or disable DDL. +func (d *ddl) SwitchMDL(enable bool) error { + isEnableBefore := variable.EnableMDL.Load() + if isEnableBefore == enable { + return nil + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + + // Check if there is any DDL running. + // This check can not cover every corner cases, so users need to guarantee that there is no DDL running by themselves. + sess, err := d.sessPool.get() + if err != nil { + return err + } + defer d.sessPool.put(sess) + se := newSession(sess) + rows, err := se.execute(ctx, "select 1 from mysql.tidb_ddl_job", "check job") + if err != nil { + return err + } + if len(rows) != 0 { + return errors.New("please wait for all jobs done") + } + + variable.EnableMDL.Store(enable) + logutil.BgLogger().Info("[ddl] switch metadata lock feature", zap.Bool("enable", enable), zap.Error(err)) + return nil +} + func (d *ddl) wait4Switch(ctx context.Context) error { for { select { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 80dc0fad41637..e5dbbe149cd37 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -485,6 +485,56 @@ func (w *worker) updateDDLJob(t *meta.Meta, job *model.Job, meetErr bool) error return errors.Trace(err) } +// registerMDLInfo registers metadata lock info. +func (w *worker) registerMDLInfo(job *model.Job, ver int64) error { + if !variable.EnableMDL.Load() { + return nil + } + if ver == 0 { + return nil + } + rows, err := w.sess.execute(context.Background(), fmt.Sprintf("select table_ids from mysql.tidb_ddl_job where job_id = %d", job.ID), "register-mdl-info") + if err != nil { + return err + } + if len(rows) == 0 { + return errors.Errorf("can't find ddl job %d", job.ID) + } + ids := rows[0].GetString(0) + sql := fmt.Sprintf("replace into mysql.tidb_mdl_info (job_id, version, table_ids) values (%d, %d, '%s')", job.ID, ver, ids) + _, err = w.sess.execute(context.Background(), sql, "register-mdl-info") + return err +} + +// cleanMDLInfo cleans metadata lock info. +func cleanMDLInfo(pool *sessionPool, jobID int64) { + if !variable.EnableMDL.Load() { + return + } + sql := fmt.Sprintf("delete from mysql.tidb_mdl_info where job_id = %d", jobID) + sctx, _ := pool.get() + defer pool.put(sctx) + sess := newSession(sctx) + sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + _, err := sess.execute(context.Background(), sql, "delete-mdl-info") + if err != nil { + logutil.BgLogger().Warn("unexpected error when clean mdl info", zap.Error(err)) + } +} + +// checkMDLInfo checks if metadata lock info exists. It means the schema is locked by some TiDBs if exists. +func checkMDLInfo(jobID int64, pool *sessionPool) (bool, error) { + sql := fmt.Sprintf("select * from mysql.tidb_mdl_info where job_id = %d", jobID) + sctx, _ := pool.get() + defer pool.put(sctx) + sess := newSession(sctx) + rows, err := sess.execute(context.Background(), sql, "check-mdl-info") + if err != nil { + return false, err + } + return len(rows) > 0, nil +} + func needUpdateRawArgs(job *model.Job, meetErr bool) bool { // If there is an error when running job and the RawArgs hasn't been decoded by DecodeArgs, // we shouldn't replace RawArgs with the marshaling Args. @@ -770,6 +820,12 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) (int64, error) { schemaVer = 0 } + err = w.registerMDLInfo(job, schemaVer) + if err != nil { + w.sess.rollback() + d.unlockSchemaVersion(job.ID) + return 0, err + } err = w.updateDDLJob(t, job, runJobErr != nil) if err = w.handleUpdateJobError(t, job, err); err != nil { w.sess.rollback() @@ -868,7 +924,7 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { } if once { - err = w.waitSchemaSynced(d, job, waitTime) + err = waitSchemaSynced(d, job, waitTime) if err == nil { once = false } @@ -1282,13 +1338,13 @@ func waitSchemaChanged(ctx context.Context, d *ddlCtx, waitTime time.Duration, l } } - // OwnerCheckAllVersions returns only when all TiDB schemas are synced(exclude the isolated TiDB). - err = d.schemaSyncer.OwnerCheckAllVersions(ctx, latestSchemaVersion) + // OwnerCheckAllVersions returns only when all TiDB schemas are synced(exclude the isolated TiDB). + err = d.schemaSyncer.OwnerCheckAllVersions(context.Background(), job.ID, latestSchemaVersion) if err != nil { logutil.Logger(d.ctx).Info("[ddl] wait latest schema version encounter error", zap.Int64("ver", latestSchemaVersion), zap.Error(err)) return } - logutil.Logger(d.ctx).Info("[ddl] wait latest schema version changed", + logutil.Logger(d.ctx).Info("[ddl] wait latest schema version changed(get the metadata lock if tidb_enable_metadata_lock is true)", zap.Int64("ver", latestSchemaVersion), zap.Duration("take time", time.Since(timeStart)), zap.String("job", job.String())) @@ -1300,17 +1356,17 @@ func waitSchemaChanged(ctx context.Context, d *ddlCtx, waitTime time.Duration, l // but in this case we don't wait enough 2 * lease time to let other servers update the schema. // So here we get the latest schema version to make sure all servers' schema version update to the latest schema version // in a cluster, or to wait for 2 * lease time. -func (w *worker) waitSchemaSynced(d *ddlCtx, job *model.Job, waitTime time.Duration) error { +func waitSchemaSynced(d *ddlCtx, job *model.Job, waitTime time.Duration) error { if !job.IsRunning() && !job.IsRollingback() && !job.IsDone() && !job.IsRollbackDone() { return nil } - ver, _ := w.store.CurrentVersion(kv.GlobalTxnScope) - snapshot := w.store.GetSnapshot(ver) + ver, _ := d.store.CurrentVersion(kv.GlobalTxnScope) + snapshot := d.store.GetSnapshot(ver) m := meta.NewSnapshotMeta(snapshot) latestSchemaVersion, err := m.GetSchemaVersionWithNonEmptyDiff() if err != nil { - logutil.Logger(w.logCtx).Warn("[ddl] get global version failed", zap.Error(err)) + logutil.Logger(d.ctx).Warn("[ddl] get global version failed", zap.Error(err)) return err } diff --git a/ddl/job_table.go b/ddl/job_table.go index c9d7c78f7aa82..040ff971de72e 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -127,10 +127,11 @@ func (d *ddl) getJob(sess *session, tp jobType, filter func(*model.Job) (bool, e func (d *ddl) getGeneralJob(sess *session) (*model.Job, error) { return d.getJob(sess, general, func(job *model.Job) (bool, error) { if job.Type == model.ActionDropSchema { - sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', '%s', ',') != 0 and processing limit 1", strconv.Quote(strconv.FormatInt(job.SchemaID, 10))) + // Check if there is any reorg job on this schema. + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', %s, ',') != 0 and processing limit 1", strconv.Quote(strconv.FormatInt(job.SchemaID, 10))) return d.checkJobIsRunnable(sess, sql) } - + // Check if there is any running job works on the same table. sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job t1, (select table_ids from mysql.tidb_ddl_job where job_id = %d) t2 where processing and CONCAT(',', t2.table_ids, ',') REGEXP CONCAT(',', REPLACE(t1.table_ids, ',', '|'), ',') != 0", job.ID) return d.checkJobIsRunnable(sess, sql) }) @@ -143,7 +144,8 @@ func (d *ddl) checkJobIsRunnable(sess *session, sql string) (bool, error) { func (d *ddl) getReorgJob(sess *session) (*model.Job, error) { return d.getJob(sess, reorg, func(job *model.Job) (bool, error) { - sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where (CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', '%s', ',') != 0 and type = %d and processing) or (CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', '%s', ',') != 0 and processing) limit 1", + // Check if there is any drop schema ddl running. + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where (CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', %s, ',') != 0 and type = %d and processing) or (CONCAT(',', schema_ids, ',') REGEXP CONCAT(',', %s, ',') != 0 and processing) limit 1", strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), model.ActionDropSchema, strconv.Quote(strconv.FormatInt(job.TableID, 10))) return d.checkJobIsRunnable(sess, sql) }) @@ -215,6 +217,7 @@ func (d *ddl) loadDDLJobAndRun(sess *session, pool *workerPool, getJob func(*ses d.delivery2worker(wk, pool, job) } +// delivery2worker owns the worker, need to put it back to the pool in this function. func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { injectFailPointForGetJob(job) d.insertRunningDDLJobMap(job.ID) @@ -225,18 +228,42 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { asyncNotify(d.ddlJobCh) metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Dec() }() - // we should wait 2 * d.lease time to guarantee all TiDB server have finished the schema change. - // see waitSchemaSynced for more details. + // check if this ddl job is synced to all servers. if !d.isSynced(job) || d.once.Load() { - err := wk.waitSchemaSynced(d.ddlCtx, job, 2*d.lease) - if err == nil { - d.once.Store(false) + if variable.EnableMDL.Load() { + exist, err := checkMDLInfo(job.ID, d.sessPool) + if err != nil { + logutil.BgLogger().Warn("[ddl] check MDL info failed", zap.Error(err), zap.String("job", job.String())) + // Release the worker resource. + pool.put(wk) + return + } else if exist { + // Release the worker resource. + pool.put(wk) + err = waitSchemaSynced(d.ddlCtx, job, 2*d.lease) + if err != nil { + logutil.BgLogger().Warn("[ddl] wait ddl job sync failed", zap.Error(err), zap.String("job", job.String())) + time.Sleep(time.Second) + return + } + d.once.Store(false) + cleanMDLInfo(d.sessPool, job.ID) + // Don't have a worker now. + return + } } else { - logutil.BgLogger().Warn("[ddl] wait ddl job sync failed", zap.Error(err), zap.String("job", job.String())) - time.Sleep(time.Second) - return + err := waitSchemaSynced(d.ddlCtx, job, 2*d.lease) + if err != nil { + logutil.BgLogger().Warn("[ddl] wait ddl job sync failed", zap.Error(err), zap.String("job", job.String())) + time.Sleep(time.Second) + // Release the worker resource. + pool.put(wk) + return + } + d.once.Store(false) } } + schemaVer, err := wk.HandleDDLJobTable(d.ddlCtx, job) pool.put(wk) if err != nil { @@ -255,6 +282,7 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { // If the job is done or still running or rolling back, we will wait 2 * lease time to guarantee other servers to update // the newest schema. waitSchemaChanged(context.Background(), d.ddlCtx, d.lease*2, schemaVer, job) + cleanMDLInfo(d.sessPool, job.ID) d.synced(job) if RunInGoTest { diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index 4a8b6fae32fa2..caf80baf0f945 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -15,23 +15,17 @@ package ddl_test import ( - "context" - "fmt" - "math/rand" "sync" "testing" "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" - "go.uber.org/atomic" "golang.org/x/exp/slices" ) @@ -183,111 +177,3 @@ func check(t *testing.T, record []int64, ids ...int64) { } } } - -func TestConcurrentDDLSwitch(t *testing.T) { - store := testkit.CreateMockStore(t) - - type table struct { - columnIdx int - indexIdx int - } - - var tables []*table - tblCount := 20 - for i := 0; i < tblCount; i++ { - tables = append(tables, &table{1, 0}) - } - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt=1") - tk.MustExec("set @@global.tidb_ddl_reorg_batch_size=32") - - for i := range tables { - tk.MustExec(fmt.Sprintf("create table t%d (col0 int) partition by range columns (col0) ("+ - "partition p1 values less than (100), "+ - "partition p2 values less than (300), "+ - "partition p3 values less than (500), "+ - "partition p4 values less than (700), "+ - "partition p5 values less than (1000), "+ - "partition p6 values less than maxvalue);", - i)) - for j := 0; j < 1000; j++ { - tk.MustExec(fmt.Sprintf("insert into t%d values (%d)", i, j)) - } - } - - ddls := make([]string, 0, tblCount) - ddlCount := 100 - for i := 0; i < ddlCount; i++ { - tblIdx := rand.Intn(tblCount) - if rand.Intn(2) == 0 { - ddls = append(ddls, fmt.Sprintf("alter table t%d add index idx%d (col0)", tblIdx, tables[tblIdx].indexIdx)) - tables[tblIdx].indexIdx++ - } else { - ddls = append(ddls, fmt.Sprintf("alter table t%d add column col%d int", tblIdx, tables[tblIdx].columnIdx)) - tables[tblIdx].columnIdx++ - } - } - - c := atomic.NewInt32(0) - ch := make(chan struct{}) - go func() { - var wg util.WaitGroupWrapper - for i := range ddls { - wg.Add(1) - go func(idx int) { - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(ddls[idx]) - c.Add(1) - wg.Done() - }(i) - } - wg.Wait() - ch <- struct{}{} - }() - - ticker := time.NewTicker(time.Second * 2) - count := 0 - done := false - for !done { - select { - case <-ch: - done = true - case <-ticker.C: - var b bool - var err error - err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, false, func(ctx context.Context, txn kv.Transaction) error { - b, err = meta.NewMeta(txn).IsConcurrentDDL() - return err - }) - require.NoError(t, err) - rs, err := testkit.NewTestKit(t, store).Exec(fmt.Sprintf("set @@global.tidb_enable_concurrent_ddl=%t", !b)) - if rs != nil { - require.NoError(t, rs.Close()) - } - if err == nil { - count++ - if b { - tk := testkit.NewTestKit(t, store) - tk.MustQuery("select count(*) from mysql.tidb_ddl_job").Check(testkit.Rows("0")) - tk.MustQuery("select count(*) from mysql.tidb_ddl_reorg").Check(testkit.Rows("0")) - } - } - } - } - - require.Equal(t, int32(ddlCount), c.Load()) - require.Greater(t, count, 0) - - tk = testkit.NewTestKit(t, store) - tk.MustExec("use test") - for i, tbl := range tables { - tk.MustQuery(fmt.Sprintf("select count(*) from information_schema.columns where TABLE_SCHEMA = 'test' and TABLE_NAME = 't%d'", i)).Check(testkit.Rows(fmt.Sprintf("%d", tbl.columnIdx))) - tk.MustExec(fmt.Sprintf("admin check table t%d", i)) - for j := 0; j < tbl.indexIdx; j++ { - tk.MustExec(fmt.Sprintf("admin check index t%d idx%d", i, j)) - } - } -} diff --git a/ddl/metadatalocktest/BUILD.bazel b/ddl/metadatalocktest/BUILD.bazel new file mode 100644 index 0000000000000..7f1fc31f9987a --- /dev/null +++ b/ddl/metadatalocktest/BUILD.bazel @@ -0,0 +1,20 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "metadatalocktest_test", + srcs = [ + "main_test.go", + "mdl_test.go", + ], + flaky = True, + deps = [ + "//config", + "//ddl", + "//errno", + "//server", + "//testkit", + "//testkit/testsetup", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/ddl/metadatalocktest/main_test.go b/ddl/metadatalocktest/main_test.go new file mode 100644 index 0000000000000..62dbb9a626287 --- /dev/null +++ b/ddl/metadatalocktest/main_test.go @@ -0,0 +1,44 @@ +// 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 metadatalocktest + +import ( + "testing" + "time" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + }) + + ddl.SetWaitTimeWhenErrorOccurred(time.Microsecond) + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + goleak.VerifyTestMain(m, opts...) +} diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go new file mode 100644 index 0000000000000..5a9a8302ba5a7 --- /dev/null +++ b/ddl/metadatalocktest/mdl_test.go @@ -0,0 +1,995 @@ +// 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. + +//go:build !featuretag + +package metadatalocktest + +import ( + "fmt" + "sync" + "testing" + "time" + + mysql "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/server" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestMDLBasicSelect(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("begin") + tk.MustQuery("select * from t;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLBasicInsert(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("begin") + tk.MustExec("insert into t values (2);") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLBasicUpdate(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("begin") + tk.MustExec("update t set a = 2;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLBasicDelete(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("begin") + tk.MustExec("delete from t;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLBasicPointGet(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int, unique key(a));") + tk.MustExec("insert into t values(1), (2), (3);") + + tk.MustExec("begin") + tk.MustQuery("select * from t where a = 1;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLBasicBatchPointGet(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int, unique key(a));") + tk.MustExec("insert into t values(1), (2), (3);") + + tk.MustExec("begin") + + tk.MustQuery("select * from t where a in (12, 22);") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLRRUpdateSchema(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + // Add a new column. + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t add column b int;") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + + // Add a new index. + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t add index idx(a);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + tk.MustGetErrCode("select * from t use index(idx)", mysql.ErrKeyDoesNotExist) + tk.MustExec("commit") + tk.MustQuery("select * from t use index(idx)").Check(testkit.Rows("1 ")) + + // Modify column(reorg). + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t modify column a char(10);") + tk.MustGetErrCode("select * from t", mysql.ErrSchemaChanged) + tk.MustExec("commit") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + + // Modify column(non-reorg). + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t modify column a char(20);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) +} + +func TestMDLRCUpdateSchema(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("set @@transaction_isolation='READ-COMMITTED';") + + // Add a new column. + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t add column b int;") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + + // Add a new index. + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t add index idx(a);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + tk.MustQuery("select * from t use index(idx)").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t use index(idx)").Check(testkit.Rows("1 ")) + + // Modify column(reorg). + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t modify column a char(10);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + + // Modify column(non-reorg). + tk.MustExec("begin") + tkDDL.MustExec("alter table test.t modify column a char(20);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) +} + +func TestMDLAutoCommitReadOnly(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + var wg sync.WaitGroup + wg.Add(2) + var ts2 time.Time + var ts1 time.Time + + go func() { + tk.MustQuery("select sleep(2) from t;") + ts1 = time.Now() + wg.Done() + }() + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + wg.Wait() + require.Greater(t, ts1, ts2) +} + +func TestMDLAutoCommitNonReadOnly(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + var wg sync.WaitGroup + wg.Add(2) + var ts2 time.Time + var ts1 time.Time + + go func() { + tk.MustExec("insert into t select sleep(2) from t;") + ts1 = time.Now() + wg.Done() + }() + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLLocalTemporaryTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("use test") + + tk.MustExec("create temporary table t(a int);") + tk.MustExec("insert into t values(1)") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + var ts1 time.Time + + tk.MustExec("begin") + tk.MustExec("insert into t values (2)") + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + ts1 = time.Now() + tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) + tk.MustExec("commit") + + wg.Wait() + require.Greater(t, ts1, ts2) +} + +func TestMDLGlobalTemporaryTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create global temporary table t(a int) ON COMMIT DELETE ROWS;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + var ts1 time.Time + + tk.MustExec("begin") + tk.MustExec("insert into t values (2)") + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + ts1 = time.Now() + tk.MustQuery("select * from t").Check(testkit.Rows("2")) + tk.MustExec("commit") + + wg.Wait() + require.Greater(t, ts1, ts2) + + tk.MustExec("begin") + + tkDDL.MustExec("alter table test.t add column c int;") + tk.MustExec("insert into t values (2, null, null)") + + tk.MustQuery("select * from t").Check(testkit.Rows("2 ")) + tk.MustExec("commit") +} + +func TestMDLCacheTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("alter table t cache") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + var ts1 time.Time + + tk.MustExec("begin") + tk.MustQuery("select * from t") + tk.MustQuery("select * from t") + + go func() { + tkDDL.MustExec("alter table test.t nocache;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + ts1 = time.Now() + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLStealRead(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + var ts1 time.Time + + time.Sleep(2 * time.Second) + + tk.MustExec("start transaction read only as of timestamp NOW() - INTERVAL 1 SECOND") + tk.MustQuery("select * from t") + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + ts1 = time.Now() + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + tk.MustExec("commit") + + wg.Wait() + require.Greater(t, ts1, ts2) +} + +func TestMDLTiDBSnapshot(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20060102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + var ts1 time.Time + + time.Sleep(2 * time.Second) + + tk.MustExec("begin") + tk.MustExec("set @@tidb_snapshot = NOW() - INTERVAL 1 SECOND") + tk.MustQuery("select * from t") + + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + ts1 = time.Now() + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + tk.MustExec("commit") + + wg.Wait() + require.Greater(t, ts1, ts2) +} + +func TestMDLPartitionTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int) partition by hash(a) partitions 10;") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec("begin") + tk.MustQuery("select * from t;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLPreparePlanBlockDDL(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec(`prepare stmt_test_1 from 'select * from t where a >= ?';`) + + tk.MustExec("begin") + tk.MustExec(`set @a = 1;`) + tk.MustQuery(`execute stmt_test_1 using @a;`).Check(testkit.Rows("1", "2", "3", "4")) + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + tk.MustExec(`prepare stmt_test_1 from 'select * from t where a >= ?';`) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) + + tk.MustQuery(`execute stmt_test_1 using @a;`).Check(testkit.Rows("1 ", "2 ", "3 ", "4 ")) +} + +func TestMDLPreparePlanCacheInvalid(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec("begin") + tk.MustQuery("select * from t;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + tk.MustExec(`prepare stmt_test_1 from 'select * from t where a >= ?';`) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) + + tk.MustExec(`set @a = 1;`) + tk.MustQuery(`execute stmt_test_1 using @a;`).Check(testkit.Rows("1 ", "2 ", "3 ", "4 ")) +} + +func TestMDLDisable2Enable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + conn3 := server.CreateMockConn(t, sv) + tk3 := testkit.NewTestKitWithSession(t, store, conn3.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec("begin") + tk.MustExec("insert into t values (10);") + tk3.MustExec("use test") + tk3.MustExec("begin") + tk3.MustQuery("select * from t;") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + tkDDL.MustExec("set global tidb_enable_metadata_lock=1") + tkDDL.MustExec("alter table test.t add index idx(a);") + wg.Done() + }() + + wg.Wait() + + tk.MustGetErrCode("commit", mysql.ErrInfoSchemaChanged) + tk3.MustExec("commit") + tk.MustExec("admin check table t") +} + +func TestMDLEnable2Disable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + conn3 := server.CreateMockConn(t, sv) + tk3 := testkit.NewTestKitWithSession(t, store, conn3.Context().Session) + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("use test") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec("begin") + tk.MustExec("insert into t values (10);") + tk3.MustExec("use test") + tk3.MustExec("begin") + tk3.MustQuery("select * from t;") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + tkDDL.MustExec("set global tidb_enable_metadata_lock=0") + tkDDL.MustExec("alter table test.t add index idx(a);") + wg.Done() + }() + + wg.Wait() + + tk.MustGetErrCode("commit", mysql.ErrInfoSchemaChanged) + tk3.MustExec("commit") + tk.MustExec("admin check table t") +} + +func TestMDLViewItself(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3);") + tk.MustExec("create view v as select * from t") + + tk.MustExec("begin") + tk.MustQuery("select * from v;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("drop view test.v;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLViewBaseTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1), (2), (3);") + tk.MustExec("create view v as select * from t") + + tk.MustExec("begin") + tk.MustQuery("select * from v;") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Less(t, ts1, ts2) +} + +func TestMDLSavePoint(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1);") + + tk.MustExec("begin") + tk.MustExec("savepoint s1") + tk.MustQuery("select * from t;") + tk.MustExec("rollback to s1") + + var wg sync.WaitGroup + wg.Add(1) + var ts2 time.Time + go func() { + tkDDL.MustExec("alter table test.t add column b int;") + ts2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + tk.MustQuery("select * from t;").Check(testkit.Rows("1")) + + wg.Wait() + require.Less(t, ts1, ts2) + + tk.MustExec("alter table t drop column b") + tk.MustExec("begin") + tk.MustExec("savepoint s2") + tkDDL.MustExec("alter table test.t add column b int;") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + tk.MustExec("rollback to s2") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) + tk.MustExec("commit") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 ")) +} diff --git a/ddl/mock.go b/ddl/mock.go index 2bea1349f1f43..dbbd65c9be2c3 100644 --- a/ddl/mock.go +++ b/ddl/mock.go @@ -16,6 +16,7 @@ package ddl import ( "context" + "sync" "sync/atomic" "time" @@ -26,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" clientv3 "go.etcd.io/etcd/client/v3" ) @@ -36,6 +38,7 @@ const mockCheckVersInterval = 2 * time.Millisecond // MockSchemaSyncer is a mock schema syncer, it is exported for tesing. type MockSchemaSyncer struct { selfSchemaVersion int64 + mdlSchemaVersions sync.Map globalVerCh chan clientv3.WatchResponse mockSession chan struct{} } @@ -47,6 +50,7 @@ func NewMockSchemaSyncer() syncer.SchemaSyncer { // Init implements SchemaSyncer.Init interface. func (s *MockSchemaSyncer) Init(ctx context.Context) error { + s.mdlSchemaVersions = sync.Map{} s.globalVerCh = make(chan clientv3.WatchResponse, 1) s.mockSession = make(chan struct{}, 1) return nil @@ -61,8 +65,12 @@ func (s *MockSchemaSyncer) GlobalVersionCh() clientv3.WatchChan { func (s *MockSchemaSyncer) WatchGlobalSchemaVer(context.Context) {} // UpdateSelfVersion implements SchemaSyncer.UpdateSelfVersion interface. -func (s *MockSchemaSyncer) UpdateSelfVersion(ctx context.Context, version int64) error { - atomic.StoreInt64(&s.selfSchemaVersion, version) +func (s *MockSchemaSyncer) UpdateSelfVersion(ctx context.Context, jobID int64, version int64) error { + if variable.EnableMDL.Load() { + s.mdlSchemaVersions.Store(jobID, version) + } else { + atomic.StoreInt64(&s.selfSchemaVersion, version) + } return nil } @@ -92,7 +100,7 @@ func (s *MockSchemaSyncer) OwnerUpdateGlobalVersion(ctx context.Context, version } // OwnerCheckAllVersions implements SchemaSyncer.OwnerCheckAllVersions interface. -func (s *MockSchemaSyncer) OwnerCheckAllVersions(ctx context.Context, latestVer int64) error { +func (s *MockSchemaSyncer) OwnerCheckAllVersions(ctx context.Context, jobID int64, latestVer int64) error { ticker := time.NewTicker(mockCheckVersInterval) defer ticker.Stop() @@ -106,9 +114,16 @@ func (s *MockSchemaSyncer) OwnerCheckAllVersions(ctx context.Context, latestVer }) return errors.Trace(ctx.Err()) case <-ticker.C: - ver := atomic.LoadInt64(&s.selfSchemaVersion) - if ver >= latestVer { - return nil + if variable.EnableMDL.Load() { + ver, ok := s.mdlSchemaVersions.Load(jobID) + if ok && ver.(int64) >= latestVer { + return nil + } + } else { + ver := atomic.LoadInt64(&s.selfSchemaVersion) + if ver >= latestVer { + return nil + } } } } diff --git a/ddl/syncer/BUILD.bazel b/ddl/syncer/BUILD.bazel index dc3c98a228d0c..a608822051aaf 100644 --- a/ddl/syncer/BUILD.bazel +++ b/ddl/syncer/BUILD.bazel @@ -7,7 +7,9 @@ go_library( visibility = ["//visibility:public"], deps = [ "//ddl/util", + "//domain/infosync", "//metrics", + "//sessionctx/variable", "//util", "//util/logutil", "@com_github_pingcap_errors//:errors", @@ -28,6 +30,7 @@ go_test( "//ddl/util", "//infoschema", "//parser/terror", + "//sessionctx/variable", "//store/mockstore", "//util", "@com_github_pingcap_errors//:errors", diff --git a/ddl/syncer/syncer.go b/ddl/syncer/syncer.go index 7c55e7f882098..b2285351f83ae 100644 --- a/ddl/syncer/syncer.go +++ b/ddl/syncer/syncer.go @@ -19,6 +19,7 @@ import ( "fmt" "math" "strconv" + "strings" "sync" "sync/atomic" "time" @@ -27,7 +28,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/sessionctx/variable" tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -58,7 +61,7 @@ type SchemaSyncer interface { // then watch this path, and initializes the self schema version to etcd. Init(ctx context.Context) error // UpdateSelfVersion updates the current version to the self path on etcd. - UpdateSelfVersion(ctx context.Context, version int64) error + UpdateSelfVersion(ctx context.Context, jobID int64, version int64) error // OwnerUpdateGlobalVersion updates the latest version to the global path on etcd until updating is successful or the ctx is done. OwnerUpdateGlobalVersion(ctx context.Context, version int64) error // GlobalVersionCh gets the chan for watching global version. @@ -72,7 +75,7 @@ type SchemaSyncer interface { // OwnerCheckAllVersions checks whether all followers' schema version are equal to // the latest schema version. (exclude the isolated TiDB) // It returns until all servers' versions are equal to the latest version. - OwnerCheckAllVersions(ctx context.Context, latestVer int64) error + OwnerCheckAllVersions(ctx context.Context, jobID int64, latestVer int64) error // Close ends SchemaSyncer. Close() } @@ -85,6 +88,7 @@ type schemaVersionSyncer struct { sync.RWMutex globalVerCh clientv3.WatchChan } + ddlID string } // NewSchemaSyncer creates a new SchemaSyncer. @@ -92,6 +96,7 @@ func NewSchemaSyncer(etcdCli *clientv3.Client, id string) SchemaSyncer { return &schemaVersionSyncer{ etcdCli: etcdCli, selfSchemaVerPath: fmt.Sprintf("%s/%s", util.DDLAllSchemaVersions, id), + ddlID: id, } } @@ -199,10 +204,18 @@ func (s *schemaVersionSyncer) WatchGlobalSchemaVer(ctx context.Context) { } // UpdateSelfVersion implements SchemaSyncer.UpdateSelfVersion interface. -func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, version int64) error { +func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, jobID int64, version int64) error { startTime := time.Now() ver := strconv.FormatInt(version, 10) - err := util.PutKVToEtcd(ctx, s.etcdCli, putKeyNoRetry, s.selfSchemaVerPath, ver, + var err error + var path string + if variable.EnableMDL.Load() { + path = fmt.Sprintf("%s/%d/%s", util.DDLAllSchemaVersionsByJob, jobID, s.ddlID) + } else { + path = s.selfSchemaVerPath + } + + err = util.PutKVToEtcd(ctx, s.etcdCli, putKeyNoRetry, path, ver, clientv3.WithLease(s.loadSession().Lease())) metrics.UpdateSelfVersionHistogram.WithLabelValues(metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -233,17 +246,23 @@ func (s *schemaVersionSyncer) removeSelfVersionPath() error { } // OwnerCheckAllVersions implements SchemaSyncer.OwnerCheckAllVersions interface. -func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, latestVer int64) error { +func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, jobID int64, latestVer int64) error { startTime := time.Now() time.Sleep(CheckVersFirstWaitTime) notMatchVerCnt := 0 intervalCnt := int(time.Second / checkVersInterval) - updatedMap := make(map[string]struct{}) var err error defer func() { metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerCheckAllVersions, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() + + // If MDL is disabled, updatedMap is a cache. We need to ensure all the keys equal to the least version. + // We can skip checking the key if it is checked in the cache(set by the previous loop). + // If MDL is enabled, updatedMap is used to check if all the servers report the least version. + // updatedMap is initialed to record all the server in every loop. We delete a server from the map if it gets the metadata lock(the key version equal the given version. + // updatedMap should be empty if all the servers get the metadata lock. + updatedMap := make(map[string]struct{}) for { if util.IsContextDone(ctx) { // ctx is canceled or timeout. @@ -251,36 +270,84 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, latestV return err } - resp, err := s.etcdCli.Get(ctx, util.DDLAllSchemaVersions, clientv3.WithPrefix()) + // Prepare path and updatedMap. + path := util.DDLAllSchemaVersions + if variable.EnableMDL.Load() { + path = fmt.Sprintf("%s/%d/", util.DDLAllSchemaVersionsByJob, jobID) + serverInfos, err := infosync.GetAllServerInfo(ctx) + if err != nil { + return err + } + updatedMap = make(map[string]struct{}) + for _, info := range serverInfos { + updatedMap[info.ID] = struct{}{} + } + } + + // Get all the schema versions from ETCD. + resp, err := s.etcdCli.Get(ctx, path, clientv3.WithPrefix()) if err != nil { logutil.BgLogger().Info("[ddl] syncer check all versions failed, continue checking.", zap.Error(err)) continue } + // Check all schema versions. succ := true - for _, kv := range resp.Kvs { - if _, ok := updatedMap[string(kv.Key)]; ok { - continue + if variable.EnableMDL.Load() { + for _, kv := range resp.Kvs { + key := string(kv.Key) + ver, err := strconv.Atoi(string(kv.Value)) + if err != nil { + logutil.BgLogger().Info("[ddl] syncer check all versions, convert value to int failed, continue checking.", zap.String("ddl", string(kv.Key)), zap.String("value", string(kv.Value)), zap.Error(err)) + succ = false + break + } + if int64(ver) < latestVer { + if notMatchVerCnt%intervalCnt == 0 { + logutil.BgLogger().Info("[ddl] syncer check all versions, someone is not synced, continue checking", + zap.String("ddl", string(kv.Key)), zap.Int("currentVer", ver), zap.Int64("latestVer", latestVer)) + } + succ = false + notMatchVerCnt++ + break + } + delete(updatedMap, key[strings.LastIndex(key, "/")+1:]) } - - ver, err := strconv.Atoi(string(kv.Value)) - if err != nil { - logutil.BgLogger().Info("[ddl] syncer check all versions, convert value to int failed, continue checking.", zap.String("ddl", string(kv.Key)), zap.String("value", string(kv.Value)), zap.Error(err)) + if len(updatedMap) > 0 { succ = false - break } - if int64(ver) < latestVer { - if notMatchVerCnt%intervalCnt == 0 { - logutil.BgLogger().Info("[ddl] syncer check all versions, someone is not synced, continue checking", - zap.String("ddl", string(kv.Key)), zap.Int("currentVer", ver), zap.Int64("latestVer", latestVer)) + } else { + for _, kv := range resp.Kvs { + if _, ok := updatedMap[string(kv.Key)]; ok { + continue } - succ = false - notMatchVerCnt++ - break + + ver, err := strconv.Atoi(string(kv.Value)) + if err != nil { + logutil.BgLogger().Info("[ddl] syncer check all versions, convert value to int failed, continue checking.", zap.String("ddl", string(kv.Key)), zap.String("value", string(kv.Value)), zap.Error(err)) + succ = false + break + } + if int64(ver) < latestVer { + if notMatchVerCnt%intervalCnt == 0 { + logutil.BgLogger().Info("[ddl] syncer check all versions, someone is not synced, continue checking", + zap.String("ddl", string(kv.Key)), zap.Int("currentVer", ver), zap.Int64("latestVer", latestVer)) + } + succ = false + notMatchVerCnt++ + break + } + updatedMap[string(kv.Key)] = struct{}{} } - updatedMap[string(kv.Key)] = struct{}{} } + if succ { + if variable.EnableMDL.Load() { + _, err = s.etcdCli.Delete(ctx, path, clientv3.WithPrefix()) + if err != nil { + logutil.BgLogger().Warn("[ddl] syncer delete versions failed", zap.Any("job id", jobID), zap.Error(err)) + } + } return nil } time.Sleep(checkVersInterval) diff --git a/ddl/syncer/syncer_test.go b/ddl/syncer/syncer_test.go index 6ac6606642160..a0b9d2a051551 100644 --- a/ddl/syncer/syncer_test.go +++ b/ddl/syncer/syncer_test.go @@ -23,10 +23,11 @@ import ( "github.com/pingcap/errors" . "github.com/pingcap/tidb/ddl" - . "github.com/pingcap/tidb/ddl/syncer" + "github.com/pingcap/tidb/ddl/syncer" util2 "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" @@ -42,15 +43,16 @@ const minInterval = 10 * time.Nanosecond // It's used to test timeout. const testLease = 5 * time.Millisecond func TestSyncerSimple(t *testing.T) { + variable.EnableMDL.Store(false) if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } integration.BeforeTestExternal(t) - origin := CheckVersFirstWaitTime - CheckVersFirstWaitTime = 0 + origin := syncer.CheckVersFirstWaitTime + syncer.CheckVersFirstWaitTime = 0 defer func() { - CheckVersFirstWaitTime = origin + syncer.CheckVersFirstWaitTime = origin }() store, err := mockstore.NewMockStore() @@ -84,7 +86,7 @@ func TestSyncerSimple(t *testing.T) { defer d.SchemaSyncer().Close() key := util2.DDLAllSchemaVersions + "/" + d.OwnerManager().ID() - checkRespKV(t, 1, key, InitialVersion, resp.Kvs...) + checkRespKV(t, 1, key, syncer.InitialVersion, resp.Kvs...) ic2 := infoschema.NewCache(2) ic2.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0), 0) @@ -130,25 +132,25 @@ func TestSyncerSimple(t *testing.T) { // for CheckAllVersions childCtx, cancel := context.WithTimeout(ctx, 200*time.Millisecond) - require.Error(t, d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer)) + require.Error(t, d.SchemaSyncer().OwnerCheckAllVersions(childCtx, 0, currentVer)) cancel() // for UpdateSelfVersion - require.NoError(t, d.SchemaSyncer().UpdateSelfVersion(context.Background(), currentVer)) - require.NoError(t, d1.SchemaSyncer().UpdateSelfVersion(context.Background(), currentVer)) + require.NoError(t, d.SchemaSyncer().UpdateSelfVersion(context.Background(), 0, currentVer)) + require.NoError(t, d1.SchemaSyncer().UpdateSelfVersion(context.Background(), 0, currentVer)) childCtx, cancel = context.WithTimeout(ctx, minInterval) defer cancel() - err = d1.SchemaSyncer().UpdateSelfVersion(childCtx, currentVer) + err = d1.SchemaSyncer().UpdateSelfVersion(childCtx, 0, currentVer) require.True(t, isTimeoutError(err)) // for CheckAllVersions - require.NoError(t, d.SchemaSyncer().OwnerCheckAllVersions(context.Background(), currentVer-1)) - require.NoError(t, d.SchemaSyncer().OwnerCheckAllVersions(context.Background(), currentVer)) + require.NoError(t, d.SchemaSyncer().OwnerCheckAllVersions(context.Background(), 0, currentVer-1)) + require.NoError(t, d.SchemaSyncer().OwnerCheckAllVersions(context.Background(), 0, currentVer)) childCtx, cancel = context.WithTimeout(ctx, minInterval) defer cancel() - err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer) + err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, 0, currentVer) require.True(t, isTimeoutError(err)) // for Close diff --git a/ddl/table_modify_test.go b/ddl/table_modify_test.go index 5b78f5e95bb25..f4b273771fd46 100644 --- a/ddl/table_modify_test.go +++ b/ddl/table_modify_test.go @@ -117,6 +117,7 @@ func TestLockTableReadOnly(t *testing.T) { tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk1.MustExec("drop table if exists t1,t2") defer func() { diff --git a/ddl/tiflash_replica_test.go b/ddl/tiflash_replica_test.go index 0f3b64e1af8ed..d43cc0947b24f 100644 --- a/ddl/tiflash_replica_test.go +++ b/ddl/tiflash_replica_test.go @@ -182,7 +182,11 @@ func TestSetTableFlashReplicaForSystemTable(t *testing.T) { for _, one := range sysTables { _, err := tk.Exec(fmt.Sprintf("alter table `%s` set tiflash replica 1", one)) if db == "MySQL" { - require.Equal(t, "[ddl:8200]Unsupported ALTER TiFlash settings for system table and memory table", err.Error()) + if one == "tidb_mdl_view" { + require.EqualError(t, err, "[ddl:1347]'MySQL.tidb_mdl_view' is not BASE TABLE") + } else { + require.Equal(t, "[ddl:8200]Unsupported ALTER TiFlash settings for system table and memory table", err.Error()) + } } else { require.Equal(t, fmt.Sprintf("[planner:1142]ALTER command denied to user 'root'@'%%' for table '%s'", strings.ToLower(one)), err.Error()) } @@ -201,6 +205,7 @@ func TestSkipSchemaChecker(t *testing.T) { store := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int)") tk2 := testkit.NewTestKit(t, store) diff --git a/ddl/util/util.go b/ddl/util/util.go index 2e94cfd40010c..074791e9a9dc3 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -51,6 +51,8 @@ const ( KeyOpRetryInterval = 30 * time.Millisecond // DDLAllSchemaVersions is the path on etcd that is used to store all servers current schema versions. DDLAllSchemaVersions = "/tidb/ddl/all_schema_versions" + // DDLAllSchemaVersionsByJob is the path on etcd that is used to store all servers current schema versions. + DDLAllSchemaVersionsByJob = "/tidb/ddl/all_schema_by_job_versions" // DDLGlobalSchemaVersion is the path on etcd that is used to store the latest schema versions. DDLGlobalSchemaVersion = "/tidb/ddl/global_schema_version" // SessionTTL is the etcd session's TTL in seconds. diff --git a/domain/domain.go b/domain/domain.go index 56277c7f2ff8c..28675cffe7d8c 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -118,6 +118,15 @@ type Domain struct { sysExecutorFactory func(*Domain) (pools.Resource, error) sysProcesses SysProcesses + + mdlCheckTableInfo *mdlCheckTableInfo +} + +type mdlCheckTableInfo struct { + mu sync.Mutex + newestVer int64 + jobsVerMap map[int64]int64 + jobsIdsMap map[int64]string } // InfoCache export for test. @@ -448,7 +457,7 @@ func (do *Domain) Reload() error { // loaded newer schema if oldSchemaVersion < is.SchemaMetaVersion() { // Update self schema version to etcd. - err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), is.SchemaMetaVersion()) + err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), 0, is.SchemaMetaVersion()) if err != nil { logutil.BgLogger().Info("update self version failed", zap.Int64("oldSchemaVersion", oldSchemaVersion), @@ -607,6 +616,110 @@ func (do *Domain) topologySyncerKeeper() { } } +func (do *Domain) refreshMDLCheckTableInfo() { + se, err := do.sysSessionPool.Get() + + if err != nil { + logutil.BgLogger().Warn("get system session failed", zap.Error(err)) + return + } + defer do.sysSessionPool.Put(se) + exec := se.(sqlexec.RestrictedSQLExecutor) + domainSchemaVer := do.InfoSchema().SchemaMetaVersion() + rows, _, err := exec.ExecRestrictedSQL(kv.WithInternalSourceType(context.Background(), kv.InternalTxnTelemetry), nil, fmt.Sprintf("select job_id, version, table_ids from mysql.tidb_mdl_info where version <= %d", domainSchemaVer)) + if err != nil { + logutil.BgLogger().Warn("get mdl info from tidb_mdl_info failed", zap.Error(err)) + return + } + do.mdlCheckTableInfo.mu.Lock() + defer do.mdlCheckTableInfo.mu.Unlock() + + do.mdlCheckTableInfo.newestVer = domainSchemaVer + do.mdlCheckTableInfo.jobsVerMap = make(map[int64]int64, len(rows)) + do.mdlCheckTableInfo.jobsIdsMap = make(map[int64]string, len(rows)) + for i := 0; i < len(rows); i++ { + do.mdlCheckTableInfo.jobsVerMap[rows[i].GetInt64(0)] = rows[i].GetInt64(1) + do.mdlCheckTableInfo.jobsIdsMap[rows[i].GetInt64(0)] = rows[i].GetString(2) + } +} + +func (do *Domain) mdlCheckLoop() { + ticker := time.Tick(time.Millisecond * 50) + var saveMaxSchemaVersion int64 + jobNeedToSync := false + jobCache := make(map[int64]int64, 1000) + + for { + select { + case <-ticker: + if !variable.EnableMDL.Load() { + continue + } + + do.mdlCheckTableInfo.mu.Lock() + maxVer := do.mdlCheckTableInfo.newestVer + if maxVer > saveMaxSchemaVersion { + saveMaxSchemaVersion = maxVer + } else if !jobNeedToSync { + // Schema doesn't change, and no job to check in the last run. + do.mdlCheckTableInfo.mu.Unlock() + continue + } + + jobNeedToCheckCnt := len(do.mdlCheckTableInfo.jobsVerMap) + if jobNeedToCheckCnt == 0 { + jobNeedToSync = false + do.mdlCheckTableInfo.mu.Unlock() + continue + } + + jobsVerMap := make(map[int64]int64, len(do.mdlCheckTableInfo.jobsVerMap)) + jobsIdsMap := make(map[int64]string, len(do.mdlCheckTableInfo.jobsIdsMap)) + for k, v := range do.mdlCheckTableInfo.jobsVerMap { + jobsVerMap[k] = v + } + for k, v := range do.mdlCheckTableInfo.jobsIdsMap { + jobsIdsMap[k] = v + } + do.mdlCheckTableInfo.mu.Unlock() + + jobNeedToSync = true + + sm := do.InfoSyncer().GetSessionManager() + if sm == nil { + logutil.BgLogger().Info("session manager is nil") + } else { + sm.CheckOldRunningTxn(jobsVerMap, jobsIdsMap) + } + + if len(jobsVerMap) == jobNeedToCheckCnt { + jobNeedToSync = false + } + + // Try to gc jobCache. + if len(jobCache) > 1000 { + jobCache = make(map[int64]int64, 1000) + } + + for jobID, ver := range jobsVerMap { + if cver, ok := jobCache[jobID]; ok && cver >= ver { + // Already update, skip it. + continue + } + logutil.BgLogger().Info("mdl gets lock, update to owner", zap.Int64("jobID", jobID), zap.Int64("version", ver)) + err := do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), jobID, ver) + if err != nil { + logutil.BgLogger().Warn("update self version failed", zap.Error(err)) + } else { + jobCache[jobID] = ver + } + } + case <-do.exit: + return + } + } +} + func (do *Domain) loadSchemaInLoop(ctx context.Context, lease time.Duration) { defer util.Recover(metrics.LabelDomain, "loadSchemaInLoop", nil, true) // Lease renewal can run at any frequency. @@ -663,6 +776,7 @@ func (do *Domain) loadSchemaInLoop(ctx context.Context, lease time.Duration) { case <-do.exit: return } + do.refreshMDLCheckTableInfo() } } @@ -761,6 +875,11 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio dumpFileGcChecker: &dumpFileGcChecker{gcLease: dumpFileGcLease, paths: []string{GetPlanReplayerDirName(), GetOptimizerTraceDirName()}}, onClose: onClose, expiredTimeStamp4PC: types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, types.DefaultFsp), + mdlCheckTableInfo: &mdlCheckTableInfo{ + mu: sync.Mutex{}, + jobsVerMap: make(map[int64]int64), + jobsIdsMap: make(map[int64]string), + }, } do.SchemaValidator = NewSchemaValidator(ddlLease, do) @@ -904,6 +1023,7 @@ func (do *Domain) Init( // Local store needs to get the change information for every DDL state in each session. go do.loadSchemaInLoop(ctx, ddlLease) } + do.wg.Run(do.mdlCheckLoop) do.wg.Add(3) go do.topNSlowQueryLoop() go do.infoSyncerKeeper() diff --git a/domain/schema_checker.go b/domain/schema_checker.go index f3737d85bdb80..30e6366c9d26e 100644 --- a/domain/schema_checker.go +++ b/domain/schema_checker.go @@ -28,6 +28,7 @@ type SchemaChecker struct { SchemaValidator schemaVer int64 relatedTableIDs []int64 + needCheckSchema bool } type intSchemaVer int64 @@ -44,11 +45,12 @@ var ( ) // NewSchemaChecker creates a new schema checker. -func NewSchemaChecker(do *Domain, schemaVer int64, relatedTableIDs []int64) *SchemaChecker { +func NewSchemaChecker(do *Domain, schemaVer int64, relatedTableIDs []int64, needCheckSchema bool) *SchemaChecker { return &SchemaChecker{ SchemaValidator: do.SchemaValidator, schemaVer: schemaVer, relatedTableIDs: relatedTableIDs, + needCheckSchema: needCheckSchema, } } @@ -62,7 +64,7 @@ func (s *SchemaChecker) CheckBySchemaVer(txnTS uint64, startSchemaVer tikv.Schem schemaOutOfDateRetryInterval := SchemaOutOfDateRetryInterval.Load() schemaOutOfDateRetryTimes := int(SchemaOutOfDateRetryTimes.Load()) for i := 0; i < schemaOutOfDateRetryTimes; i++ { - relatedChange, CheckResult := s.SchemaValidator.Check(txnTS, startSchemaVer.SchemaMetaVersion(), s.relatedTableIDs) + relatedChange, CheckResult := s.SchemaValidator.Check(txnTS, startSchemaVer.SchemaMetaVersion(), s.relatedTableIDs, s.needCheckSchema) switch CheckResult { case ResultSucc: return nil, nil diff --git a/domain/schema_checker_test.go b/domain/schema_checker_test.go index ee725ae4d5bbe..b2b2b8a8d870e 100644 --- a/domain/schema_checker_test.go +++ b/domain/schema_checker_test.go @@ -26,7 +26,7 @@ import ( func TestSchemaCheckerSimple(t *testing.T) { lease := 5 * time.Millisecond validator := NewSchemaValidator(lease, nil) - checker := &SchemaChecker{SchemaValidator: validator} + checker := &SchemaChecker{SchemaValidator: validator, needCheckSchema: true} // Add some schema versions and delta table IDs. ts := uint64(time.Now().UnixNano()) @@ -65,6 +65,6 @@ func TestSchemaCheckerSimple(t *testing.T) { // Use checker.SchemaValidator.Check instead of checker.Check here because backoff make CI slow. nowTS := uint64(time.Now().UnixNano()) - _, result := checker.SchemaValidator.Check(nowTS, checker.schemaVer, checker.relatedTableIDs) + _, result := checker.SchemaValidator.Check(nowTS, checker.schemaVer, checker.relatedTableIDs, true) require.Equal(t, ResultUnknown, result) } diff --git a/domain/schema_validator.go b/domain/schema_validator.go index 9d06da1a1eb60..511553feafad8 100644 --- a/domain/schema_validator.go +++ b/domain/schema_validator.go @@ -48,7 +48,7 @@ type SchemaValidator interface { // which is produced when the oldSchemaVer is updated to the newSchemaVer. Update(leaseGrantTime uint64, oldSchemaVer, newSchemaVer int64, change *transaction.RelatedSchemaChange) // Check is it valid for a transaction to use schemaVer and related tables, at timestamp txnTS. - Check(txnTS uint64, schemaVer int64, relatedPhysicalTableIDs []int64) (*transaction.RelatedSchemaChange, checkResult) + Check(txnTS uint64, schemaVer int64, relatedPhysicalTableIDs []int64, needCheckSchema bool) (*transaction.RelatedSchemaChange, checkResult) // Stop stops checking the valid of transaction. Stop() // Restart restarts the schema validator after it is stopped. @@ -223,7 +223,7 @@ func (s *schemaValidator) findNewerDeltas(currVer int64) []deltaSchemaInfo { } // Check checks schema validity, returns true if use schemaVer and related tables at txnTS is legal. -func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTableIDs []int64) (*transaction.RelatedSchemaChange, checkResult) { +func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTableIDs []int64, needCheckSchema bool) (*transaction.RelatedSchemaChange, checkResult) { s.mux.RLock() defer s.mux.RUnlock() if !s.isStarted { @@ -245,13 +245,17 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTa return nil, ResultFail } - relatedChanges, changed := s.isRelatedTablesChanged(schemaVer, relatedPhysicalTableIDs) - if changed { - if relatedChanges.Amendable { - relatedChanges.LatestInfoSchema = s.latestInfoSchema - return &relatedChanges, ResultFail + // When disabling MDL -> enabling MDL, the old transaction's needCheckSchema is true, we need to check it. + // When enabling MDL -> disabling MDL, the old transaction's needCheckSchema is false, so still need to check it, and variable EnableMDL is false now. + if needCheckSchema || !variable.EnableMDL.Load() { + relatedChanges, changed := s.isRelatedTablesChanged(schemaVer, relatedPhysicalTableIDs) + if changed { + if relatedChanges.Amendable { + relatedChanges.LatestInfoSchema = s.latestInfoSchema + return &relatedChanges, ResultFail + } + return nil, ResultFail } - return nil, ResultFail } return nil, ResultSucc } diff --git a/domain/schema_validator_test.go b/domain/schema_validator_test.go index 8b4a1a2b08eef..a18fbcb4a435a 100644 --- a/domain/schema_validator_test.go +++ b/domain/schema_validator_test.go @@ -55,7 +55,7 @@ func subTestSchemaValidatorGeneral(t *testing.T) { item.oldVer, item.schemaVer, &transaction.RelatedSchemaChange{PhyTblIDS: []int64{10}, ActionTypes: []uint64{10}}) - _, valid := validator.Check(item.leaseGrantTS, item.schemaVer, []int64{10}) + _, valid := validator.Check(item.leaseGrantTS, item.schemaVer, []int64{10}, true) require.Equal(t, ResultSucc, valid) // Stop the validator, validator's items value is nil. @@ -63,23 +63,23 @@ func subTestSchemaValidatorGeneral(t *testing.T) { require.False(t, validator.IsStarted()) _, isTablesChanged := validator.isRelatedTablesChanged(item.schemaVer, []int64{10}) require.True(t, isTablesChanged) - _, valid = validator.Check(item.leaseGrantTS, item.schemaVer, []int64{10}) + _, valid = validator.Check(item.leaseGrantTS, item.schemaVer, []int64{10}, true) require.Equal(t, ResultUnknown, valid) validator.Restart() // Increase the current time by 2 leases, check schema is invalid. after2LeaseTime := time.Now().Add(2 * lease) ts := uint64(after2LeaseTime.UnixNano()) // Make sure that ts has timed out a lease. - _, valid = validator.Check(ts, item.schemaVer, []int64{10}) + _, valid = validator.Check(ts, item.schemaVer, []int64{10}, true) require.Equalf(t, ResultUnknown, valid, "validator latest schema ver %v, time %v, item schema ver %v, ts %v", validator.latestSchemaVer, validator.latestSchemaExpire, 0, oracle.GetTimeFromTS(ts)) // Make sure newItem's version is greater than item.schema. newItem := getGreaterVersionItem(t, leaseGrantCh, item.schemaVer) currVer := newItem.schemaVer validator.Update(newItem.leaseGrantTS, newItem.oldVer, currVer, nil) - _, valid = validator.Check(ts, item.schemaVer, nil) + _, valid = validator.Check(ts, item.schemaVer, nil, true) require.Equalf(t, ResultFail, valid, "currVer %d, newItem %v", currVer, item) - _, valid = validator.Check(ts, item.schemaVer, []int64{0}) + _, valid = validator.Check(ts, item.schemaVer, []int64{0}, true) require.Equalf(t, ResultFail, valid, "currVer %d, newItem %v", currVer, item) // Check the latest schema version must changed. @@ -101,7 +101,7 @@ func subTestSchemaValidatorGeneral(t *testing.T) { // All schema versions is expired. ts = uint64(after2LeaseTime.Add(2 * lease).UnixNano()) - _, valid = validator.Check(ts, newItem.schemaVer, nil) + _, valid = validator.Check(ts, newItem.schemaVer, nil, true) require.Equal(t, ResultUnknown, valid, "schemaVer %v, validator %#v", newItem.schemaVer, validator) close(exit) diff --git a/executor/compiler.go b/executor/compiler.go index cee5836b8d11f..1e71cfa146558 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -57,6 +57,8 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } + c.Ctx.GetSessionVars().StmtCtx.IsReadOnly = plannercore.IsReadOnly(stmtNode, c.Ctx.GetSessionVars()) + ret := &plannercore.PreprocessorReturn{} err := plannercore.Preprocess(c.Ctx, stmtNode, diff --git a/executor/ddl.go b/executor/ddl.go index d60cf603674a9..1fd2b20eb70a1 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -58,7 +58,7 @@ type DDLExec struct { func (e *DDLExec) toErr(err error) error { // The err may be cause by schema changed, here we distinguish the ErrInfoSchemaChanged error from other errors. dom := domain.GetDomain(e.ctx) - checker := domain.NewSchemaChecker(dom, e.is.SchemaMetaVersion(), nil) + checker := domain.NewSchemaChecker(dom, e.is.SchemaMetaVersion(), nil, true) txn, err1 := e.ctx.Txn(true) if err1 != nil { logutil.BgLogger().Error("active txn failed", zap.Error(err1)) diff --git a/executor/executor.go b/executor/executor.go index 73238ee3cdcb1..4bf0c43caccb4 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -764,6 +764,7 @@ func (e *ShowDDLJobsExec) Open(ctx context.Context) error { return err } e.sess = sess + sess.GetSessionVars().SetInTxn(true) err = sessiontxn.NewTxn(context.Background(), sess) if err != nil { return err @@ -772,7 +773,6 @@ func (e *ShowDDLJobsExec) Open(ctx context.Context) error { if err != nil { return err } - sess.GetSessionVars().SetInTxn(true) err = e.DDLJobRetriever.initial(txn, sess) return err } diff --git a/executor/infoschema_cluster_table_test.go b/executor/infoschema_cluster_table_test.go index f70e43da8fcdd..38fe23036e313 100644 --- a/executor/infoschema_cluster_table_test.go +++ b/executor/infoschema_cluster_table_test.go @@ -290,7 +290,7 @@ func TestTableStorageStats(t *testing.T) { "test 2", )) rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows() - result := 35 + result := 37 require.Len(t, rows, result) // More tests about the privileges. diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index a7b282e2d2da0..ff8639269c4cd 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1356,6 +1356,7 @@ func (e *DDLJobsReaderExec) Open(ctx context.Context) error { return err } e.sess = sess + sess.GetSessionVars().SetInTxn(true) err = sessiontxn.NewTxn(context.Background(), sess) if err != nil { return err @@ -1364,7 +1365,6 @@ func (e *DDLJobsReaderExec) Open(ctx context.Context) error { if err != nil { return err } - sess.GetSessionVars().SetInTxn(true) err = e.DDLJobRetriever.initial(txn, sess) if err != nil { return err diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 7d8bf04845f0b..90c6824877996 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -139,7 +139,7 @@ func TestViews(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DEFINER='root'@'localhost' VIEW test.v1 AS SELECT 1") - tk.MustQuery("select TABLE_COLLATION is null from INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE='VIEW'").Check(testkit.Rows("1")) + tk.MustQuery("select TABLE_COLLATION is null from INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE='VIEW'").Check(testkit.Rows("1", "1")) tk.MustQuery("SELECT * FROM information_schema.views WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_bin")) tk.MustQuery("SELECT table_catalog, table_schema, table_name, table_type, engine, version, row_format, table_rows, avg_row_length, data_length, max_data_length, index_length, data_free, auto_increment, update_time, check_time, table_collation, checksum, create_options, table_comment FROM information_schema.tables WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 VIEW VIEW")) } diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index 67cc805ff820c..48d85d813f205 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -1,4 +1,4 @@ -// Copyright 2021 PingCAP, Inc. +// 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. @@ -3770,6 +3770,7 @@ func TestSetVariables(t *testing.T) { tk.MustQuery("select @@global.tidb_enable_concurrent_ddl").Check(testkit.Rows("0")) require.False(t, variable.EnableConcurrentDDL.Load()) testkit.NewTestKit(t, store).MustQuery("select @@global.tidb_enable_concurrent_ddl").Check(testkit.Rows("0")) + tk.MustExec("set @@global.tidb_enable_concurrent_ddl=1") } func TestPreparePlanCache(t *testing.T) { diff --git a/expression/integration_test.go b/expression/integration_test.go index d0ec7283dd319..b551b3c5cc6d5 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4913,8 +4913,9 @@ func TestSchemaDMLNotChange(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) - tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1;") tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") + tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1;") tk2.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key, c_json json);") diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 09c9b9c4bdd73..efbb5f33d8af6 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -624,12 +624,21 @@ func (is *SessionTables) schemaTables(schema model.CIStr) *schemaTables { type SessionExtendedInfoSchema struct { InfoSchema LocalTemporaryTables *SessionTables + MdlTables *SessionTables } // TableByName implements InfoSchema.TableByName func (ts *SessionExtendedInfoSchema) TableByName(schema, table model.CIStr) (table.Table, error) { - if tbl, ok := ts.LocalTemporaryTables.TableByName(schema, table); ok { - return tbl, nil + if ts.LocalTemporaryTables != nil { + if tbl, ok := ts.LocalTemporaryTables.TableByName(schema, table); ok { + return tbl, nil + } + } + + if ts.MdlTables != nil { + if tbl, ok := ts.MdlTables.TableByName(schema, table); ok { + return tbl, nil + } } return ts.InfoSchema.TableByName(schema, table) @@ -637,8 +646,16 @@ func (ts *SessionExtendedInfoSchema) TableByName(schema, table model.CIStr) (tab // TableByID implements InfoSchema.TableByID func (ts *SessionExtendedInfoSchema) TableByID(id int64) (table.Table, bool) { - if tbl, ok := ts.LocalTemporaryTables.TableByID(id); ok { - return tbl, true + if ts.LocalTemporaryTables != nil { + if tbl, ok := ts.LocalTemporaryTables.TableByID(id); ok { + return tbl, true + } + } + + if ts.MdlTables != nil { + if tbl, ok := ts.MdlTables.TableByID(id); ok { + return tbl, true + } } return ts.InfoSchema.TableByID(id) @@ -650,14 +667,41 @@ func (ts *SessionExtendedInfoSchema) SchemaByTable(tableInfo *model.TableInfo) ( return nil, false } - if db, ok := ts.LocalTemporaryTables.SchemaByTable(tableInfo); ok { - return db, true + if ts.LocalTemporaryTables != nil { + if db, ok := ts.LocalTemporaryTables.SchemaByTable(tableInfo); ok { + return db, true + } } return ts.InfoSchema.SchemaByTable(tableInfo) } +// UpdateTableInfo implements InfoSchema.SchemaByTable. +func (ts *SessionExtendedInfoSchema) UpdateTableInfo(db *model.DBInfo, tableInfo table.Table) error { + if ts.MdlTables == nil { + ts.MdlTables = NewSessionTables() + } + err := ts.MdlTables.AddTable(db, tableInfo) + if err != nil { + return err + } + return nil +} + // HasTemporaryTable returns whether information schema has temporary table func (ts *SessionExtendedInfoSchema) HasTemporaryTable() bool { - return ts.LocalTemporaryTables.Count() > 0 || ts.InfoSchema.HasTemporaryTable() + return ts.LocalTemporaryTables != nil && ts.LocalTemporaryTables.Count() > 0 || ts.InfoSchema.HasTemporaryTable() +} + +// AttachMDLTableInfoSchema attach MDL related table information schema to is +func AttachMDLTableInfoSchema(is InfoSchema) InfoSchema { + mdlTables := NewSessionTables() + if iss, ok := is.(*SessionExtendedInfoSchema); ok { + iss.MdlTables = mdlTables + return iss + } + return &SessionExtendedInfoSchema{ + InfoSchema: is, + MdlTables: mdlTables, + } } diff --git a/infoschema/tables.go b/infoschema/tables.go index 4c4dd2940183b..209bed56cca62 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1452,6 +1452,7 @@ var tableTiDBTrxCols = []columnInfo{ {name: txninfo.UserStr, tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: txninfo.DBStr, tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, {name: txninfo.AllSQLDigestsStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, + {name: txninfo.RelatedTableIDsStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the table IDs that the transaction has accessed"}, } var tableDeadlocksCols = []columnInfo{ diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index e848acb4be481..0e315c78186fc 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1408,8 +1408,8 @@ func TestTiDBTrx(t *testing.T) { tk.Session().SetSessionManager(sm) tk.MustQuery("select * from information_schema.TIDB_TRX;").Check(testkit.Rows( - "424768545227014155 2021-05-07 12:56:48.001000 "+digest.String()+" update `test_tidb_trx` set `i` = `i` + ? Idle 1 19 2 root test []", - "425070846483628033 2021-05-20 21:16:35.778000 LockWaiting 2021-05-20 13:18:30.123456 0 0 10 user1 db1 [\"sql1\",\"sql2\",\""+digest.String()+"\"]")) + "424768545227014155 2021-05-07 12:56:48.001000 "+digest.String()+" update `test_tidb_trx` set `i` = `i` + ? Idle 1 19 2 root test [] ", + "425070846483628033 2021-05-20 21:16:35.778000 LockWaiting 2021-05-20 13:18:30.123456 0 0 10 user1 db1 [\"sql1\",\"sql2\",\""+digest.String()+"\"] ")) // Test the all_sql_digests column can be directly passed to the tidb_decode_sql_digests function. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/sqlDigestRetrieverSkipRetrieveGlobal", "return")) diff --git a/meta/meta.go b/meta/meta.go index 6dff27ba81a27..d0884f722e70e 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -546,6 +546,12 @@ func (m *Meta) SetDDLTables() error { return errors.Trace(err) } +// SetMDLTables write a key into storage. +func (m *Meta) SetMDLTables() error { + err := m.txn.Set(mDDLTableVersion, []byte("2")) + return errors.Trace(err) +} + // CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID. func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { id, err := m.GetSystemDBID() @@ -591,6 +597,15 @@ func (m *Meta) CheckDDLTableExists() (bool, error) { return len(v) != 0, nil } +// CheckMDLTableExists check if the tables related to concurrent DDL exists. +func (m *Meta) CheckMDLTableExists() (bool, error) { + v, err := m.txn.Get(mDDLTableVersion) + if err != nil { + return false, errors.Trace(err) + } + return bytes.Equal(v, []byte("2")), nil +} + // SetFlashbackClusterJobID set flashback cluster jobID func (m *Meta) SetFlashbackClusterJobID(jobID int64) error { return errors.Trace(m.txn.Set(mInFlashbackCluster, m.jobIDKey(jobID))) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index e59985c8e1e12..1abe487f8b2c0 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4347,7 +4347,12 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as return nil, err } + tbl, err = tryLockMDLAndUpdateSchemaIfNecessary(b.ctx, dbName, tbl, b.is) + if err != nil { + return nil, err + } tableInfo := tbl.Meta() + if b.isCreateView && tableInfo.TempTableType == model.TempTableLocal { return nil, ErrViewSelectTemporaryTable.GenWithStackByArgs(tn.Name) } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 288de1bd7b541..4e320c62e2e50 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -426,6 +426,7 @@ func TestDupRandJoinCondsPushDown(t *testing.T) { } func TestTablePartition(t *testing.T) { + variable.EnableMDL.Store(false) definitions := []model.PartitionDefinition{ { ID: 41, @@ -1015,6 +1016,7 @@ func TestAggPrune(t *testing.T) { } func TestVisitInfo(t *testing.T) { + variable.EnableMDL.Store(false) tests := []struct { sql string ans []visitInfo diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 32ac16e7164f8..58d254d958c86 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -2565,6 +2565,7 @@ func TestPlanCacheWithRCWhenInfoSchemaChange(t *testing.T) { tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk1.MustExec("drop table if exists t1") tk1.MustExec("create table t1(id int primary key, c int, index ic (c))") @@ -2604,6 +2605,7 @@ func TestConsistencyBetweenPrepareExecuteAndNormalSql(t *testing.T) { tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk1.MustExec("use test") @@ -2678,6 +2680,7 @@ func TestCacheHitInRc(t *testing.T) { tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk1.MustExec("use test") diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 059224715a5d4..09f3f09fc3a6a 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -21,6 +21,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta/autoid" @@ -44,7 +45,9 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/domainutil" + "github.com/pingcap/tidb/util/logutil" utilparser "github.com/pingcap/tidb/util/parser" + "go.uber.org/zap" ) // PreprocessOpt presents optional parameters to `Preprocess` method. @@ -1480,6 +1483,15 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { p.err = err return } + currentDB := p.ctx.GetSessionVars().CurrentDB + if tn.Schema.String() != "" { + currentDB = tn.Schema.L + } + table, err = tryLockMDLAndUpdateSchemaIfNecessary(p.ctx, model.NewCIStr(currentDB), table, p.ensureInfoSchema()) + if err != nil { + p.err = err + return + } tableInfo := table.Meta() dbInfo, _ := p.ensureInfoSchema().SchemaByTable(tableInfo) @@ -1685,3 +1697,118 @@ func (p *preprocessor) hasAutoConvertWarning(colDef *ast.ColumnDef) bool { } return false } + +func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model.CIStr, tbl table.Table, is infoschema.InfoSchema) (table.Table, error) { + if !sctx.GetSessionVars().TxnCtx.EnableMDL { + return tbl, nil + } + if is.SchemaMetaVersion() == 0 { + return tbl, nil + } + skipLock := false + if sctx.GetSessionVars().SnapshotInfoschema != nil { + return tbl, nil + } + if sctx.GetSessionVars().TxnCtx.IsStaleness { + return tbl, nil + } + if tbl.Meta().TempTableType == model.TempTableLocal { + // Don't attach, don't lock. + return tbl, nil + } else if tbl.Meta().TempTableType == model.TempTableGlobal { + skipLock = true + } + if IsAutoCommitTxn(sctx) && sctx.GetSessionVars().StmtCtx.IsReadOnly { + return tbl, nil + } + tableInfo := tbl.Meta() + if _, ok := sctx.GetSessionVars().GetRelatedTableForMDL().Load(tableInfo.ID); !ok { + if se, ok := is.(*infoschema.SessionExtendedInfoSchema); ok && skipLock { + if se.MdlTables == nil { + return tbl, nil + } + if _, ok := se.MdlTables.TableByID(tableInfo.ID); ok { + // Already attach. + return tbl, nil + } + } + + // We need to write 0 to the map to block the txn. + // If we don't write 0, consider the following case: + // the background mdl check loop gets the mdl lock from this txn. But the domain infoSchema may be changed before writing the ver to the map. + // In this case, this TiDB wrongly gets the mdl lock. + if !skipLock { + sctx.GetSessionVars().GetRelatedTableForMDL().Store(tableInfo.ID, int64(0)) + } + domainSchema := domain.GetDomain(sctx).InfoSchema() + domainSchemaVer := domainSchema.SchemaMetaVersion() + if !skipLock { + sctx.GetSessionVars().GetRelatedTableForMDL().Store(tableInfo.ID, domainSchemaVer) + } + + var err error + tbl, err = domainSchema.TableByName(dbName, tableInfo.Name) + if err != nil { + return nil, err + } + // Check the table change, if adding new public index or modify a column, we need to handle them. + if !sctx.GetSessionVars().IsPessimisticReadConsistency() { + var copyTableInfo *model.TableInfo + for i, idx := range tbl.Meta().Indices { + if idx.State != model.StatePublic { + continue + } + found := false + for _, idxx := range tableInfo.Indices { + if idx.Name.L == idxx.Name.L && idx.ID == idxx.ID { + found = true + break + } + } + if !found { + if copyTableInfo == nil { + copyTableInfo = tbl.Meta().Clone() + } + copyTableInfo.Indices[i].State = model.StateWriteReorganization + dbInfo, _ := domainSchema.SchemaByName(dbName) + allocs := autoid.NewAllocatorsFromTblInfo(sctx.GetStore(), dbInfo.ID, copyTableInfo) + tbl, err = table.TableFromMeta(allocs, copyTableInfo) + if err != nil { + return nil, err + } + } + } + // Check the column change. + for _, col := range tbl.Meta().Columns { + if col.State != model.StatePublic { + continue + } + found := false + for _, coll := range tableInfo.Columns { + if col.Name.L == coll.Name.L && col.ID != coll.ID { + logutil.BgLogger().Info("public column changed", zap.String("column", col.Name.L), zap.String("old_col", coll.Name.L), zap.Int64("new id", col.ID), zap.Int64("old id", coll.ID)) + found = true + break + } + } + if found { + return nil, ErrSchemaChanged.GenWithStack("public column %s has changed", col.Name) + } + } + } + + se, ok := is.(*infoschema.SessionExtendedInfoSchema) + if !ok { + se = infoschema.AttachMDLTableInfoSchema(is).(*infoschema.SessionExtendedInfoSchema) + sessiontxn.GetTxnManager(sctx).SetTxnInfoSchema(se) + sctx.GetSessionVars().TxnCtx.InfoSchema = se + } + db, _ := domainSchema.SchemaByTable(tbl.Meta()) + err = se.UpdateTableInfo(db, tbl) + if err != nil { + return nil, err + } + return tbl, nil + } + return tbl, nil +} diff --git a/planner/memo/BUILD.bazel b/planner/memo/BUILD.bazel index c0523d2600e3e..f64aab9f76e91 100644 --- a/planner/memo/BUILD.bazel +++ b/planner/memo/BUILD.bazel @@ -38,6 +38,7 @@ go_test( "//parser/model", "//planner/core", "//planner/property", + "//sessionctx/variable", "//testkit/testsetup", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", diff --git a/planner/memo/group_test.go b/planner/memo/group_test.go index c7d377f38d08b..e601fea20536e 100644 --- a/planner/memo/group_test.go +++ b/planner/memo/group_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/parser/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/stretchr/testify/require" ) @@ -88,6 +89,7 @@ func TestGroupExists(t *testing.T) { } func TestGroupFingerPrint(t *testing.T) { + variable.EnableMDL.Store(false) p := parser.New() stmt1, err := p.ParseOneStmt("select * from t where a > 1 and a < 100", "", "") require.NoError(t, err) @@ -223,6 +225,7 @@ func TestFirstElemAfterDelete(t *testing.T) { } func TestBuildKeyInfo(t *testing.T) { + variable.EnableMDL.Store(false) p := parser.New() ctx := plannercore.MockContext() is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()}) diff --git a/server/server.go b/server/server.go index 576a50eac9177..7602fec4e05ba 100644 --- a/server/server.go +++ b/server/server.go @@ -871,3 +871,14 @@ func setSystemTimeZoneVariable() { variable.SetSysVar("system_time_zone", tz) }) } + +// CheckOldRunningTxn implements SessionManager interface. +func (s *Server) CheckOldRunningTxn(job2ver map[int64]int64, job2ids map[int64]string) { + s.rwlock.RLock() + defer s.rwlock.RUnlock() + for _, client := range s.clients { + if client.ctx.Session != nil { + session.RemoveLockDDLJobs(client.ctx.Session, job2ver, job2ids) + } + } +} diff --git a/server/tidb_test.go b/server/tidb_test.go index 5bcf04f14a13c..92c249041dd32 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -874,6 +874,9 @@ func TestInternalSessionTxnStartTS(t *testing.T) { se, err := session.CreateSession4Test(ts.store) require.NoError(t, err) + _, err = se.Execute(context.Background(), "set global tidb_enable_metadata_lock=0") + require.NoError(t, err) + count := 10 stmts := make([]ast.StmtNode, count) for i := 0; i < count; i++ { diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 0131d75792919..0f78838b63fd4 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -137,6 +137,7 @@ go_test( "//parser/mysql", "//parser/terror", "//planner/core", + "//server", "//sessionctx", "//sessionctx/variable", "//sessiontxn", @@ -144,6 +145,7 @@ go_test( "//store/mockstore", "//table", "//tablecodec", + "//telemetry", "//testkit", "//testkit/external", "//testkit/testdata", diff --git a/session/bootstrap.go b/session/bootstrap.go index 5518fbde9be9a..1a9c79170615f 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -424,6 +424,10 @@ const ( CreateAdvisoryLocks = `CREATE TABLE IF NOT EXISTS mysql.advisory_locks ( lock_name VARCHAR(64) NOT NULL PRIMARY KEY );` + // CreateMDLView is a view about metadata locks. + CreateMDLView = `CREATE OR REPLACE VIEW mysql.tidb_mdl_view as ( + select JOB_ID, DB_NAME, TABLE_NAME, QUERY, SESSION_ID, TxnStart, TIDB_DECODE_SQL_DIGESTS(ALL_SQL_DIGESTS, 4096) AS SQL_DIGESTS from information_schema.ddl_jobs, information_schema.CLUSTER_TIDB_TRX, information_schema.CLUSTER_PROCESSLIST where ddl_jobs.STATE = 'running' and find_in_set(ddl_jobs.table_id, CLUSTER_TIDB_TRX.RELATED_TABLE_IDS) and CLUSTER_TIDB_TRX.SESSION_ID=CLUSTER_PROCESSLIST.ID + );` ) // bootstrap initiates system DB for a store. @@ -627,11 +631,12 @@ const ( version92 = 92 // version93 converts oom-use-tmp-storage to a sysvar version93 = 93 + version94 = 94 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version93 +var currentBootstrapVersion int64 = version94 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -730,6 +735,7 @@ var ( upgradeToVer90, upgradeToVer91, upgradeToVer93, + upgradeToVer94, } ) @@ -1918,6 +1924,13 @@ func upgradeToVer93(s Session, ver int64) { importConfigOption(s, "oom-use-tmp-storage", variable.TiDBEnableTmpStorageOnOOM, valStr) } +func upgradeToVer94(s Session, ver int64) { + if ver >= version94 { + return + } + mustExecute(s, CreateMDLView) +} + func writeOOMAction(s Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, @@ -2012,6 +2025,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateAnalyzeJobs) // Create advisory_locks table. mustExecute(s, CreateAdvisoryLocks) + // Create mdl view. + mustExecute(s, CreateMDLView) } // inTestSuite checks if we are bootstrapping in the context of tests. diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 7adc0b58666cd..6c3b459d8ee72 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/telemetry" "github.com/stretchr/testify/require" ) @@ -142,11 +143,14 @@ func TestBootstrapWithError(t *testing.T) { store: store, sessionVars: variable.NewSessionVars(), } + se.functionUsageMu.builtinFunctionUsage = make(telemetry.BuiltinFunctionsUsage) se.txn.init() se.mu.values = make(map[fmt.Stringer]interface{}) se.SetValue(sessionctx.Initing, true) err := InitDDLJobTables(store) require.NoError(t, err) + err = InitMDLTable(store) + require.NoError(t, err) dom, err := domap.Get(store) require.NoError(t, err) domain.BindDomain(se, dom) diff --git a/session/schema_test.go b/session/schema_test.go index bc3fe1731b7b2..f312d253418f2 100644 --- a/session/schema_test.go +++ b/session/schema_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" @@ -44,6 +45,10 @@ func createMockStoreForSchemaTest(t *testing.T, opts ...mockstore.MockTiKVStoreO dom.SetStatsUpdating(true) + sv := server.CreateMockServer(t, store) + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + t.Cleanup(func() { dom.Close() require.NoError(t, store.Close()) @@ -60,6 +65,7 @@ func TestPrepareStmtCommitWhenSchemaChanged(t *testing.T) { tk2 := testkit.NewTestKit(t, store) tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk1.MustExec("create table t (a int, b int)") @@ -85,6 +91,7 @@ func TestCommitWhenSchemaChanged(t *testing.T) { setTxnTk := testkit.NewTestKit(t, store) setTxnTk.MustExec("set global tidb_txn_mode=''") + setTxnTk.MustExec("set global tidb_enable_metadata_lock=0") tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) @@ -142,6 +149,7 @@ func TestRetrySchemaChange(t *testing.T) { setTxnTk := testkit.NewTestKit(t, store) setTxnTk.MustExec("set global tidb_txn_mode=''") + setTxnTk.MustExec("set global tidb_enable_metadata_lock=0") tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) diff --git a/session/session.go b/session/session.go index 29d3809af4798..d61e2ebfba93d 100644 --- a/session/session.go +++ b/session/session.go @@ -570,6 +570,11 @@ func (s *session) TxnInfo() *txninfo.TxnInfo { txnInfo.ConnectionID = processInfo.ID txnInfo.Username = processInfo.User txnInfo.CurrentDB = processInfo.DB + txnInfo.RelatedTableIDs = make(map[int64]struct{}) + s.GetSessionVars().GetRelatedTableForMDL().Range(func(key, value interface{}) bool { + txnInfo.RelatedTableIDs[key.(int64)] = struct{}{} + return true + }) return &txnInfo } @@ -649,8 +654,12 @@ func (s *session) doCommit(ctx context.Context) error { } physicalTableIDs = append(physicalTableIDs, id) } + needCheckSchema := true // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) + if s.GetSessionVars().TxnCtx != nil { + needCheckSchema = !s.GetSessionVars().TxnCtx.EnableMDL + } + s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs, needCheckSchema)) s.txn.SetOption(kv.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) s.txn.SetOption(kv.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if sessVars.EnableAmendPessimisticTxn { @@ -2618,6 +2627,7 @@ var ( {ddl.ReorgTableSQL, ddl.ReorgTableID}, {ddl.HistoryTableSQL, ddl.HistoryTableID}, } + mdlTable = "create table mysql.tidb_mdl_info(job_id BIGINT NOT NULL PRIMARY KEY, version BIGINT NOT NULL, table_ids text(65535));" ) // InitDDLJobTables is to create tidb_ddl_job, tidb_ddl_reorg and tidb_ddl_history. @@ -2639,7 +2649,7 @@ func InitDDLJobTables(store kv.Storage) error { return errors.Trace(err) } if id >= meta.MaxGlobalID { - return errors.Errorf("It is unreasonable that the global ID grows such a big value: %d, please concat TiDB team", id) + return errors.Errorf("It is unreasonable that the global ID grows such a big value: %d, please contact TiDB team", id) } stmt, err := p.ParseOneStmt(tbl.SQL, "", "") if err != nil { @@ -2661,6 +2671,46 @@ func InitDDLJobTables(store kv.Storage) error { }) } +// InitMDLTable is to create tidb_mdl_info, which is used for metadata lock. +func InitMDLTable(store kv.Storage) error { + return kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + exists, err := t.CheckMDLTableExists() + if err != nil || exists { + return errors.Trace(err) + } + dbID, err := t.CreateMySQLDatabaseIfNotExists() + if err != nil { + return err + } + p := parser.New() + id, err := t.GetGlobalID() + if err != nil { + return errors.Trace(err) + } + if id >= meta.MaxGlobalID { + return errors.Errorf("It is unreasonable that the global ID grows such a big value: %d, please contact TiDB team", id) + } + stmt, err := p.ParseOneStmt(mdlTable, "", "") + if err != nil { + return errors.Trace(err) + } + tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + if err != nil { + return errors.Trace(err) + } + tblInfo.State = model.StatePublic + tblInfo.ID = ddl.MDLTableID + tblInfo.UpdateTS = t.StartTS + err = t.CreateTableOrView(dbID, tblInfo) + if err != nil { + return errors.Trace(err) + } + + return t.SetMDLTables() + }) +} + // BootstrapSession runs the first time when the TiDB server start. func BootstrapSession(store kv.Storage) (*domain.Domain, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) @@ -2678,6 +2728,10 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { if err != nil { return nil, err } + err = InitMDLTable(store) + if err != nil { + return nil, err + } ver := getStoreBootstrapVersion(store) if ver == notBootstrapped { runInBootstrapSession(store, bootstrap) @@ -3476,3 +3530,20 @@ func (s *session) setRequestSource(ctx context.Context, stmtLabel string, stmtNo } } } + +// RemoveLockDDLJobs removes the DDL jobs which doesn't get the metadata lock from job2ver. +func RemoveLockDDLJobs(s Session, job2ver map[int64]int64, job2ids map[int64]string) { + if s.GetSessionVars().InRestrictedSQL { + return + } + s.GetSessionVars().GetRelatedTableForMDL().Range(func(tblID, value any) bool { + for jobID, ver := range job2ver { + ids := util.Str2Int64Map(job2ids[jobID]) + if _, ok := ids[tblID.(int64)]; ok && value.(int64) < ver { + delete(job2ver, jobID) + logutil.BgLogger().Debug("old running transaction block DDL", zap.Int64("table ID", tblID.(int64)), zap.Uint64("conn ID", s.GetSessionVars().ConnectionID)) + } + } + return true + }) +} diff --git a/session/session_test/session_test.go b/session/session_test/session_test.go index 48519d9a39868..b2e5f956e51b9 100644 --- a/session/session_test/session_test.go +++ b/session/session_test/session_test.go @@ -60,6 +60,7 @@ func TestSchemaCheckerSQL(t *testing.T) { store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second) setTxnTk := testkit.NewTestKit(t, store) + setTxnTk.MustExec("set global tidb_enable_metadata_lock=0") setTxnTk.MustExec("set global tidb_txn_mode=''") tk := testkit.NewTestKit(t, store) tk1 := testkit.NewTestKit(t, store) @@ -138,6 +139,7 @@ func TestSchemaCheckerTempTable(t *testing.T) { tk2 := testkit.NewTestKit(t, store) tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") // create table diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index 8bd5bf99cecc2..640e16474b4bd 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -16,6 +16,8 @@ package txninfo import ( "encoding/json" + "fmt" + "strings" "time" "github.com/pingcap/tidb/metrics" @@ -124,6 +126,8 @@ const ( DBStr = "DB" // AllSQLDigestsStr is the column name of the TIDB_TRX table's AllSQLDigests column. AllSQLDigestsStr = "ALL_SQL_DIGESTS" + // RelatedTableIDsStr is the table id of the TIDB_TRX table's RelatedTableIDs column. + RelatedTableIDsStr = "RELATED_TABLE_IDS" ) // TxnRunningStateStrs is the names of the TxnRunningStates @@ -168,6 +172,8 @@ type TxnInfo struct { Username string // The schema this transaction works on CurrentDB string + // The related table IDs. + RelatedTableIDs map[int64]struct{} } var columnValueGetterMap = map[string]func(*TxnInfo) types.Datum{ @@ -227,6 +233,20 @@ var columnValueGetterMap = map[string]func(*TxnInfo) types.Datum{ } return types.NewDatum(string(res)) }, + RelatedTableIDsStr: func(info *TxnInfo) types.Datum { + relatedTableIDs := info.RelatedTableIDs + str := strings.Builder{} + first := true + for tblID := range relatedTableIDs { + if !first { + str.Write([]byte(",")) + } else { + first = false + } + str.WriteString(fmt.Sprintf("%d", tblID)) + } + return types.NewDatum(str.String()) + }, } // ToDatum Converts the `TxnInfo`'s specified column to `Datum` to show in the `TIDB_TRX` table. diff --git a/session/txnmanager.go b/session/txnmanager.go index 59313a8687b9f..65652428f7481 100644 --- a/session/txnmanager.go +++ b/session/txnmanager.go @@ -70,6 +70,13 @@ func (m *txnManager) GetTxnInfoSchema() infoschema.InfoSchema { return nil } +func (m *txnManager) SetTxnInfoSchema(is infoschema.InfoSchema) { + if m.ctxProvider == nil { + return + } + m.ctxProvider.SetTxnInfoSchema(is) +} + func (m *txnManager) GetStmtReadTS() (uint64, error) { if m.ctxProvider == nil { return 0, errors.New("context provider not set") diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 470ff975e7724..2b6423f5a9b79 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -312,7 +312,8 @@ type StatementContext struct { IsSQLRegistered atomic2.Bool // IsSQLAndPlanRegistered uses to indicate whether the SQL and plan has been registered for TopSQL. IsSQLAndPlanRegistered atomic2.Bool - + // IsReadOnly uses to indicate whether the SQL is read-only. + IsReadOnly bool // StatsLoadStatus records StatsLoadedStatus for the index/column which is used in query StatsLoadStatus map[model.TableItemID]string // IsSyncStatsFailed indicates whether any failure happened during sync stats diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e64e740795114..9f1e1ff05c45d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -221,6 +221,10 @@ type TxnCtxNoNeedToRestore struct { // TemporaryTables is used to store transaction-specific information for global temporary tables. // It can also be stored in sessionCtx with local temporary tables, but it's easier to clean this data after transaction ends. TemporaryTables map[int64]tableutil.TempTable + // EnableMDL indicates whether to enable the MDL lock for the transaction. + EnableMDL bool + // relatedTableForMDL records the `lock` table for metadata lock. It maps from int64 to int64(version). + relatedTableForMDL *sync.Map } // SavepointRecord indicates a transaction's savepoint record. @@ -320,6 +324,8 @@ func (tc *TransactionContext) Cleanup() { tc.pessimisticLockCache = nil tc.IsStaleness = false tc.Savepoints = nil + tc.EnableMDL = false + tc.relatedTableForMDL = nil } // ClearDelta clears the delta map. @@ -3042,6 +3048,16 @@ func (s *SessionVars) GetNegateStrMatchDefaultSelectivity() float64 { return 1 - s.GetStrMatchDefaultSelectivity() } +// GetRelatedTableForMDL gets the related table for metadata lock. +func (s *SessionVars) GetRelatedTableForMDL() *sync.Map { + s.TxnCtx.tdmLock.Lock() + defer s.TxnCtx.tdmLock.Unlock() + if s.TxnCtx.relatedTableForMDL == nil { + s.TxnCtx.relatedTableForMDL = new(sync.Map) + } + return s.TxnCtx.relatedTableForMDL +} + // EnableForceInlineCTE returns the session variable enableForceInlineCTE func (s *SessionVars) EnableForceInlineCTE() bool { return s.enableForceInlineCTE diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index edd9a4b82043a..ce3106469bd60 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -861,6 +861,17 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(s *SessionVars) (string, error) { return BoolToOnOff(EnableConcurrentDDL.Load()), nil }}, + {Scope: ScopeGlobal, Name: TiDBEnableMDL, Value: BoolToOnOff(DefTiDBEnableMDL), Type: TypeBool, SetGlobal: func(vars *SessionVars, val string) error { + if EnableMDL.Load() != TiDBOptOn(val) { + err := SwitchMDL(TiDBOptOn(val)) + if err != nil { + return err + } + } + return nil + }, GetGlobal: func(vars *SessionVars) (string, error) { + return BoolToOnOff(EnableMDL.Load()), nil + }}, {Scope: ScopeGlobal, Name: TiDBEnableNoopVariables, Value: BoolToOnOff(DefTiDBEnableNoopVariables), Type: TypeEnum, PossibleValues: []string{Off, On}, SetGlobal: func(s *SessionVars, val string) error { EnableNoopVariables.Store(TiDBOptOn(val)) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 806bf9ccca002..2c8f6cdb0beb9 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -645,6 +645,9 @@ const ( // TiDBEnableLocalTxn indicates whether to enable Local Txn. TiDBEnableLocalTxn = "tidb_enable_local_txn" + // TiDBEnableMDL indicates whether to enable MDL. + TiDBEnableMDL = "tidb_enable_metadata_lock" + // TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client. TiDBTSOClientBatchMaxWaitTime = "tidb_tso_client_batch_max_wait_time" @@ -1048,6 +1051,7 @@ const ( DefEnableTiDBGCAwareMemoryTrack = true DefTiDBDefaultStrMatchSelectivity = 0.8 DefTiDBEnableTmpStorageOnOOM = true + DefTiDBEnableMDL = false DefTiFlashFastScan = false DefTiDBEnableFastReorg = false DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB @@ -1107,6 +1111,7 @@ var ( EnableConcurrentDDL = atomic.NewBool(DefTiDBEnableConcurrentDDL) DDLForce2Queue = atomic.NewBool(false) EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) + EnableMDL = atomic.NewBool(DefTiDBEnableMDL) // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) // DDLDiskQuota is the temporary variable for set disk quota for lightning @@ -1125,6 +1130,8 @@ var ( SetStatsCacheCapacity atomic.Value // SwitchConcurrentDDL is the func registered by DDL to switch concurrent DDL. SwitchConcurrentDDL func(bool) error = nil + // SwitchMDL is the func registered by DDL to switch MDL. + SwitchMDL func(bool2 bool) error = nil // EnableDDL is the func registered by ddl to enable running ddl in this instance. EnableDDL func() error = nil // DisableDDL is the func registered by ddl to disable running ddl in this instance. diff --git a/sessiontxn/interface.go b/sessiontxn/interface.go index b76c0a020481a..85d9217a90c0f 100644 --- a/sessiontxn/interface.go +++ b/sessiontxn/interface.go @@ -117,6 +117,8 @@ type TxnContextProvider interface { TxnAdvisable // GetTxnInfoSchema returns the information schema used by txn GetTxnInfoSchema() infoschema.InfoSchema + // SetTxnInfoSchema sets the information schema used by txn. + SetTxnInfoSchema(is infoschema.InfoSchema) // GetTxnScope returns the current txn scope GetTxnScope() string // GetReadReplicaScope returns the read replica scope @@ -151,6 +153,8 @@ type TxnManager interface { // If the session is not in any transaction, for example: between two autocommit statements, // this method will return the latest information schema in session that is same with `sessionctx.GetDomainInfoSchema()` GetTxnInfoSchema() infoschema.InfoSchema + // SetTxnInfoSchema sets the information schema used by txn. + SetTxnInfoSchema(infoschema.InfoSchema) // GetTxnScope returns the current txn scope GetTxnScope() string // GetReadReplicaScope returns the read replica scope diff --git a/sessiontxn/isolation/base.go b/sessiontxn/isolation/base.go index 8ffcee0f453d6..e7e6891be1f0d 100644 --- a/sessiontxn/isolation/base.go +++ b/sessiontxn/isolation/base.go @@ -120,6 +120,9 @@ func (p *baseTxnContextProvider) OnInitialize(ctx context.Context, tp sessiontxn p.onInitializeTxnCtx(txnCtx) } sessVars.TxnCtx = txnCtx + if variable.EnableMDL.Load() { + sessVars.TxnCtx.EnableMDL = true + } txn, err := p.sctx.Txn(false) if err != nil { @@ -141,6 +144,10 @@ func (p *baseTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema { return p.infoSchema } +func (p *baseTxnContextProvider) SetTxnInfoSchema(is infoschema.InfoSchema) { + p.infoSchema = is +} + // GetTxnScope returns the current txn scope func (p *baseTxnContextProvider) GetTxnScope() string { return p.sctx.GetSessionVars().TxnCtx.TxnScope diff --git a/sessiontxn/staleread/provider.go b/sessiontxn/staleread/provider.go index 4d0eb4a1b9619..84904ab18feb9 100644 --- a/sessiontxn/staleread/provider.go +++ b/sessiontxn/staleread/provider.go @@ -53,6 +53,11 @@ func (p *StalenessTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema { return p.is } +// SetTxnInfoSchema sets the information schema used by txn. +func (p *StalenessTxnContextProvider) SetTxnInfoSchema(is infoschema.InfoSchema) { + p.is = is +} + // GetTxnScope returns the current txn scope func (p *StalenessTxnContextProvider) GetTxnScope() string { return p.sctx.GetSessionVars().TxnCtx.TxnScope diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 68b7e4ef66206..726782fe30088 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -151,6 +151,7 @@ func TestTxnContextInExplicitTxn(t *testing.T) { store, do := setupTxnContextTest(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("use test") se := tk.Session() @@ -252,6 +253,7 @@ func TestTxnContextWithAutocommitFalse(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") se := tk.Session() tk2 := testkit.NewTestKit(t, store) @@ -677,6 +679,7 @@ func TestTxnContextForStaleReadInPrepare(t *testing.T) { func TestTxnContextPreparedStmtWithForUpdate(t *testing.T) { store, do := setupTxnContextTest(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("use test") se := tk.Session() diff --git a/table/temptable/infoschema.go b/table/temptable/infoschema.go index aff7e97ba7855..b8523a1ed5595 100644 --- a/table/temptable/infoschema.go +++ b/table/temptable/infoschema.go @@ -25,8 +25,8 @@ func AttachLocalTemporaryTableInfoSchema(sctx sessionctx.Context, is infoschema. if localTemporaryTables == nil { return is } - - if _, ok := is.(*infoschema.SessionExtendedInfoSchema); ok { + if se, ok := is.(*infoschema.SessionExtendedInfoSchema); ok { + se.LocalTemporaryTables = localTemporaryTables return is } @@ -39,7 +39,9 @@ func AttachLocalTemporaryTableInfoSchema(sctx sessionctx.Context, is infoschema. // DetachLocalTemporaryTableInfoSchema detach local temporary table information schema from is func DetachLocalTemporaryTableInfoSchema(is infoschema.InfoSchema) infoschema.InfoSchema { if attachedInfoSchema, ok := is.(*infoschema.SessionExtendedInfoSchema); ok { - return attachedInfoSchema.InfoSchema + newIs := *attachedInfoSchema + newIs.LocalTemporaryTables = nil + return &newIs } return is diff --git a/testkit/mocksessionmanager.go b/testkit/mocksessionmanager.go index 23238619e2f3e..080f5b7c56d80 100644 --- a/testkit/mocksessionmanager.go +++ b/testkit/mocksessionmanager.go @@ -83,3 +83,10 @@ func (*MockSessionManager) DeleteInternalSession(interface{}) {} func (*MockSessionManager) GetInternalSessionStartTSList() []uint64 { return nil } + +// CheckOldRunningTxn is to get all startTS of every transactions running in the current internal sessions +func (msm *MockSessionManager) CheckOldRunningTxn(job2ver map[int64]int64, job2ids map[int64]string) { + for _, se := range msm.conn { + session.RemoveLockDDLJobs(se, job2ver, job2ids) + } +} diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 0f7a107389836..12895bc217df5 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -1839,6 +1839,7 @@ func TestPessimisticTxnWithDDLAddDropColumn(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_metadata_lock=0") tk.MustExec("use test") tk2.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -1870,6 +1871,7 @@ func TestPessimisticTxnWithDDLChangeColumn(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") @@ -2136,6 +2138,7 @@ func TestAmendTxnVariable(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk3 := testkit.NewTestKit(t, store) @@ -2541,6 +2544,7 @@ func TestAmendWithColumnTypeChange(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1;") @@ -2893,6 +2897,7 @@ func TestAmendForIndexChange(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk.MustExec("set tidb_enable_amend_pessimistic_txn = ON;") @@ -2967,6 +2972,7 @@ func TestAmendForColumnChange(t *testing.T) { tk := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=0") tk2.MustExec("use test") tk.MustExec("set tidb_enable_amend_pessimistic_txn = ON;") diff --git a/util/processinfo.go b/util/processinfo.go index 57a1a0e92abc1..f3ac796bb4dbb 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -180,6 +180,8 @@ type SessionManager interface { DeleteInternalSession(se interface{}) // GetInternalSessionStartTSList gets all startTS of every transactions running in the current internal sessions. GetInternalSessionStartTSList() []uint64 + // CheckOldRunningTxn checks if there is an old transaction running in the current sessions + CheckOldRunningTxn(job2ver map[int64]int64, job2ids map[int64]string) } // GlobalConnID is the global connection ID, providing UNIQUE connection IDs across the whole TiDB cluster. diff --git a/util/util.go b/util/util.go index 9bf6b085882de..be43a3ca4ad90 100644 --- a/util/util.go +++ b/util/util.go @@ -18,6 +18,8 @@ import ( "encoding/json" "io/ioutil" "net/http" + "strconv" + "strings" "github.com/pingcap/errors" ) @@ -86,3 +88,14 @@ func ChanMap[T, R any](c <-chan T, f func(T) R) <-chan R { }() return outCh } + +// Str2Int64Map converts a string to a map[int64]struct{}. +func Str2Int64Map(str string) map[int64]struct{} { + strs := strings.Split(str, ",") + res := make(map[int64]struct{}, len(strs)) + for _, s := range strs { + id, _ := strconv.ParseInt(s, 10, 64) + res[id] = struct{}{} + } + return res +}