Skip to content

Commit

Permalink
ddl: change system variables name (#42200)
Browse files Browse the repository at this point in the history
close #42186
  • Loading branch information
Benjamin2037 authored Mar 25, 2023
1 parent d7fe314 commit 922a5d3
Show file tree
Hide file tree
Showing 18 changed files with 52 additions and 52 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -420,7 +420,7 @@ bazel_coverage_test: check-bazel-prepare failpoint-enable bazel_ci_prepare
-- //... -//cmd/... -//tests/graceshutdown/... \
-//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/...
bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) --build_tests_only --test_keep_going=false \
--@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest,distributereorg \
--@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest,disttask \
-- //... -//cmd/... -//tests/graceshutdown/... \
-//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/...

Expand Down
2 changes: 1 addition & 1 deletion ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -991,7 +991,7 @@ func TestAddIndexFailOnCaseWhenCanExit(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int)")
tk.MustExec("insert into t values(1, 1)")
if variable.DDLEnableDistributeReorg.Load() {
if variable.EnableDistTask.Load() {
tk.MustGetErrMsg("alter table t add index idx(b)", "[ddl:-1]job.ErrCount:0, mock unknown type: ast.whenClause.")
} else {
tk.MustGetErrMsg("alter table t add index idx(b)", "[ddl:-1]DDL job rollback, error msg: job.ErrCount:1, mock unknown type: ast.whenClause.")
Expand Down
2 changes: 1 addition & 1 deletion ddl/dist_owner.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ const (
)

func initDistReorg(reorgMeta *model.DDLReorgMeta) {
isDistReorg := variable.DDLEnableDistributeReorg.Load()
isDistReorg := variable.EnableDistTask.Load()
reorgMeta.IsDistReorg = isDistReorg
if isDistReorg {
metrics.TelemetryDistReorgCnt.Inc()
Expand Down
4 changes: 2 additions & 2 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,7 @@ func TestAddIndexFailed(t *testing.T) {
}

func TestAddIndexCanceledInDistReorg(t *testing.T) {
if !variable.DDLEnableDistributeReorg.Load() {
if !variable.EnableDistTask.Load() {
// Non-dist-reorg hasn't this fail-point.
return
}
Expand Down Expand Up @@ -371,7 +371,7 @@ func TestRunDDLJobPanicDisableClusteredIndex(t *testing.T) {
}

func testAddIndexWorkerNum(t *testing.T, s *failedSuite, test func(*testkit.TestKit)) {
if variable.DDLEnableDistributeReorg.Load() {
if variable.EnableDistTask.Load() {
t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test")
}

Expand Down
2 changes: 1 addition & 1 deletion ddl/job_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -374,7 +374,7 @@ func (d *ddl) getTableByTxn(store kv.Storage, schemaID, tableID int64) (*model.D
}

func (d *ddl) loadBackfillJobAndRun() {
isDistReorg := variable.DDLEnableDistributeReorg.Load()
isDistReorg := variable.EnableDistTask.Load()
if !isDistReorg {
return
}
Expand Down
8 changes: 4 additions & 4 deletions ddl/job_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -733,10 +733,10 @@ func TestSimpleExecBackfillJobs(t *testing.T) {
}

func TestGetTasks(t *testing.T) {
// TODO: update the variable of `enableDistReorg`
isDistReorg := variable.DDLEnableDistributeReorg.Load()
variable.DDLEnableDistributeReorg.Store(false)
defer func() { variable.DDLEnableDistributeReorg.Store(isDistReorg) }()
// TODO: update the variable of `EnableDistTask`
isDistReorg := variable.EnableDistTask.Load()
variable.EnableDistTask.Store(false)
defer func() { variable.EnableDistTask.Store(isDistReorg) }()

store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand Down
2 changes: 1 addition & 1 deletion ddl/modify_column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ func TestModifyColumnReorgInfo(t *testing.T) {
// During the period, the old TiDB version(do not exist the element information) is upgraded to the new TiDB version.
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/MockGetIndexRecordErr", `return("addIdxNotOwnerErr")`))
// TODO: Remove this check after "err" isn't nil in runReorgJobAndHandleErr.
if variable.DDLEnableDistributeReorg.Load() {
if variable.EnableDistTask.Load() {
err = tk.ExecToErr("alter table t1 add index idx2(c1)")
require.EqualError(t, err, "[ddl:8201]TiDB server is not a DDL owner")
} else {
Expand Down
2 changes: 1 addition & 1 deletion executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -391,7 +391,7 @@ go_test(
"//sessionctx/binloginfo",
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//sessionctx/variable/featuretag/distributereorg",
"//sessionctx/variable/featuretag/disttask",
"//sessiontxn",
"//sessiontxn/staleread",
"//statistics",
Expand Down
18 changes: 9 additions & 9 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import (
"github.com/pingcap/tidb/parser/terror"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/sessionctx/variable/featuretag/distributereorg"
"github.com/pingcap/tidb/sessionctx/variable/featuretag/disttask"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -1322,14 +1322,14 @@ func TestLoadDDLDistributeVars(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
require.Equal(t, variable.DefTiDBDDLEnableDistributeReorg, distributereorg.TiDBEnableDistributeReorg)

tk.MustGetDBError("set @@global.tidb_ddl_distribute_reorg = invalid_val", variable.ErrWrongValueForVar)
require.Equal(t, distributereorg.TiDBEnableDistributeReorg, variable.DDLEnableDistributeReorg.Load())
tk.MustExec("set @@global.tidb_ddl_distribute_reorg = 'on'")
require.Equal(t, true, variable.DDLEnableDistributeReorg.Load())
tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_distribute_reorg = %v", distributereorg.TiDBEnableDistributeReorg))
require.Equal(t, distributereorg.TiDBEnableDistributeReorg, variable.DDLEnableDistributeReorg.Load())
require.Equal(t, variable.DefTiDBEnableDistTask, disttask.TiDBEnableDistTask)

tk.MustGetDBError("set @@global.tidb_enable_dist_task = invalid_val", variable.ErrWrongValueForVar)
require.Equal(t, disttask.TiDBEnableDistTask, variable.EnableDistTask.Load())
tk.MustExec("set @@global.tidb_enable_dist_task = 'on'")
require.Equal(t, true, variable.EnableDistTask.Load())
tk.MustExec(fmt.Sprintf("set @@global.tidb_enable_dist_task = %v", disttask.TiDBEnableDistTask))
require.Equal(t, disttask.TiDBEnableDistTask, variable.EnableDistTask.Load())
}

// Test issue #9205, fix the precision problem for time type default values
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ go_library(
"//parser/types",
"//sessionctx/sessionstates",
"//sessionctx/stmtctx",
"//sessionctx/variable/featuretag/distributereorg",
"//sessionctx/variable/featuretag/disttask",
"//tidb-binlog/pump_client",
"//types",
"//types/parser_driver",
Expand Down
Original file line number Diff line number Diff line change
@@ -1,11 +1,11 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "distributereorg",
name = "disttask",
srcs = [
"default.go",
"non_default.go", #keep
],
importpath = "github.com/pingcap/tidb/sessionctx/variable/featuretag/distributereorg",
importpath = "github.com/pingcap/tidb/sessionctx/variable/featuretag/disttask",
visibility = ["//visibility:public"],
)
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.

//go:build !distributereorg
//go:build !disttask

package distributereorg
package disttask

// TiDBEnableDistributeReorg is a feature tag
const TiDBEnableDistributeReorg bool = false
// TiDBEnableDistTask is a feature tag
const TiDBEnableDistTask bool = false
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.

//go:build distributereorg
//go:build disttask

package distributereorg
package disttask

// TiDBEnableDistributeReorg is a feature tag
const TiDBEnableDistributeReorg bool = true
// TiDBEnableDistTask is a feature tag
const TiDBEnableDistTask bool = true
8 changes: 4 additions & 4 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1159,13 +1159,13 @@ var defaultSysVars = []*SysVar{
}, GetGlobal: func(_ context.Context, vars *SessionVars) (string, error) {
return BoolToOnOff(EnableMDL.Load()), nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLEnableDistributeReorg, Value: BoolToOnOff(DefTiDBDDLEnableDistributeReorg), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
if DDLEnableDistributeReorg.Load() != TiDBOptOn(val) {
DDLEnableDistributeReorg.Store(TiDBOptOn(val))
{Scope: ScopeGlobal, Name: TiDBEnableDistTask, Value: BoolToOnOff(DefTiDBEnableDistTask), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
if EnableDistTask.Load() != TiDBOptOn(val) {
EnableDistTask.Store(TiDBOptOn(val))
}
return nil
}, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) {
return BoolToOnOff(DDLEnableDistributeReorg.Load()), nil
return BoolToOnOff(EnableDistTask.Load()), nil
}},
{Scope: ScopeGlobal, Name: TiDBEnableNoopVariables, Value: BoolToOnOff(DefTiDBEnableNoopVariables), Type: TypeEnum, PossibleValues: []string{Off, On}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
EnableNoopVariables.Store(TiDBOptOn(val))
Expand Down
8 changes: 4 additions & 4 deletions sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -713,16 +713,16 @@ func TestSetTIDBDistributeReorg(t *testing.T) {
vars.GlobalVarsAccessor = mock

// Set to on
err := mock.SetGlobalSysVar(context.Background(), TiDBDDLEnableDistributeReorg, On)
err := mock.SetGlobalSysVar(context.Background(), TiDBEnableDistTask, On)
require.NoError(t, err)
val, err := mock.GetGlobalSysVar(TiDBDDLEnableDistributeReorg)
val, err := mock.GetGlobalSysVar(TiDBEnableDistTask)
require.NoError(t, err)
require.Equal(t, On, val)

// Set to off
err = mock.SetGlobalSysVar(context.Background(), TiDBDDLEnableDistributeReorg, Off)
err = mock.SetGlobalSysVar(context.Background(), TiDBEnableDistTask, Off)
require.NoError(t, err)
val, err = mock.GetGlobalSysVar(TiDBDDLEnableDistributeReorg)
val, err = mock.GetGlobalSysVar(TiDBEnableDistTask)
require.NoError(t, err)
require.Equal(t, Off, val)
}
Expand Down
14 changes: 7 additions & 7 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/variable/featuretag/distributereorg"
"github.com/pingcap/tidb/sessionctx/variable/featuretag/disttask"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/paging"
"github.com/pingcap/tidb/util/size"
Expand Down Expand Up @@ -904,8 +904,8 @@ const (
// TiDBMaxAutoAnalyzeTime is the max time that auto analyze can run. If auto analyze runs longer than the value, it
// will be killed. 0 indicates that there is no time limit.
TiDBMaxAutoAnalyzeTime = "tidb_max_auto_analyze_time"
// TiDBDDLEnableDistributeReorg indicates whether to enable the new Reorg framework.
TiDBDDLEnableDistributeReorg = "tidb_ddl_distribute_reorg"
// TiDBEnableDistTask indicates whether to enable the distributed execute background tasks(For example DDL, Import etc).
TiDBEnableDistTask = "tidb_enable_dist_task"
// TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary.
TiDBGenerateBinaryPlan = "tidb_generate_binary_plan"
// TiDBEnableGCAwareMemoryTrack indicates whether to turn-on GC-aware memory track.
Expand Down Expand Up @@ -1174,7 +1174,7 @@ const (
DefTiDBPrepPlanCacheSize = 100
DefTiDBEnablePrepPlanCacheMemoryMonitor = true
DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1
DefTiDBDDLEnableDistributeReorg = distributereorg.TiDBEnableDistributeReorg
DefTiDBEnableDistTask = disttask.TiDBEnableDistTask
DefTiDBSimplifiedMetrics = false
DefTiDBEnablePaging = true
DefTiFlashFineGrainedShuffleStreamCount = 0
Expand Down Expand Up @@ -1289,7 +1289,7 @@ var (
MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime)
// variables for plan cache
PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio)
DDLEnableDistributeReorg = atomic.NewBool(DefTiDBDDLEnableDistributeReorg)
EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask)
DDLForce2Queue = atomic.NewBool(false)
EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables)
EnableMDL = atomic.NewBool(false)
Expand Down Expand Up @@ -1356,9 +1356,9 @@ var (
// Hooks functions for Cluster Resource Control.
var (
// EnableGlobalResourceControlFunc is the function registered by tikv_driver to set cluster resource control.
EnableGlobalResourceControlFunc func() = func() {}
EnableGlobalResourceControlFunc = func() {}
// DisableGlobalResourceControlFunc is the function registered by tikv_driver to unset cluster resource control.
DisableGlobalResourceControlFunc func() = func() {}
DisableGlobalResourceControlFunc = func() {}
)

func serverMemoryLimitDefaultValue() string {
Expand Down
8 changes: 4 additions & 4 deletions telemetry/data_feature_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -597,8 +597,8 @@ func TestDistReorgUsage(t *testing.T) {
require.NoError(t, err)
initCount := usage.DDLUsageCounter.DistReorgUsed

tk.MustExec("set @@global.tidb_ddl_distribute_reorg = off")
allow := variable.DDLEnableDistributeReorg.Load()
tk.MustExec("set @@global.tidb_enable_dist_task = off")
allow := variable.EnableDistTask.Load()
require.Equal(t, false, allow)
tk.MustExec("use test")
tk.MustExec("drop table if exists tele_t")
Expand All @@ -609,8 +609,8 @@ func TestDistReorgUsage(t *testing.T) {
require.NoError(t, err)
require.Equal(t, initCount, usage.DDLUsageCounter.DistReorgUsed)

tk.MustExec("set @@global.tidb_ddl_distribute_reorg = on")
allow = variable.DDLEnableDistributeReorg.Load()
tk.MustExec("set @@global.tidb_enable_dist_task = on")
allow = variable.EnableDistTask.Load()
require.Equal(t, true, allow)
usage, err = telemetry.GetFeatureUsage(tk.Session())
require.NoError(t, err)
Expand Down
2 changes: 1 addition & 1 deletion tests/realtikvtest/addindextest/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ func TestIngestMVIndexOnPartitionTable(t *testing.T) {
}

func TestAddIndexIngestAdjustBackfillWorker(t *testing.T) {
if variable.DDLEnableDistributeReorg.Load() {
if variable.EnableDistTask.Load() {
t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
Expand Down

0 comments on commit 922a5d3

Please sign in to comment.