diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index f71cd3c877..21b2bc909e 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -59,7 +59,7 @@ go_test( embed = [":bindinfo"], flaky = True, race = "on", - shard_count = 35, + shard_count = 36, deps = [ "//pkg/parser", "//pkg/parser/ast", diff --git a/pkg/bindinfo/binding_auto.go b/pkg/bindinfo/binding_auto.go index 4666bc9a7e..98ab4f5685 100644 --- a/pkg/bindinfo/binding_auto.go +++ b/pkg/bindinfo/binding_auto.go @@ -30,8 +30,13 @@ import ( "go.uber.org/zap" ) -// PlanDigestFunc is used to get the plan digest of this SQL. -var PlanDigestFunc func(sctx sessionctx.Context, stmt ast.StmtNode) (planDigest string, err error) +// CalculatePlanDigest is used to get the plan digest of this SQL. +// This function will call the optimizer. +var CalculatePlanDigest func(sctx sessionctx.Context, stmt ast.StmtNode) (planDigest string, err error) + +// RecordRelevantOptVarsAndFixes is used to get the relevant optimizer variables for this SQL. +// This function will call the optimizer. +var RecordRelevantOptVarsAndFixes func(sctx sessionctx.Context, stmt ast.StmtNode) (varNames []string, fixIDs []uint64, err error) // BindingPlanInfo contains the binding info and its corresponding plan execution info, which is used by // "SHOW PLAN FOR " to help users understand the historical plans for a specific SQL. diff --git a/pkg/bindinfo/binding_auto_test.go b/pkg/bindinfo/binding_auto_test.go index 31ac14477c..1bdfb5b294 100644 --- a/pkg/bindinfo/binding_auto_test.go +++ b/pkg/bindinfo/binding_auto_test.go @@ -15,9 +15,11 @@ package bindinfo_test import ( + "fmt" "testing" "github.com/pingcap/tidb/pkg/bindinfo" + "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -82,3 +84,42 @@ func TestIsSimplePointPlan(t *testing.T) { HashJoin root 1 plus(test.t.a, 1)->Column#3 0 time:173µs, open:24.9µs, close:8.92µs, loops:1, Concurrency:OFF 380 Bytes N/A └─Point_Get_5 root 1 table:t, handle:2 0 time:143.2µs, open:1.71µs, close:5.92µs, loops:1, Get:{num_rpc:1, total_time:40µs} N/A N/A`)) } + +func TestRelevantOptVarsAndFixes(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec(`create table t1 (a int, b int, c varchar(10), key(a), key(b))`) + tk.MustExec(`create table t2 (a int, b int, c varchar(10), key(a), key(b))`) + + type testCase struct { + query string + vars string + fixes string + } + cases := []testCase{ + {"select 1 from t1", "[tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_table_full_scan_cost_factor tidb_opt_table_reader_cost_factor]", "[45132 52869]"}, + {"select 1 from t1 where a=1", "[tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor]", "[52869]"}, + {"select * from t1 where a=1", "[tidb_opt_index_lookup_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_table_rowid_scan_cost_factor]", "[52869]"}, + {"select * from t1 where a>1", "[tidb_opt_index_lookup_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_table_full_scan_cost_factor tidb_opt_table_reader_cost_factor tidb_opt_table_rowid_scan_cost_factor]", "[45132 52869]"}, + {"select a from t1 where a=1", "[tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor]", "[52869]"}, + {"select max(a) from t1 where a=1", "[tidb_opt_hash_agg_cost_factor tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_ordering_index_selectivity_ratio tidb_opt_stream_agg_cost_factor tidb_opt_topn_cost_factor]", "[52869]"}, + {"select sum(b) from t1 where a=1", "[tidb_opt_hash_agg_cost_factor tidb_opt_index_lookup_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_stream_agg_cost_factor tidb_opt_table_rowid_scan_cost_factor]", "[52869]"}, + {"select a from t1 where a=1 order by b", "[tidb_opt_index_lookup_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_sort_cost_factor tidb_opt_table_rowid_scan_cost_factor]", "[45132 52869]"}, + {"select a from t1 where a=1 order by b limit 10", "[tidb_opt_index_lookup_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_table_range_scan_cost_factor tidb_opt_table_rowid_scan_cost_factor tidb_opt_topn_cost_factor]", "[52869]"}, + {"select 1 from t1, t2 where t1.a=t2.a", "[tidb_opt_hash_join_cost_factor tidb_opt_index_join_cost_factor tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_merge_join_cost_factor]", "[44855 45132 52869]"}, + {"select 1 from t1, t2 where t1.a=t2.b", "[tidb_opt_hash_join_cost_factor tidb_opt_index_join_cost_factor tidb_opt_index_reader_cost_factor tidb_opt_index_scan_cost_factor tidb_opt_merge_join_cost_factor]", "[44855 45132 52869]"}, + {"select 1 from t1, t2 where t1.c=t2.c", "[tidb_opt_hash_join_cost_factor tidb_opt_table_full_scan_cost_factor tidb_opt_table_reader_cost_factor]", "[52869]"}, + } + + for _, c := range cases { + p := parser.New() + stmt, err := p.ParseOneStmt(c.query, "", "") + require.NoError(t, err) + vars, fixes, err := bindinfo.RecordRelevantOptVarsAndFixes(tk.Session(), stmt) + require.NoError(t, err) + require.Equal(t, fmt.Sprintf("%v", vars), c.vars) + require.Equal(t, fmt.Sprintf("%v", fixes), c.fixes) + } +} diff --git a/pkg/bindinfo/utils.go b/pkg/bindinfo/utils.go index 5a81bf357c..ec4906d537 100644 --- a/pkg/bindinfo/utils.go +++ b/pkg/bindinfo/utils.go @@ -220,7 +220,7 @@ func getBindingPlanDigest(sctx sessionctx.Context, schema, bindingSQL string) (p if !hasParam(stmt) { // if there is '?' from `create binding using select a from t where a=?`, // the final plan digest might be incorrect. - planDigest, _ = PlanDigestFunc(sctx, stmt) + planDigest, _ = CalculatePlanDigest(sctx, stmt) } } return diff --git a/pkg/executor/select.go b/pkg/executor/select.go index b55ce7b8ae..98eb552dc6 100644 --- a/pkg/executor/select.go +++ b/pkg/executor/select.go @@ -955,6 +955,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.DiskTracker.Killer = &vars.SQLKiller vars.SQLKiller.Reset() vars.SQLKiller.ConnID.Store(vars.ConnectionID) + vars.ResetRelevantOptVarsAndFixes(false) isAnalyze := false if execStmt, ok := s.(*ast.ExecuteStmt); ok { diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel index e8c9776a1f..75630e3306 100644 --- a/pkg/planner/cardinality/BUILD.bazel +++ b/pkg/planner/cardinality/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/planner/util/debugtrace", "//pkg/planner/util/fixcontrol", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/tablecodec", "//pkg/types", diff --git a/pkg/planner/cardinality/cross_estimation.go b/pkg/planner/cardinality/cross_estimation.go index 1bb8df3527..bfa902d740 100644 --- a/pkg/planner/cardinality/cross_estimation.go +++ b/pkg/planner/cardinality/cross_estimation.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/ranger" @@ -95,6 +96,7 @@ func AdjustRowCountForIndexScanByLimit(sctx planctx.PlanContext, // This formula is to bias away from non-filtering (or poorly filtering) indexes that provide order due, where filtering exists // outside of that index. Such plans have high risk since we cannot estimate when rows will be found. orderRatio := sctx.GetSessionVars().OptOrderingIdxSelRatio + sctx.GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptOrderingIdxSelRatio) if dsStatsInfo.RowCount < path.CountAfterAccess && orderRatio >= 0 { rowsToMeetFirst := (((path.CountAfterAccess - path.CountAfterIndex) * orderRatio) + (path.CountAfterIndex - dsStatsInfo.RowCount)) * orderRatio rowCount = rowsToMeetFirst + expectedCnt diff --git a/pkg/planner/cardinality/row_count_index.go b/pkg/planner/cardinality/row_count_index.go index 297894ca78..517fd3d9e7 100644 --- a/pkg/planner/cardinality/row_count_index.go +++ b/pkg/planner/cardinality/row_count_index.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -440,6 +441,7 @@ func equalRowCountOnIndex(sctx planctx.PlanContext, idx *statistics.Index, b []b skewEstimate := float64(0) // skewRatio determines how much of the potential skew should be considered skewRatio := sctx.GetSessionVars().RiskEqSkewRatio + sctx.GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptRiskEqSkewRatio) if skewRatio > 0 { // Calculate the worst case selectivity assuming the value is skewed within the remaining values not in TopN. skewEstimate = idx.Histogram.NotNullCount() - (histNDV - 1) diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index a0029488cd..3d278eafff 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -1798,6 +1798,7 @@ func constructDS2IndexScanTask( // We can calculate the lower bound of the NDV therefore we can get an upper bound of the row count here. rowCountUpperBound := -1.0 fixControlOK := fixcontrol.GetBoolWithDefault(ds.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix44855, false) + ds.SCtx().GetSessionVars().RecordRelevantOptFix(fixcontrol.Fix44855) if fixControlOK && ds.TableStats != nil { usedColIDs := make([]int64, 0) // We only consider columns in this index that (1) are used to probe as join key, diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 028b17822a..d6d48d7a9a 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -871,6 +871,7 @@ func compareCandidates(sctx base.PlanContext, statsTbl *statistics.Table, tableI len(lhs.path.PartialIndexPaths) == 0 && len(rhs.path.PartialIndexPaths) == 0 && // not IndexMerge since its row count estimation is not accurate enough prop.ExpectedCnt == math.MaxFloat64 { // Limit may affect access row count threshold := float64(fixcontrol.GetIntWithDefault(sctx.GetSessionVars().OptimizerFixControl, fixcontrol.Fix45132, 1000)) + sctx.GetSessionVars().RecordRelevantOptFix(fixcontrol.Fix45132) if threshold > 0 { // set it to 0 to disable this rule // corrResult is included to ensure we don't preference to a higher risk plan given that // this rule does not check the other criteria included below. @@ -1321,6 +1322,7 @@ func skylinePruning(ds *logicalop.DataSource, prop *property.PhysicalProperty) [ fixcontrol.Fix52869, false, ) + ds.SCtx().GetSessionVars().RecordRelevantOptFix(fixcontrol.Fix52869) if preferRange { // Override preferRange with the following limitations to scope preferRange = preferMerge || idxMissingStats || ds.TableStats.HistColl.Pseudo || ds.TableStats.RowCount < 1 diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index 134f406546..ea8f8346dc 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -533,6 +533,7 @@ func generateOtherIndexMerge(ds *logicalop.DataSource, regularPathCount int, ind fixcontrol.Fix52869, false, ) + ds.SCtx().GetSessionVars().RecordRelevantOptFix(fixcontrol.Fix52869) if !skipRangeScanCheck { for i := 1; i < len(ds.PossibleAccessPaths); i++ { if len(ds.PossibleAccessPaths[i].AccessConds) != 0 { diff --git a/pkg/planner/core/plan_cost_ver2.go b/pkg/planner/core/plan_cost_ver2.go index 7aca6975c2..73cdd9c773 100644 --- a/pkg/planner/core/plan_cost_ver2.go +++ b/pkg/planner/core/plan_cost_ver2.go @@ -133,6 +133,7 @@ func (p *PhysicalIndexScan) GetPlanCostVer2(taskType property.TaskType, option * p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexScanCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptIndexScanCostFactor) return p.PlanCostVer2, nil } @@ -202,15 +203,19 @@ func (p *PhysicalTableScan) GetPlanCostVer2(taskType property.TaskType, option * if p.isChildOfIndexLookUp { // This is a RowID table scan (child of IndexLookUp) p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableRowIDScanCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTableRowIDScanCostFactor) } else if !hasFullRangeScan { // This is a table range scan (predicate exists on the PK) p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableRangeScanCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTableRangeScanCostFactor) } else { // This is a table full scan if p.StoreType == kv.TiFlash { p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableTiFlashScanCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTableTiFlashScanCostFactor) } else { p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableFullScanCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTableFullScanCostFactor) } } return p.PlanCostVer2, nil @@ -240,6 +245,7 @@ func (p *PhysicalIndexReader) GetPlanCostVer2(taskType property.TaskType, option p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexReaderCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptIndexReaderCostFactor) return p.PlanCostVer2, nil } @@ -277,6 +283,7 @@ func (p *PhysicalTableReader) GetPlanCostVer2(taskType property.TaskType, option } // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableReaderCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTableReaderCostFactor) return p.PlanCostVer2, nil } @@ -341,9 +348,11 @@ func (p *PhysicalIndexLookUpReader) GetPlanCostVer2(taskType property.TaskType, if p.PushedLimit != nil && tableRows <= float64(p.PushedLimit.Count) { // Multiply by limit cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().LimitCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptLimitCostFactor) } // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexLookupCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptIndexLookupCostFactor) return p.PlanCostVer2, nil } @@ -403,10 +412,12 @@ func (p *PhysicalIndexMergeReader) GetPlanCostVer2(taskType property.TaskType, o p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, 0.99) // Multiply by limit cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().LimitCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptLimitCostFactor) } p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexMergeCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptIndexMergeCostFactor) return p.PlanCostVer2, nil } @@ -462,6 +473,7 @@ func (p *PhysicalSort) GetPlanCostVer2(taskType property.TaskType, option *optim p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().SortCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptSortCostFactor) return p.PlanCostVer2, nil } @@ -499,6 +511,7 @@ func (p *PhysicalTopN) GetPlanCostVer2(taskType property.TaskType, option *optim p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TopNCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptTopNCostFactor) return p.PlanCostVer2, nil } @@ -524,6 +537,7 @@ func (p *PhysicalStreamAgg) GetPlanCostVer2(taskType property.TaskType, option * p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().StreamAggCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptStreamAggCostFactor) return p.PlanCostVer2, nil } @@ -558,6 +572,7 @@ func (p *PhysicalHashAgg) GetPlanCostVer2(taskType property.TaskType, option *op p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().HashAggCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptHashAggCostFactor) return p.PlanCostVer2, nil } @@ -590,6 +605,7 @@ func (p *PhysicalMergeJoin) GetPlanCostVer2(taskType property.TaskType, option * p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().MergeJoinCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptMergeJoinCostFactor) return p.PlanCostVer2, nil } @@ -645,6 +661,7 @@ func (p *PhysicalHashJoin) GetPlanCostVer2(taskType property.TaskType, option *o p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().HashJoinCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptHashJoinCostFactor) return p.PlanCostVer2, nil } @@ -714,6 +731,7 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt p.PlanCostInit = true // Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexJoinCostFactor) + p.SCtx().GetSessionVars().RecordRelevantOptVar(vardef.TiDBOptIndexJoinCostFactor) return p.PlanCostVer2, nil } diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index bef3651960..92cde80501 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -18,6 +18,7 @@ import ( "context" "math" "math/rand" + "sort" "sync" "time" @@ -631,7 +632,7 @@ func queryPlanCost(sctx sessionctx.Context, stmt ast.StmtNode) (float64, error) return core.GetPlanCost(pp, property.RootTaskType, optimizetrace.NewDefaultPlanCostOption()) } -func planDigestFunc(sctx sessionctx.Context, stmt ast.StmtNode) (planDigest string, err error) { +func calculatePlanDigestFunc(sctx sessionctx.Context, stmt ast.StmtNode) (planDigest string, err error) { ret := &core.PreprocessorReturn{} nodeW := resolve.NewNodeW(stmt) err = core.Preprocess( @@ -654,6 +655,41 @@ func planDigestFunc(sctx sessionctx.Context, stmt ast.StmtNode) (planDigest stri return digest.String(), nil } +func recordRelevantOptVarsAndFixes(sctx sessionctx.Context, stmt ast.StmtNode) (varNames []string, fixIDs []uint64, err error) { + sctx.GetSessionVars().ResetRelevantOptVarsAndFixes(true) + defer sctx.GetSessionVars().ResetRelevantOptVarsAndFixes(false) + ret := &core.PreprocessorReturn{} + nodeW := resolve.NewNodeW(stmt) + err = core.Preprocess( + context.Background(), + sctx, + nodeW, + core.WithPreprocessorReturn(ret), + core.InitTxnContextProvider, + ) + if err != nil { + return nil, nil, err + } + + _, _, err = Optimize(context.Background(), sctx, nodeW, sctx.GetDomainInfoSchema().(infoschema.InfoSchema)) + if err != nil { + return nil, nil, err + } + + for varName := range sctx.GetSessionVars().RelevantOptVars { + varNames = append(varNames, varName) + } + sort.Strings(varNames) + + for fixID := range sctx.GetSessionVars().RelevantOptFixes { + fixIDs = append(fixIDs, fixID) + } + sort.Slice(fixIDs, func(i, j int) bool { + return fixIDs[i] < fixIDs[j] + }) + return +} + func init() { core.OptimizeAstNode = Optimize core.IsReadOnly = IsReadOnly @@ -661,5 +697,6 @@ func init() { bindinfo.GetBindingHandle = func(sctx sessionctx.Context) bindinfo.BindingHandle { return domain.GetDomain(sctx).BindingHandle() } - bindinfo.PlanDigestFunc = planDigestFunc + bindinfo.CalculatePlanDigest = calculatePlanDigestFunc + bindinfo.RecordRelevantOptVarsAndFixes = recordRelevantOptVarsAndFixes } diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index 0f1f2cdf82..a678ada6a9 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -1640,6 +1640,15 @@ type SessionVars struct { // 0 > value <= 1 applies that percentage as the estimate when rows are found. For example 0.1 = 10%. OptOrderingIdxSelRatio float64 + // RecordRelevantOptVarsAndFixes indicates whether to record optimizer variables/fixes relevant to this query. + RecordRelevantOptVarsAndFixes bool + + // RelevantOptVars is a map of relevant optimizer variables to be recorded. + RelevantOptVars map[string]struct{} + + // RelevantOptFixes is a map of relevant optimizer fixes to be recorded. + RelevantOptFixes map[uint64]struct{} + // EnableMPPSharedCTEExecution indicates whether we enable the shared CTE execution strategy on MPP side. EnableMPPSharedCTEExecution bool @@ -1723,6 +1732,35 @@ type SessionVars struct { BulkDMLEnabled bool } +// ResetRelevantOptVarsAndFixes resets the relevant optimizer variables and fixes. +func (s *SessionVars) ResetRelevantOptVarsAndFixes(record bool) { + s.RecordRelevantOptVarsAndFixes = record + s.RelevantOptVars = nil + s.RelevantOptFixes = nil +} + +// RecordRelevantOptVar records the optimizer variable that is relevant to the current query. +func (s *SessionVars) RecordRelevantOptVar(varName string) { + if !s.RecordRelevantOptVarsAndFixes { + return + } + if s.RelevantOptVars == nil { + s.RelevantOptVars = make(map[string]struct{}) + } + s.RelevantOptVars[varName] = struct{}{} +} + +// RecordRelevantOptFix records the optimizer fix that is relevant to the current query. +func (s *SessionVars) RecordRelevantOptFix(fixID uint64) { + if !s.RecordRelevantOptVarsAndFixes { + return + } + if s.RelevantOptFixes == nil { + s.RelevantOptFixes = make(map[uint64]struct{}) + } + s.RelevantOptFixes[fixID] = struct{}{} +} + // GetSessionVars implements the `SessionVarsProvider` interface. func (s *SessionVars) GetSessionVars() *SessionVars { return s