planner: a new function to get all relevant optimizer variables related to a certain SQL for SPM (#61119)
ref pingcap/tidb#60148
This commit is contained in:
@ -59,7 +59,7 @@ go_test(
|
||||
embed = [":bindinfo"],
|
||||
flaky = True,
|
||||
race = "on",
|
||||
shard_count = 35,
|
||||
shard_count = 36,
|
||||
deps = [
|
||||
"//pkg/parser",
|
||||
"//pkg/parser/ast",
|
||||
|
||||
@ -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 <SQL>" to help users understand the historical plans for a specific SQL.
|
||||
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
Reference in New Issue
Block a user