diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 536a06eda8993..2c6955853bf90 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -30,7 +30,7 @@ import ( ) func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader) bool { - if !ctx.GetSessionVars().AllowMPPExecution { + if !ctx.GetSessionVars().IsMPPAllowed() { return false } _, ok := tr.GetTablePlan().(*plannercore.PhysicalExchangeSender) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index dedf980ee063f..f293eb12f2899 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1700,7 +1700,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } joins := make([]PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) - if p.ctx.GetSessionVars().AllowMPPExecution && canPushToTiFlash { + if p.ctx.GetSessionVars().IsMPPAllowed() && canPushToTiFlash { if p.shouldUseMPPBCJ() { mppJoins := p.tryToGetMppHashJoin(prop, true) if (p.preferJoinType & preferBCJoin) > 0 { @@ -2028,7 +2028,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !lt.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if lt.ctx.GetSessionVars().AllowMPPExecution { + if lt.ctx.GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -2437,7 +2437,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) } canPushDownToTiFlash := la.canPushToCop(kv.TiFlash) - canPushDownToMPP := la.ctx.GetSessionVars().AllowMPPExecution && la.checkCanPushDownToMPP() && canPushDownToTiFlash + canPushDownToMPP := la.ctx.GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() && canPushDownToTiFlash if la.HasDistinct() { // TODO: remove after the cost estimation of distinct pushdown is implemented. if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index e7cbf8d88bfd7..db5eceab8ccfd 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3601,3 +3601,129 @@ func (s *testIntegrationSuite) TestIssue23839(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=2000001") tk.Exec("explain SELECT OUTR . col2 AS X FROM (SELECT INNR . col1 as col1, SUM( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR2 INNER JOIN (SELECT INNR . col1 as col1, MAX( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR ON OUTR2.col1 = OUTR.col1 GROUP BY OUTR . col1, OUTR2 . col1 HAVING X <> 'b'") } + +func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // test value limit of tidb_opt_tiflash_concurrency_factor + err := tk.ExecToErr("set @@tidb_opt_tiflash_concurrency_factor = 0") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_opt_tiflash_concurrency_factor' can't be set to the value of '0'`) + + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 1") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("1")) + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 24") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("24")) + + // test set tidb_allow_mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@session.tidb_allow_mpp = off") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = oN") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = enForcE") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@global.tidb_allow_mpp = faLsE") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@global.tidb_allow_mpp = True") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("ON")) + + err = tk.ExecToErr("set @@global.tidb_allow_mpp = enforceWithTypo") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_allow_mpp' can't be set to the value of 'enforceWithTypo'`) + + // test query + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("create index idx on t(a)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + // ban mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + + // read from tiflash, batch cop. + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "StreamAgg_20 1.00 285050.00 root funcs:count(Column#5)->Column#3", + "└─TableReader_21 1.00 19003.88 root data:StreamAgg_9", + " └─StreamAgg_9 1.00 19006.88 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_19 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_18 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // open mpp + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + + // should use tikv to index read + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_30 1.00 485.00 root funcs:count(Column#6)->Column#3", + "└─IndexReader_31 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_29 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with large cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 11910.68 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 11877.08 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // enforce mpp + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + // should use mpp + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "HashAgg_24 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_26 1.00 0.00 root data:ExchangeSender_25", + " └─ExchangeSender_25 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_23 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with little cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 0.00 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) +} diff --git a/planner/core/task.go b/planner/core/task.go index 4b2ab61c8c413..0e842adb2d751 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2053,11 +2053,15 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() + + p.cost = t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + if p.ctx.GetSessionVars().IsMPPEnforced() { + p.cost = 0 + } rt := &rootTask{ p: p, - cst: t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor, + cst: p.cost, } - p.cost = rt.cost() return rt } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 7721ce0292ae9..56a592d8b8bac 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -501,11 +501,12 @@ type SessionVars struct { AllowWriteRowID bool // AllowBatchCop means if we should send batch coprocessor to TiFlash. Default value is 1, means to use batch cop in case of aggregation and join. - // If value is set to 2 , which means to force to send batch cop for any query. Value is set to 0 means never use batch cop. + // Value set to 2 means to force to send batch cop for any query. Value set to 0 means never use batch cop. AllowBatchCop int - // AllowMPPExecution will prefer using mpp way to execute a query. - AllowMPPExecution bool + // AllowMPPExecution means if we should use mpp way to execute query. Default value is "ON", means to be determined by the optimizer. + // Value set to "ENFORCE" means to use mpp whenever possible. Value set to means never use mpp. + allowMPPExecution string // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. AllowAutoRandExplicitInsert bool @@ -850,6 +851,16 @@ func (s *SessionVars) AllocMPPTaskID(startTS uint64) int64 { return 1 } +// IsMPPAllowed returns whether mpp execution is allowed. +func (s *SessionVars) IsMPPAllowed() bool { + return s.allowMPPExecution != "OFF" +} + +// IsMPPEnforced returns whether mpp execution is enforced. +func (s *SessionVars) IsMPPEnforced() bool { + return s.allowMPPExecution == "ENFORCE" +} + // CheckAndGetTxnScope will return the transaction scope we should use in the current session. func (s *SessionVars) CheckAndGetTxnScope() string { if s.InRestrictedSQL { @@ -1077,7 +1088,7 @@ func NewSessionVars() *SessionVars { terror.Log(vars.SetSystemVar(TiDBEnableStreaming, enableStreaming)) vars.AllowBatchCop = DefTiDBAllowBatchCop - vars.AllowMPPExecution = DefTiDBAllowMPPExecution + vars.allowMPPExecution = DefTiDBAllowMPPExecution var enableChunkRPC string if config.GetGlobalConfig().TiKVClient.EnableChunkRPC { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 3c8affc8c0b20..267f015d74423 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -578,13 +578,40 @@ var defaultSysVars = []*SysVar{ return oracle.GlobalTxnScope } return oracle.LocalTxnScope - }()}, - /* TiDB specific variables */ - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdCount, Value: strconv.Itoa(DefBroadcastJoinThresholdCount), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdSize, Value: strconv.Itoa(DefBroadcastJoinThresholdSize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64}, - {Scope: ScopeSession, Name: TiDBSnapshot, Value: ""}, - {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToOnOff(DefOptAggPushDown), Type: TypeBool}, + }(), SetSession: func(s *SessionVars, val string) error { + switch val { + case oracle.GlobalTxnScope: + s.TxnScope = oracle.NewGlobalTxnScope() + case oracle.LocalTxnScope: + s.TxnScope = oracle.GetTxnScope() + default: + return ErrWrongValueForVar.GenWithStack("@@txn_scope value should be global or local") + } + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { + s.allowMPPExecution = val + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdCount, Value: strconv.Itoa(DefBroadcastJoinThresholdCount), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.BroadcastJoinThresholdCount = tidbOptInt64(val, DefBroadcastJoinThresholdCount) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdSize, Value: strconv.Itoa(DefBroadcastJoinThresholdSize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.BroadcastJoinThresholdSize = tidbOptInt64(val, DefBroadcastJoinThresholdSize) + return nil + }}, + {Scope: ScopeSession, Name: TiDBSnapshot, Value: "", SetSession: func(s *SessionVars, val string) error { + err := setSnapshotTS(s, val) + if err != nil { + return err + } + return nil + }}, + {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToOnOff(DefOptAggPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.AllowAggPushDown = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptBCJ, Value: BoolToOnOff(DefOptBCJ), Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) && vars.AllowBatchCop == 0 { return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs("Can't set Broadcast Join to 1 but tidb_allow_batch_cop is 0, please active batch cop at first.") @@ -600,33 +627,120 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, Value: strconv.Itoa(DefChecksumTableConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptInSubqToJoinAndAgg, Value: BoolToOnOff(DefOptInSubqToJoinAndAgg), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBOptPreferRangeScan, Value: BoolToOnOff(DefOptPreferRangeScan), Type: TypeBool, IsHintUpdatable: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationThreshold, Value: strconv.FormatFloat(DefOptCorrelationThreshold, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 1}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCPUFactor, Value: strconv.FormatFloat(DefOptCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCopCPUFactor, Value: strconv.FormatFloat(DefOptCopCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptNetworkFactor, Value: strconv.FormatFloat(DefOptNetworkFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptScanFactor, Value: strconv.FormatFloat(DefOptScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDescScanFactor, Value: strconv.FormatFloat(DefOptDescScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMemoryFactor, Value: strconv.FormatFloat(DefOptMemoryFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDiskFactor, Value: strconv.FormatFloat(DefOptDiskFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptConcurrencyFactor, Value: strconv.FormatFloat(DefOptConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipUTF8Check, Value: BoolToOnOff(DefSkipUTF8Check), Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipASCIICheck, Value: BoolToOnOff(DefSkipASCIICheck), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToOnOff(DefBatchDelete), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToOnOff(DefBatchCommit), Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.ExecutorConcurrency = tidbOptPositiveInt32(val, DefExecutorConcurrency) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.distSQLScanConcurrency = tidbOptPositiveInt32(val, DefDistSQLScanConcurrency) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptInSubqToJoinAndAgg, Value: BoolToOnOff(DefOptInSubqToJoinAndAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.SetAllowInSubqToJoinAndAgg(TiDBOptOn(val)) + return nil + }}, + {Scope: ScopeSession, Name: TiDBOptPreferRangeScan, Value: BoolToOnOff(DefOptPreferRangeScan), Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { + s.SetAllowPreferRangeScan(TiDBOptOn(val)) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationThreshold, Value: strconv.FormatFloat(DefOptCorrelationThreshold, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { + s.CorrelationThreshold = tidbOptFloat64(val, DefOptCorrelationThreshold) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.CorrelationExpFactor = int(tidbOptInt64(val, DefOptCorrelationExpFactor)) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCPUFactor, Value: strconv.FormatFloat(DefOptCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCopCPUFactor, Value: strconv.FormatFloat(DefOptCopCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.CopCPUFactor = tidbOptFloat64(val, DefOptCopCPUFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptNetworkFactor, Value: strconv.FormatFloat(DefOptNetworkFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.NetworkFactor = tidbOptFloat64(val, DefOptNetworkFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptScanFactor, Value: strconv.FormatFloat(DefOptScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.ScanFactor = tidbOptFloat64(val, DefOptScanFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDescScanFactor, Value: strconv.FormatFloat(DefOptDescScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.DescScanFactor = tidbOptFloat64(val, DefOptDescScanFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.SeekFactor = tidbOptFloat64(val, DefOptSeekFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMemoryFactor, Value: strconv.FormatFloat(DefOptMemoryFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.MemoryFactor = tidbOptFloat64(val, DefOptMemoryFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDiskFactor, Value: strconv.FormatFloat(DefOptDiskFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.DiskFactor = tidbOptFloat64(val, DefOptDiskFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptConcurrencyFactor, Value: strconv.FormatFloat(DefOptConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.ConcurrencyFactor = tidbOptFloat64(val, DefOptConcurrencyFactor) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.IndexJoinBatchSize = tidbOptPositiveInt32(val, DefIndexJoinBatchSize) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.IndexLookupSize = tidbOptPositiveInt32(val, DefIndexLookupSize) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexLookupConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipUTF8Check, Value: BoolToOnOff(DefSkipUTF8Check), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.SkipUTF8Check = TiDBOptOn(val) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipASCIICheck, Value: BoolToOnOff(DefSkipASCIICheck), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.SkipASCIICheck = TiDBOptOn(val) + return nil + }}, + {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.BatchInsert = TiDBOptOn(val) + return nil + }}, + {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToOnOff(DefBatchDelete), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.BatchDelete = TiDBOptOn(val) + return nil + }}, + {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToOnOff(DefBatchCommit), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.BatchCommit = TiDBOptOn(val) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.DMLBatchSize = int(tidbOptInt64(val, DefOptCorrelationExpFactor)) + return nil + }}, {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index a642b99238eaa..6deeaca47c1e0 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -299,6 +299,8 @@ const ( // The default value is 0 TiDBAllowBatchCop = "tidb_allow_batch_cop" + // TiDBAllowMPPExecution means if we should use mpp way to execute query. Default value is 1 (or 'ON'), means to be determined by the optimizer. + // Value set to 2 (or 'ENFORCE') which means to use mpp whenever possible. Value set to 2 (or 'OFF') means never use mpp. TiDBAllowMPPExecution = "tidb_allow_mpp" // TiDBInitChunkSize is used to control the init chunk size during query execution. @@ -622,7 +624,7 @@ const ( DefBroadcastJoinThresholdCount = 10 * 1024 DefTiDBOptimizerSelectivityLevel = 0 DefTiDBAllowBatchCop = 1 - DefTiDBAllowMPPExecution = true + DefTiDBAllowMPPExecution = "ON" DefTiDBTxnMode = "" DefTiDBRowFormatV1 = 1 DefTiDBRowFormatV2 = 2