Skip to content

Commit

Permalink
Change maximum to 256
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo committed Oct 15, 2021
1 parent d7c2557 commit 397eb52
Show file tree
Hide file tree
Showing 7 changed files with 43 additions and 45 deletions.
6 changes: 3 additions & 3 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1218,9 +1218,9 @@ func (s *testSuite6) TestSetDDLReorgWorkerCnt(c *C) {
res = tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt")
res.Check(testkit.Rows("100"))

tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 129")
tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '129'"))
tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("128"))
tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 257")
tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '257'"))
tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("256"))
}

func (s *testSuite6) TestSetDDLReorgBatchSize(c *C) {
Expand Down
4 changes: 2 additions & 2 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,8 @@ func (s *testSuite) TestJoinInDisk(c *C) {
func (s *testSuiteJoin2) TestJoin(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("set @@tidb_index_lookup_join_concurrency = 100")
c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency(), Equals, 100)
tk.MustExec("set @@tidb_index_lookup_join_concurrency = 200")
c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency(), Equals, 200)

tk.MustExec("set @@tidb_index_lookup_join_concurrency = 4")
c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency(), Equals, 4)
Expand Down
4 changes: 2 additions & 2 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1306,7 +1306,7 @@ var defaultSysVars = []*SysVar{
}
return config.HideConfig(string(j)), nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error {
SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount)))
return nil
}},
Expand Down Expand Up @@ -1729,7 +1729,7 @@ var defaultSysVars = []*SysVar{
if err := setTiDBTableValue(s, "tikv_gc_auto_concurrency", autoConcurrency, "Let TiDB pick the concurrency automatically. If set false, tikv_gc_concurrency will be used"); err != nil {
return err
}
return setTiDBTableValue(s, "tikv_gc_concurrency", val, "How many goroutines used to do GC parallel, [1, 128], default 2")
return setTiDBTableValue(s, "tikv_gc_concurrency", val, "How many goroutines used to do GC parallel, [1, 256], default 2")
}},
{Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "LEGACY", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}, GetGlobal: func(s *SessionVars) (string, error) {
return getTiDBTableValue(s, "tikv_gc_scan_lock_mode", "LEGACY")
Expand Down
26 changes: 26 additions & 0 deletions sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -751,3 +751,29 @@ func TestIdentity(t *testing.T) {
require.NoError(t, err)
require.Equal(t, val, "21")
}

func TestDDLWorkers(t *testing.T) {
svWorkerCount, svBatchSize := GetSysVar(TiDBDDLReorgWorkerCount), GetSysVar(TiDBDDLReorgBatchSize)
vars := NewSessionVars()
vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests()

val, err := svWorkerCount.Validate(vars, "-100", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, "1") // converts it to min value
val, err = svWorkerCount.Validate(vars, "1234", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, "256") // converts it to max value
val, err = svWorkerCount.Validate(vars, "100", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, "100") // unchanged

val, err = svBatchSize.Validate(vars, "10", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, fmt.Sprint(MinDDLReorgBatchSize)) // converts it to min value
val, err = svBatchSize.Validate(vars, "999999", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, fmt.Sprint(MaxDDLReorgBatchSize)) // converts it to max value
val, err = svBatchSize.Validate(vars, "100", ScopeGlobal)
require.NoError(t, err)
require.Equal(t, val, "100") // unchanged
}
17 changes: 8 additions & 9 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -609,7 +609,7 @@ const (
const (
// MaxConfigurableConcurrency is the maximum number of "threads" (goroutines) that can be specified
// for any type of configuration item that has concurrent workers.
MaxConfigurableConcurrency = 128
MaxConfigurableConcurrency = 256
)

// Default TiDB system variable values.
Expand Down Expand Up @@ -757,14 +757,13 @@ const (

// Process global variables.
var (
ProcessGeneralLog = atomic.NewBool(false)
EnablePProfSQLCPU = atomic.NewBool(false)
ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount
maxDDLReorgWorkerCount int32 = 128
ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize
ddlErrorCountlimit int64 = DefTiDBDDLErrorCountLimit
ddlReorgRowFormat int64 = DefTiDBRowFormatV2
maxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount
ProcessGeneralLog = atomic.NewBool(false)
EnablePProfSQLCPU = atomic.NewBool(false)
ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount
ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize
ddlErrorCountlimit int64 = DefTiDBDDLErrorCountLimit
ddlReorgRowFormat int64 = DefTiDBRowFormatV2
maxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount
// Export for testing.
MaxDDLReorgBatchSize int32 = 10240
MinDDLReorgBatchSize int32 = 32
Expand Down
13 changes: 2 additions & 11 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,11 +35,8 @@ import (
const secondsPerYear = 60 * 60 * 24 * 365

// SetDDLReorgWorkerCounter sets ddlReorgWorkerCounter count.
// Max worker count is maxDDLReorgWorkerCount.
// Sysvar validation enforces the range to already be correct.
func SetDDLReorgWorkerCounter(cnt int32) {
if cnt > maxDDLReorgWorkerCount {
cnt = maxDDLReorgWorkerCount
}
atomic.StoreInt32(&ddlReorgWorkerCounter, cnt)
}

Expand All @@ -49,14 +46,8 @@ func GetDDLReorgWorkerCounter() int32 {
}

// SetDDLReorgBatchSize sets ddlReorgBatchSize size.
// Max batch size is MaxDDLReorgBatchSize.
// Sysvar validation enforces the range to already be correct.
func SetDDLReorgBatchSize(cnt int32) {
if cnt > MaxDDLReorgBatchSize {
cnt = MaxDDLReorgBatchSize
}
if cnt < MinDDLReorgBatchSize {
cnt = MinDDLReorgBatchSize
}
atomic.StoreInt32(&ddlReorgBatchSize, cnt)
}

Expand Down
18 changes: 0 additions & 18 deletions sessionctx/variable/varsutil_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,24 +496,6 @@ func TestVarsutil(t *testing.T) {
require.Regexp(t, ".*Truncated incorrect tidb_analyze_version value", warn.Err.Error())
}

func TestSetOverflowBehave(t *testing.T) {
ddRegWorker := maxDDLReorgWorkerCount + 1
SetDDLReorgWorkerCounter(ddRegWorker)
require.Equal(t, GetDDLReorgWorkerCounter(), maxDDLReorgWorkerCount)

ddlReorgBatchSize := MaxDDLReorgBatchSize + 1
SetDDLReorgBatchSize(ddlReorgBatchSize)
require.Equal(t, GetDDLReorgBatchSize(), MaxDDLReorgBatchSize)
ddlReorgBatchSize = MinDDLReorgBatchSize - 1
SetDDLReorgBatchSize(ddlReorgBatchSize)
require.Equal(t, GetDDLReorgBatchSize(), MinDDLReorgBatchSize)

val := tidbOptInt64("a", 1)
require.Equal(t, int64(1), val)
val2 := tidbOptFloat64("b", 1.2)
require.Equal(t, 1.2, val2)
}

func TestValidate(t *testing.T) {
v := NewSessionVars()
v.GlobalVarsAccessor = NewMockGlobalAccessor4Tests()
Expand Down

0 comments on commit 397eb52

Please sign in to comment.