From a4e8b1874e62414abe37cf7ab178fd23601d17d2 Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Wed, 16 Dec 2020 15:25:00 +0800 Subject: [PATCH 1/5] add a switch for alter partition alter placement Signed-off-by: AilinKid <314806019@qq.com> --- ddl/ddl_api.go | 6 +- ddl/placement_sql_test.go | 35 ++++++- session/session.go | 1 + sessionctx/variable/session.go | 158 ++++++++++++++++--------------- sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 + 6 files changed, 124 insertions(+), 81 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6095d1d80c773..c3fdf95b740ac 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2430,7 +2430,11 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A isAlterTable := true err = d.RenameTable(ctx, ident, newIdent, isAlterTable) case ast.AlterTableAlterPartition: - err = d.AlterTablePartition(ctx, ident, spec) + if ctx.GetSessionVars().EnableAlterPartitionPlacement { + err = d.AlterTablePartition(ctx, ident, spec) + } else { + err = errors.New("alter table partition alter partition is switched off by tidb_enable_alter_partition_placement") + } case ast.AlterTablePartition: // Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ... err = errors.New("alter table partition is unsupported") diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 4b324cc569218..0571448d73115 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -29,7 +29,12 @@ func (s *testDBSuite1) TestAlterTableAlterPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") - defer tk.MustExec("drop table if exists t1") + + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk.MustExec("drop table if exists t1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() tk.MustExec(`create table t1 (c int) PARTITION BY RANGE (c) ( @@ -290,7 +295,11 @@ add placement policy func (s *testDBSuite1) TestPlacementPolicyCache(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - defer tk.MustExec("drop table if exists t1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk.MustExec("drop table if exists t1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() initTable := func() []string { bundles := make(map[string]*placement.Bundle) @@ -333,7 +342,11 @@ func (s *testSerialDBSuite) TestTxnScopeConstraint(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") - defer tk.MustExec("drop table if exists t1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk.MustExec("drop table if exists t1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() tk.MustExec(`create table t1 (c int) PARTITION BY RANGE (c) ( @@ -474,7 +487,11 @@ func (s *testDBSuite1) TestAbortTxnIfPlacementChanged(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tp1") - defer tk.MustExec("drop table if exists tp1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk.MustExec("drop table if exists tp1") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() tk.MustExec(`create table tp1 (c int) PARTITION BY RANGE (c) ( @@ -485,6 +502,11 @@ PARTITION BY RANGE (c) ( c.Assert(err, IsNil) tk1 := testkit.NewTestKitWithSession(c, s.store, se1) tk1.MustExec("use test") + + tk1.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk1.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() _, err = tk.Exec(`alter table tp1 alter partition p0 add placement policy constraints='["+ zone = sh "]' @@ -529,6 +551,11 @@ func (s *testDBSuite1) TestGlobalTxnState(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + defer func() { + tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + }() + tk.MustExec(`create table t1 (c int) PARTITION BY RANGE (c) ( PARTITION p0 VALUES LESS THAN (6), diff --git a/session/session.go b/session/session.go index 3a27feac86417..89415ede51e2f 100644 --- a/session/session.go +++ b/session/session.go @@ -2286,6 +2286,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableChangeColumnType, variable.TiDBEnableChangeMultiSchema, variable.TiDBEnablePointGetCache, + variable.TiDBEnableAlterPartitionPlacement, variable.TiDBEnableAmendPessimisticTxn, variable.TiDBMemQuotaApplyCache, variable.TiDBEnableParallelApply, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0cc8a1b730f48..0cca9ddc05bbb 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -578,6 +578,9 @@ type SessionVars struct { // EnablePointGetCache is used to cache value for point get for read only scenario. EnablePointGetCache bool + // EnableAlterPartitionPlacement indicates whether a user can alter table partition placement rules. + EnableAlterPartitionPlacement bool + // WaitSplitRegionFinish defines the split region behaviour is sync or async. WaitSplitRegionFinish bool @@ -851,82 +854,83 @@ type ConnectionInfo struct { // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]types.Datum), - UserVarTypes: make(map[string]*types.FieldType), - systems: make(map[string]string), - stmtVars: make(map[string]string), - PreparedStmts: make(map[uint32]interface{}), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - RetryInfo: &RetryInfo{}, - ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - StrictSQLMode: true, - AutoIncrementIncrement: DefAutoIncrementIncrement, - AutoIncrementOffset: DefAutoIncrementOffset, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - AllowBCJ: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - preferRangeScan: DefOptPreferRangeScan, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - CPUFactor: DefOptCPUFactor, - CopCPUFactor: DefOptCopCPUFactor, - CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - NetworkFactor: DefOptNetworkFactor, - ScanFactor: DefOptScanFactor, - DescScanFactor: DefOptDescScanFactor, - SeekFactor: DefOptSeekFactor, - MemoryFactor: DefOptMemoryFactor, - DiskFactor: DefOptDiskFactor, - ConcurrencyFactor: DefOptConcurrencyFactor, - EnableRadixJoin: false, - EnableVectorizedExpression: DefEnableVectorizedExpression, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, - WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, - WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, - enableIndexMerge: false, - EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: kv.ReplicaReadLeader, - AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, - UsePlanBaselines: DefTiDBUsePlanBaselines, - EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, - EnableExtendedStats: false, - IsolationReadEngines: make(map[kv.StoreType]struct{}), - LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, - MetricSchemaStep: DefTiDBMetricSchemaStep, - MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, - SequenceState: NewSequenceState(), - WindowingUseHighPrecision: true, - PrevFoundInPlanCache: DefTiDBFoundInPlanCache, - FoundInPlanCache: DefTiDBFoundInPlanCache, - PrevFoundInBinding: DefTiDBFoundInBinding, - FoundInBinding: DefTiDBFoundInBinding, - SelectLimit: math.MaxUint64, - AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, - EnableClusteredIndex: DefTiDBEnableClusteredIndex, - EnableParallelApply: DefTiDBEnableParallelApply, - ShardAllocateStep: DefTiDBShardAllocateStep, - EnableChangeColumnType: DefTiDBChangeColumnType, - EnableChangeMultiSchema: DefTiDBChangeMultiSchema, - EnablePointGetCache: DefTiDBPointGetCache, - EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, - PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), - TxnScope: config.GetGlobalConfig().TxnScope, - EnabledRateLimitAction: DefTiDBEnableRateLimitAction, - EnableAsyncCommit: DefTiDBEnableAsyncCommit, - Enable1PC: DefTiDBEnable1PC, - GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, - AnalyzeVersion: DefTiDBAnalyzeVersion, + Users: make(map[string]types.Datum), + UserVarTypes: make(map[string]*types.FieldType), + systems: make(map[string]string), + stmtVars: make(map[string]string), + PreparedStmts: make(map[uint32]interface{}), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + RetryInfo: &RetryInfo{}, + ActiveRoles: make([]*auth.RoleIdentity, 0, 10), + StrictSQLMode: true, + AutoIncrementIncrement: DefAutoIncrementIncrement, + AutoIncrementOffset: DefAutoIncrementOffset, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + AllowBCJ: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + preferRangeScan: DefOptPreferRangeScan, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + CPUFactor: DefOptCPUFactor, + CopCPUFactor: DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, + NetworkFactor: DefOptNetworkFactor, + ScanFactor: DefOptScanFactor, + DescScanFactor: DefOptDescScanFactor, + SeekFactor: DefOptSeekFactor, + MemoryFactor: DefOptMemoryFactor, + DiskFactor: DefOptDiskFactor, + ConcurrencyFactor: DefOptConcurrencyFactor, + EnableRadixJoin: false, + EnableVectorizedExpression: DefEnableVectorizedExpression, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, + WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, + enableIndexMerge: false, + EnableNoopFuncs: DefTiDBEnableNoopFuncs, + replicaRead: kv.ReplicaReadLeader, + AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, + UsePlanBaselines: DefTiDBUsePlanBaselines, + EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, + EnableExtendedStats: false, + IsolationReadEngines: make(map[kv.StoreType]struct{}), + LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, + MetricSchemaStep: DefTiDBMetricSchemaStep, + MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, + SequenceState: NewSequenceState(), + WindowingUseHighPrecision: true, + PrevFoundInPlanCache: DefTiDBFoundInPlanCache, + FoundInPlanCache: DefTiDBFoundInPlanCache, + PrevFoundInBinding: DefTiDBFoundInBinding, + FoundInBinding: DefTiDBFoundInBinding, + SelectLimit: math.MaxUint64, + AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, + EnableClusteredIndex: DefTiDBEnableClusteredIndex, + EnableParallelApply: DefTiDBEnableParallelApply, + ShardAllocateStep: DefTiDBShardAllocateStep, + EnableChangeColumnType: DefTiDBChangeColumnType, + EnableChangeMultiSchema: DefTiDBChangeMultiSchema, + EnablePointGetCache: DefTiDBPointGetCache, + EnableAlterPartitionPlacement: DefTiDBAlterPartitionPlacement, + EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, + PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), + TxnScope: config.GetGlobalConfig().TxnScope, + EnabledRateLimitAction: DefTiDBEnableRateLimitAction, + EnableAsyncCommit: DefTiDBEnableAsyncCommit, + Enable1PC: DefTiDBEnable1PC, + GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, + AnalyzeVersion: DefTiDBAnalyzeVersion, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1629,6 +1633,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableChangeMultiSchema = TiDBOptOn(val) case TiDBEnablePointGetCache: s.EnablePointGetCache = TiDBOptOn(val) + case TiDBEnableAlterPartitionPlacement: + s.EnableAlterPartitionPlacement = TiDBOptOn(val) case TiDBEnableAmendPessimisticTxn: s.EnableAmendPessimisticTxn = TiDBOptOn(val) case TiDBTxnScope: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 2b97549270a73..31aaf2bc29486 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1047,6 +1047,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToOnOff(DefTiDBChangeColumnType), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnablePointGetCache, Value: BoolToOnOff(DefTiDBPointGetCache), Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBEnableAlterPartitionPlacement, Value: BoolToOnOff(DefTiDBAlterPartitionPlacement), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority]}, {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 991c74f097d49..a6978a1130c9c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -355,6 +355,9 @@ const ( // TiDBEnablePointGetCache is used to control whether to enable the point get cache for special scenario. TiDBEnablePointGetCache = "tidb_enable_point_get_cache" + // TiDBEnableAlterPartitionPlacement is used to control whether to enable alter table partition. + TiDBEnableAlterPartitionPlacement = "tidb_enable_alter_partition_placement" + // tidb_max_delta_schema_count defines the max length of deltaSchemaInfos. // deltaSchemaInfos is a queue that maintains the history of schema changes. TiDBMaxDeltaSchemaCount = "tidb_max_delta_schema_count" @@ -569,6 +572,7 @@ const ( DefTiDBChangeColumnType = false DefTiDBChangeMultiSchema = false DefTiDBPointGetCache = false + DefTiDBAlterPartitionPlacement = false DefTiDBHashAggPartialConcurrency = ConcurrencyUnset DefTiDBHashAggFinalConcurrency = ConcurrencyUnset DefTiDBWindowConcurrency = ConcurrencyUnset From e5f35d7f916225f86e746767fc6db956f715668c Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Wed, 16 Dec 2020 15:55:34 +0800 Subject: [PATCH 2/5] . Signed-off-by: AilinKid <314806019@qq.com> --- ddl/ddl_api.go | 4 +- ddl/placement_sql_test.go | 24 ++--- session/session.go | 2 +- sessionctx/variable/session.go | 162 +++++++++++++++---------------- sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 6 +- 6 files changed, 100 insertions(+), 100 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c3fdf95b740ac..8c56edc9498c5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2430,10 +2430,10 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A isAlterTable := true err = d.RenameTable(ctx, ident, newIdent, isAlterTable) case ast.AlterTableAlterPartition: - if ctx.GetSessionVars().EnableAlterPartitionPlacement { + if ctx.GetSessionVars().EnableAlterPlacement { err = d.AlterTablePartition(ctx, ident, spec) } else { - err = errors.New("alter table partition alter partition is switched off by tidb_enable_alter_partition_placement") + err = errors.New("alter table partition alter partition is switched off by tidb_enable_alter_placement") } case ast.AlterTablePartition: // Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ... diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 0571448d73115..b09621ce0aeb8 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -30,10 +30,10 @@ func (s *testDBSuite1) TestAlterTableAlterPartition(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk.Se.GetSessionVars().EnableAlterPlacement = true defer func() { tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk.Se.GetSessionVars().EnableAlterPlacement = false }() tk.MustExec(`create table t1 (c int) @@ -295,10 +295,10 @@ add placement policy func (s *testDBSuite1) TestPlacementPolicyCache(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk.Se.GetSessionVars().EnableAlterPlacement = true defer func() { tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk.Se.GetSessionVars().EnableAlterPlacement = false }() initTable := func() []string { @@ -342,10 +342,10 @@ func (s *testSerialDBSuite) TestTxnScopeConstraint(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk.Se.GetSessionVars().EnableAlterPlacement = true defer func() { tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk.Se.GetSessionVars().EnableAlterPlacement = false }() tk.MustExec(`create table t1 (c int) @@ -487,10 +487,10 @@ func (s *testDBSuite1) TestAbortTxnIfPlacementChanged(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tp1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk.Se.GetSessionVars().EnableAlterPlacement = true defer func() { tk.MustExec("drop table if exists tp1") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk.Se.GetSessionVars().EnableAlterPlacement = false }() tk.MustExec(`create table tp1 (c int) @@ -503,9 +503,9 @@ PARTITION BY RANGE (c) ( tk1 := testkit.NewTestKitWithSession(c, s.store, se1) tk1.MustExec("use test") - tk1.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk1.Se.GetSessionVars().EnableAlterPlacement = true defer func() { - tk1.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk1.Se.GetSessionVars().EnableAlterPlacement = false }() _, err = tk.Exec(`alter table tp1 alter partition p0 add placement policy @@ -551,9 +551,9 @@ func (s *testDBSuite1) TestGlobalTxnState(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = true + tk.Se.GetSessionVars().EnableAlterPlacement = true defer func() { - tk.Se.GetSessionVars().EnableAlterPartitionPlacement = false + tk.Se.GetSessionVars().EnableAlterPlacement = false }() tk.MustExec(`create table t1 (c int) diff --git a/session/session.go b/session/session.go index 35e3b35dac817..569a937d88d06 100644 --- a/session/session.go +++ b/session/session.go @@ -2286,7 +2286,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableChangeColumnType, variable.TiDBEnableChangeMultiSchema, variable.TiDBEnablePointGetCache, - variable.TiDBEnableAlterPartitionPlacement, + variable.TiDBEnableAlterPlacement, variable.TiDBEnableAmendPessimisticTxn, variable.TiDBMemQuotaApplyCache, variable.TiDBEnableParallelApply, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 9f217f3e1dc21..c7b0991ba5f00 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -578,8 +578,8 @@ type SessionVars struct { // EnablePointGetCache is used to cache value for point get for read only scenario. EnablePointGetCache bool - // EnableAlterPartitionPlacement indicates whether a user can alter table partition placement rules. - EnableAlterPartitionPlacement bool + // EnableAlterPlacement indicates whether a user can alter table partition placement rules. + EnableAlterPlacement bool // WaitSplitRegionFinish defines the split region behaviour is sync or async. WaitSplitRegionFinish bool @@ -857,83 +857,83 @@ type ConnectionInfo struct { // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]types.Datum), - UserVarTypes: make(map[string]*types.FieldType), - systems: make(map[string]string), - stmtVars: make(map[string]string), - PreparedStmts: make(map[uint32]interface{}), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - RetryInfo: &RetryInfo{}, - ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - StrictSQLMode: true, - AutoIncrementIncrement: DefAutoIncrementIncrement, - AutoIncrementOffset: DefAutoIncrementOffset, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - AllowBCJ: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - preferRangeScan: DefOptPreferRangeScan, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - CPUFactor: DefOptCPUFactor, - CopCPUFactor: DefOptCopCPUFactor, - CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - NetworkFactor: DefOptNetworkFactor, - ScanFactor: DefOptScanFactor, - DescScanFactor: DefOptDescScanFactor, - SeekFactor: DefOptSeekFactor, - MemoryFactor: DefOptMemoryFactor, - DiskFactor: DefOptDiskFactor, - ConcurrencyFactor: DefOptConcurrencyFactor, - EnableRadixJoin: false, - EnableVectorizedExpression: DefEnableVectorizedExpression, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, - WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, - WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, - enableIndexMerge: false, - EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: kv.ReplicaReadLeader, - AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, - UsePlanBaselines: DefTiDBUsePlanBaselines, - EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, - EnableExtendedStats: false, - IsolationReadEngines: make(map[kv.StoreType]struct{}), - LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, - MetricSchemaStep: DefTiDBMetricSchemaStep, - MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, - SequenceState: NewSequenceState(), - WindowingUseHighPrecision: true, - PrevFoundInPlanCache: DefTiDBFoundInPlanCache, - FoundInPlanCache: DefTiDBFoundInPlanCache, - PrevFoundInBinding: DefTiDBFoundInBinding, - FoundInBinding: DefTiDBFoundInBinding, - SelectLimit: math.MaxUint64, - AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, - EnableClusteredIndex: DefTiDBEnableClusteredIndex, - EnableParallelApply: DefTiDBEnableParallelApply, - ShardAllocateStep: DefTiDBShardAllocateStep, - EnableChangeColumnType: DefTiDBChangeColumnType, - EnableChangeMultiSchema: DefTiDBChangeMultiSchema, - EnablePointGetCache: DefTiDBPointGetCache, - EnableAlterPartitionPlacement: DefTiDBAlterPartitionPlacement, - EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, - PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), - TxnScope: config.GetGlobalConfig().TxnScope, - EnabledRateLimitAction: DefTiDBEnableRateLimitAction, - EnableAsyncCommit: DefTiDBEnableAsyncCommit, - Enable1PC: DefTiDBEnable1PC, - GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, - AnalyzeVersion: DefTiDBAnalyzeVersion, + Users: make(map[string]types.Datum), + UserVarTypes: make(map[string]*types.FieldType), + systems: make(map[string]string), + stmtVars: make(map[string]string), + PreparedStmts: make(map[uint32]interface{}), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + RetryInfo: &RetryInfo{}, + ActiveRoles: make([]*auth.RoleIdentity, 0, 10), + StrictSQLMode: true, + AutoIncrementIncrement: DefAutoIncrementIncrement, + AutoIncrementOffset: DefAutoIncrementOffset, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + AllowBCJ: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + preferRangeScan: DefOptPreferRangeScan, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + CPUFactor: DefOptCPUFactor, + CopCPUFactor: DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, + NetworkFactor: DefOptNetworkFactor, + ScanFactor: DefOptScanFactor, + DescScanFactor: DefOptDescScanFactor, + SeekFactor: DefOptSeekFactor, + MemoryFactor: DefOptMemoryFactor, + DiskFactor: DefOptDiskFactor, + ConcurrencyFactor: DefOptConcurrencyFactor, + EnableRadixJoin: false, + EnableVectorizedExpression: DefEnableVectorizedExpression, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, + WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, + enableIndexMerge: false, + EnableNoopFuncs: DefTiDBEnableNoopFuncs, + replicaRead: kv.ReplicaReadLeader, + AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, + UsePlanBaselines: DefTiDBUsePlanBaselines, + EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, + EnableExtendedStats: false, + IsolationReadEngines: make(map[kv.StoreType]struct{}), + LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, + MetricSchemaStep: DefTiDBMetricSchemaStep, + MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, + SequenceState: NewSequenceState(), + WindowingUseHighPrecision: true, + PrevFoundInPlanCache: DefTiDBFoundInPlanCache, + FoundInPlanCache: DefTiDBFoundInPlanCache, + PrevFoundInBinding: DefTiDBFoundInBinding, + FoundInBinding: DefTiDBFoundInBinding, + SelectLimit: math.MaxUint64, + AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, + EnableClusteredIndex: DefTiDBEnableClusteredIndex, + EnableParallelApply: DefTiDBEnableParallelApply, + ShardAllocateStep: DefTiDBShardAllocateStep, + EnableChangeColumnType: DefTiDBChangeColumnType, + EnableChangeMultiSchema: DefTiDBChangeMultiSchema, + EnablePointGetCache: DefTiDBPointGetCache, + EnableAlterPlacement: DefTiDBEnableAlterPlacement, + EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, + PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), + TxnScope: config.GetGlobalConfig().TxnScope, + EnabledRateLimitAction: DefTiDBEnableRateLimitAction, + EnableAsyncCommit: DefTiDBEnableAsyncCommit, + Enable1PC: DefTiDBEnable1PC, + GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, + AnalyzeVersion: DefTiDBAnalyzeVersion, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1636,8 +1636,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableChangeMultiSchema = TiDBOptOn(val) case TiDBEnablePointGetCache: s.EnablePointGetCache = TiDBOptOn(val) - case TiDBEnableAlterPartitionPlacement: - s.EnableAlterPartitionPlacement = TiDBOptOn(val) + case TiDBEnableAlterPlacement: + s.EnableAlterPlacement = TiDBOptOn(val) case TiDBEnableAmendPessimisticTxn: s.EnableAmendPessimisticTxn = TiDBOptOn(val) case TiDBTxnScope: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 420133c6b21e3..27852ef076b99 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1047,7 +1047,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToOnOff(DefTiDBChangeColumnType), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnablePointGetCache, Value: BoolToOnOff(DefTiDBPointGetCache), Type: TypeBool}, - {Scope: ScopeGlobal, Name: TiDBEnableAlterPartitionPlacement, Value: BoolToOnOff(DefTiDBAlterPartitionPlacement), Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBEnableAlterPlacement, Value: BoolToOnOff(DefTiDBEnableAlterPlacement), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority]}, {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 0d05f269e3278..c013eb2a24d95 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -355,8 +355,8 @@ const ( // TiDBEnablePointGetCache is used to control whether to enable the point get cache for special scenario. TiDBEnablePointGetCache = "tidb_enable_point_get_cache" - // TiDBEnableAlterPartitionPlacement is used to control whether to enable alter table partition. - TiDBEnableAlterPartitionPlacement = "tidb_enable_alter_partition_placement" + // TiDBEnableAlterPlacement is used to control whether to enable alter table partition. + TiDBEnableAlterPlacement = "tidb_enable_alter_placement" // tidb_max_delta_schema_count defines the max length of deltaSchemaInfos. // deltaSchemaInfos is a queue that maintains the history of schema changes. @@ -575,7 +575,7 @@ const ( DefTiDBChangeColumnType = false DefTiDBChangeMultiSchema = false DefTiDBPointGetCache = false - DefTiDBAlterPartitionPlacement = false + DefTiDBEnableAlterPlacement = false DefTiDBHashAggPartialConcurrency = ConcurrencyUnset DefTiDBHashAggFinalConcurrency = ConcurrencyUnset DefTiDBWindowConcurrency = ConcurrencyUnset From 2d4626a1091dc84a00c4b8385703f25f98b628dd Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Thu, 17 Dec 2020 15:19:50 +0800 Subject: [PATCH 3/5] address comment Signed-off-by: AilinKid <314806019@qq.com> --- session/session.go | 1 - sessionctx/variable/sysvar.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/session/session.go b/session/session.go index 569a937d88d06..21d98b0a10662 100644 --- a/session/session.go +++ b/session/session.go @@ -2286,7 +2286,6 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableChangeColumnType, variable.TiDBEnableChangeMultiSchema, variable.TiDBEnablePointGetCache, - variable.TiDBEnableAlterPlacement, variable.TiDBEnableAmendPessimisticTxn, variable.TiDBMemQuotaApplyCache, variable.TiDBEnableParallelApply, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 27852ef076b99..b4215ba8f9e0b 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1047,7 +1047,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToOnOff(DefTiDBChangeColumnType), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnablePointGetCache, Value: BoolToOnOff(DefTiDBPointGetCache), Type: TypeBool}, - {Scope: ScopeGlobal, Name: TiDBEnableAlterPlacement, Value: BoolToOnOff(DefTiDBEnableAlterPlacement), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBEnableAlterPlacement, Value: BoolToOnOff(DefTiDBEnableAlterPlacement), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority]}, {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63}, From 77c12968542f1db6b4b1a2ab495067a33966f74a Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Thu, 17 Dec 2020 15:58:14 +0800 Subject: [PATCH 4/5] . Signed-off-by: AilinKid <314806019@qq.com> --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 8c56edc9498c5..4d8556472ba82 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2433,7 +2433,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A if ctx.GetSessionVars().EnableAlterPlacement { err = d.AlterTablePartition(ctx, ident, spec) } else { - err = errors.New("alter table partition alter partition is switched off by tidb_enable_alter_placement") + err = errors.New("alter partition alter placement is experimental and it is switched off by tidb_enable_alter_placement") } case ast.AlterTablePartition: // Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ... From 79ff523633cad066a368f7ae08201b115f9cdfd7 Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Thu, 17 Dec 2020 17:29:33 +0800 Subject: [PATCH 5/5] . Signed-off-by: AilinKid <314806019@qq.com> --- session/session.go | 1 + sessionctx/variable/sysvar.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 21d98b0a10662..569a937d88d06 100644 --- a/session/session.go +++ b/session/session.go @@ -2286,6 +2286,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableChangeColumnType, variable.TiDBEnableChangeMultiSchema, variable.TiDBEnablePointGetCache, + variable.TiDBEnableAlterPlacement, variable.TiDBEnableAmendPessimisticTxn, variable.TiDBMemQuotaApplyCache, variable.TiDBEnableParallelApply, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b4215ba8f9e0b..27852ef076b99 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1047,7 +1047,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToOnOff(DefTiDBChangeColumnType), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnablePointGetCache, Value: BoolToOnOff(DefTiDBPointGetCache), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBEnableAlterPlacement, Value: BoolToOnOff(DefTiDBEnableAlterPlacement), Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBEnableAlterPlacement, Value: BoolToOnOff(DefTiDBEnableAlterPlacement), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority]}, {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63},