From fa8e2d62a4e7db66dece3d7fc17b54044f540beb Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Thu, 11 Jan 2024 18:52:55 +0800 Subject: [PATCH] This is an automated cherry-pick of #49524 Signed-off-by: ti-chi-bot --- executor/infoschema_reader.go | 6 +- executor/set.go | 3 +- pkg/session/test/variable/variable_test.go | 390 ++++++++++++++++++ pkg/sessionctx/variable/BUILD.bazel | 128 ++++++ session/bootstrap.go | 32 +- sessionctx/variable/sysvar.go | 64 +++ sessionctx/variable/sysvar_test.go | 164 ++++++++ .../r/executor/infoschema_reader.result | 313 ++++++++++++++ .../t/executor/infoschema_reader.test | 253 ++++++++++++ 9 files changed, 1320 insertions(+), 33 deletions(-) create mode 100644 pkg/session/test/variable/variable_test.go create mode 100644 pkg/sessionctx/variable/BUILD.bazel create mode 100644 tests/integrationtest/r/executor/infoschema_reader.result create mode 100644 tests/integrationtest/t/executor/infoschema_reader.test diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index f4bdfe2a7a460..9ab0d066403f7 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -423,10 +423,14 @@ func (e *memtableRetriever) setDataForVariablesInfo(ctx sessionctx.Context) erro if sv.IsNoop { isNoop = "YES" } + defVal := sv.Value + if sv.HasGlobalScope() { + defVal = variable.GlobalSystemVariableInitialValue(sv.Name, defVal) + } row := types.MakeDatums( sv.Name, // VARIABLE_NAME sv.Scope.String(), // VARIABLE_SCOPE - sv.Value, // DEFAULT_VALUE + defVal, // DEFAULT_VALUE currentVal, // CURRENT_VALUE sv.MinValue, // MIN_VALUE sv.MaxValue, // MAX_VALUE diff --git a/executor/set.go b/executor/set.go index f9ac8abd38a1d..e45ddb830d293 100644 --- a/executor/set.go +++ b/executor/set.go @@ -277,7 +277,8 @@ func (e *SetExecutor) getVarValue(ctx context.Context, v *expression.VarAssignme // to the compiled-in MySQL default value, use the DEFAULT keyword. // See http://dev.mysql.com/doc/refman/5.7/en/set-statement.html if sysVar != nil { - return sysVar.Value, nil + defVal := variable.GlobalSystemVariableInitialValue(sysVar.Name, sysVar.Value) + return defVal, nil } return e.ctx.GetSessionVars().GetGlobalSystemVar(ctx, v.Name) } diff --git a/pkg/session/test/variable/variable_test.go b/pkg/session/test/variable/variable_test.go new file mode 100644 index 0000000000000..005d9eb122688 --- /dev/null +++ b/pkg/session/test/variable/variable_test.go @@ -0,0 +1,390 @@ +// 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 new file mode 100644 index 0000000000000..d7ad78566993b --- /dev/null +++ b/pkg/sessionctx/variable/BUILD.bazel @@ -0,0 +1,128 @@ +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/session/bootstrap.go b/session/bootstrap.go index 8af2369c209ec..72c1ee533e6f7 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -2737,37 +2737,7 @@ func doDMLWorks(s Session) { if !v.HasGlobalScope() { continue } - vVal := v.Value - switch v.Name { - case variable.TiDBTxnMode: - if config.GetGlobalConfig().Store == "tikv" || config.GetGlobalConfig().Store == "unistore" { - vVal = "pessimistic" - } - case variable.TiDBEnableAsyncCommit, variable.TiDBEnable1PC: - if config.GetGlobalConfig().Store == "tikv" { - vVal = variable.On - } - case variable.TiDBMemOOMAction: - if intest.InTest { - vVal = variable.OOMActionLog - } - case variable.TiDBEnableAutoAnalyze: - if intest.InTest { - vVal = variable.Off - } - // For the following sysvars, we change the default - // FOR NEW INSTALLS ONLY. In most cases you don't want to do this. - // It is better to change the value in the Sysvar struct, so that - // all installs will have the same value. - case variable.TiDBRowFormatVersion: - vVal = strconv.Itoa(variable.DefTiDBRowFormatV2) - case variable.TiDBTxnAssertionLevel: - vVal = variable.AssertionFastStr - case variable.TiDBEnableMutationChecker: - vVal = variable.On - case variable.TiDBPessimisticTransactionFairLocking: - vVal = variable.On - } + vVal := variable.GlobalSystemVariableInitialValue(v.Name, v.Value) // sanitize k and vVal value := fmt.Sprintf(`("%s", "%s")`, sqlexec.EscapeString(k), sqlexec.EscapeString(vVal)) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 835103bac9f2e..f6617e29f2936 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -26,6 +26,7 @@ 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" @@ -48,6 +49,38 @@ 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" @@ -2706,6 +2739,37 @@ var defaultSysVars = []*SysVar{ }, } +// GlobalSystemVariableInitialValue gets the default value for a system variable including ones that are dynamically set (e.g. based on the store) +func GlobalSystemVariableInitialValue(varName, varVal string) string { + switch varName { + case TiDBEnableAsyncCommit, TiDBEnable1PC: + if config.GetGlobalConfig().Store == "tikv" { + varVal = On + } + case TiDBMemOOMAction: + if intest.InTest { + varVal = OOMActionLog + } + case TiDBEnableAutoAnalyze: + if intest.InTest { + varVal = Off + } + // For the following sysvars, we change the default + // FOR NEW INSTALLS ONLY. In most cases you don't want to do this. + // It is better to change the value in the Sysvar struct, so that + // all installs will have the same value. + case TiDBRowFormatVersion: + varVal = strconv.Itoa(DefTiDBRowFormatV2) + case TiDBTxnAssertionLevel: + varVal = AssertionFastStr + case TiDBEnableMutationChecker: + varVal = On + case TiDBPessimisticTransactionFairLocking: + varVal = On + } + return varVal +} + func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { p, err := tiflashcompute.GetDispatchPolicyByStr(val) if err != nil { diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 568d7a45691dd..57cfeec8a798e 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -1238,3 +1238,167 @@ 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 { + name string + val string + initVal string + }{ + { + TiDBTxnMode, + DefTiDBTxnMode, + "pessimistic", + }, + { + TiDBEnableAsyncCommit, + BoolToOnOff(DefTiDBEnableAsyncCommit), + BoolToOnOff(DefTiDBEnableAsyncCommit), + }, + { + TiDBEnable1PC, + BoolToOnOff(DefTiDBEnable1PC), + BoolToOnOff(DefTiDBEnable1PC), + }, + { + TiDBMemOOMAction, + DefTiDBMemOOMAction, + OOMActionLog, + }, + { + TiDBEnableAutoAnalyze, + BoolToOnOff(DefTiDBEnableAutoAnalyze), + Off, + }, + { + TiDBRowFormatVersion, + strconv.Itoa(DefTiDBRowFormatV1), + strconv.Itoa(DefTiDBRowFormatV2), + }, + { + TiDBTxnAssertionLevel, + DefTiDBTxnAssertionLevel, + AssertionFastStr, + }, + { + TiDBEnableMutationChecker, + BoolToOnOff(DefTiDBEnableMutationChecker), + On, + }, + { + TiDBPessimisticTransactionFairLocking, + BoolToOnOff(DefTiDBPessimisticTransactionFairLocking), + On, + }, + } + for _, v := range vars { + initVal := GlobalSystemVariableInitialValue(v.name, v.val) + 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 new file mode 100644 index 0000000000000..4745b06b79a02 --- /dev/null +++ b/tests/integrationtest/r/executor/infoschema_reader.result @@ -0,0 +1,313 @@ +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 new file mode 100644 index 0000000000000..f996f0f990c67 --- /dev/null +++ b/tests/integrationtest/t/executor/infoschema_reader.test @@ -0,0 +1,253 @@ +# 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;