Skip to content

Commit

Permalink
*: remove system var tidb_enable_change_multi_schema (#36167)
Browse files Browse the repository at this point in the history
ref #14766
  • Loading branch information
tangenta authored Jul 14, 2022
1 parent 81cf12e commit 2193bfe
Show file tree
Hide file tree
Showing 12 changed files with 31 additions and 104 deletions.
7 changes: 2 additions & 5 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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 {
Expand All @@ -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
}
Expand Down
13 changes: 0 additions & 13 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
7 changes: 0 additions & 7 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
46 changes: 11 additions & 35 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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 {
Expand Down Expand Up @@ -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)
Expand All @@ -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.
Expand Down Expand Up @@ -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)
Expand All @@ -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
}
Expand Down
6 changes: 4 additions & 2 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
Expand Down
14 changes: 4 additions & 10 deletions ddl/multi_schema_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
15 changes: 0 additions & 15 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
10 changes: 8 additions & 2 deletions parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"`

Expand Down Expand Up @@ -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
}
Expand Down
4 changes: 0 additions & 4 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/removed.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand All @@ -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
Expand Down
6 changes: 0 additions & 6 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
5 changes: 0 additions & 5 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -854,7 +851,6 @@ const (
DefTiDBDDLReorgBatchSize = 256
DefTiDBDDLErrorCountLimit = 512
DefTiDBMaxDeltaSchemaCount = 1024
DefTiDBChangeMultiSchema = false
DefTiDBPointGetCache = false
DefTiDBPlacementMode = PlacementModeStrict
DefTiDBEnableAutoIncrementInGenerated = false
Expand Down Expand Up @@ -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)
Expand Down

0 comments on commit 2193bfe

Please sign in to comment.