From c4908e5edaa67e3b27dc3ed74b421174a7539563 Mon Sep 17 00:00:00 2001 From: Yang Keao Date: Fri, 12 Apr 2024 17:53:03 +0800 Subject: [PATCH] fix conflicts Signed-off-by: Yang Keao --- executor/infoschema_reader_test.go | 33 ++ pkg/session/test/variable/variable_test.go | 390 ------------------ pkg/sessionctx/variable/BUILD.bazel | 128 ------ sessionctx/variable/sysvar.go | 34 +- sessionctx/variable/sysvar_test.go | 108 +---- .../r/executor/infoschema_reader.result | 313 -------------- .../t/executor/infoschema_reader.test | 253 ------------ 7 files changed, 35 insertions(+), 1224 deletions(-) delete mode 100644 pkg/session/test/variable/variable_test.go delete mode 100644 pkg/sessionctx/variable/BUILD.bazel delete mode 100644 tests/integrationtest/r/executor/infoschema_reader.result delete mode 100644 tests/integrationtest/t/executor/infoschema_reader.test diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 64b423f3dbe24..58719179632a5 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -867,6 +867,39 @@ func TestNullColumns(t *testing.T) { Check(testkit.Rows("def test v_test type 1 YES binary 0 0 binary(0) select,insert,update,references ")) } +func TestDefaultValueOfVariablesInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select VARIABLE_NAME from information_schema.VARIABLES_INFO where DEFAULT_VALUE = CURRENT_VALUE and variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME;").Check(testkit.Rows( + "tidb_enable_1pc", + "tidb_enable_async_commit", + "tidb_enable_auto_analyze", + "tidb_enable_mutation_checker", + "tidb_mem_oom_action", + "tidb_pessimistic_txn_fair_locking", + "tidb_row_format_version", + "tidb_txn_assertion_level", + )) + tk.MustExec("set global tidb_enable_async_commit = default;") + tk.MustExec("set global tidb_enable_1pc = default;") + tk.MustExec("set global tidb_mem_oom_action = default;") + tk.MustExec("set global tidb_enable_auto_analyze = default;") + tk.MustExec("set global tidb_row_format_version = default;") + tk.MustExec("set global tidb_txn_assertion_level = default;") + tk.MustExec("set global tidb_enable_mutation_checker = default;") + tk.MustExec("set global tidb_pessimistic_txn_fair_locking = default;") + tk.MustQuery("select a.VARIABLE_NAME from information_schema.VARIABLES_INFO as a, mysql.GLOBAL_VARIABLES as b where a.VARIABLE_NAME = b.VARIABLE_NAME and a.DEFAULT_VALUE = b.VARIABLE_VALUE and a.CURRENT_VALUE = b.VARIABLE_VALUE and a.variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME;").Check(testkit.Rows( + "tidb_enable_1pc", + "tidb_enable_async_commit", + "tidb_enable_auto_analyze", + "tidb_enable_mutation_checker", + "tidb_mem_oom_action", + "tidb_pessimistic_txn_fair_locking", + "tidb_row_format_version", + "tidb_txn_assertion_level", + )) +} + // Code below are helper utilities for the test cases. type getTiFlashSystemTableRequestMocker struct { diff --git a/pkg/session/test/variable/variable_test.go b/pkg/session/test/variable/variable_test.go deleted file mode 100644 index 005d9eb122688..0000000000000 --- a/pkg/session/test/variable/variable_test.go +++ /dev/null @@ -1,390 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package variable - -import ( - "context" - "fmt" - "strings" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" - "github.com/pingcap/tidb/pkg/store/copr" - "github.com/pingcap/tidb/pkg/testkit" - "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" - "github.com/pingcap/tidb/pkg/util/memory" - "github.com/stretchr/testify/require" -) - -func TestForbidSettingBothTSVariable(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - // For mock tikv, safe point is not initialized, we manually insert it for snapshot to use. - safePointName := "tikv_gc_safe_point" - safePointValue := "20060102-15:04:05 -0700" - safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" - updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') - ON DUPLICATE KEY - UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) - tk.MustExec(updateSafePoint) - - // Set tidb_snapshot and assert tidb_read_staleness - tk.MustExec("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'") - tk.MustGetErrMsg("set @@tidb_read_staleness='-5'", "tidb_snapshot should be clear before setting tidb_read_staleness") - tk.MustExec("set @@tidb_snapshot = ''") - tk.MustExec("set @@tidb_read_staleness='-5'") - - // Set tidb_read_staleness and assert tidb_snapshot - tk.MustExec("set @@tidb_read_staleness='-5'") - tk.MustGetErrMsg("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'", "tidb_read_staleness should be clear before setting tidb_snapshot") - tk.MustExec("set @@tidb_read_staleness = ''") - tk.MustExec("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'") -} - -func TestCoprocessorOOMAction(t *testing.T) { - // Assert Coprocessor OOMAction - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("set @@tidb_enable_rate_limit_action=true") - tk.MustExec("create database testoom") - tk.MustExec("use testoom") - tk.MustExec(`set @@tidb_wait_split_region_finish=1`) - // create table for non keep-order case - tk.MustExec("drop table if exists t5") - tk.MustExec("create table t5(id int)") - tk.MustQuery(`split table t5 between (0) and (10000) regions 10`).Check(testkit.Rows("9 1")) - // create table for keep-order case - tk.MustExec("drop table if exists t6") - tk.MustExec("create table t6(id int, index(id))") - tk.MustQuery(`split table t6 between (0) and (10000) regions 10`).Check(testkit.Rows("10 1")) - tk.MustQuery("split table t6 INDEX id between (0) and (10000) regions 10;").Check(testkit.Rows("10 1")) - count := 10 - for i := 0; i < count; i++ { - tk.MustExec(fmt.Sprintf("insert into t5 (id) values (%v)", i)) - tk.MustExec(fmt.Sprintf("insert into t6 (id) values (%v)", i)) - } - - testcases := []struct { - name string - sql string - }{ - { - name: "keep Order", - sql: "select id from t6 order by id", - }, - { - name: "non keep Order", - sql: "select id from t5", - }, - } - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockConsumeAndAssert", `return(true)`)) - defer func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockConsumeAndAssert")) - }() - - enableOOM := func(tk *testkit.TestKit, name, sql string) { - t.Logf("enable OOM, testcase: %v", name) - // larger than 4 copResponse, smaller than 5 copResponse - quota := 5*copr.MockResponseSizeForTest - 100 - defer tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") - tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") - tk.MustExec("use testoom") - tk.MustExec("set @@tidb_enable_rate_limit_action=1") - tk.MustExec("set @@tidb_distsql_scan_concurrency = 10") - tk.MustExec(fmt.Sprintf("set @@tidb_mem_quota_query=%v;", quota)) - var expect []string - for i := 0; i < count; i++ { - expect = append(expect, fmt.Sprintf("%v", i)) - } - tk.MustQuery(sql).Sort().Check(testkit.Rows(expect...)) - // assert oom action worked by max consumed > memory quota - require.Greater(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), int64(quota)) - } - - disableOOM := func(tk *testkit.TestKit, name, sql string) { - t.Logf("disable OOM, testcase: %v", name) - quota := 5*copr.MockResponseSizeForTest - 100 - tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") - defer tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") - tk.MustExec("use testoom") - tk.MustExec("set @@tidb_enable_rate_limit_action=0") - tk.MustExec("set @@tidb_distsql_scan_concurrency = 10") - tk.MustExec(fmt.Sprintf("set @@tidb_mem_quota_query=%v;", quota)) - err := tk.QueryToErr(sql) - require.Error(t, err) - require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(err)) - } - - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockWaitMax", `return(true)`)) - // assert oom action and switch - for _, testcase := range testcases { - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - tk.SetSession(se) - enableOOM(tk, testcase.name, testcase.sql) - tk.MustExec("set @@tidb_enable_rate_limit_action = 0") - disableOOM(tk, testcase.name, testcase.sql) - tk.MustExec("set @@tidb_enable_rate_limit_action = 1") - enableOOM(tk, testcase.name, testcase.sql) - se.Close() - } - globaltk := testkit.NewTestKit(t, store) - globaltk.MustExec("use testoom") - globaltk.MustExec("set global tidb_enable_rate_limit_action= 0") - for _, testcase := range testcases { - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - tk.SetSession(se) - disableOOM(tk, testcase.name, testcase.sql) - se.Close() - } - globaltk.MustExec("set global tidb_enable_rate_limit_action= 1") - for _, testcase := range testcases { - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - tk.SetSession(se) - enableOOM(tk, testcase.name, testcase.sql) - se.Close() - } - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockWaitMax")) - - // assert oom fallback - for _, testcase := range testcases { - t.Log(testcase.name) - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - tk.SetSession(se) - tk.MustExec("use testoom") - tk.MustExec("set tidb_distsql_scan_concurrency = 1") - tk.MustExec("set @@tidb_mem_quota_query=1;") - tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") - err = tk.QueryToErr(testcase.sql) - require.Error(t, err) - require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(err)) - tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") - se.Close() - } -} - -func TestStatementCountLimit(t *testing.T) { - store := testkit.CreateMockStore(t) - setTxnTk := testkit.NewTestKit(t, store) - setTxnTk.MustExec("set global tidb_txn_mode=''") - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table stmt_count_limit (id int)") - defer config.RestoreFunc()() - config.UpdateGlobal(func(conf *config.Config) { - conf.Performance.StmtCountLimit = 3 - }) - tk.MustExec("set tidb_disable_txn_auto_retry = 0") - tk.MustExec("begin") - tk.MustExec("insert into stmt_count_limit values (1)") - tk.MustExec("insert into stmt_count_limit values (2)") - _, err := tk.Exec("insert into stmt_count_limit values (3)") - require.Error(t, err) - - // begin is counted into history but this one is not. - tk.MustExec("SET SESSION autocommit = false") - tk.MustExec("insert into stmt_count_limit values (1)") - tk.MustExec("insert into stmt_count_limit values (2)") - tk.MustExec("insert into stmt_count_limit values (3)") - _, err = tk.Exec("insert into stmt_count_limit values (4)") - require.Error(t, err) -} - -func TestCorrectScopeError(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeNone, Name: "sv_none", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal, Name: "sv_global", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeSession, Name: "sv_session", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "sv_both", Value: "acdc"}) - - // check set behavior - - // none - _, err := tk.Exec("SET sv_none='acdc'") - require.Equal(t, "[variable:1238]Variable 'sv_none' is a read only variable", err.Error()) - _, err = tk.Exec("SET GLOBAL sv_none='acdc'") - require.Equal(t, "[variable:1238]Variable 'sv_none' is a read only variable", err.Error()) - - // global - tk.MustExec("SET GLOBAL sv_global='acdc'") - _, err = tk.Exec("SET sv_global='acdc'") - require.Equal(t, "[variable:1229]Variable 'sv_global' is a GLOBAL variable and should be set with SET GLOBAL", err.Error()) - - // session - _, err = tk.Exec("SET GLOBAL sv_session='acdc'") - require.Equal(t, "[variable:1228]Variable 'sv_session' is a SESSION variable and can't be used with SET GLOBAL", err.Error()) - tk.MustExec("SET sv_session='acdc'") - - // both - tk.MustExec("SET GLOBAL sv_both='acdc'") - tk.MustExec("SET sv_both='acdc'") - - // unregister - variable.UnregisterSysVar("sv_none") - variable.UnregisterSysVar("sv_global") - variable.UnregisterSysVar("sv_session") - variable.UnregisterSysVar("sv_both") -} - -func TestReadDMLBatchSize(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("set global tidb_dml_batch_size=1000") - se, err := session.CreateSession(store) - require.NoError(t, err) - - // `select 1` to load the global variables. - _, _ = se.Execute(context.TODO(), "select 1") - require.Equal(t, 1000, se.GetSessionVars().DMLBatchSize) -} - -func TestSetEnableRateLimitAction(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("set @@tidb_enable_rate_limit_action=true") - // assert default value - result := tk.MustQuery("select @@tidb_enable_rate_limit_action;") - result.Check(testkit.Rows("1")) - tk.MustExec("use test") - tk.MustExec("create table tmp123(id int)") - rs, err := tk.Exec("select * from tmp123;") - require.NoError(t, err) - haveRateLimitAction := false - action := tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false) - for ; action != nil; action = action.GetFallback() { - if action.GetPriority() == memory.DefRateLimitPriority { - haveRateLimitAction = true - break - } - } - require.True(t, haveRateLimitAction) - err = rs.Close() - require.NoError(t, err) - - // assert set sys variable - tk.MustExec("set global tidb_enable_rate_limit_action= '0';") - tk.Session().Close() - - tk.RefreshSession() - result = tk.MustQuery("select @@tidb_enable_rate_limit_action;") - result.Check(testkit.Rows("0")) - - haveRateLimitAction = false - action = tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false) - for ; action != nil; action = action.GetFallback() { - if action.GetPriority() == memory.DefRateLimitPriority { - haveRateLimitAction = true - break - } - } - require.False(t, haveRateLimitAction) -} - -func TestMaxExecutionTime(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("use test") - tk.MustExec("create table MaxExecTime( id int,name varchar(128),age int);") - tk.MustExec("begin") - tk.MustExec("insert into MaxExecTime (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18);") - - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) MAX_EXECUTION_TIME(500) */ * FROM MaxExecTime;") - require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) - require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(500)") - require.True(t, tk.Session().GetSessionVars().StmtCtx.HasMaxExecutionTime) - require.Equal(t, uint64(500), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) - - tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) - tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) */ * FROM MaxExecTime;") - - tk.MustExec("set @@global.MAX_EXECUTION_TIME = 300;") - tk.MustQuery("select * FROM MaxExecTime;") - - tk.MustExec("set @@MAX_EXECUTION_TIME = 150;") - tk.MustQuery("select * FROM MaxExecTime;") - - tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("300")) - tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("150")) - - tk.MustExec("set @@global.MAX_EXECUTION_TIME = 0;") - tk.MustExec("set @@MAX_EXECUTION_TIME = 0;") - tk.MustExec("commit") - tk.MustExec("drop table if exists MaxExecTime;") -} - -func TestReplicaRead(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) - tk.MustExec("set @@tidb_replica_read = 'follower';") - require.Equal(t, kv.ReplicaReadFollower, tk.Session().GetSessionVars().GetReplicaRead()) - tk.MustExec("set @@tidb_replica_read = 'leader';") - require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) -} - -func TestIsolationRead(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - require.Len(t, tk.Session().GetSessionVars().GetIsolationReadEngines(), 3) - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") - engines := tk.Session().GetSessionVars().GetIsolationReadEngines() - require.Len(t, engines, 1) - _, hasTiFlash := engines[kv.TiFlash] - _, hasTiKV := engines[kv.TiKV] - require.True(t, hasTiFlash) - require.False(t, hasTiKV) -} - -func TestLastQueryInfo(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mockRUConsumption", `return()`)) - defer func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockRUConsumption")) - }() - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, index idx(a))") - tk.MustExec(`prepare stmt1 from 'select * from t'`) - tk.MustExec("execute stmt1") - checkMatch := func(actual []string, expected []interface{}) bool { - return strings.Contains(actual[0], expected[0].(string)) - } - tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"ru_consumption":15`), checkMatch) - tk.MustExec("select a from t where a = 1") - tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"ru_consumption":27`), checkMatch) - tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"ru_consumption":30`), checkMatch) -} diff --git a/pkg/sessionctx/variable/BUILD.bazel b/pkg/sessionctx/variable/BUILD.bazel deleted file mode 100644 index d7ad78566993b..0000000000000 --- a/pkg/sessionctx/variable/BUILD.bazel +++ /dev/null @@ -1,128 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "variable", - srcs = [ - "error.go", - "mock_globalaccessor.go", - "noop.go", - "removed.go", - "sequence_state.go", - "session.go", - "setvar_affect.go", - "statusvar.go", - "sysvar.go", - "tidb_vars.go", - "variable.go", - "varsutil.go", - ], - importpath = "github.com/pingcap/tidb/pkg/sessionctx/variable", - visibility = ["//visibility:public"], - deps = [ - "//pkg/config", - "//pkg/domain/resourcegroup", - "//pkg/errctx", - "//pkg/errno", - "//pkg/keyspace", - "//pkg/kv", - "//pkg/meta/autoid", - "//pkg/metrics", - "//pkg/parser", - "//pkg/parser/ast", - "//pkg/parser/auth", - "//pkg/parser/charset", - "//pkg/parser/model", - "//pkg/parser/mysql", - "//pkg/parser/types", - "//pkg/planner/util/fixcontrol", - "//pkg/privilege/privileges/ldap", - "//pkg/sessionctx/sessionstates", - "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable/featuretag/disttask", - "//pkg/tidb-binlog/pump_client", - "//pkg/types", - "//pkg/types/parser_driver", - "//pkg/util", - "//pkg/util/chunk", - "//pkg/util/collate", - "//pkg/util/dbterror", - "//pkg/util/disk", - "//pkg/util/distrole", - "//pkg/util/execdetails", - "//pkg/util/gctuner", - "//pkg/util/intest", - "//pkg/util/kvcache", - "//pkg/util/logutil", - "//pkg/util/mathutil", - "//pkg/util/memory", - "//pkg/util/paging", - "//pkg/util/replayer", - "//pkg/util/rowcodec", - "//pkg/util/size", - "//pkg/util/sqlkiller", - "//pkg/util/stmtsummary/v2:stmtsummary", - "//pkg/util/stringutil", - "//pkg/util/tableutil", - "//pkg/util/tiflash", - "//pkg/util/tiflashcompute", - "//pkg/util/tikvutil", - "//pkg/util/timeutil", - "//pkg/util/tls", - "//pkg/util/topsql/state", - "//pkg/util/versioninfo", - "@com_github_pingcap_errors//:errors", - "@com_github_tikv_client_go_v2//config", - "@com_github_tikv_client_go_v2//kv", - "@com_github_tikv_client_go_v2//oracle", - "@com_github_tikv_client_go_v2//tikv", - "@com_github_tikv_client_go_v2//util", - "@com_github_twmb_murmur3//:murmur3", - "@org_golang_x_exp//maps", - "@org_uber_go_atomic//:atomic", - "@org_uber_go_zap//:zap", - ], -) - -go_test( - name = "variable_test", - timeout = "short", - srcs = [ - "main_test.go", - "mock_globalaccessor_test.go", - "removed_test.go", - "session_test.go", - "statusvar_test.go", - "sysvar_test.go", - "variable_test.go", - "varsutil_test.go", - ], - embed = [":variable"], - flaky = True, - shard_count = 50, - deps = [ - "//pkg/config", - "//pkg/kv", - "//pkg/parser", - "//pkg/parser/auth", - "//pkg/parser/mysql", - "//pkg/parser/terror", - "//pkg/planner/core", - "//pkg/sessionctx/sessionstates", - "//pkg/sessionctx/stmtctx", - "//pkg/testkit", - "//pkg/testkit/testsetup", - "//pkg/types", - "//pkg/util", - "//pkg/util/chunk", - "//pkg/util/execdetails", - "//pkg/util/gctuner", - "//pkg/util/memory", - "//pkg/util/mock", - "//pkg/util/timeutil", - "@com_github_pingcap_failpoint//:failpoint", - "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//util", - "@io_opencensus_go//stats/view", - "@org_uber_go_goleak//:goleak", - ], -) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index f6617e29f2936..44b0191218218 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -26,7 +26,6 @@ import ( "time" "github.com/pingcap/errors" -<<<<<<< HEAD:sessionctx/variable/sysvar.go "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/kv" @@ -40,6 +39,7 @@ import ( _ "github.com/pingcap/tidb/types/parser_driver" // for parser driver "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gctuner" + "github.com/pingcap/tidb/util/intest" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" @@ -49,38 +49,6 @@ import ( "github.com/pingcap/tidb/util/tls" topsqlstate "github.com/pingcap/tidb/util/topsql/state" "github.com/pingcap/tidb/util/versioninfo" -======= - "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/keyspace" - "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/parser" - "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" - "github.com/pingcap/tidb/pkg/privilege/privileges/ldap" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/types" - _ "github.com/pingcap/tidb/pkg/types/parser_driver" // for parser driver - "github.com/pingcap/tidb/pkg/util" - "github.com/pingcap/tidb/pkg/util/collate" - distroleutil "github.com/pingcap/tidb/pkg/util/distrole" - "github.com/pingcap/tidb/pkg/util/gctuner" - "github.com/pingcap/tidb/pkg/util/intest" - "github.com/pingcap/tidb/pkg/util/logutil" - "github.com/pingcap/tidb/pkg/util/mathutil" - "github.com/pingcap/tidb/pkg/util/memory" - stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" - "github.com/pingcap/tidb/pkg/util/tiflash" - "github.com/pingcap/tidb/pkg/util/tiflashcompute" - "github.com/pingcap/tidb/pkg/util/tikvutil" - "github.com/pingcap/tidb/pkg/util/timeutil" - "github.com/pingcap/tidb/pkg/util/tls" - topsqlstate "github.com/pingcap/tidb/pkg/util/topsql/state" - "github.com/pingcap/tidb/pkg/util/versioninfo" ->>>>>>> 205b5bbd210 (variable: fix information_schema.VARIABLES_INFO DEFAULT_VALUE not right problem (#49524)):pkg/sessionctx/variable/sysvar.go tikvcfg "github.com/tikv/client-go/v2/config" tikvstore "github.com/tikv/client-go/v2/kv" atomic2 "go.uber.org/atomic" diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 57cfeec8a798e..28a4c21bb04a5 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -1238,111 +1238,6 @@ func TestTiDBEnableRowLevelChecksum(t *testing.T) { require.NoError(t, err) require.Equal(t, Off, val) } -<<<<<<< HEAD:sessionctx/variable/sysvar_test.go -======= - -func TestTiDBTiFlashReplicaRead(t *testing.T) { - vars := NewSessionVars(nil) - mock := NewMockGlobalAccessor4Tests() - mock.SessionVars = vars - vars.GlobalVarsAccessor = mock - tidbTiFlashReplicaRead := GetSysVar(TiFlashReplicaRead) - // Check default value - require.Equal(t, DefTiFlashReplicaRead, tidbTiFlashReplicaRead.Value) - - err := mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "all_replicas") - require.NoError(t, err) - val, err := mock.GetGlobalSysVar(TiFlashReplicaRead) - require.NoError(t, err) - require.Equal(t, "all_replicas", val) - - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "closest_adaptive") - require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) - require.NoError(t, err) - require.Equal(t, "closest_adaptive", val) - - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "closest_replicas") - require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) - require.NoError(t, err) - require.Equal(t, "closest_replicas", val) - - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, DefTiFlashReplicaRead) - require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "random") - require.Error(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) - require.NoError(t, err) - require.Equal(t, DefTiFlashReplicaRead, val) -} - -func TestSetTiDBCloudStorageURI(t *testing.T) { - vars := NewSessionVars(nil) - mock := NewMockGlobalAccessor4Tests() - mock.SessionVars = vars - vars.GlobalVarsAccessor = mock - cloudStorageURI := GetSysVar(TiDBCloudStorageURI) - require.Len(t, CloudStorageURI.Load(), 0) - defer func() { - CloudStorageURI.Store("") - }() - - // Default empty - require.Len(t, cloudStorageURI.Value, 0) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - // Set to noop - noopURI := "noop://blackhole?access-key=hello&secret-access-key=world" - err := mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, noopURI) - require.NoError(t, err) - val, err1 := mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) - require.NoError(t, err1) - require.Equal(t, noopURI, val) - require.Equal(t, noopURI, CloudStorageURI.Load()) - - // Set to s3, should fail - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, "s3://blackhole") - require.ErrorContains(t, err, "bucket blackhole") - - s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - w.WriteHeader(200) - })) - defer s.Close() - - // Set to s3, should return uri without variable - s3URI := "s3://tiflow-test/?access-key=testid&secret-access-key=testkey8&session-token=testtoken&endpoint=" + s.URL - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, s3URI) - require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) - require.NoError(t, err1) - require.True(t, strings.HasPrefix(val, "s3://tiflow-test/")) - require.Contains(t, val, "access-key=xxxxxx") - require.Contains(t, val, "secret-access-key=xxxxxx") - require.Contains(t, val, "session-token=xxxxxx") - require.Equal(t, s3URI, CloudStorageURI.Load()) - - // ks3 is like s3 - ks3URI := "ks3://tiflow-test/?region=test&access-key=testid&secret-access-key=testkey8&session-token=testtoken&endpoint=" + s.URL - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, ks3URI) - require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) - require.NoError(t, err1) - require.True(t, strings.HasPrefix(val, "ks3://tiflow-test/")) - require.Contains(t, val, "access-key=xxxxxx") - require.Contains(t, val, "secret-access-key=xxxxxx") - require.Contains(t, val, "session-token=xxxxxx") - require.Equal(t, ks3URI, CloudStorageURI.Load()) - - // Set to empty, should return no error - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, "") - require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) - require.NoError(t, err1) - require.Len(t, val, 0) - cancel() -} func TestGlobalSystemVariableInitialValue(t *testing.T) { vars := []struct { @@ -1353,7 +1248,7 @@ func TestGlobalSystemVariableInitialValue(t *testing.T) { { TiDBTxnMode, DefTiDBTxnMode, - "pessimistic", + "", }, { TiDBEnableAsyncCommit, @@ -1401,4 +1296,3 @@ func TestGlobalSystemVariableInitialValue(t *testing.T) { require.Equal(t, v.initVal, initVal) } } ->>>>>>> 205b5bbd210 (variable: fix information_schema.VARIABLES_INFO DEFAULT_VALUE not right problem (#49524)):pkg/sessionctx/variable/sysvar_test.go diff --git a/tests/integrationtest/r/executor/infoschema_reader.result b/tests/integrationtest/r/executor/infoschema_reader.result deleted file mode 100644 index 4745b06b79a02..0000000000000 --- a/tests/integrationtest/r/executor/infoschema_reader.result +++ /dev/null @@ -1,313 +0,0 @@ -select * from information_schema.profiling; -QUERY_ID SEQ STATE DURATION CPU_USER CPU_SYSTEM CONTEXT_VOLUNTARY CONTEXT_INVOLUNTARY BLOCK_OPS_IN BLOCK_OPS_OUT MESSAGES_SENT MESSAGES_RECEIVED PAGE_FAULTS_MAJOR PAGE_FAULTS_MINOR SWAPS SOURCE_FUNCTION SOURCE_FILE SOURCE_LINE -set @@profiling=1; -select * from information_schema.profiling; -QUERY_ID SEQ STATE DURATION CPU_USER CPU_SYSTEM CONTEXT_VOLUNTARY CONTEXT_INVOLUNTARY BLOCK_OPS_IN BLOCK_OPS_OUT MESSAGES_SENT MESSAGES_RECEIVED PAGE_FAULTS_MAJOR PAGE_FAULTS_MINOR SWAPS SOURCE_FUNCTION SOURCE_FILE SOURCE_LINE -0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 -select * from information_schema.SCHEMATA where schema_name='mysql'; -CATALOG_NAME SCHEMA_NAME DEFAULT_CHARACTER_SET_NAME DEFAULT_COLLATION_NAME SQL_PATH TIDB_PLACEMENT_POLICY_NAME -def mysql utf8mb4 utf8mb4_bin NULL NULL -drop user if exists schemata_tester; -create user schemata_tester; -select count(*) from information_schema.SCHEMATA; -count(*) -1 -select * from information_schema.SCHEMATA where schema_name='mysql'; -CATALOG_NAME SCHEMA_NAME DEFAULT_CHARACTER_SET_NAME DEFAULT_COLLATION_NAME SQL_PATH TIDB_PLACEMENT_POLICY_NAME -select * from information_schema.SCHEMATA where schema_name='INFORMATION_SCHEMA'; -CATALOG_NAME SCHEMA_NAME DEFAULT_CHARACTER_SET_NAME DEFAULT_COLLATION_NAME SQL_PATH TIDB_PLACEMENT_POLICY_NAME -def INFORMATION_SCHEMA utf8mb4 utf8mb4_bin NULL NULL -CREATE ROLE r_mysql_priv; -GRANT ALL PRIVILEGES ON mysql.* TO r_mysql_priv; -GRANT r_mysql_priv TO schemata_tester; -set role r_mysql_priv; -select count(*) from information_schema.SCHEMATA; -count(*) -2 -select * from information_schema.SCHEMATA; -CATALOG_NAME SCHEMA_NAME DEFAULT_CHARACTER_SET_NAME DEFAULT_COLLATION_NAME SQL_PATH TIDB_PLACEMENT_POLICY_NAME -def INFORMATION_SCHEMA utf8mb4 utf8mb4_bin NULL NULL -def mysql utf8mb4 utf8mb4_bin NULL NULL -drop table if exists executor__infoschema_reader.t; -create table executor__infoschema_reader.t (a int, b int, primary key(a), key k1(b)); -select index_id from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 't'; -index_id -0 -1 -select tidb_table_id > 0 from information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't'; -tidb_table_id > 0 -1 -drop database if exists `foo`; -CREATE DATABASE `foo` DEFAULT CHARACTER SET = 'utf8mb4'; -select default_character_set_name, default_collation_name FROM information_schema.SCHEMATA WHERE schema_name = 'foo'; -default_character_set_name default_collation_name -utf8mb4 utf8mb4_bin -drop database `foo`; -drop view if exists executor__infoschema_reader.v1; -CREATE DEFINER='root'@'localhost' VIEW executor__infoschema_reader.v1 AS SELECT 1; -select TABLE_COLLATION is null from INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE='VIEW'; -TABLE_COLLATION is null -1 -1 -SELECT * FROM information_schema.views WHERE table_schema='executor__infoschema_reader' AND table_name='v1'; -TABLE_CATALOG TABLE_SCHEMA TABLE_NAME VIEW_DEFINITION CHECK_OPTION IS_UPDATABLE DEFINER SECURITY_TYPE CHARACTER_SET_CLIENT COLLATION_CONNECTION -def executor__infoschema_reader v1 SELECT 1 AS `1` CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_general_ci -SELECT table_catalog, table_schema, table_name, table_type, engine, version, row_format, table_rows, avg_row_length, data_length, max_data_length, index_length, data_free, auto_increment, update_time, check_time, table_collation, checksum, create_options, table_comment FROM information_schema.tables WHERE table_schema='executor__infoschema_reader' AND table_name='v1'; -table_catalog table_schema table_name table_type engine version row_format table_rows avg_row_length data_length max_data_length index_length data_free auto_increment update_time check_time table_collation checksum create_options table_comment -def executor__infoschema_reader v1 VIEW NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL VIEW -drop table if exists t; -create table t (bit bit(10) DEFAULT b'100'); -SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'executor__infoschema_reader' AND TABLE_NAME = 't'; -TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION COLUMN_DEFAULT IS_NULLABLE DATA_TYPE CHARACTER_MAXIMUM_LENGTH CHARACTER_OCTET_LENGTH NUMERIC_PRECISION NUMERIC_SCALE DATETIME_PRECISION CHARACTER_SET_NAME COLLATION_NAME COLUMN_TYPE COLUMN_KEY EXTRA PRIVILEGES COLUMN_COMMENT GENERATION_EXPRESSION -def executor__infoschema_reader t bit 1 b'100' YES bit NULL NULL 10 0 NULL NULL NULL bit(10) select,insert,update,references -drop table if exists t; -set time_zone='+08:00'; -drop table if exists t; -create table t (b timestamp(3) NOT NULL DEFAULT '1970-01-01 08:00:01.000'); -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -column_default -1970-01-01 08:00:01.000 -set time_zone='+04:00'; -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -column_default -1970-01-01 04:00:01.000 -set time_zone=default; -drop table if exists t; -create table t (a bit DEFAULT (rand())); -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -column_default -rand() -drop table if exists t; -CREATE TABLE t (`COL3` bit(1) NOT NULL,b year) ; -select column_type from information_schema.columns where TABLE_SCHEMA = 'executor__infoschema_reader' and TABLE_NAME = 't'; -column_type -bit(1) -year(4) -select ordinal_position from information_schema.columns where table_schema=database() and table_name='t' and column_name='b'; -ordinal_position -2 -select * from information_schema.ENGINES; -ENGINE SUPPORT COMMENT TRANSACTIONS XA SAVEPOINTS -InnoDB DEFAULT Supports transactions, row-level locking, and foreign keys YES YES YES -drop table if exists t; -create table t (a varchar(255) collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -255 255 -drop table t; -create table t (a varchar(255) collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -255 1020 -drop table t; -create table t (a varchar(255) collate utf8_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -255 765 -drop table t; -create table t (a char(10) collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -10 10 -drop table t; -create table t (a char(10) collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -10 40 -drop table t; -create table t (a set('a', 'b', 'cccc') collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -8 8 -drop table t; -create table t (a set('a', 'b', 'cccc') collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -8 32 -drop table t; -create table t (a enum('a', 'b', 'cccc') collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -4 4 -drop table t; -create table t (a enum('a', 'b', 'cccc') collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -character_maximum_length character_octet_length -4 16 -drop table t; -set global tidb_ddl_enable_fast_reorg = false; -drop database if exists test_ddl_jobs; -create database test_ddl_jobs; -select db_name, job_type from information_schema.DDL_JOBS limit 1; -db_name job_type -test_ddl_jobs create schema -use test_ddl_jobs; -create table t (a int); -select db_name, table_name, job_type from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and table_name = 't'; -db_name table_name job_type -test_ddl_jobs t create table -select job_type from information_schema.DDL_JOBS group by job_type having job_type = 'create table'; -job_type -create table -select distinct job_type from information_schema.DDL_JOBS where job_type = 'create table' and start_time > str_to_date('20190101','%Y%m%d%H%i%s'); -job_type -create table -drop user if exists DDL_JOBS_tester; -create user DDL_JOBS_tester; -select DB_NAME, TABLE_NAME from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and TABLE_NAME = 't'; -DB_NAME TABLE_NAME -CREATE ROLE r_priv; -GRANT ALL PRIVILEGES ON test_ddl_jobs.* TO r_priv; -GRANT r_priv TO DDL_JOBS_tester; -set role r_priv; -select DB_NAME, TABLE_NAME from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and TABLE_NAME = 't'; -DB_NAME TABLE_NAME -test_ddl_jobs t -create table tt (a int); -alter table tt add index t(a), add column b int; -select db_name, table_name, job_type from information_schema.DDL_JOBS limit 3; -db_name table_name job_type -test_ddl_jobs tt alter table multi-schema change -test_ddl_jobs tt add column /* subjob */ -test_ddl_jobs tt add index /* subjob */ /* txn */ -drop database test_ddl_jobs; -use executor__infoschema_reader; -set global tidb_ddl_enable_fast_reorg = default; -select * from information_schema.KEY_COLUMN_USAGE where TABLE_NAME='stats_meta' and COLUMN_NAME='table_id'; -CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME -def mysql tbl def mysql stats_meta table_id 1 NULL NULL NULL NULL -create user key_column_tester; -select * from information_schema.KEY_COLUMN_USAGE where TABLE_NAME != 'CLUSTER_SLOW_QUERY'; -CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION POSITION_IN_UNIQUE_CONSTRAINT REFERENCED_TABLE_SCHEMA REFERENCED_TABLE_NAME REFERENCED_COLUMN_NAME -CREATE ROLE r_stats_meta ; -GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta; -GRANT r_stats_meta TO key_column_tester; -set role r_stats_meta; -select count(*)>0 from information_schema.KEY_COLUMN_USAGE where TABLE_NAME='stats_meta'; -count(*)>0 -1 -drop table if exists e, e2; -CREATE TABLE e ( id INT NOT NULL, fname VARCHAR(30), lname VARCHAR(30)) PARTITION BY RANGE (id) ( -PARTITION p0 VALUES LESS THAN (50), -PARTITION p1 VALUES LESS THAN (100), -PARTITION p2 VALUES LESS THAN (150), -PARTITION p3 VALUES LESS THAN (MAXVALUE)); -CREATE TABLE e2 ( id INT NOT NULL, fname VARCHAR(30), lname VARCHAR(30)); -SELECT PARTITION_NAME, TABLE_ROWS FROM INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_NAME = 'e' and table_schema=(select database()); -PARTITION_NAME TABLE_ROWS -p0 0 -p1 0 -p2 0 -p3 0 -INSERT INTO e VALUES (1669, "Jim", "Smith"), (337, "Mary", "Jones"), (16, "Frank", "White"), (2005, "Linda", "Black"); -set tidb_enable_exchange_partition='on'; -ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2; -INSERT INTO e VALUES (41, "Michael", "Green"); -analyze table e; -SELECT PARTITION_NAME, TABLE_ROWS FROM INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_NAME = 'e'; -PARTITION_NAME TABLE_ROWS -p0 1 -p1 0 -p2 0 -p3 3 -set tidb_enable_exchange_partition=default; -select count(*) > 0 from information_schema.`METRICS_TABLES`; -count(*) > 0 -1 -select * from information_schema.`METRICS_TABLES` where table_name='tidb_qps'; -TABLE_NAME PROMQL LABELS QUANTILE COMMENT -tidb_qps sum(rate(tidb_server_query_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (result,type,instance) instance,type,result 0 TiDB query processing numbers per second -select * from information_schema.TABLE_CONSTRAINTS where TABLE_NAME='gc_delete_range'; -CONSTRAINT_CATALOG CONSTRAINT_SCHEMA CONSTRAINT_NAME TABLE_SCHEMA TABLE_NAME CONSTRAINT_TYPE -def mysql delete_range_index mysql gc_delete_range UNIQUE -select * from information_schema.SESSION_VARIABLES where VARIABLE_NAME='tidb_retry_limit'; -VARIABLE_NAME VARIABLE_VALUE -tidb_retry_limit 10 -drop sequence if exists seq, seq2; -CREATE SEQUENCE seq maxvalue 10000000; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq'; -TABLE_CATALOG SEQUENCE_SCHEMA SEQUENCE_NAME CACHE CACHE_VALUE CYCLE INCREMENT MAX_VALUE MIN_VALUE START COMMENT -def executor__infoschema_reader seq 1 1000 0 1 10000000 1 1 -DROP SEQUENCE seq; -CREATE SEQUENCE seq start = -1 minvalue -1 maxvalue 10 increment 1 cache 10; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq'; -TABLE_CATALOG SEQUENCE_SCHEMA SEQUENCE_NAME CACHE CACHE_VALUE CYCLE INCREMENT MAX_VALUE MIN_VALUE START COMMENT -def executor__infoschema_reader seq 1 10 0 1 10 -1 -1 -CREATE SEQUENCE seq2 start = -9 minvalue -10 maxvalue 10 increment -1 cache 15; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq2'; -TABLE_CATALOG SEQUENCE_SCHEMA SEQUENCE_NAME CACHE CACHE_VALUE CYCLE INCREMENT MAX_VALUE MIN_VALUE START COMMENT -def executor__infoschema_reader seq2 1 15 0 -1 10 -10 -9 -SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME , TABLE_TYPE, ENGINE, TABLE_ROWS FROM information_schema.tables WHERE TABLE_TYPE='SEQUENCE' AND TABLE_NAME='seq2' and table_schema='executor__infoschema_reader'; -TABLE_CATALOG TABLE_SCHEMA TABLE_NAME TABLE_TYPE ENGINE TABLE_ROWS -def executor__infoschema_reader seq2 SEQUENCE InnoDB 1 -drop table if exists t_int, t_implicit, t_common; -create table t_int (a int primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_int'; -TIDB_PK_TYPE -CLUSTERED -set tidb_enable_clustered_index=int_only; -create table t_implicit (a varchar(64) primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_implicit'; -TIDB_PK_TYPE -NONCLUSTERED -set tidb_enable_clustered_index=on; -create table t_common (a varchar(64) primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_common'; -TIDB_PK_TYPE -CLUSTERED -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'INFORMATION_SCHEMA' and table_name = 'TABLES'; -TIDB_PK_TYPE -NONCLUSTERED -set tidb_enable_clustered_index=default; -drop table if exists t; -CREATE TABLE t ( id int DEFAULT NULL); -CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`1.1.1.1` SQL SECURITY DEFINER VIEW `v_test` (`type`) AS SELECT NULL AS `type` FROM `t` AS `f`; -select * from information_schema.columns where TABLE_SCHEMA = 'executor__infoschema_reader' and TABLE_NAME = 'v_test'; -TABLE_CATALOG TABLE_SCHEMA TABLE_NAME COLUMN_NAME ORDINAL_POSITION COLUMN_DEFAULT IS_NULLABLE DATA_TYPE CHARACTER_MAXIMUM_LENGTH CHARACTER_OCTET_LENGTH NUMERIC_PRECISION NUMERIC_SCALE DATETIME_PRECISION CHARACTER_SET_NAME COLLATION_NAME COLUMN_TYPE COLUMN_KEY EXTRA PRIVILEGES COLUMN_COMMENT GENERATION_EXPRESSION -def executor__infoschema_reader v_test type 1 NULL YES binary 0 0 NULL NULL NULL NULL NULL binary(0) select,insert,update,references -drop user if exists usageuser; -create user usageuser; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; -GRANTEE TABLE_CATALOG PRIVILEGE_TYPE IS_GRANTABLE -'usageuser'@'%' def USAGE NO -GRANT SELECT ON *.* to usageuser; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; -GRANTEE TABLE_CATALOG PRIVILEGE_TYPE IS_GRANTABLE -'usageuser'@'%' def SELECT NO -GRANT SELECT ON *.* to usageuser WITH GRANT OPTION; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; -GRANTEE TABLE_CATALOG PRIVILEGE_TYPE IS_GRANTABLE -'usageuser'@'%' def SELECT YES -GRANT BACKUP_ADMIN ON *.* to usageuser; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'" ORDER BY privilege_type; -GRANTEE TABLE_CATALOG PRIVILEGE_TYPE IS_GRANTABLE -'usageuser'@'%' def BACKUP_ADMIN NO -'usageuser'@'%' def SELECT YES -select VARIABLE_NAME from information_schema.VARIABLES_INFO where DEFAULT_VALUE = CURRENT_VALUE and variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME; -VARIABLE_NAME -tidb_enable_1pc -tidb_enable_async_commit -tidb_enable_auto_analyze -tidb_enable_mutation_checker -tidb_mem_oom_action -tidb_pessimistic_txn_fair_locking -tidb_row_format_version -tidb_txn_assertion_level -set global tidb_enable_async_commit = default; -set global tidb_enable_1pc = default; -set global tidb_mem_oom_action = default; -set global tidb_enable_auto_analyze = default; -set global tidb_row_format_version = default; -set global tidb_txn_assertion_level = default; -set global tidb_enable_mutation_checker = default; -set global tidb_pessimistic_txn_fair_locking = default; -select a.VARIABLE_NAME from information_schema.VARIABLES_INFO as a, mysql.GLOBAL_VARIABLES as b where a.VARIABLE_NAME = b.VARIABLE_NAME and a.DEFAULT_VALUE = b.VARIABLE_VALUE and a.CURRENT_VALUE = b.VARIABLE_VALUE and a.variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME; -VARIABLE_NAME -tidb_enable_1pc -tidb_enable_async_commit -tidb_enable_auto_analyze -tidb_enable_mutation_checker -tidb_mem_oom_action -tidb_pessimistic_txn_fair_locking -tidb_row_format_version -tidb_txn_assertion_level diff --git a/tests/integrationtest/t/executor/infoschema_reader.test b/tests/integrationtest/t/executor/infoschema_reader.test deleted file mode 100644 index f996f0f990c67..0000000000000 --- a/tests/integrationtest/t/executor/infoschema_reader.test +++ /dev/null @@ -1,253 +0,0 @@ -# TestProfiling -select * from information_schema.profiling; -set @@profiling=1; -select * from information_schema.profiling; - -# TestSchemataTables -select * from information_schema.SCHEMATA where schema_name='mysql'; -drop user if exists schemata_tester; -create user schemata_tester; - -connect (conn1, localhost, schemata_tester,, information_schema); -select count(*) from information_schema.SCHEMATA; -select * from information_schema.SCHEMATA where schema_name='mysql'; -select * from information_schema.SCHEMATA where schema_name='INFORMATION_SCHEMA'; - -connection default; -CREATE ROLE r_mysql_priv; -GRANT ALL PRIVILEGES ON mysql.* TO r_mysql_priv; -GRANT r_mysql_priv TO schemata_tester; - -connection conn1; -set role r_mysql_priv; -select count(*) from information_schema.SCHEMATA; -select * from information_schema.SCHEMATA; - -connection default; -disconnect conn1; - -# TestTableIDAndIndexID -drop table if exists executor__infoschema_reader.t; -create table executor__infoschema_reader.t (a int, b int, primary key(a), key k1(b)); -select index_id from information_schema.tidb_indexes where table_schema = 'executor__infoschema_reader' and table_name = 't'; -select tidb_table_id > 0 from information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't'; - -# TestSchemataCharacterSet -drop database if exists `foo`; -CREATE DATABASE `foo` DEFAULT CHARACTER SET = 'utf8mb4'; -select default_character_set_name, default_collation_name FROM information_schema.SCHEMATA WHERE schema_name = 'foo'; -drop database `foo`; - -# TestViews -drop view if exists executor__infoschema_reader.v1; -CREATE DEFINER='root'@'localhost' VIEW executor__infoschema_reader.v1 AS SELECT 1; -select TABLE_COLLATION is null from INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE='VIEW'; -SELECT * FROM information_schema.views WHERE table_schema='executor__infoschema_reader' AND table_name='v1'; -SELECT table_catalog, table_schema, table_name, table_type, engine, version, row_format, table_rows, avg_row_length, data_length, max_data_length, index_length, data_free, auto_increment, update_time, check_time, table_collation, checksum, create_options, table_comment FROM information_schema.tables WHERE table_schema='executor__infoschema_reader' AND table_name='v1'; - -# TestColumnsTables -drop table if exists t; -create table t (bit bit(10) DEFAULT b'100'); -SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'executor__infoschema_reader' AND TABLE_NAME = 't'; -drop table if exists t; -set time_zone='+08:00'; -drop table if exists t; -create table t (b timestamp(3) NOT NULL DEFAULT '1970-01-01 08:00:01.000'); -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -set time_zone='+04:00'; -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -set time_zone=default; -drop table if exists t; -create table t (a bit DEFAULT (rand())); -select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='executor__infoschema_reader'; -drop table if exists t; -CREATE TABLE t (`COL3` bit(1) NOT NULL,b year) ; -select column_type from information_schema.columns where TABLE_SCHEMA = 'executor__infoschema_reader' and TABLE_NAME = 't'; -## For issue: https://github.com/pingcap/tidb/issues/43379 -select ordinal_position from information_schema.columns where table_schema=database() and table_name='t' and column_name='b'; - -# TestEngines -select * from information_schema.ENGINES; - -# TestDataTypesMaxLengthAndOctLength -# https://github.com/pingcap/tidb/issues/25467 -drop table if exists t; -create table t (a varchar(255) collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a varchar(255) collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a varchar(255) collate utf8_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a char(10) collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a char(10) collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a set('a', 'b', 'cccc') collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a set('a', 'b', 'cccc') collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a enum('a', 'b', 'cccc') collate ascii_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; -create table t (a enum('a', 'b', 'cccc') collate utf8mb4_bin); -select character_maximum_length, character_octet_length from information_schema.columns where table_schema=(select database()) and table_name='t'; -drop table t; - -# TestDDLJobs -set global tidb_ddl_enable_fast_reorg = false; -drop database if exists test_ddl_jobs; -create database test_ddl_jobs; -select db_name, job_type from information_schema.DDL_JOBS limit 1; -use test_ddl_jobs; -create table t (a int); -select db_name, table_name, job_type from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and table_name = 't'; -select job_type from information_schema.DDL_JOBS group by job_type having job_type = 'create table'; -select distinct job_type from information_schema.DDL_JOBS where job_type = 'create table' and start_time > str_to_date('20190101','%Y%m%d%H%i%s'); -drop user if exists DDL_JOBS_tester; -create user DDL_JOBS_tester; - -connect(conn1, localhost, DDL_JOBS_tester,, information_schema); -select DB_NAME, TABLE_NAME from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and TABLE_NAME = 't'; - -connection default; -CREATE ROLE r_priv; -GRANT ALL PRIVILEGES ON test_ddl_jobs.* TO r_priv; -GRANT r_priv TO DDL_JOBS_tester; - -connection conn1; -set role r_priv; -select DB_NAME, TABLE_NAME from information_schema.DDL_JOBS where DB_NAME = 'test_ddl_jobs' and TABLE_NAME = 't'; - -connection default; -create table tt (a int); -alter table tt add index t(a), add column b int; -select db_name, table_name, job_type from information_schema.DDL_JOBS limit 3; - -disconnect conn1; -drop database test_ddl_jobs; -use executor__infoschema_reader; -set global tidb_ddl_enable_fast_reorg = default; - -# TestKeyColumnUsage -select * from information_schema.KEY_COLUMN_USAGE where TABLE_NAME='stats_meta' and COLUMN_NAME='table_id'; -create user key_column_tester; - -connect (conn1, localhost, key_column_tester,, information_schema); -select * from information_schema.KEY_COLUMN_USAGE where TABLE_NAME != 'CLUSTER_SLOW_QUERY'; - -connection default; -CREATE ROLE r_stats_meta ; -GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta; -GRANT r_stats_meta TO key_column_tester; - -connection conn1; -set role r_stats_meta; -select count(*)>0 from information_schema.KEY_COLUMN_USAGE where TABLE_NAME='stats_meta'; - -connection default; -disconnect conn1; - -# TestPartitionTablesStatsCache -# https://github.com/pingcap/tidb/issues/32693 -drop table if exists e, e2; -CREATE TABLE e ( id INT NOT NULL, fname VARCHAR(30), lname VARCHAR(30)) PARTITION BY RANGE (id) ( - PARTITION p0 VALUES LESS THAN (50), - PARTITION p1 VALUES LESS THAN (100), - PARTITION p2 VALUES LESS THAN (150), - PARTITION p3 VALUES LESS THAN (MAXVALUE)); -CREATE TABLE e2 ( id INT NOT NULL, fname VARCHAR(30), lname VARCHAR(30)); -SELECT PARTITION_NAME, TABLE_ROWS FROM INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_NAME = 'e' and table_schema=(select database()); -INSERT INTO e VALUES (1669, "Jim", "Smith"), (337, "Mary", "Jones"), (16, "Frank", "White"), (2005, "Linda", "Black"); -set tidb_enable_exchange_partition='on'; -ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2; -INSERT INTO e VALUES (41, "Michael", "Green"); -analyze table e; -SELECT PARTITION_NAME, TABLE_ROWS FROM INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_NAME = 'e'; -set tidb_enable_exchange_partition=default; - -# TestMetricTables -select count(*) > 0 from information_schema.`METRICS_TABLES`; -select * from information_schema.`METRICS_TABLES` where table_name='tidb_qps'; - -# TestTableConstraintsTable -select * from information_schema.TABLE_CONSTRAINTS where TABLE_NAME='gc_delete_range'; - -# TestTableSessionVar -select * from information_schema.SESSION_VARIABLES where VARIABLE_NAME='tidb_retry_limit'; - -# TestSequences -drop sequence if exists seq, seq2; -CREATE SEQUENCE seq maxvalue 10000000; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq'; -DROP SEQUENCE seq; -CREATE SEQUENCE seq start = -1 minvalue -1 maxvalue 10 increment 1 cache 10; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq'; -CREATE SEQUENCE seq2 start = -9 minvalue -10 maxvalue 10 increment -1 cache 15; -SELECT * FROM information_schema.sequences WHERE sequence_schema='executor__infoschema_reader' AND sequence_name='seq2'; -SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME , TABLE_TYPE, ENGINE, TABLE_ROWS FROM information_schema.tables WHERE TABLE_TYPE='SEQUENCE' AND TABLE_NAME='seq2' and table_schema='executor__infoschema_reader'; - -# TestTablesPKType -drop table if exists t_int, t_implicit, t_common; -create table t_int (a int primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_int'; -set tidb_enable_clustered_index=int_only; -create table t_implicit (a varchar(64) primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_implicit'; -set tidb_enable_clustered_index=on; -create table t_common (a varchar(64) primary key, b int); -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'executor__infoschema_reader' and table_name = 't_common'; -SELECT TIDB_PK_TYPE FROM information_schema.tables where table_schema = 'INFORMATION_SCHEMA' and table_name = 'TABLES'; -set tidb_enable_clustered_index=default; - -# TestNullColumns -drop table if exists t; -CREATE TABLE t ( id int DEFAULT NULL); -CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`1.1.1.1` SQL SECURITY DEFINER VIEW `v_test` (`type`) AS SELECT NULL AS `type` FROM `t` AS `f`; -select * from information_schema.columns where TABLE_SCHEMA = 'executor__infoschema_reader' and TABLE_NAME = 'v_test'; - -# TestUserPrivilegesTable -drop user if exists usageuser; -create user usageuser; - -connect (conn1, localhost, usageuser,, information_schema); -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; - -connection default; -GRANT SELECT ON *.* to usageuser; - -connection conn1; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; - -connection default; -GRANT SELECT ON *.* to usageuser WITH GRANT OPTION; - -connection conn1; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"; - -connection default; -GRANT BACKUP_ADMIN ON *.* to usageuser; - -connection conn1; -SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'" ORDER BY privilege_type; - -connection default; -disconnect conn1; - -# test information_schema.VARIABLES_INFO DEFAULT_VALUE -select VARIABLE_NAME from information_schema.VARIABLES_INFO where DEFAULT_VALUE = CURRENT_VALUE and variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME; -set global tidb_enable_async_commit = default; -set global tidb_enable_1pc = default; -set global tidb_mem_oom_action = default; -set global tidb_enable_auto_analyze = default; -set global tidb_row_format_version = default; -set global tidb_txn_assertion_level = default; -set global tidb_enable_mutation_checker = default; -set global tidb_pessimistic_txn_fair_locking = default; -select a.VARIABLE_NAME from information_schema.VARIABLES_INFO as a, mysql.GLOBAL_VARIABLES as b where a.VARIABLE_NAME = b.VARIABLE_NAME and a.DEFAULT_VALUE = b.VARIABLE_VALUE and a.CURRENT_VALUE = b.VARIABLE_VALUE and a.variable_name in ('tidb_enable_async_commit','tidb_enable_1pc', 'tidb_mem_oom_action', 'tidb_enable_auto_analyze', 'tidb_row_format_version', 'tidb_txn_assertion_level', 'tidb_enable_mutation_checker', 'tidb_pessimistic_txn_fair_locking') order by VARIABLE_NAME;