diff --git a/errors.toml b/errors.toml index cab50305f90be..2553e73407b1e 100644 --- a/errors.toml +++ b/errors.toml @@ -2946,6 +2946,16 @@ error = ''' Cannot set resource group for a role ''' +["schema:8258"] +error = ''' +Table %s is in mode %s +''' + +["schema:8259"] +error = ''' +Invalid mode set from (or by default) %s to %s for table %s +''' + ["server:1040"] error = ''' Too many connections diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 2dc23911008ce..82ab3005840a4 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -66,6 +66,7 @@ go_library( "stat.go", "table.go", "table_lock.go", + "table_mode.go", "ttl.go", ], importpath = "github.com/pingcap/tidb/pkg/ddl", @@ -269,6 +270,7 @@ go_test( "schema_test.go", "sequence_test.go", "stat_test.go", + "table_mode_test.go", "table_modify_test.go", "table_split_test.go", "table_test.go", diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 936fcb325cd80..9bfa1fbc134ed 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -118,6 +118,7 @@ type Executor interface { RenameTable(ctx sessionctx.Context, stmt *ast.RenameTableStmt) error LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error UnlockTables(ctx sessionctx.Context, lockedTables []model.TableLockTpInfo) error + AlterTableMode(ctx sessionctx.Context, args *model.AlterTableModeArgs) error CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error RepairTable(ctx sessionctx.Context, createStmt *ast.CreateTableStmt) error @@ -1059,8 +1060,18 @@ func (e *executor) createTableWithInfoJob( if oldTable, err := is.TableByName(e.ctx, schema.Name, tbInfo.Name); err == nil { err = infoschema.ErrTableExists.GenWithStackByArgs(ast.Ident{Schema: schema.Name, Name: tbInfo.Name}) switch cfg.OnExist { + // If table not exists, we will set tableMode as specified in tbInfo. If table exists: case OnExistIgnore: ctx.GetSessionVars().StmtCtx.AppendNote(err) + // If table exists, and if target TableMode is ModeRestore, we check if the existing table is consistent + if tbInfo.TableMode == model.TableModeRestore { + oldTableMode := oldTable.Meta().TableMode + if oldTableMode != model.TableModeRestore { + // Indeed this is not a conversion problem but an inconsistency problem. + return nil, infoschema.ErrInvalidTableModeSet.GenWithStackByArgs(oldTableMode, tbInfo.TableMode) + } + } + // Currently, target TableMode will NEVER be ModeImport because ImportInto does not use this function return nil, nil case OnExistReplace: // only CREATE OR REPLACE VIEW is supported at the moment. @@ -5650,6 +5661,31 @@ func (e *executor) UnlockTables(ctx sessionctx.Context, unlockTables []model.Tab return errors.Trace(err) } +func (e *executor) AlterTableMode(ctx sessionctx.Context, args *model.AlterTableModeArgs) error { + is := e.infoCache.GetLatest() + + schema, ok := is.SchemaByID(args.SchemaID) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(fmt.Sprintf("SchemaID: %v", args.SchemaID)) + } + + _, ok = is.TableByID(e.ctx, args.TableID) + if !ok { + return infoschema.ErrTableNotExists.GenWithStackByArgs(schema.Name, args.TableID) + } + + job := &model.Job{ + Version: model.JobVersion2, + SchemaID: args.SchemaID, + TableID: args.TableID, + Type: model.ActionAlterTableMode, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + } + err := e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + func throwErrIfInMemOrSysDB(ctx sessionctx.Context, dbLowerName string) error { if util.IsMemOrSysDB(dbLowerName) { if ctx.GetSessionVars().User != nil { diff --git a/pkg/ddl/job_worker.go b/pkg/ddl/job_worker.go index 3ec69b9d056df..7c2230b7e51f4 100644 --- a/pkg/ddl/job_worker.go +++ b/pkg/ddl/job_worker.go @@ -981,6 +981,8 @@ func (w *worker) runOneJobStep( ver, err = onLockTables(jobCtx, job) case model.ActionUnlockTable: ver, err = onUnlockTables(jobCtx, job) + case model.ActionAlterTableMode: + ver, err = onAlterTableMode(jobCtx, job) case model.ActionSetTiFlashReplica: ver, err = w.onSetTableFlashReplica(jobCtx, job) case model.ActionUpdateTiFlashReplicaStatus: diff --git a/pkg/ddl/schematracker/checker.go b/pkg/ddl/schematracker/checker.go index 5aac03ebcc6dd..727137f0f196f 100644 --- a/pkg/ddl/schematracker/checker.go +++ b/pkg/ddl/schematracker/checker.go @@ -399,6 +399,11 @@ func (d *Checker) UnlockTables(ctx sessionctx.Context, lockedTables []model.Tabl return d.realExecutor.UnlockTables(ctx, lockedTables) } +// AlterTableMode implements the DDL interface. +func (d *Checker) AlterTableMode(ctx sessionctx.Context, args *model.AlterTableModeArgs) error { + return d.realExecutor.AlterTableMode(ctx, args) +} + // CleanupTableLock implements the DDL interface. func (d *Checker) CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error { return d.realExecutor.CleanupTableLock(ctx, tables) diff --git a/pkg/ddl/schematracker/dm_tracker.go b/pkg/ddl/schematracker/dm_tracker.go index d44f68fcb47d1..c9ebe13e1ee7e 100644 --- a/pkg/ddl/schematracker/dm_tracker.go +++ b/pkg/ddl/schematracker/dm_tracker.go @@ -1118,6 +1118,11 @@ func (*SchemaTracker) UnlockTables(_ sessionctx.Context, _ []model.TableLockTpIn return nil } +// AlterTableMode implements the DDL interface, it's no-op in DM's case. +func (*SchemaTracker) AlterTableMode(_ sessionctx.Context, _ *model.AlterTableModeArgs) error { + return nil +} + // CleanupTableLock implements the DDL interface, it's no-op in DM's case. func (*SchemaTracker) CleanupTableLock(_ sessionctx.Context, _ []*ast.TableName) error { return nil diff --git a/pkg/ddl/table_mode.go b/pkg/ddl/table_mode.go new file mode 100644 index 0000000000000..79642f5432894 --- /dev/null +++ b/pkg/ddl/table_mode.go @@ -0,0 +1,78 @@ +// Copyright 2025 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 ddl + +import ( + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/meta/model" +) + +// onAlterTableMode should only be called by alterTableMode, will call updateVersionAndTableInfo +func onAlterTableMode(jobCtx *jobContext, job *model.Job) (ver int64, err error) { + args, err := model.GetAlterTableModeArgs(job) + if err != nil { + return ver, err + } + + var tbInfo *model.TableInfo + metaMut := jobCtx.metaMut + tbInfo, err = GetTableInfoAndCancelFaultJob(metaMut, job, job.SchemaID) + if err != nil { + return ver, err + } + + switch tbInfo.TableMode { + case model.TableModeNormal, model.TableModeImport, model.TableModeRestore: + // directly change table mode to target mode + err = alterTableMode(tbInfo, args) + if err != nil { + job.State = model.JobStateCancelled + } else { + // update table info and schema version + ver, err = updateVersionAndTableInfo(jobCtx, job, tbInfo, true) + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + } + default: + job.State = model.JobStateCancelled + err = infoschema.ErrInvalidTableModeSet.GenWithStackByArgs(tbInfo.TableMode, args.TableMode, tbInfo.Name.O) + } + + return ver, err +} + +// alterTableMode first checks if the change is valid and changes table mode to target mode +func alterTableMode(tbInfo *model.TableInfo, args *model.AlterTableModeArgs) error { + // Currently we can assume args.TableMode will NEVER be model.TableModeRestore. + // Because BR will NOT use this function to set a table into ModeRestore, + // instead BR will use (batch)CreateTableWithInfo. + + if args.TableMode == model.TableModeImport { + // only transition from ModeNormal to ModeImport is allowed + if tbInfo.TableMode != model.TableModeNormal { + return infoschema.ErrInvalidTableModeSet.GenWithStackByArgs(tbInfo.TableMode, args.TableMode, tbInfo.Name.O) + } + } + + if args.TableMode == model.TableModeRestore { + // Currently this branch will never be executed except for testing. + // only transition from ModeNormal to ModeRestore is allowed + if tbInfo.TableMode != model.TableModeNormal { + return infoschema.ErrInvalidTableModeSet.GenWithStackByArgs(tbInfo.TableMode, args.TableMode, tbInfo.Name.O) + } + } + + tbInfo.TableMode = args.TableMode + return nil +} diff --git a/pkg/ddl/table_mode_test.go b/pkg/ddl/table_mode_test.go new file mode 100644 index 0000000000000..7f16f829a4159 --- /dev/null +++ b/pkg/ddl/table_mode_test.go @@ -0,0 +1,272 @@ +// Copyright 2025 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 ddl_test + +import ( + "context" + "sync" + "testing" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func getClonedTableInfoFromDomain( + t *testing.T, + dbName string, + tableName string, + dom *domain.Domain, +) *model.TableInfo { + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) + require.NoError(t, err) + return tbl.Meta().Clone() +} + +func setTableModeTest( + ctx sessionctx.Context, + t *testing.T, + store kv.Storage, + de ddl.ExecutorForTest, + dbInfo *model.DBInfo, + tblInfo *model.TableInfo, + mode model.TableModeState, +) error { + args := &model.AlterTableModeArgs{ + TableMode: mode, + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + } + job := &model.Job{ + Version: model.JobVersion2, + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + Type: model.ActionAlterTableMode, + BinlogInfo: &model.HistoryInfo{}, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ + { + Database: dbInfo.Name.L, + Table: tblInfo.Name.L, + }, + }, + } + ctx.SetValue(sessionctx.QueryString, "skip") + err := de.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, args, true)) + + if err == nil { + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + + testCheckTableState(t, store, dbInfo, tblInfo, model.StatePublic) + testCheckJobDone(t, store, job.ID, true) + checkTableModeTest(t, store, dbInfo, tblInfo, mode) + } + + return err +} + +func checkTableModeTest(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo, mode model.TableModeState) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err := kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + tt := meta.NewMutator(txn) + info, err := tt.GetTable(dbInfo.ID, tblInfo.ID) + require.NoError(t, err) + require.NotNil(t, info) + require.Equal(t, mode, info.TableMode) + return nil + }) + require.NoError(t, err) +} + +func checkErrorCode(t *testing.T, err error, expected int) { + originErr := errors.Cause(err) + tErr, ok := originErr.(*terror.Error) + require.True(t, ok) + sqlErr := terror.ToSQLError(tErr) + require.Equal(t, expected, int(sqlErr.Code)) +} + +func TestTableModeBasic(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomain(t) + de := domain.DDLExecutor() + tk := testkit.NewTestKit(t, store) + + ctx := testkit.NewTestKit(t, store).Session() + + // init test + tk.MustExec("use test") + tk.MustExec("create table t1(id int)") + + // get cloned table info for creating new table t1_restore + tblInfo := getClonedTableInfoFromDomain(t, "test", "t1", domain) + + // For testing create table as ModeRestore + tblInfo.Name = ast.NewCIStr("t1_restore") + tblInfo.TableMode = model.TableModeRestore + err := de.CreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), tblInfo, nil, ddl.WithOnExist(ddl.OnExistIgnore)) + require.NoError(t, err) + dbInfo, ok := domain.InfoSchema().SchemaByName(ast.NewCIStr("test")) + require.True(t, ok) + checkTableModeTest(t, store, dbInfo, tblInfo, model.TableModeRestore) + + // For testing select is not allowed when table is in ModeImport + tk.MustGetErrCode("select * from t1_restore", errno.ErrProtectedTableMode) + + // For testing insert is not allowed when table is in ModeImport + tk.MustGetErrCode("insert into t1_restore values(1)", errno.ErrProtectedTableMode) + + // For testing accessing table metadata is allowed when table is in ModeRestore + tk.MustExec("show create table t1_restore") + tk.MustExec("describe t1_restore") + + // For testing AlterTable ModeRestore -> ModeImport is not allowed + err = setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, tblInfo, model.TableModeImport) + checkErrorCode(t, err, errno.ErrInvalidTableModeSet) + + // For testing AlterTableMode ModeRestore -> ModeNormal + err = setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, tblInfo, model.TableModeNormal) + require.NoError(t, err) + + // For testing AlterTableMode ModeNormal -> ModeRestore + err = setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, tblInfo, model.TableModeRestore) + require.NoError(t, err) + + // For testing batch create tables with info + var tblInfo1, tblInfo2, tblInfo3 *model.TableInfo + tblInfo1 = getClonedTableInfoFromDomain(t, "test", "t1", domain) + tblInfo1.Name = ast.NewCIStr("t1_1") + tblInfo1.TableMode = model.TableModeNormal + tblInfo2 = getClonedTableInfoFromDomain(t, "test", "t1", domain) + tblInfo2.Name = ast.NewCIStr("t1_2") + tblInfo2.TableMode = model.TableModeImport + tblInfo3 = getClonedTableInfoFromDomain(t, "test", "t1", domain) + tblInfo3.Name = ast.NewCIStr("t1_3") + tblInfo3.TableMode = model.TableModeRestore + err = de.BatchCreateTableWithInfo( + ctx, + ast.NewCIStr("test"), + []*model.TableInfo{tblInfo1, tblInfo2, tblInfo3}, + ddl.WithOnExist(ddl.OnExistIgnore), + ) + require.NoError(t, err) + checkTableModeTest(t, store, dbInfo, tblInfo1, model.TableModeNormal) + checkTableModeTest(t, store, dbInfo, tblInfo2, model.TableModeImport) + checkTableModeTest(t, store, dbInfo, tblInfo3, model.TableModeRestore) +} + +func TestTableModeConcurrent(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomain(t) + de := domain.DDLExecutor() + tk := testkit.NewTestKit(t, store) + ctx := testkit.NewTestKit(t, store).Session() + + tk.MustExec("use test") + tk.MustExec("create table t1(id int)") + dbInfo, ok := domain.InfoSchema().SchemaByName(ast.NewCIStr("test")) + require.True(t, ok) + + // Concurrency test1: concurrently alter t1 to ModeImport, expecting one success, one failure. + t1Infos := []*model.TableInfo{ + getClonedTableInfoFromDomain(t, "test", "t1", domain), + getClonedTableInfoFromDomain(t, "test", "t1", domain), + } + var wg sync.WaitGroup + wg.Add(len(t1Infos)) + errs := make(chan error, len(t1Infos)) + for _, info := range t1Infos { + go func(info *model.TableInfo) { + defer wg.Done() + errs <- setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, info, model.TableModeImport) + }(info) + } + wg.Wait() + close(errs) + var successCount int + var failedErr error + for e := range errs { + if e == nil { + successCount++ + } else { + failedErr = e + } + } + require.Equal(t, 1, successCount) + require.NotNil(t, failedErr) + checkErrorCode(t, failedErr, errno.ErrInvalidTableModeSet) + checkTableModeTest(t, store, dbInfo, t1Infos[0], model.TableModeImport) + + // Concurrency test2: concurrently alter t1 to ModeNormal, expecting both success. + t1NormalInfos := []*model.TableInfo{ + getClonedTableInfoFromDomain(t, "test", "t1", domain), + getClonedTableInfoFromDomain(t, "test", "t1", domain), + } + var wg2 sync.WaitGroup + wg2.Add(len(t1NormalInfos)) + errs2 := make(chan error, len(t1NormalInfos)) + for _, info := range t1NormalInfos { + go func(info *model.TableInfo) { + defer wg2.Done() + errs2 <- setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, info, model.TableModeNormal) + }(info) + } + wg2.Wait() + close(errs2) + for e := range errs2 { + require.NoError(t, e) + } + checkTableModeTest(t, store, dbInfo, t1NormalInfos[0], model.TableModeNormal) + + // Concurrency test3: concurrently alter t1 to ModeRestore and ModeImport, expecting one success, one failure. + modes := []model.TableModeState{ + model.TableModeRestore, + model.TableModeImport, + } + clones := make([]*model.TableInfo, len(modes)) + for i := range modes { + clones[i] = getClonedTableInfoFromDomain(t, "test", "t1", domain) + } + var wg3 sync.WaitGroup + wg3.Add(len(modes)) + errs3 := make(chan error, len(modes)) + for i, mode := range modes { + go func(clone *model.TableInfo, m model.TableModeState) { + defer wg3.Done() + errs3 <- setTableModeTest(ctx, t, store, de.(ddl.ExecutorForTest), dbInfo, clone, m) + }(clones[i], mode) + } + wg3.Wait() + close(errs3) + var successCount3 int + var failedErr3 error + for e := range errs3 { + if e == nil { + successCount3++ + } else { + failedErr3 = e + } + } + require.Equal(t, 1, successCount3) + require.NotNil(t, failedErr3) + checkErrorCode(t, failedErr3, errno.ErrInvalidTableModeSet) +} diff --git a/pkg/errno/errcode.go b/pkg/errno/errcode.go index dadf801c74534..2622729a3faf6 100644 --- a/pkg/errno/errcode.go +++ b/pkg/errno/errcode.go @@ -1137,6 +1137,8 @@ const ( ErrDDLSetting = 8246 ErrIngestFailed = 8247 ErrIngestCheckEnvFailed = 8256 + ErrProtectedTableMode = 8258 + ErrInvalidTableModeSet = 8259 ErrCannotPauseDDLJob = 8260 ErrCannotResumeDDLJob = 8261 diff --git a/pkg/errno/errname.go b/pkg/errno/errname.go index 1d4f673a7bbb9..6b6aaf5065dd7 100644 --- a/pkg/errno/errname.go +++ b/pkg/errno/errname.go @@ -1080,6 +1080,8 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrMemoryExceedForInstance: mysql.Message("Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory. Please try narrowing your query scope or increase the tidb_server_memory_limit and try again.[conn=%d]", nil), ErrDeleteNotFoundColumn: mysql.Message("Delete can not find column %s for table %s", nil), ErrKeyTooLarge: mysql.Message("key is too large, the size of given key is %d", nil), + ErrProtectedTableMode: mysql.Message("Table %s is in mode %s", nil), + ErrInvalidTableModeSet: mysql.Message("Invalid mode set from (or by default) %s to %s for table %s", nil), ErrHTTPServiceError: mysql.Message("HTTP request failed with status %s", nil), diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 6310cc8c07a8d..1bc365921396e 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -700,31 +700,32 @@ func (e *memtableRetriever) setDataFromOneTable( } record := types.MakeDatums( - infoschema.CatalogVal, // TABLE_CATALOG - schema.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - tableType, // TABLE_TYPE - "InnoDB", // ENGINE - uint64(10), // VERSION - "Compact", // ROW_FORMAT - rowCount, // TABLE_ROWS - avgRowLength, // AVG_ROW_LENGTH - dataLength, // DATA_LENGTH - uint64(0), // MAX_DATA_LENGTH - indexLength, // INDEX_LENGTH - uint64(0), // DATA_FREE - autoIncID, // AUTO_INCREMENT - createTime, // CREATE_TIME - nil, // UPDATE_TIME - nil, // CHECK_TIME - collation, // TABLE_COLLATION - nil, // CHECKSUM - createOptions, // CREATE_OPTIONS - table.Comment, // TABLE_COMMENT - table.ID, // TIDB_TABLE_ID - shardingInfo, // TIDB_ROW_ID_SHARDING_INFO - pkType, // TIDB_PK_TYPE - policyName, // TIDB_PLACEMENT_POLICY_NAME + infoschema.CatalogVal, // TABLE_CATALOG + schema.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + tableType, // TABLE_TYPE + "InnoDB", // ENGINE + uint64(10), // VERSION + "Compact", // ROW_FORMAT + rowCount, // TABLE_ROWS + avgRowLength, // AVG_ROW_LENGTH + dataLength, // DATA_LENGTH + uint64(0), // MAX_DATA_LENGTH + indexLength, // INDEX_LENGTH + uint64(0), // DATA_FREE + autoIncID, // AUTO_INCREMENT + createTime, // CREATE_TIME + nil, // UPDATE_TIME + nil, // CHECK_TIME + collation, // TABLE_COLLATION + nil, // CHECKSUM + createOptions, // CREATE_OPTIONS + table.Comment, // TABLE_COMMENT + table.ID, // TIDB_TABLE_ID + shardingInfo, // TIDB_ROW_ID_SHARDING_INFO + pkType, // TIDB_PK_TYPE + policyName, // TIDB_PLACEMENT_POLICY_NAME + table.TableMode.String(), // TIDB_TABLE_MODE ) rows = append(rows, record) e.recordMemoryConsume(record) @@ -755,6 +756,7 @@ func (e *memtableRetriever) setDataFromOneTable( nil, // TIDB_ROW_ID_SHARDING_INFO pkType, // TIDB_PK_TYPE nil, // TIDB_PLACEMENT_POLICY_NAME + nil, // TIDB_TABLE_MODE ) rows = append(rows, record) e.recordMemoryConsume(record) @@ -859,6 +861,7 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc nil, // TIDB_ROW_ID_SHARDING_INFO nil, // TIDB_PK_TYPE nil, // TIDB_PLACEMENT_POLICY_NAME + nil, // TIDB_TABLE_MODE ) rows = append(rows, record) e.recordMemoryConsume(record) diff --git a/pkg/executor/infoschema_reader_test.go b/pkg/executor/infoschema_reader_test.go index 47cc9d558bf75..bc8e880ee580a 100644 --- a/pkg/executor/infoschema_reader_test.go +++ b/pkg/executor/infoschema_reader_test.go @@ -560,6 +560,10 @@ func TestTablesTable(t *testing.T) { } } + // test table mode + tk.MustQuery(`select tidb_table_mode from information_schema.tables where table_schema = 'db1' and + table_name = 't1'`).Check(testkit.Rows("ModeNormal")) + // Predicates are extracted in CNF, so we separate the test cases by the number of disjunctions in the predicate. // predicate covers one disjunction diff --git a/pkg/infoschema/error.go b/pkg/infoschema/error.go index 0dacfca35da00..807f2ea694154 100644 --- a/pkg/infoschema/error.go +++ b/pkg/infoschema/error.go @@ -108,4 +108,8 @@ var ( ErrResourceGroupSupportDisabled = dbterror.ClassSchema.NewStd(mysql.ErrResourceGroupSupportDisabled) // ErrCheckConstraintDupName returns for duplicate constraint names. ErrCheckConstraintDupName = dbterror.ClassSchema.NewStd(mysql.ErrCheckConstraintDupName) + // ErrProtectedTableMode returns for accessing table in import/restore mode. + ErrProtectedTableMode = dbterror.ClassSchema.NewStd(mysql.ErrProtectedTableMode) + // ErrInvalidTableModeSet returns for invalid TableMode conversion. + ErrInvalidTableModeSet = dbterror.ClassSchema.NewStd(mysql.ErrInvalidTableModeSet) ) diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index dbf001e26aa3b..8b8c6a87c924b 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -466,6 +466,7 @@ var tablesCols = []columnInfo{ {name: "TIDB_ROW_ID_SHARDING_INFO", tp: mysql.TypeVarchar, size: 255}, {name: "TIDB_PK_TYPE", tp: mysql.TypeVarchar, size: 64}, {name: "TIDB_PLACEMENT_POLICY_NAME", tp: mysql.TypeVarchar, size: 64}, + {name: "TIDB_TABLE_MODE", tp: mysql.TypeVarchar, size: 16}, } // See: http://dev.mysql.com/doc/refman/5.7/en/information-schema-columns-table.html diff --git a/pkg/meta/meta_test.go b/pkg/meta/meta_test.go index 6b2c1c0068697..c6aac82ad3b20 100644 --- a/pkg/meta/meta_test.go +++ b/pkg/meta/meta_test.go @@ -738,7 +738,7 @@ func TestIsTableInfoMustLoadSubStringsOrder(t *testing.T) { tableInfo := &model.TableInfo{} b, err := json.Marshal(tableInfo) require.NoError(t, err) - expect := `{"id":0,"name":{"O":"","L":""},"charset":"","collate":"","cols":null,"index_info":null,"constraint_info":null,"fk_info":null,"state":0,"pk_is_handle":false,"is_common_handle":false,"common_handle_version":0,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":0,"max_idx_id":0,"max_fk_id":0,"max_cst_id":0,"update_timestamp":0,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"auto_random_range_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":0,"tiflash_replica":null,"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null,"exchange_partition_info":null,"ttl_info":null,"revision":0}` + expect := `{"id":0,"name":{"O":"","L":""},"charset":"","collate":"","cols":null,"index_info":null,"constraint_info":null,"fk_info":null,"state":0,"pk_is_handle":false,"is_common_handle":false,"common_handle_version":0,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":0,"max_idx_id":0,"max_fk_id":0,"max_cst_id":0,"update_timestamp":0,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"auto_random_range_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":0,"tiflash_replica":null,"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null,"exchange_partition_info":null,"ttl_info":null,"revision":0,"table_mode":0}` require.Equal(t, string(b), expect) } diff --git a/pkg/meta/model/bdr.go b/pkg/meta/model/bdr.go index 6103c5345c437..4965cc800139a 100644 --- a/pkg/meta/model/bdr.go +++ b/pkg/meta/model/bdr.go @@ -103,6 +103,7 @@ var BDRActionMap = map[DDLBDRType][]ActionType{ ActionRemovePartitioning, ActionAddVectorIndex, ActionModifyEngineAttribute, + ActionAlterTableMode, }, UnmanagementDDL: { ActionCreatePlacementPolicy, diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index d3fd19a26cb37..524285948a156 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -113,6 +113,7 @@ const ( ActionRemovePartitioning ActionType = 72 ActionAddVectorIndex ActionType = 73 ActionModifyEngineAttribute ActionType = 74 + ActionAlterTableMode ActionType = 75 ) // ActionMap is the map of DDL ActionType to string. @@ -186,6 +187,7 @@ var ActionMap = map[ActionType]string{ ActionRemovePartitioning: "alter table remove partitioning", ActionAddVectorIndex: "add vector index", ActionModifyEngineAttribute: "modify engine attribute", + ActionAlterTableMode: "alter table mode", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility. diff --git a/pkg/meta/model/job_args.go b/pkg/meta/model/job_args.go index 1c62c726d9184..cc8d1fb1e16ab 100644 --- a/pkg/meta/model/job_args.go +++ b/pkg/meta/model/job_args.go @@ -1083,6 +1083,26 @@ func GetLockTablesArgs(job *Job) (*LockTablesArgs, error) { return getOrDecodeArgs[*LockTablesArgs](&LockTablesArgs{}, job) } +// AlterTableModeArgs is the argument for AlterTableMode. +type AlterTableModeArgs struct { + TableMode TableModeState + SchemaID int64 + TableID int64 +} + +func (a *AlterTableModeArgs) getArgsV1(*Job) []any { + return []any{a} +} + +func (a *AlterTableModeArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(a)) +} + +// GetAlterTableModeArgs get the AlterTableModeArgs argument. +func GetAlterTableModeArgs(job *Job) (*AlterTableModeArgs, error) { + return getOrDecodeArgs[*AlterTableModeArgs](&AlterTableModeArgs{}, job) +} + // RepairTableArgs is the argument for repair table type RepairTableArgs struct { TableInfo *TableInfo `json:"table_info"` diff --git a/pkg/meta/model/table.go b/pkg/meta/model/table.go index 9fb6bf3760f15..0c2c575250417 100644 --- a/pkg/meta/model/table.go +++ b/pkg/meta/model/table.go @@ -196,6 +196,8 @@ type TableInfo struct { Revision uint64 `json:"revision"` DBID int64 `json:"-"` + + TableMode TableModeState `json:"table_mode"` } // Hash64 implement HashEquals interface. @@ -660,6 +662,32 @@ func (t TableLockState) String() string { } } +// TableModeState is the state for table mode. +type TableModeState byte + +const ( + // TableModeNormal means the table is in normal mode. + TableModeNormal TableModeState = iota + // TableModeImport means the table is in import mode. + TableModeImport + // TableModeRestore means the table is in restore mode. + TableModeRestore +) + +// String implements fmt.Stringer interface. +func (t TableModeState) String() string { + switch t { + case TableModeNormal: + return "ModeNormal" + case TableModeImport: + return "ModeImport" + case TableModeRestore: + return "ModeRestore" + default: + return "" + } +} + // TiFlashReplicaInfo means the flash replica info. type TiFlashReplicaInfo struct { Count uint64 diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 6d95e5d1a84bb..6043c1907ea99 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -248,6 +248,32 @@ func CheckTableLock(ctx tablelock.TableLockReadContext, is infoschema.InfoSchema return nil } +// CheckTableMode checks if the table is accessible by table mode. +func CheckTableMode(is infoschema.InfoSchema, node ast.Node, vs []visitInfo) error { + // First make exceptions for stmt that only visit table meta; + // For example, `describe ` and `show create table `; + // These exceptions can be simply categorized as `ast.ShowStmt`; + if _, ok := node.(*ast.ShowStmt); ok { + return nil + } + + for i := range vs { + tb, err := is.TableByName(context.Background(), ast.NewCIStr(vs[i].db), ast.NewCIStr(vs[i].table)) + if infoschema.ErrTableNotExists.Equal(err) { + return nil + } + if err != nil { + return err + } + if tb.Meta().TableMode == model.TableModeImport { + return infoschema.ErrProtectedTableMode.GenWithStackByArgs(tb.Meta().Name.O, model.TableModeImport) + } else if tb.Meta().TableMode == model.TableModeRestore { + return infoschema.ErrProtectedTableMode.GenWithStackByArgs(tb.Meta().Name.O, model.TableModeRestore) + } + } + return nil +} + func checkStableResultMode(sctx base.PlanContext) bool { s := sctx.GetSessionVars() st := s.StmtCtx diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index d2b9a4b6b16bf..9e0297c69b0be 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -475,6 +475,10 @@ func optimize(ctx context.Context, sctx planctx.PlanContext, node *resolve.NodeW return nil, nil, 0, err } + if err := core.CheckTableMode(is, node.Node, builder.GetVisitInfo()); err != nil { + return nil, nil, 0, err + } + names := p.OutputNames() // Handle the non-logical plan statement. diff --git a/tests/integrationtest/r/infoschema/infoschema.result b/tests/integrationtest/r/infoschema/infoschema.result index c2e4ad2c96155..7fc320d26cea6 100644 --- a/tests/integrationtest/r/infoschema/infoschema.result +++ b/tests/integrationtest/r/infoschema/infoschema.result @@ -54,11 +54,11 @@ count(*) 3 desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA= 'mysql' or TABLE_SCHEMA = 'test') and (TABLE_NAME = 't1' or TABLE_NAME = 't2'); id estRows task access object operator info -HashAgg 1.00 root funcs:count(1)->Column#26 +HashAgg 1.00 root funcs:count(1)->Column#27 └─MemTableScan 10000.00 root table:TABLES table_name:["t1","t2"], table_schema:["mysql","test"] desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA in ('mysql', 'test') and TABLE_NAME in ('t1', 't2'); id estRows task access object operator info -HashAgg 1.00 root funcs:count(1)->Column#26 +HashAgg 1.00 root funcs:count(1)->Column#27 └─MemTableScan 10000.00 root table:TABLES table_name:["t1","t2"], table_schema:["mysql","test"] SELECT count(*) FROM information_schema.TABLES WHERE TABLE_NAME in ('t1', 't2') and TABLE_SCHEMA = 'mysql'; count(*) diff --git a/tests/integrationtest/r/planner/core/integration.result b/tests/integrationtest/r/planner/core/integration.result index f090176e415b0..c5b9c8f70388c 100644 --- a/tests/integrationtest/r/planner/core/integration.result +++ b/tests/integrationtest/r/planner/core/integration.result @@ -3595,9 +3595,9 @@ Projection_16 10000.00 root planner__core__integration.t1.a └─TableFullScan_24 20000000.00 cop[tikv] table:two keep order:false, stats:pseudo explain select rank() over (partition by table_name) from information_schema.tables; id estRows task access object operator info -Projection_7 10000.00 root Column#27->Column#28 +Projection_7 10000.00 root Column#28->Column#29 └─Shuffle_11 10000.00 root execution info: concurrency:5, data sources:[MemTableScan_9] - └─Window_8 10000.00 root rank()->Column#27 over(partition by Column#3) + └─Window_8 10000.00 root rank()->Column#28 over(partition by Column#3) └─Sort_10 10000.00 root Column#3 └─ShuffleReceiver_12 10000.00 root └─MemTableScan_9 10000.00 root table:TABLES