Skip to content

Commit

Permalink
Merge branch 'master' into options-replica-read-type
Browse files Browse the repository at this point in the history
  • Loading branch information
disksing authored May 12, 2021
2 parents e6d79ef + 0179854 commit cf0e684
Show file tree
Hide file tree
Showing 16 changed files with 187 additions and 29 deletions.
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1020,7 +1020,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrBuildExecutor: mysql.Message("Failed to build executor", nil),
ErrBatchInsertFail: mysql.Message("Batch insert failed, please clean the table and try again.", nil),
ErrGetStartTS: mysql.Message("Can not get start ts", nil),
ErrPrivilegeCheckFail: mysql.Message("privilege check fail", nil), // this error message should begin lowercased to be compatible with the test
ErrPrivilegeCheckFail: mysql.Message("privilege check for '%s' fail", nil), // this error message should begin lowercased to be compatible with the test
ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil),
ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil),
ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil),
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1133,7 +1133,7 @@ Schema has changed

["planner:8121"]
error = '''
privilege check fail
privilege check for '%s' fail
'''

["planner:8122"]
Expand Down
2 changes: 1 addition & 1 deletion executor/mpp_gather.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 6 additions & 0 deletions executor/revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,14 @@ func (e *RevokeExec) Next(ctx context.Context, req *chunk.Chunk) error {
return err
}

sessVars := e.ctx.GetSessionVars()
// Revoke for each user.
for _, user := range e.Users {
if user.User.CurrentUser {
user.User.Username = sessVars.User.AuthUsername
user.User.Hostname = sessVars.User.AuthHostname
}

// Check if user exists.
exists, err := userExists(e.ctx, user.User.Username, user.User.Hostname)
if err != nil {
Expand Down
6 changes: 3 additions & 3 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1662,7 +1662,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 {
Expand Down Expand Up @@ -1965,7 +1965,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))
Expand Down Expand Up @@ -2355,7 +2355,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 {
Expand Down
126 changes: 126 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3706,3 +3706,129 @@ func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) {
res.Check(testkit.Rows(output[i].Plan...))
}
}

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"))
}
4 changes: 2 additions & 2 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,13 +103,13 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [
if v.privilege == mysql.ExtendedPriv {
if !pm.RequestDynamicVerification(activeRoles, v.dynamicPriv, v.dynamicWithGrant) {
if v.err == nil {
return ErrPrivilegeCheckFail
return ErrPrivilegeCheckFail.GenWithStackByArgs(v.dynamicPriv)
}
return v.err
}
} else if !pm.RequestVerification(activeRoles, v.db, v.table, v.column, v.privilege) {
if v.err == nil {
return ErrPrivilegeCheckFail
return ErrPrivilegeCheckFail.GenWithStackByArgs(v.privilege.String())
}
return v.err
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -994,7 +994,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch
var visitInfos []visitInfo
for _, checkType := range checkTypes {
if pm != nil && !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) {
return errors.New("privilege check fail")
return ErrPrivilegeCheckFail.GenWithStackByArgs(checkType.String())
}
// This visitInfo is only for table lock check, so we do not need column field,
// just fill it empty string.
Expand Down
8 changes: 6 additions & 2 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -2026,11 +2026,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
}

Expand Down
16 changes: 11 additions & 5 deletions privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -843,6 +843,12 @@ func (s *testPrivilegeSuite) TestRevokePrivileges(c *C) {
c.Assert(se.Auth(&auth.UserIdentity{Username: "hasgrant", Hostname: "localhost", AuthUsername: "hasgrant", AuthHostname: "%"}, nil, nil), IsTrue)
mustExec(c, se, "REVOKE SELECT ON mysql.* FROM 'withoutgrant'")
mustExec(c, se, "REVOKE ALL ON mysql.* FROM withoutgrant")

// For issue https://github.com/pingcap/tidb/issues/23850
mustExec(c, se, "CREATE USER u4")
mustExec(c, se, "GRANT ALL ON *.* TO u4 WITH GRANT OPTION")
c.Assert(se.Auth(&auth.UserIdentity{Username: "u4", Hostname: "localhost", AuthUsername: "u4", AuthHostname: "%"}, nil, nil), IsTrue)
mustExec(c, se, "REVOKE ALL ON *.* FROM CURRENT_USER()")
}

func (s *testPrivilegeSuite) TestSetGlobal(c *C) {
Expand Down Expand Up @@ -1006,14 +1012,14 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) {
_, err = se.ExecuteInternal(context.Background(), "drop table information_schema.tables")
c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "update information_schema.tables set table_name = 'tst' where table_name = 'mysql'")
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)

// Test performance_schema.
mustExec(c, se, `select * from performance_schema.events_statements_summary_by_digest`)
_, err = se.ExecuteInternal(context.Background(), "drop table performance_schema.events_statements_summary_by_digest")
c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "update performance_schema.events_statements_summary_by_digest set schema_name = 'tst'")
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "delete from performance_schema.events_statements_summary_by_digest")
c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "create table performance_schema.t(a int)")
Expand All @@ -1025,7 +1031,7 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) {
_, err = se.ExecuteInternal(context.Background(), "drop table metrics_schema.tidb_query_duration")
c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "update metrics_schema.tidb_query_duration set instance = 'tst'")
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "delete from metrics_schema.tidb_query_duration")
c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "create table metric_schema.t(a int)")
Expand All @@ -1041,9 +1047,9 @@ func (s *testPrivilegeSuite) TestAdminCommand(c *C) {

c.Assert(se.Auth(&auth.UserIdentity{Username: "test_admin", Hostname: "localhost"}, nil, nil), IsTrue)
_, err := se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS")
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "ADMIN CHECK TABLE t")
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)

c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue)
_, err = se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS")
Expand Down
2 changes: 1 addition & 1 deletion session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2892,7 +2892,7 @@ func (s *testSessionSuite2) TestUpdatePrivilege(c *C) {

_, err := tk1.Exec("update t2 set id = 666 where id = 1;")
c.Assert(err, NotNil)
c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue)
c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue)

// Cover a bug that t1 and t2 both require update privilege.
// In fact, the privlege check for t1 should be update, and for t2 should be select.
Expand Down
19 changes: 15 additions & 4 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -492,11 +492,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
Expand Down Expand Up @@ -845,6 +846,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 {
Expand Down Expand Up @@ -1094,7 +1105,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 {
Expand Down
6 changes: 3 additions & 3 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -720,8 +720,8 @@ var defaultSysVars = []*SysVar{
}
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), SetSession: func(s *SessionVars, val string) error {
s.AllowMPPExecution = TiDBOptOn(val)
{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 {
Expand Down Expand Up @@ -793,7 +793,7 @@ var defaultSysVars = []*SysVar{
s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
{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
}},
Expand Down
Loading

0 comments on commit cf0e684

Please sign in to comment.