diff --git a/ddl/column.go b/ddl/column.go index b31300d07e0ae..792b1ca54c057 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -329,7 +329,7 @@ func checkDropColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Col job.State = model.JobStateCancelled return nil, nil, nil, ifExists, dbterror.ErrCantDropFieldOrKey.GenWithStack("column %s doesn't exist", colName) } - if err = isDroppableColumn(job.MultiSchemaInfo != nil, tblInfo, colName); err != nil { + if err = isDroppableColumn(tblInfo, colName); err != nil { job.State = model.JobStateCancelled return nil, nil, nil, false, errors.Trace(err) } @@ -1560,7 +1560,7 @@ func isColumnWithIndex(colName string, indices []*model.IndexInfo) bool { return false } -func isColumnCanDropWithIndex(isMultiSchemaChange bool, colName string, indices []*model.IndexInfo) error { +func isColumnCanDropWithIndex(colName string, indices []*model.IndexInfo) error { for _, indexInfo := range indices { if indexInfo.Primary || len(indexInfo.Columns) > 1 { for _, col := range indexInfo.Columns { @@ -1569,9 +1569,6 @@ func isColumnCanDropWithIndex(isMultiSchemaChange bool, colName string, indices } } } - if len(indexInfo.Columns) == 1 && indexInfo.Columns[0].Name.L == colName && !isMultiSchemaChange { - return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s with tidb_enable_change_multi_schema is disable", colName) - } } return nil } diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 5efbd6dc3b688..6f655327e5216 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1961,19 +1961,6 @@ func TestExpressionIndexDDLError(t *testing.T) { tk.MustGetErrCode("alter table t drop column b", errno.ErrDependentByFunctionalIndex) } -func TestRestrainDropColumnWithIndex(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t (a int, b int, index(a))") - tk.MustExec("set @@GLOBAL.tidb_enable_change_multi_schema=0") - tk.MustQuery("select @@tidb_enable_change_multi_schema").Check(testkit.Rows("0")) - tk.MustGetErrCode("alter table t drop column a", errno.ErrUnsupportedDDLOperation) - tk.MustExec("set @@GLOBAL.tidb_enable_change_multi_schema=1") - tk.MustExec("alter table t drop column a") -} - func TestParallelRenameTable(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 4b1b510bfe1db..abbdc25775182 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -631,13 +631,6 @@ func TestErrnoErrorCode(t *testing.T) { tk.MustExec("create table test_error_code_null(c1 char(100) not null);") sql = "insert into test_error_code_null (c1) values(null);" tk.MustGetErrCode(sql, errno.ErrBadNull) - // disable tidb_enable_change_multi_schema - tk.MustExec("set global tidb_enable_change_multi_schema = false") - sql = "alter table test_error_code_null add column (x1 int, x2 int)" - tk.MustGetErrCode(sql, errno.ErrUnsupportedDDLOperation) - sql = "alter table test_error_code_null add column (x1 int, x2 int)" - tk.MustGetErrCode(sql, errno.ErrUnsupportedDDLOperation) - tk.MustExec("set global tidb_enable_change_multi_schema = true") } func TestTableDDLWithFloatType(t *testing.T) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 96348445a9320..d5cf5ef2efd26 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3062,18 +3062,6 @@ func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) return validSpecs, nil } -func checkMultiSpecs(sctx sessionctx.Context, specs []*ast.AlterTableSpec) error { - if !variable.EnableChangeMultiSchema.Load() { - if len(specs) > 1 { - return dbterror.ErrRunMultiSchemaChanges - } - if len(specs) == 1 && len(specs[0].NewColumns) > 1 && specs[0].Tp == ast.AlterTableAddColumns { - return dbterror.ErrRunMultiSchemaChanges - } - } - return nil -} - func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast.AlterTableStmt) (err error) { ident := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} validSpecs, err := resolveAlterTableSpec(sctx, stmt.Specs) @@ -3098,15 +3086,9 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast } } - err = checkMultiSpecs(sctx, validSpecs) - if err != nil { - return err - } - if len(validSpecs) > 1 { sctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() } - for _, spec := range validSpecs { var handledCharsetOrCollate bool switch spec.Tp { @@ -4005,21 +3987,15 @@ func (d *ddl) DropColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTa return err } - var multiSchemaInfo *model.MultiSchemaInfo - if variable.EnableChangeMultiSchema.Load() { - multiSchemaInfo = &model.MultiSchemaInfo{} - } - job := &model.Job{ - SchemaID: schema.ID, - TableID: t.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: model.StatePublic, - TableName: t.Meta().Name.L, - Type: model.ActionDropColumn, - BinlogInfo: &model.HistoryInfo{}, - MultiSchemaInfo: multiSchemaInfo, - Args: []interface{}{colName, spec.IfExists}, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: model.StatePublic, + TableName: t.Meta().Name.L, + Type: model.ActionDropColumn, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{colName, spec.IfExists}, } err = d.DoDDLJob(ctx, job) @@ -4041,7 +4017,7 @@ func checkIsDroppableColumn(ctx sessionctx.Context, t table.Table, spec *ast.Alt return false, err } - if err = isDroppableColumn(variable.EnableChangeMultiSchema.Load(), tblInfo, colName); err != nil { + if err = isDroppableColumn(tblInfo, colName); err != nil { return false, errors.Trace(err) } // We don't support dropping column with PK handle covered now. @@ -6159,7 +6135,7 @@ func checkIsDropPrimaryKey(indexName model.CIStr, indexInfo *model.IndexInfo, t return isPK, nil } -func isDroppableColumn(multiSchemaChange bool, tblInfo *model.TableInfo, colName model.CIStr) error { +func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { if ok, dep, isHidden := hasDependentByGeneratedColumn(tblInfo, colName); ok { if isHidden { return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(dep) @@ -6172,7 +6148,7 @@ func isDroppableColumn(multiSchemaChange bool, tblInfo *model.TableInfo, colName colName, tblInfo.Name) } // We only support dropping column with single-value none Primary Key index covered now. - err := isColumnCanDropWithIndex(multiSchemaChange, colName.L, tblInfo.Indices) + err := isColumnCanDropWithIndex(colName.L, tblInfo.Indices) if err != nil { return err } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 830579fab895f..1c0e465414717 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -786,7 +786,7 @@ func writeBinlog(binlogCli *pumpcli.PumpsClient, txn kv.Transaction, job *model. // When this column is in the "delete only" and "delete reorg" states, the binlog of "drop column" has not been written yet, // but the column has been removed from the binlog of the write operation. // So we add this binlog to enable downstream components to handle DML correctly in this schema state. - ((job.Type == model.ActionDropColumn) && job.SchemaState == model.StateDeleteOnly) { + (job.Type == model.ActionDropColumn && job.SchemaState == model.StateDeleteOnly) { if skipWriteBinlog(job) { return } @@ -877,7 +877,9 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, // Mock for run ddl job panic. failpoint.Inject("mockPanicInRunDDLJob", func(val failpoint.Value) {}) - logutil.Logger(w.logCtx).Info("[ddl] run DDL job", zap.String("job", job.String())) + if job.Type != model.ActionMultiSchemaChange { + logutil.Logger(w.logCtx).Info("[ddl] run DDL job", zap.String("job", job.String())) + } timeStart := time.Now() if job.RealStartTS == 0 { job.RealStartTS = t.StartTS diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index a7ce46fc2aae7..8f5654bda0994 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -323,20 +323,14 @@ func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { } func appendMultiChangeWarningsToOwnerCtx(ctx sessionctx.Context, job *model.Job) { - if job.MultiSchemaInfo == nil { + if job.MultiSchemaInfo == nil || job.Type != model.ActionMultiSchemaChange { return } - if job.Type == model.ActionMultiSchemaChange { - for _, sub := range job.MultiSchemaInfo.SubJobs { - if sub.Warning != nil { - ctx.GetSessionVars().StmtCtx.AppendNote(sub.Warning) - } + for _, sub := range job.MultiSchemaInfo.SubJobs { + if sub.Warning != nil { + ctx.GetSessionVars().StmtCtx.AppendNote(sub.Warning) } } - for _, w := range job.MultiSchemaInfo.Warnings { - ctx.GetSessionVars().StmtCtx.AppendNote(w) - } - } // rollingBackMultiSchemaChange updates a multi-schema change job diff --git a/executor/executor_test.go b/executor/executor_test.go index 3d68a859c6dff..2c13530dad3fe 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1576,21 +1576,6 @@ func TestPlanReplayerDumpSingle(t *testing.T) { } } -func TestDropColWithPrimaryKey(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id int primary key, c1 int, c2 int, c3 int, index idx1(c1, c2), index idx2(c3))") - tk.MustExec("set global tidb_enable_change_multi_schema = off") - tk.MustGetErrMsg("alter table t drop column id", "[ddl:8200]Unsupported drop integer primary key") - tk.MustGetErrMsg("alter table t drop column c1", "[ddl:8200]can't drop column c1 with composite index covered or Primary Key covered now") - tk.MustGetErrMsg("alter table t drop column c3", "[ddl:8200]can't drop column c3 with tidb_enable_change_multi_schema is disable") - tk.MustExec("set global tidb_enable_change_multi_schema = on") - tk.MustExec("alter table t drop column c3") -} - func TestUnsignedFeedback(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 62f2fe5239e62..3dcd74cebe7e4 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -254,8 +254,6 @@ func NewDDLReorgMeta() *DDLReorgMeta { // MultiSchemaInfo keeps some information for multi schema change. type MultiSchemaInfo struct { - Warnings []*errors.Error - SubJobs []*SubJob `json:"sub_jobs"` Revertible bool `json:"revertible"` @@ -661,6 +659,14 @@ func (job *Job) MayNeedReorg() bool { return ok && needReorg } return false + case ActionMultiSchemaChange: + for _, sub := range job.MultiSchemaInfo.SubJobs { + proxyJob := Job{Type: sub.Type, CtxVars: sub.CtxVars} + if proxyJob.MayNeedReorg() { + return true + } + } + return false default: return false } diff --git a/session/bootstrap.go b/session/bootstrap.go index e9db7a08c3ed7..4c7b03a80d8ec 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -2030,10 +2030,6 @@ func doDMLWorks(s Session) { if inTestSuite() || config.CheckTableBeforeDrop { vVal = string(variable.Dynamic) } - case variable.TiDBEnableChangeMultiSchema: - if inTestSuite() { - vVal = variable.On - } case variable.TiDBMemOOMAction: if inTestSuite() { vVal = variable.OOMActionLog diff --git a/sessionctx/variable/removed.go b/sessionctx/variable/removed.go index fbaae51b04a40..f540f3894abe0 100644 --- a/sessionctx/variable/removed.go +++ b/sessionctx/variable/removed.go @@ -34,6 +34,7 @@ const ( placementChecks = "placement_checks" tiDBEnableStreaming = "tidb_enable_streaming" tiDBOptBCJ = "tidb_opt_broadcast_join" + tidbEnableChangeMultiSchema = "tidb_enable_change_multi_schema" ) var removedSysVars = map[string]string{ @@ -49,6 +50,7 @@ var removedSysVars = map[string]string{ tiDBMemQuotaIndexLookupJoin: "use tidb_mem_quota_query instead", tiDBEnableStreaming: "streaming is no longer supported", tiDBOptBCJ: "tidb_opt_broadcast_join is removed and use tidb_allow_mpp instead", + tidbEnableChangeMultiSchema: "alter multiple schema objects in a table is now always enabled", } // IsRemovedSysVar returns true if the sysvar has been removed diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index fe83218698719..e8cb285d518fe 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -496,12 +496,6 @@ var defaultSysVars = []*SysVar{ SetMaxDeltaSchemaCount(TidbOptInt64(val, DefTiDBMaxDeltaSchemaCount)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Hidden: true, Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { - EnableChangeMultiSchema.Store(TiDBOptOn(val)) - return nil - }, GetGlobal: func(s *SessionVars) (string, error) { - return BoolToOnOff(EnableChangeMultiSchema.Load()), nil - }}, {Scope: ScopeGlobal, Name: TiDBEnablePointGetCache, Value: BoolToOnOff(DefTiDBPointGetCache), Hidden: true, Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { EnablePointGetCache.Store(TiDBOptOn(val)) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ff3cfdb41dd4a..1993746c1d0aa 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -435,9 +435,6 @@ const ( // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" - // TiDBEnableChangeMultiSchema is used to control whether to enable the change multi schema. - TiDBEnableChangeMultiSchema = "tidb_enable_change_multi_schema" - // TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in // expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated" @@ -854,7 +851,6 @@ const ( DefTiDBDDLReorgBatchSize = 256 DefTiDBDDLErrorCountLimit = 512 DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBChangeMultiSchema = false DefTiDBPointGetCache = false DefTiDBPlacementMode = PlacementModeStrict DefTiDBEnableAutoIncrementInGenerated = false @@ -992,7 +988,6 @@ var ( MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Instance.MemoryUsageAlarmRatio) EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) EnablePointGetCache = atomic.NewBool(DefTiDBPointGetCache) - EnableChangeMultiSchema = atomic.NewBool(DefTiDBChangeMultiSchema) MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly)