Skip to content

Commit 61f5903

Browse files
committed
planner: add optimizer cost factors
1 parent ec6b882 commit 61f5903

File tree

4 files changed

+196
-11
lines changed

4 files changed

+196
-11
lines changed

pkg/planner/core/plan_cost_ver2.go

Lines changed: 55 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -131,6 +131,8 @@ func (p *PhysicalIndexScan) GetPlanCostVer2(taskType property.TaskType, option *
131131

132132
p.PlanCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor)
133133
p.PlanCostInit = true
134+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
135+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexScanCostFactor)
134136
return p.PlanCostVer2, nil
135137
}
136138

@@ -185,6 +187,30 @@ func (p *PhysicalTableScan) GetPlanCostVer2(taskType property.TaskType, option *
185187
}
186188

187189
p.PlanCostInit = true
190+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
191+
if p.isChildOfIndexLookUp {
192+
// This is a RowID table scan (child of IndexLookUp)
193+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableRowIDScanCostFactor)
194+
} else {
195+
var unsignedIntHandle bool
196+
if p.Table.PKIsHandle {
197+
if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil {
198+
unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag())
199+
}
200+
}
201+
hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle)
202+
if !hasFullRangeScan {
203+
// This is a table range scan (predicate exists on the PK)
204+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableRangeScanCostFactor)
205+
} else {
206+
// This is a table full scan
207+
if p.StoreType == kv.TiFlash {
208+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableTiFlashScanCostFactor)
209+
} else {
210+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableFullScanCostFactor)
211+
}
212+
}
213+
}
188214
return p.PlanCostVer2, nil
189215
}
190216

@@ -210,6 +236,8 @@ func (p *PhysicalIndexReader) GetPlanCostVer2(taskType property.TaskType, option
210236

211237
p.PlanCostVer2 = costusage.DivCostVer2(costusage.SumCostVer2(childCost, netCost), concurrency)
212238
p.PlanCostInit = true
239+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
240+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexReaderCostFactor)
213241
return p.PlanCostVer2, nil
214242
}
215243

@@ -245,6 +273,8 @@ func (p *PhysicalTableReader) GetPlanCostVer2(taskType property.TaskType, option
245273
!hasCostFlag(option.CostFlag, costusage.CostFlagRecalculate) { // show the real cost in explain-statements
246274
p.PlanCostVer2 = costusage.DivCostVer2(p.PlanCostVer2, 1000000000)
247275
}
276+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
277+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TableReaderCostFactor)
248278
return p.PlanCostVer2, nil
249279
}
250280

@@ -306,6 +336,12 @@ func (p *PhysicalIndexLookUpReader) GetPlanCostVer2(taskType property.TaskType,
306336
}
307337

308338
p.PlanCostInit = true
339+
if p.PushedLimit != nil && tableRows <= float64(p.PushedLimit.Count) {
340+
// Multiply by limit cost factor - defaults to 1, but can be increased/decreased to influence the cost model
341+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().LimitCostFactor)
342+
}
343+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
344+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexLookupCostFactor)
309345
return p.PlanCostVer2, nil
310346
}
311347

@@ -363,8 +399,12 @@ func (p *PhysicalIndexMergeReader) GetPlanCostVer2(taskType property.TaskType, o
363399
// todo: refine the cost computation out from cost model.
364400
if p.PushedLimit != nil {
365401
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, 0.99)
402+
// Multiply by limit cost factor - defaults to 1, but can be increased/decreased to influence the cost model
403+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().LimitCostFactor)
366404
}
367405
p.PlanCostInit = true
406+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
407+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexMergeCostFactor)
368408
return p.PlanCostVer2, nil
369409
}
370410

@@ -418,6 +458,8 @@ func (p *PhysicalSort) GetPlanCostVer2(taskType property.TaskType, option *optim
418458

419459
p.PlanCostVer2 = costusage.SumCostVer2(childCost, sortCPUCost, sortMemCost, sortDiskCost)
420460
p.PlanCostInit = true
461+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
462+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().SortCostFactor)
421463
return p.PlanCostVer2, nil
422464
}
423465

@@ -453,6 +495,8 @@ func (p *PhysicalTopN) GetPlanCostVer2(taskType property.TaskType, option *optim
453495

454496
p.PlanCostVer2 = costusage.SumCostVer2(childCost, topNCPUCost, topNMemCost)
455497
p.PlanCostInit = true
498+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
499+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().TopNCostFactor)
456500
return p.PlanCostVer2, nil
457501
}
458502

@@ -476,6 +520,8 @@ func (p *PhysicalStreamAgg) GetPlanCostVer2(taskType property.TaskType, option *
476520

477521
p.PlanCostVer2 = costusage.SumCostVer2(childCost, aggCost, groupCost)
478522
p.PlanCostInit = true
523+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
524+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().StreamAggCostFactor)
479525
return p.PlanCostVer2, nil
480526
}
481527

@@ -508,6 +554,8 @@ func (p *PhysicalHashAgg) GetPlanCostVer2(taskType property.TaskType, option *op
508554

509555
p.PlanCostVer2 = costusage.SumCostVer2(startCost, childCost, costusage.DivCostVer2(costusage.SumCostVer2(aggCost, groupCost, hashBuildCost, hashProbeCost), concurrency))
510556
p.PlanCostInit = true
557+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
558+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().HashAggCostFactor)
511559
return p.PlanCostVer2, nil
512560
}
513561

@@ -538,6 +586,8 @@ func (p *PhysicalMergeJoin) GetPlanCostVer2(taskType property.TaskType, option *
538586

539587
p.PlanCostVer2 = costusage.SumCostVer2(leftChildCost, rightChildCost, filterCost, groupCost)
540588
p.PlanCostInit = true
589+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
590+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().MergeJoinCostFactor)
541591
return p.PlanCostVer2, nil
542592
}
543593

@@ -591,6 +641,8 @@ func (p *PhysicalHashJoin) GetPlanCostVer2(taskType property.TaskType, option *o
591641
costusage.DivCostVer2(costusage.SumCostVer2(probeFilterCost, probeHashCost), tidbConcurrency))
592642
}
593643
p.PlanCostInit = true
644+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
645+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().HashJoinCostFactor)
594646
return p.PlanCostVer2, nil
595647
}
596648

@@ -658,6 +710,8 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt
658710

659711
p.PlanCostVer2 = costusage.SumCostVer2(startCost, buildChildCost, buildFilterCost, buildTaskCost, costusage.DivCostVer2(costusage.SumCostVer2(doubleReadCost, probeCost, probeFilterCost, hashTableCost), probeConcurrency))
660712
p.PlanCostInit = true
713+
// Multiply by cost factor - defaults to 1, but can be increased/decreased to influence the cost model
714+
p.PlanCostVer2 = costusage.MulCostVer2(p.PlanCostVer2, p.SCtx().GetSessionVars().IndexJoinCostFactor)
661715
return p.PlanCostVer2, nil
662716
}
663717

@@ -919,16 +973,6 @@ func getTableScanPenalty(p *PhysicalTableScan, rows float64) (rowPenalty float64
919973
if len(p.rangeInfo) > 0 {
920974
return float64(0)
921975
}
922-
var unsignedIntHandle bool
923-
if p.Table.PKIsHandle {
924-
if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil {
925-
unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag())
926-
}
927-
}
928-
hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle)
929-
if !hasFullRangeScan {
930-
return float64(0)
931-
}
932976

933977
sessionVars := p.SCtx().GetSessionVars()
934978
allowPreferRangeScan := sessionVars.GetAllowPreferRangeScan()
@@ -956,7 +1000,7 @@ func getTableScanPenalty(p *PhysicalTableScan, rows float64) (rowPenalty float64
9561000
// penalty is applied to a full table scan (not range scan). This may also penalize a
9571001
// full table scan where USE/FORCE was applied to the primary key.
9581002
hasIndexForce := sessionVars.StmtCtx.GetIndexForce()
959-
shouldApplyPenalty := hasFullRangeScan && (hasIndexForce || preferRangeScanCondition)
1003+
shouldApplyPenalty := hasIndexForce || preferRangeScanCondition
9601004
if shouldApplyPenalty {
9611005
// MySQL will increase the cost of table scan if FORCE index is used. TiDB takes this one
9621006
// step further - because we don't differentiate USE/FORCE - the added penalty applies to

pkg/sessionctx/variable/session.go

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1021,6 +1021,25 @@ type SessionVars struct {
10211021
// concurrencyFactor is the CPU cost of additional one goroutine.
10221022
concurrencyFactor float64
10231023

1024+
// Optimizer cost model factors for each physical operator
1025+
IndexScanCostFactor float64
1026+
IndexReaderCostFactor float64
1027+
TableReaderCostFactor float64
1028+
TableFullScanCostFactor float64
1029+
TableRangeScanCostFactor float64
1030+
TableRowIDScanCostFactor float64
1031+
TableTiFlashScanCostFactor float64
1032+
IndexLookupCostFactor float64
1033+
IndexMergeCostFactor float64
1034+
SortCostFactor float64
1035+
TopNCostFactor float64
1036+
LimitCostFactor float64
1037+
StreamAggCostFactor float64
1038+
HashAggCostFactor float64
1039+
MergeJoinCostFactor float64
1040+
HashJoinCostFactor float64
1041+
IndexJoinCostFactor float64
1042+
10241043
// enableForceInlineCTE is used to enable/disable force inline CTE.
10251044
enableForceInlineCTE bool
10261045

@@ -2126,6 +2145,23 @@ func NewSessionVars(hctx HookContext) *SessionVars {
21262145
memoryFactor: DefOptMemoryFactor,
21272146
diskFactor: DefOptDiskFactor,
21282147
concurrencyFactor: DefOptConcurrencyFactor,
2148+
IndexScanCostFactor: DefOptIndexScanCostFactor,
2149+
IndexReaderCostFactor: DefOptIndexReaderCostFactor,
2150+
TableReaderCostFactor: DefOptTableReaderCostFactor,
2151+
TableFullScanCostFactor: DefOptTableFullScanCostFactor,
2152+
TableRangeScanCostFactor: DefOptTableRangeScanCostFactor,
2153+
TableRowIDScanCostFactor: DefOptTableRowIDScanCostFactor,
2154+
TableTiFlashScanCostFactor: DefOptTableTiFlashScanCostFactor,
2155+
IndexLookupCostFactor: DefOptIndexLookupCostFactor,
2156+
IndexMergeCostFactor: DefOptIndexMergeCostFactor,
2157+
SortCostFactor: DefOptSortCostFactor,
2158+
TopNCostFactor: DefOptTopNCostFactor,
2159+
LimitCostFactor: DefOptLimitCostFactor,
2160+
StreamAggCostFactor: DefOptStreamAggCostFactor,
2161+
HashAggCostFactor: DefOptHashAggCostFactor,
2162+
MergeJoinCostFactor: DefOptMergeJoinCostFactor,
2163+
HashJoinCostFactor: DefOptHashJoinCostFactor,
2164+
IndexJoinCostFactor: DefOptIndexJoinCostFactor,
21292165
enableForceInlineCTE: DefOptForceInlineCTE,
21302166
EnableVectorizedExpression: DefEnableVectorizedExpression,
21312167
CommandValue: uint32(mysql.ComSleep),

pkg/sessionctx/variable/sysvar.go

Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2054,6 +2054,74 @@ var defaultSysVars = []*SysVar{
20542054
s.concurrencyFactor = tidbOptFloat64(val, DefOptConcurrencyFactor)
20552055
return nil
20562056
}},
2057+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptIndexScanCostFactor, Value: strconv.FormatFloat(DefOptIndexScanCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2058+
s.IndexScanCostFactor = tidbOptFloat64(val, DefOptIndexScanCostFactor)
2059+
return nil
2060+
}},
2061+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptIndexReaderCostFactor, Value: strconv.FormatFloat(DefOptIndexReaderCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2062+
s.IndexReaderCostFactor = tidbOptFloat64(val, DefOptIndexReaderCostFactor)
2063+
return nil
2064+
}},
2065+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTableReaderCostFactor, Value: strconv.FormatFloat(DefOptTableReaderCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2066+
s.TableReaderCostFactor = tidbOptFloat64(val, DefOptTableReaderCostFactor)
2067+
return nil
2068+
}},
2069+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTableFullScanCostFactor, Value: strconv.FormatFloat(DefOptTableFullScanCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2070+
s.TableFullScanCostFactor = tidbOptFloat64(val, DefOptTableFullScanCostFactor)
2071+
return nil
2072+
}},
2073+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTableRangeScanCostFactor, Value: strconv.FormatFloat(DefOptTableRangeScanCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2074+
s.TableRangeScanCostFactor = tidbOptFloat64(val, DefOptTableRangeScanCostFactor)
2075+
return nil
2076+
}},
2077+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTableRowIDScanCostFactor, Value: strconv.FormatFloat(DefOptTableRowIDScanCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2078+
s.TableRowIDScanCostFactor = tidbOptFloat64(val, DefOptTableRowIDScanCostFactor)
2079+
return nil
2080+
}},
2081+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTableTiFlashScanCostFactor, Value: strconv.FormatFloat(DefOptTableTiFlashScanCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2082+
s.TableTiFlashScanCostFactor = tidbOptFloat64(val, DefOptTableTiFlashScanCostFactor)
2083+
return nil
2084+
}},
2085+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptIndexLookupCostFactor, Value: strconv.FormatFloat(DefOptIndexLookupCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2086+
s.IndexLookupCostFactor = tidbOptFloat64(val, DefOptIndexLookupCostFactor)
2087+
return nil
2088+
}},
2089+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptIndexMergeCostFactor, Value: strconv.FormatFloat(DefOptIndexMergeCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2090+
s.IndexMergeCostFactor = tidbOptFloat64(val, DefOptIndexMergeCostFactor)
2091+
return nil
2092+
}},
2093+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSortCostFactor, Value: strconv.FormatFloat(DefOptSortCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2094+
s.SortCostFactor = tidbOptFloat64(val, DefOptSortCostFactor)
2095+
return nil
2096+
}},
2097+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTopNCostFactor, Value: strconv.FormatFloat(DefOptTopNCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2098+
s.TopNCostFactor = tidbOptFloat64(val, DefOptTopNCostFactor)
2099+
return nil
2100+
}},
2101+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptLimitCostFactor, Value: strconv.FormatFloat(DefOptLimitCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2102+
s.LimitCostFactor = tidbOptFloat64(val, DefOptLimitCostFactor)
2103+
return nil
2104+
}},
2105+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptStreamAggCostFactor, Value: strconv.FormatFloat(DefOptStreamAggCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2106+
s.StreamAggCostFactor = tidbOptFloat64(val, DefOptStreamAggCostFactor)
2107+
return nil
2108+
}},
2109+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptHashAggCostFactor, Value: strconv.FormatFloat(DefOptHashAggCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2110+
s.HashAggCostFactor = tidbOptFloat64(val, DefOptHashAggCostFactor)
2111+
return nil
2112+
}},
2113+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMergeJoinCostFactor, Value: strconv.FormatFloat(DefOptMergeJoinCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2114+
s.MergeJoinCostFactor = tidbOptFloat64(val, DefOptMergeJoinCostFactor)
2115+
return nil
2116+
}},
2117+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptHashJoinCostFactor, Value: strconv.FormatFloat(DefOptHashJoinCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2118+
s.HashJoinCostFactor = tidbOptFloat64(val, DefOptHashJoinCostFactor)
2119+
return nil
2120+
}},
2121+
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptIndexJoinCostFactor, Value: strconv.FormatFloat(DefOptIndexJoinCostFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
2122+
s.IndexJoinCostFactor = tidbOptFloat64(val, DefOptIndexJoinCostFactor)
2123+
return nil
2124+
}},
20572125
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptForceInlineCTE, Value: BoolToOnOff(DefOptForceInlineCTE), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
20582126
s.enableForceInlineCTE = TiDBOptOn(val)
20592127
return nil

pkg/sessionctx/variable/tidb_vars.go

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -350,6 +350,26 @@ const (
350350
// TiDBOptForceInlineCTE is used to enable/disable inline CTE
351351
TiDBOptForceInlineCTE = "tidb_opt_force_inline_cte"
352352

353+
// The following optimizer cost factors represent a multiplier for each optimizer physical operator.
354+
// These factors are used to adjust the cost of each operator to influence the optimizer's plan selection.
355+
TiDBOptIndexScanCostFactor = "tidb_opt_index_scan_cost_factor"
356+
TiDBOptIndexReaderCostFactor = "tidb_opt_index_reader_cost_factor"
357+
TiDBOptTableReaderCostFactor = "tidb_opt_table_reader_cost_factor"
358+
TiDBOptTableFullScanCostFactor = "tidb_opt_table_full_scan_cost_factor"
359+
TiDBOptTableRangeScanCostFactor = "tidb_opt_table_range_scan_cost_factor"
360+
TiDBOptTableRowIDScanCostFactor = "tidb_opt_table_rowid_scan_cost_factor"
361+
TiDBOptTableTiFlashScanCostFactor = "tidb_opt_table_tiflash_scan_cost_factor"
362+
TiDBOptIndexLookupCostFactor = "tidb_opt_index_lookup_cost_factor"
363+
TiDBOptIndexMergeCostFactor = "tidb_opt_index_merge_cost_factor"
364+
TiDBOptSortCostFactor = "tidb_opt_sort_cost_factor"
365+
TiDBOptTopNCostFactor = "tidb_opt_topn_cost_factor"
366+
TiDBOptLimitCostFactor = "tidb_opt_limit_cost_factor"
367+
TiDBOptStreamAggCostFactor = "tidb_opt_stream_agg_cost_factor"
368+
TiDBOptHashAggCostFactor = "tidb_opt_hash_agg_cost_factor"
369+
TiDBOptMergeJoinCostFactor = "tidb_opt_merge_join_cost_factor"
370+
TiDBOptHashJoinCostFactor = "tidb_opt_hash_join_cost_factor"
371+
TiDBOptIndexJoinCostFactor = "tidb_opt_index_join_cost_factor"
372+
353373
// TiDBIndexJoinBatchSize is used to set the batch size of an index lookup join.
354374
// The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor.
355375
// This value controls how much of data in a batch to do the index join.
@@ -1274,6 +1294,23 @@ const (
12741294
DefOptMemoryFactor = 0.001
12751295
DefOptDiskFactor = 1.5
12761296
DefOptConcurrencyFactor = 3.0
1297+
DefOptIndexScanCostFactor = 1.0
1298+
DefOptIndexReaderCostFactor = 1.0
1299+
DefOptTableReaderCostFactor = 1.0
1300+
DefOptTableFullScanCostFactor = 1.0
1301+
DefOptTableRangeScanCostFactor = 1.0
1302+
DefOptTableRowIDScanCostFactor = 1.0
1303+
DefOptTableTiFlashScanCostFactor = 1.0
1304+
DefOptIndexLookupCostFactor = 1.0
1305+
DefOptIndexMergeCostFactor = 1.0
1306+
DefOptSortCostFactor = 1.0
1307+
DefOptTopNCostFactor = 1.0
1308+
DefOptLimitCostFactor = 1.0
1309+
DefOptStreamAggCostFactor = 1.0
1310+
DefOptHashAggCostFactor = 1.0
1311+
DefOptMergeJoinCostFactor = 1.0
1312+
DefOptHashJoinCostFactor = 1.0
1313+
DefOptIndexJoinCostFactor = 1.0
12771314
DefOptForceInlineCTE = false
12781315
DefOptInSubqToJoinAndAgg = true
12791316
DefOptPreferRangeScan = true

0 commit comments

Comments
 (0)