diff --git a/executor/adapter.go b/executor/adapter.go index 1d1d15b5262f7..42ce555c6b3c3 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -341,7 +341,7 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) { if raw, ok := sctx.(processinfoSetter); ok { pi = raw sql := a.OriginText() - maxExecutionTime := sctx.GetSessionVars().GetMaxExecutionTime() + maxExecutionTime := getMaxExecutionTime(sctx) // Update processinfo, ShowProcess() will use it. pi.SetProcessInfo(sql, time.Now(), cmd, maxExecutionTime) if sctx.GetSessionVars().StmtCtx.StmtType == "" { @@ -538,7 +538,6 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { var pi processinfoSetter if raw, ok := sctx.(processinfoSetter); ok { pi = raw -<<<<<<< HEAD:executor/adapter.go sql := a.OriginText() if simple, ok := a.Plan.(*plannercore.Simple); ok && simple.Statement != nil { if ss, ok := simple.Statement.(ast.SensitiveStmtNode); ok { @@ -547,10 +546,6 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { } } maxExecutionTime := getMaxExecutionTime(sctx) -======= - sql := a.getSQLForProcessInfo() - maxExecutionTime := sctx.GetSessionVars().GetMaxExecutionTime() ->>>>>>> 13bff87d08c (variable: unifiy MaxExecuteTime usage and fix some problem (#50915)):pkg/executor/adapter.go // Update processinfo, ShowProcess() will use it. if a.Ctx.GetSessionVars().StmtCtx.StmtType == "" { a.Ctx.GetSessionVars().StmtCtx.StmtType = ast.GetStmtLabel(a.StmtNode) @@ -809,6 +804,14 @@ func isNoResultPlan(p plannercore.Plan) bool { return false } +// getMaxExecutionTime get the max execution timeout value. +func getMaxExecutionTime(sctx sessionctx.Context) uint64 { + if sctx.GetSessionVars().StmtCtx.HasMaxExecutionTime { + return sctx.GetSessionVars().StmtCtx.MaxExecutionTime + } + return sctx.GetSessionVars().MaxExecutionTime +} + type chunkRowRecordSet struct { rows []chunk.Row idx int diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 4a538fefb0091..e492dfc561e32 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -922,11 +922,7 @@ func (b *builtinTiDBDecodeSQLDigestsSig) evalString(row chunk.Row) (string, bool // Querying may take some time and it takes a context.Context as argument, which is not available here. // We simply create a context with a timeout here. -<<<<<<< HEAD:expression/builtin_info.go timeout := time.Duration(b.ctx.GetSessionVars().MaxExecutionTime) * time.Millisecond -======= - timeout := time.Duration(ctx.GetSessionVars().GetMaxExecutionTime()) * time.Millisecond ->>>>>>> 13bff87d08c (variable: unifiy MaxExecuteTime usage and fix some problem (#50915)):pkg/expression/builtin_info.go if timeout == 0 || timeout > 20*time.Second { timeout = 20 * time.Second } diff --git a/pkg/planner/core/plan_stats.go b/pkg/planner/core/plan_stats.go deleted file mode 100644 index a3850bce9449d..0000000000000 --- a/pkg/planner/core/plan_stats.go +++ /dev/null @@ -1,309 +0,0 @@ -// Copyright 2021 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 core - -import ( - "context" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" - "github.com/pingcap/tidb/pkg/statistics" - "github.com/pingcap/tidb/pkg/table" - "github.com/pingcap/tidb/pkg/util/logutil" - "go.uber.org/zap" -) - -type collectPredicateColumnsPoint struct{} - -func (collectPredicateColumnsPoint) optimize(_ context.Context, plan LogicalPlan, _ *logicalOptimizeOp) (LogicalPlan, bool, error) { - planChanged := false - if plan.SCtx().GetSessionVars().InRestrictedSQL { - return plan, planChanged, nil - } - predicateNeeded := variable.EnableColumnTracking.Load() - syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait - histNeeded := syncWait > 0 - predicateColumns, histNeededColumns := CollectColumnStatsUsage(plan, predicateNeeded, histNeeded) - if len(predicateColumns) > 0 { - plan.SCtx().UpdateColStatsUsage(predicateColumns) - } - if !histNeeded { - return plan, planChanged, nil - } - - // Prepare the table metadata to avoid repeatedly fetching from the infoSchema below. - is := plan.SCtx().GetInfoSchema().(infoschema.InfoSchema) - tblID2Tbl := make(map[int64]table.Table) - for _, neededCol := range histNeededColumns { - tbl, _ := infoschema.FindTableByTblOrPartID(is, neededCol.TableID) - if tbl == nil { - continue - } - tblID2Tbl[neededCol.TableID] = tbl - } - - // collect needed virtual columns from already needed columns - // Note that we use the dependingVirtualCols only to collect needed index stats, but not to trigger stats loading on - // the virtual columns themselves. It's because virtual columns themselves don't have statistics, while expression - // indexes, which are indexes on virtual columns, have statistics. We don't waste the resource here now. - dependingVirtualCols := CollectDependingVirtualCols(tblID2Tbl, histNeededColumns) - - histNeededIndices := collectSyncIndices(plan.SCtx(), append(histNeededColumns, dependingVirtualCols...), tblID2Tbl) - histNeededItems := collectHistNeededItems(histNeededColumns, histNeededIndices) - if histNeeded && len(histNeededItems) > 0 { - err := RequestLoadStats(plan.SCtx(), histNeededItems, syncWait) - return plan, planChanged, err - } - return plan, planChanged, nil -} - -func (collectPredicateColumnsPoint) name() string { - return "collect_predicate_columns_point" -} - -type syncWaitStatsLoadPoint struct{} - -func (syncWaitStatsLoadPoint) optimize(_ context.Context, plan LogicalPlan, _ *logicalOptimizeOp) (LogicalPlan, bool, error) { - planChanged := false - if plan.SCtx().GetSessionVars().InRestrictedSQL { - return plan, planChanged, nil - } - if plan.SCtx().GetSessionVars().StmtCtx.IsSyncStatsFailed { - return plan, planChanged, nil - } - err := SyncWaitStatsLoad(plan) - return plan, planChanged, err -} - -func (syncWaitStatsLoadPoint) name() string { - return "sync_wait_stats_load_point" -} - -// RequestLoadStats send load column/index stats requests to stats handle -func RequestLoadStats(ctx PlanContext, neededHistItems []model.TableItemID, syncWait int64) error { - maxExecutionTime := ctx.GetSessionVars().GetMaxExecutionTime() - if maxExecutionTime > 0 && maxExecutionTime < uint64(syncWait) { - syncWait = int64(maxExecutionTime) - } - failpoint.Inject("assertSyncWaitFailed", func(val failpoint.Value) { - if val.(bool) { - if syncWait != 1 { - panic("syncWait should be 1(ms)") - } - } - }) - var timeout = time.Duration(syncWait * time.Millisecond.Nanoseconds()) - stmtCtx := ctx.GetSessionVars().StmtCtx - err := domain.GetDomain(ctx).StatsHandle().SendLoadRequests(stmtCtx, neededHistItems, timeout) - if err != nil { - stmtCtx.IsSyncStatsFailed = true - if variable.StatsLoadPseudoTimeout.Load() { - logutil.BgLogger().Warn("RequestLoadStats failed", zap.Error(err)) - stmtCtx.AppendWarning(err) - return nil - } - logutil.BgLogger().Error("RequestLoadStats failed", zap.Error(err)) - return err - } - return nil -} - -// SyncWaitStatsLoad sync-wait for stats load until timeout -func SyncWaitStatsLoad(plan LogicalPlan) error { - stmtCtx := plan.SCtx().GetSessionVars().StmtCtx - if len(stmtCtx.StatsLoad.NeededItems) <= 0 { - return nil - } - err := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) - if err != nil { - stmtCtx.IsSyncStatsFailed = true - if variable.StatsLoadPseudoTimeout.Load() { - logutil.BgLogger().Warn("SyncWaitStatsLoad failed", zap.Error(err)) - stmtCtx.AppendWarning(err) - return nil - } - logutil.BgLogger().Error("SyncWaitStatsLoad failed", zap.Error(err)) - return err - } - return nil -} - -// CollectDependingVirtualCols collects the virtual columns that depend on the needed columns, and returns them in a new slice. -// -// Why do we need this? -// It's mainly for stats sync loading. -// Currently, virtual columns themselves don't have statistics. But expression indexes, which are indexes on virtual -// columns, have statistics. We need to collect needed virtual columns, then needed expression index stats can be -// collected for sync loading. -// In normal cases, if a virtual column can be used, which means related statistics may be needed, the corresponding -// expressions in the query must have already been replaced with the virtual column before here. So we just need to treat -// them like normal columns in stats sync loading, which means we just extract the Column from the expressions, the -// virtual columns we want will be there. -// However, in some cases (the mv index case now), the expressions are not replaced with the virtual columns before here. -// Instead, we match the expression in the query against the expression behind the virtual columns after here when -// building the access paths. This means we are unable to known what virtual columns will be needed by just extracting -// the Column from the expressions here. So we need to manually collect the virtual columns that may be needed. -// -// Note 1: As long as a virtual column depends on the needed columns, it will be collected. This could collect some virtual -// columns that are not actually needed. -// It's OK because that's how sync loading is expected. Sync loading only needs to ensure all actually needed stats are -// triggered to be loaded. Other logic of sync loading also works like this. -// If we want to collect only the virtual columns that are actually needed, we need to make the checking logic here exactly -// the same as the logic for generating the access paths, which will make the logic here very complicated. -// -// Note 2: Only direct dependencies are considered here. -// If a virtual column depends on another virtual column, and the latter depends on the needed columns, then the former -// will not be collected. -// For example: create table t(a int, b int, c int as (a+b), d int as (c+1)); If a is needed, then c will be collected, -// but d will not be collected. -// It's because currently it's impossible that statistics related to indirectly depending columns are actually needed. -// If we need to check indirect dependency some day, we can easily extend the logic here. -func CollectDependingVirtualCols(tblID2Tbl map[int64]table.Table, neededItems []model.TableItemID) []model.TableItemID { - generatedCols := make([]model.TableItemID, 0) - - // group the neededItems by table id - tblID2neededColIDs := make(map[int64][]int64, len(tblID2Tbl)) - for _, item := range neededItems { - if item.IsIndex { - continue - } - tblID2neededColIDs[item.TableID] = append(tblID2neededColIDs[item.TableID], item.ID) - } - - // process them by table id - for tblID, colIDs := range tblID2neededColIDs { - tbl := tblID2Tbl[tblID] - if tbl == nil { - continue - } - // collect the needed columns on this table into a set for faster lookup - colNameSet := make(map[string]struct{}, len(colIDs)) - for _, colID := range colIDs { - name := tbl.Meta().FindColumnNameByID(colID) - if name == "" { - continue - } - colNameSet[name] = struct{}{} - } - // iterate columns in this table, and collect the virtual columns that depend on the needed columns - for _, col := range tbl.Cols() { - // only handles virtual columns - if !col.IsVirtualGenerated() { - continue - } - // If this column is already needed, then skip it. - if _, ok := colNameSet[col.Name.L]; ok { - continue - } - // If there exists a needed column that is depended on by this virtual column, - // then we think this virtual column is needed. - for depCol := range col.Dependences { - if _, ok := colNameSet[depCol]; ok { - generatedCols = append(generatedCols, model.TableItemID{TableID: tblID, ID: col.ID, IsIndex: false}) - break - } - } - } - } - return generatedCols -} - -// collectSyncIndices will collect the indices which includes following conditions: -// 1. the indices contained the any one of histNeededColumns, eg: histNeededColumns contained A,B columns, and idx_a is -// composed up by A column, then we thought the idx_a should be collected -// 2. The stats condition of idx_a can't meet IsFullLoad, which means its stats was evicted previously -func collectSyncIndices(ctx PlanContext, - histNeededColumns []model.TableItemID, - tblID2Tbl map[int64]table.Table, -) map[model.TableItemID]struct{} { - histNeededIndices := make(map[model.TableItemID]struct{}) - stats := domain.GetDomain(ctx).StatsHandle() - for _, column := range histNeededColumns { - if column.IsIndex { - continue - } - tbl := tblID2Tbl[column.TableID] - if tbl == nil { - continue - } - colName := tbl.Meta().FindColumnNameByID(column.ID) - if colName == "" { - continue - } - for _, idx := range tbl.Indices() { - if idx.Meta().State != model.StatePublic { - continue - } - idxCol := idx.Meta().FindColumnByName(colName) - idxID := idx.Meta().ID - if idxCol != nil { - tblStats := stats.GetTableStats(tbl.Meta()) - if tblStats == nil || tblStats.Pseudo { - continue - } - idxStats, ok := tblStats.Indices[idx.Meta().ID] - if ok && idxStats.IsStatsInitialized() && !idxStats.IsFullLoad() { - histNeededIndices[model.TableItemID{TableID: column.TableID, ID: idxID, IsIndex: true}] = struct{}{} - } - } - } - } - return histNeededIndices -} - -func collectHistNeededItems(histNeededColumns []model.TableItemID, histNeededIndices map[model.TableItemID]struct{}) (histNeededItems []model.TableItemID) { - for idx := range histNeededIndices { - histNeededItems = append(histNeededItems, idx) - } - histNeededItems = append(histNeededItems, histNeededColumns...) - return -} - -func recordTableRuntimeStats(sctx PlanContext, tbls map[int64]struct{}) { - tblStats := sctx.GetSessionVars().StmtCtx.TableStats - if tblStats == nil { - tblStats = map[int64]any{} - } - for tblID := range tbls { - tblJSONStats, skip, err := recordSingleTableRuntimeStats(sctx, tblID) - if err != nil { - logutil.BgLogger().Warn("record table json stats failed", zap.Int64("tblID", tblID), zap.Error(err)) - } - if tblJSONStats == nil && !skip { - logutil.BgLogger().Warn("record table json stats failed due to empty", zap.Int64("tblID", tblID)) - } - tblStats[tblID] = tblJSONStats - } - sctx.GetSessionVars().StmtCtx.TableStats = tblStats -} - -func recordSingleTableRuntimeStats(sctx PlanContext, tblID int64) (stats *statistics.Table, skip bool, err error) { - dom := domain.GetDomain(sctx) - statsHandle := dom.StatsHandle() - is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) - tbl, ok := is.TableByID(tblID) - if !ok { - return nil, false, nil - } - tableInfo := tbl.Meta() - stats = statsHandle.GetTableStats(tableInfo) - // Skip the warning if the table is a temporary table because the temporary table doesn't have stats. - skip = tableInfo.TempTableType != model.TempTableNone - return stats, skip, nil -} diff --git a/pkg/session/test/variable/variable_test.go b/pkg/session/test/variable/variable_test.go deleted file mode 100644 index c4078af451fdb..0000000000000 --- a/pkg/session/test/variable/variable_test.go +++ /dev/null @@ -1,366 +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/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 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) - require.Equal(t, uint64(500), tk.Session().GetSessionVars().GetMaxExecutionTime()) - - 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;") - require.Equal(t, uint64(150), tk.Session().GetSessionVars().GetMaxExecutionTime()) - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) */ * FROM MaxExecTime;") - require.Equal(t, uint64(1000), tk.Session().GetSessionVars().GetMaxExecutionTime()) - - 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 []any) 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/session/test/vars/vars_test.go b/pkg/session/test/vars/vars_test.go deleted file mode 100644 index 8968ce7a65dcd..0000000000000 --- a/pkg/session/test/vars/vars_test.go +++ /dev/null @@ -1,404 +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 vars - -import ( - "context" - "fmt" - "strconv" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/domain" - tikv "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" - "github.com/pingcap/tidb/pkg/testkit" - "github.com/pingcap/tidb/pkg/util/hint" - "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/txnkv/transaction" -) - -func TestKVVars(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@tidb_backoff_lock_fast = 1") - tk.MustExec("set @@tidb_backoff_weight = 100") - tk.MustExec("create table if not exists kvvars (a int key)") - tk.MustExec("insert into kvvars values (1)") - tk.MustExec("begin") - txn, err := tk.Session().Txn(false) - require.NoError(t, err) - vars := txn.GetVars().(*tikv.Variables) - require.Equal(t, 1, vars.BackoffLockFast) - require.Equal(t, 100, vars.BackOffWeight) - tk.MustExec("rollback") - tk.MustExec("set @@tidb_backoff_weight = 50") - tk.MustExec("set @@autocommit = 0") - tk.MustExec("select * from kvvars") - require.True(t, tk.Session().GetSessionVars().InTxn()) - txn, err = tk.Session().Txn(false) - require.NoError(t, err) - vars = txn.GetVars().(*tikv.Variables) - require.Equal(t, 50, vars.BackOffWeight) - - tk.MustExec("set @@autocommit = 1") - require.Nil(t, failpoint.Enable("tikvclient/probeSetVars", `return(true)`)) - tk.MustExec("select * from kvvars where a = 1") - require.Nil(t, failpoint.Disable("tikvclient/probeSetVars")) - require.True(t, transaction.SetSuccess.Load()) - transaction.SetSuccess.Store(false) -} - -func TestRemovedSysVars(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "bogus_var", Value: "acdc"}) - result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'bogus_var'") - result.Check(testkit.Rows("bogus_var acdc")) - result = tk.MustQuery("SELECT @@GLOBAL.bogus_var") - result.Check(testkit.Rows("acdc")) - tk.MustExec("SET GLOBAL bogus_var = 'newvalue'") - - // unregister - variable.UnregisterSysVar("bogus_var") - - result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'bogus_var'") - result.Check(testkit.Rows()) // empty - tk.MustContainErrMsg("SET GLOBAL bogus_var = 'newvalue'", "[variable:1193]Unknown system variable 'bogus_var'") - tk.MustContainErrMsg("SELECT @@GLOBAL.bogus_var", "[variable:1193]Unknown system variable 'bogus_var'") -} - -func TestTiKVSystemVars(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_enable'") // default is on from the sysvar - result.Check(testkit.Rows("tidb_gc_enable ON")) - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'") - result.Check(testkit.Rows()) // but no value in the table (yet) because the value has not been set and the GC has never been run - - // update will set a value in the table - tk.MustExec("SET GLOBAL tidb_gc_enable = 1") - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'") - result.Check(testkit.Rows("true")) - - tk.MustExec("UPDATE mysql.tidb SET variable_value = 'false' WHERE variable_name='tikv_gc_enable'") - result = tk.MustQuery("SELECT @@tidb_gc_enable;") - result.Check(testkit.Rows("0")) // reads from mysql.tidb value and changes to false - - tk.MustExec("SET GLOBAL tidb_gc_concurrency = -1") // sets auto concurrency and concurrency - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'") - result.Check(testkit.Rows("true")) - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'") - result.Check(testkit.Rows("-1")) - - tk.MustExec("SET GLOBAL tidb_gc_concurrency = 5") // sets auto concurrency and concurrency - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'") - result.Check(testkit.Rows("false")) - result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'") - result.Check(testkit.Rows("5")) - - tk.MustExec("UPDATE mysql.tidb SET variable_value = 'true' WHERE variable_name='tikv_gc_auto_concurrency'") - result = tk.MustQuery("SELECT @@tidb_gc_concurrency;") - result.Check(testkit.Rows("-1")) // because auto_concurrency is turned on it takes precedence - - tk.MustExec("REPLACE INTO mysql.tidb (variable_value, variable_name) VALUES ('15m', 'tikv_gc_run_interval')") - result = tk.MustQuery("SELECT @@GLOBAL.tidb_gc_run_interval;") - result.Check(testkit.Rows("15m0s")) - result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_run_interval'") - result.Check(testkit.Rows("tidb_gc_run_interval 15m0s")) - - tk.MustExec("SET GLOBAL tidb_gc_run_interval = '9m'") // too small - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_gc_run_interval value: '9m'")) - result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_run_interval'") - result.Check(testkit.Rows("tidb_gc_run_interval 10m0s")) - - tk.MustExec("SET GLOBAL tidb_gc_run_interval = '700000000000ns'") // specified in ns, also valid - - _, err := tk.Exec("SET GLOBAL tidb_gc_run_interval = '11mins'") - require.Equal(t, "[variable:1232]Incorrect argument type to variable 'tidb_gc_run_interval'", err.Error()) -} - -func TestUpgradeSysvars(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - se := tk.Session().(variable.GlobalVarAccessor) - - // Set the global var to a non-canonical form of the value - // i.e. implying that it was set from an earlier version of TiDB. - - tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_enable_noop_functions', '0')`) - domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache - v, err := se.GetGlobalSysVar("tidb_enable_noop_functions") - require.NoError(t, err) - require.Equal(t, "OFF", v) - - // Set the global var to "" which is the invalid version of this from TiDB 4.0.16 - // the err is quashed by the GetGlobalSysVar, and the default value is restored. - // This helps callers of GetGlobalSysVar(), which can't individually be expected - // to handle upgrade/downgrade issues correctly. - - tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('rpl_semi_sync_slave_enabled', '')`) - domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache - v, err = se.GetGlobalSysVar("rpl_semi_sync_slave_enabled") - require.NoError(t, err) - require.Equal(t, "OFF", v) // the default value is restored. - result := tk.MustQuery("SHOW VARIABLES LIKE 'rpl_semi_sync_slave_enabled'") - result.Check(testkit.Rows("rpl_semi_sync_slave_enabled OFF")) - - // Ensure variable out of range is converted to in range after upgrade. - // This further helps for https://github.com/pingcap/tidb/pull/28842 - - tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_executor_concurrency', '999')`) - domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache - v, err = se.GetGlobalSysVar("tidb_executor_concurrency") - require.NoError(t, err) - require.Equal(t, "256", v) // the max value is restored. - - // Handle the case of a completely bogus value from an earlier version of TiDB. - // This could be the case if an ENUM sysvar removes a value. - - tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_enable_noop_functions', 'SOMEVAL')`) - domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache - v, err = se.GetGlobalSysVar("tidb_enable_noop_functions") - require.NoError(t, err) - require.Equal(t, "OFF", v) // the default value is restored. -} - -func TestSetInstanceSysvarBySetGlobalSysVar(t *testing.T) { - varName := "tidb_general_log" - defaultValue := "OFF" // This is the default value for tidb_general_log - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - se := tk.Session().(variable.GlobalVarAccessor) - - // Get globalSysVar twice and get the same default value - v, err := se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, defaultValue, v) - v, err = se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, defaultValue, v) - - // session.GetGlobalSysVar would not get the value which session.SetGlobalSysVar writes, - // because SetGlobalSysVar calls SetGlobalFromHook, which uses TiDBGeneralLog's SetGlobal, - // but GetGlobalSysVar could not access TiDBGeneralLog's GetGlobal. - - // set to "1" - err = se.SetGlobalSysVar(context.Background(), varName, "ON") - require.NoError(t, err) - v, err = se.GetGlobalSysVar(varName) - tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("1")) - require.NoError(t, err) - require.Equal(t, defaultValue, v) - - // set back to "0" - err = se.SetGlobalSysVar(context.Background(), varName, defaultValue) - require.NoError(t, err) - v, err = se.GetGlobalSysVar(varName) - tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("0")) - require.NoError(t, err) - require.Equal(t, defaultValue, v) -} - -func TestTimeZone(t *testing.T) { - store := testkit.CreateMockStore(t) - - // TestCastTimeToDate - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set time_zone = '-8:00'") - date := time.Now().In(time.FixedZone("", -8*int(time.Hour/time.Second))) - tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format(time.DateOnly))) - tk.MustExec("set time_zone = '+08:00'") - date = time.Now().In(time.FixedZone("", 8*int(time.Hour/time.Second))) - tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format(time.DateOnly))) - - // TestSetGlobalTZ - tk = testkit.NewTestKit(t, store) - tk.MustExec("set time_zone = '+08:00'") - tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - tk.MustExec("set global time_zone = '+00:00'") - tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - tk1 := testkit.NewTestKit(t, store) - tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) -} - -func TestGlobalVarAccessor(t *testing.T) { - varName := "max_allowed_packet" - varValue := strconv.FormatUint(variable.DefMaxAllowedPacket, 10) // This is the default value for max_allowed_packet - - // The value of max_allowed_packet should be a multiple of 1024, - // so the setting of varValue1 and varValue2 would be truncated to varValue0 - varValue0 := "4194304" - varValue1 := "4194305" - varValue2 := "4194306" - - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - se := tk.Session().(variable.GlobalVarAccessor) - // Get globalSysVar twice and get the same value - v, err := se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue, v) - v, err = se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue, v) - // Set global var to another value - err = se.SetGlobalSysVar(context.Background(), varName, varValue1) - require.NoError(t, err) - v, err = se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue0, v) - require.NoError(t, tk.Session().CommitTxn(context.TODO())) - - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - se1 := tk1.Session().(variable.GlobalVarAccessor) - v, err = se1.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue0, v) - err = se1.SetGlobalSysVar(context.Background(), varName, varValue2) - require.NoError(t, err) - v, err = se1.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue0, v) - require.NoError(t, tk1.Session().CommitTxn(context.TODO())) - - // Make sure the change is visible to any client that accesses that global variable. - v, err = se.GetGlobalSysVar(varName) - require.NoError(t, err) - require.Equal(t, varValue0, v) - - // For issue 10955, make sure the new session load `max_execution_time` into sessionVars. - tk1.MustExec("set @@global.max_execution_time = 100") - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - require.Equal(t, uint64(100), tk2.Session().GetSessionVars().MaxExecutionTime) - require.Equal(t, uint64(100), tk2.Session().GetSessionVars().GetMaxExecutionTime()) - tk1.MustExec("set @@global.max_execution_time = 0") - - result := tk.MustQuery("show global variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) - result = tk.MustQuery("show session variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) - tk.MustExec("set session sql_select_limit=100000000000;") - result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) - result = tk.MustQuery("show session variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 100000000000")) - tk.MustExec("set @@global.sql_select_limit = 1") - result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 1")) - tk.MustExec("set @@global.sql_select_limit = default") - result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") - result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) - - result = tk.MustQuery("select @@global.autocommit;") - result.Check(testkit.Rows("1")) - result = tk.MustQuery("select @@autocommit;") - result.Check(testkit.Rows("1")) - tk.MustExec("set @@global.autocommit = 0;") - result = tk.MustQuery("select @@global.autocommit;") - result.Check(testkit.Rows("0")) - result = tk.MustQuery("select @@autocommit;") - result.Check(testkit.Rows("1")) - tk.MustExec("set @@global.autocommit=1") - - err = tk.ExecToErr("set global time_zone = 'timezone'") - require.Error(t, err) - require.True(t, terror.ErrorEqual(err, variable.ErrUnknownTimeZone)) -} - -func TestPrepareExecuteWithSQLHints(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - se := tk.Session() - se.SetConnectionID(1) - tk.MustExec("use test") - tk.MustExec("create table t(a int primary key)") - - type hintCheck struct { - hint string - check func(*hint.StmtHints) - } - - hintChecks := []hintCheck{ - { - hint: "MEMORY_QUOTA(1024 MB)", - check: func(stmtHint *hint.StmtHints) { - require.True(t, stmtHint.HasMemQuotaHint) - require.Equal(t, int64(1024*1024*1024), stmtHint.MemQuotaQuery) - }, - }, - { - hint: "READ_CONSISTENT_REPLICA()", - check: func(stmtHint *hint.StmtHints) { - require.True(t, stmtHint.HasReplicaReadHint) - require.Equal(t, byte(tikv.ReplicaReadFollower), stmtHint.ReplicaRead) - }, - }, - { - hint: "MAX_EXECUTION_TIME(1000)", - check: func(stmtHint *hint.StmtHints) { - require.True(t, stmtHint.HasMaxExecutionTime) - require.Equal(t, uint64(1000), stmtHint.MaxExecutionTime) - }, - }, - { - hint: "USE_TOJA(TRUE)", - check: func(stmtHint *hint.StmtHints) { - require.True(t, stmtHint.HasAllowInSubqToJoinAndAggHint) - require.True(t, stmtHint.AllowInSubqToJoinAndAgg) - }, - }, - { - hint: "RESOURCE_GROUP(rg1)", - check: func(stmtHint *hint.StmtHints) { - require.True(t, stmtHint.HasResourceGroup) - require.Equal(t, "rg1", stmtHint.ResourceGroup) - }, - }, - } - - for i, check := range hintChecks { - // common path - tk.MustExec(fmt.Sprintf("prepare stmt%d from 'select /*+ %s */ * from t'", i, check.hint)) - for j := 0; j < 10; j++ { - tk.MustQuery(fmt.Sprintf("execute stmt%d", i)) - check.check(&tk.Session().GetSessionVars().StmtCtx.StmtHints) - } - // fast path - tk.MustExec(fmt.Sprintf("prepare fast%d from 'select /*+ %s */ * from t where a = 1'", i, check.hint)) - for j := 0; j < 10; j++ { - tk.MustQuery(fmt.Sprintf("execute fast%d", i)) - check.check(&tk.Session().GetSessionVars().StmtCtx.StmtHints) - } - } -} diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index a97cdd07b632f..82a82d2072265 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -272,7 +272,7 @@ func TestPlanStatsLoadTimeout(t *testing.T) { ToTimeout: time.Now().Local().Add(timeout), } dom.StatsHandle().AppendNeededItem(task, timeout) // make channel queue full - sql := "select /*+ MAX_EXECUTION_TIME(1000) */ * from t where c>1" + sql := "select * from t where c>1" stmt, err := p.ParseOneStmt(sql, "", "") require.NoError(t, err) tk.MustExec("set global tidb_stats_load_pseudo_timeout=false") @@ -284,11 +284,6 @@ func TestPlanStatsLoadTimeout(t *testing.T) { tk.MustExec(sql) // not fail sql for timeout when pseudo=true failpoint.Disable("github.com/pingcap/executor/assertSyncStatsFailed") - // Test Issue #50872. - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/assertSyncWaitFailed", `return(true)`)) - tk.MustExec(sql) - failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/assertSyncWaitFailed") - plan, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) require.NoError(t, err) // not fail sql for timeout when pseudo=true switch pp := plan.(type) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index f99976d6cc163..dc09defed1662 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -3443,150 +3443,3 @@ func (s *SessionVars) GetRelatedTableForMDL() *sync.Map { func (s *SessionVars) EnableForceInlineCTE() bool { return s.enableForceInlineCTE } -<<<<<<< HEAD:sessionctx/variable/session.go -======= - -// IsRuntimeFilterEnabled return runtime filter mode whether OFF -func (s *SessionVars) IsRuntimeFilterEnabled() bool { - return s.runtimeFilterMode != RFOff -} - -// GetRuntimeFilterTypes return the session variable runtimeFilterTypes -func (s *SessionVars) GetRuntimeFilterTypes() []RuntimeFilterType { - return s.runtimeFilterTypes -} - -// GetRuntimeFilterMode return the session variable runtimeFilterMode -func (s *SessionVars) GetRuntimeFilterMode() RuntimeFilterMode { - return s.runtimeFilterMode -} - -// GetMaxExecutionTime get the max execution timeout value. -func (s *SessionVars) GetMaxExecutionTime() uint64 { - if s.StmtCtx.HasMaxExecutionTime { - return s.StmtCtx.MaxExecutionTime - } - return s.MaxExecutionTime -} - -// GetTiKVClientReadTimeout returns readonly kv request timeout, prefer query hint over session variable -func (s *SessionVars) GetTiKVClientReadTimeout() uint64 { - return s.TiKVClientReadTimeout -} - -// RuntimeFilterType type of runtime filter "IN" -type RuntimeFilterType int64 - -// In type of runtime filter, like "t.k1 in (?)" -// MinMax type of runtime filter, like "t.k1 < ? and t.k1 > ?" -const ( - In RuntimeFilterType = iota - MinMax - // todo BloomFilter, bf/in -) - -// String convert Runtime Filter Type to String name -func (rfType RuntimeFilterType) String() string { - switch rfType { - case In: - return "IN" - case MinMax: - return "MIN_MAX" - default: - return "" - } -} - -// RuntimeFilterTypeStringToType convert RuntimeFilterTypeNameString to RuntimeFilterType -// If name is legal, it will return Runtime Filter Type and true -// Else, it will return -1 and false -// The second param means the convert is ok or not. Ture is ok, false means it is illegal name -// At present, we only support two names: "IN" and "MIN_MAX" -func RuntimeFilterTypeStringToType(name string) (RuntimeFilterType, bool) { - switch name { - case "IN": - return In, true - case "MIN_MAX": - return MinMax, true - default: - return -1, false - } -} - -// ToRuntimeFilterType convert session var value to RuntimeFilterType list -// If sessionVarValue is legal, it will return RuntimeFilterType list and true -// The second param means the convert is ok or not. Ture is ok, false means it is illegal value -// The legal value should be comma-separated, eg: "IN,MIN_MAX" -func ToRuntimeFilterType(sessionVarValue string) ([]RuntimeFilterType, bool) { - typeNameList := strings.Split(sessionVarValue, ",") - rfTypeMap := make(map[RuntimeFilterType]bool) - for _, typeName := range typeNameList { - rfType, ok := RuntimeFilterTypeStringToType(strings.ToUpper(typeName)) - if !ok { - return nil, ok - } - rfTypeMap[rfType] = true - } - rfTypeList := make([]RuntimeFilterType, 0, len(rfTypeMap)) - for rfType := range rfTypeMap { - rfTypeList = append(rfTypeList, rfType) - } - return rfTypeList, true -} - -// RuntimeFilterMode the mode of runtime filter "OFF", "LOCAL" -type RuntimeFilterMode int64 - -// RFOff disable runtime filter -// RFLocal enable local runtime filter -// RFGlobal enable local and global runtime filter -const ( - RFOff RuntimeFilterMode = iota + 1 - RFLocal - RFGlobal -) - -// String convert Runtime Filter Mode to String name -func (rfMode RuntimeFilterMode) String() string { - switch rfMode { - case RFOff: - return "OFF" - case RFLocal: - return "LOCAL" - case RFGlobal: - return "GLOBAL" - default: - return "" - } -} - -// RuntimeFilterModeStringToMode convert RuntimeFilterModeString to RuntimeFilterMode -// If name is legal, it will return Runtime Filter Mode and true -// Else, it will return -1 and false -// The second param means the convert is ok or not. Ture is ok, false means it is illegal name -// At present, we only support one name: "OFF", "LOCAL" -func RuntimeFilterModeStringToMode(name string) (RuntimeFilterMode, bool) { - switch name { - case "OFF": - return RFOff, true - case "LOCAL": - return RFLocal, true - default: - return -1, false - } -} - -const ( - // OptObjectiveModerate is a possible value and the default value for TiDBOptObjective. - // Please see comments of SessionVars.OptObjective for details. - OptObjectiveModerate string = "moderate" - // OptObjectiveDeterminate is a possible value for TiDBOptObjective. - OptObjectiveDeterminate = "determinate" -) - -// GetOptObjective return the session variable "tidb_opt_objective". -// Please see comments of SessionVars.OptObjective for details. -func (s *SessionVars) GetOptObjective() string { - return s.OptObjective -} ->>>>>>> 13bff87d08c (variable: unifiy MaxExecuteTime usage and fix some problem (#50915)):pkg/sessionctx/variable/session.go