Skip to content

Commit

Permalink
planner: support cost model for Chunk encode format. (pingcap#13461)
Browse files Browse the repository at this point in the history
  • Loading branch information
wshwsh12 authored and sre-bot committed Dec 18, 2019
1 parent dddb85c commit 02408db
Show file tree
Hide file tree
Showing 10 changed files with 77 additions and 35 deletions.
11 changes: 6 additions & 5 deletions cmd/explaintest/r/explain_complex_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -160,14 +160,15 @@ explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as
id count task operator info
Projection_10 428.32 root test.dt.id, test.dt.aid, test.dt.pt, test.dt.dic, test.dt.cm, test.rr.gid, test.rr.acd, test.rr.t, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5
└─Limit_13 428.32 root offset:0, count:2000
└─IndexJoin_19 428.32 root inner join, inner:IndexLookUp_18, outer key:test.dt.aid, test.dt.dic, inner key:test.rr.aid, test.rr.dic
└─IndexMergeJoin_26 428.32 root inner join, inner:Projection_24, outer key:test.dt.aid, test.dt.dic, inner key:test.rr.aid, test.rr.dic
├─TableReader_58 428.32 root data:Selection_57
│ └─Selection_57 428.32 cop[tikv] eq(test.dt.bm, 0), eq(test.dt.pt, "ios"), gt(test.dt.t, 1478185592), not(isnull(test.dt.dic))
│ └─TableScan_56 2000.00 cop[tikv] table:dt, range:[0,+inf], keep order:false
└─IndexLookUp_18 1.00 root
├─IndexScan_15 1.00 cop[tikv] table:rr, index:aid, dic, range: decided by [eq(test.rr.aid, test.dt.aid) eq(test.rr.dic, test.dt.dic)], keep order:false
└─Selection_17 1.00 cop[tikv] eq(test.rr.pt, "ios"), gt(test.rr.t, 1478185592)
└─TableScan_16 1.00 cop[tikv] table:rr, keep order:false
└─Projection_24 1.00 root test.rr.aid, test.rr.pt, test.rr.dic, test.rr.gid, test.rr.acd, test.rr.t
└─IndexLookUp_23 1.00 root
├─IndexScan_20 1.00 cop[tikv] table:rr, index:aid, dic, range: decided by [eq(test.rr.aid, test.dt.aid) eq(test.rr.dic, test.dt.dic)], keep order:true
└─Selection_22 1.00 cop[tikv] eq(test.rr.pt, "ios"), gt(test.rr.t, 1478185592)
└─TableScan_21 1.00 cop[tikv] table:rr, keep order:false
explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr;
id count task operator info
Projection_5 207.86 root test.pp.pc, test.pp.cr, Column#22, Column#23, Column#24
Expand Down
11 changes: 6 additions & 5 deletions cmd/explaintest/r/topn_push_down.result
Original file line number Diff line number Diff line change
Expand Up @@ -169,16 +169,17 @@ LIMIT 0, 5;
id count task operator info
Projection_12 0.00 root test.te.expect_time
└─TopN_15 0.00 root test.te.expect_time:asc, offset:0, count:5
└─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:test.tr.id, inner key:test.te.trade_id
└─IndexMergeJoin_31 0.00 root inner join, inner:Projection_29, outer key:test.tr.id, inner key:test.te.trade_id
├─IndexLookUp_78 0.00 root
│ ├─Selection_76 0.00 cop[tikv] eq(test.tr.business_type, 18), eq(test.tr.trade_type, 1)
│ │ └─IndexScan_74 10.00 cop[tikv] table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo
│ └─Selection_77 0.00 cop[tikv] eq(test.tr.brand_identy, 32314), eq(test.tr.domain_type, 2)
│ └─TableScan_75 0.00 cop[tikv] table:tr, keep order:false, stats:pseudo
└─IndexLookUp_23 1.25 root
├─IndexScan_20 50.00 cop[tikv] table:te, index:trade_id, range: decided by [eq(test.te.trade_id, test.tr.id)], keep order:false, stats:pseudo
└─Selection_22 1.25 cop[tikv] ge(test.te.expect_time, 2018-04-23 00:00:00.000000), le(test.te.expect_time, 2018-04-23 23:59:59.000000)
└─TableScan_21 50.00 cop[tikv] table:te, keep order:false, stats:pseudo
└─Projection_29 1.25 root test.te.trade_id, test.te.expect_time
└─IndexLookUp_28 1.25 root
├─IndexScan_25 50.00 cop[tikv] table:te, index:trade_id, range: decided by [eq(test.te.trade_id, test.tr.id)], keep order:true, stats:pseudo
└─Selection_27 1.25 cop[tikv] ge(test.te.expect_time, 2018-04-23 00:00:00.000000), le(test.te.expect_time, 2018-04-23 23:59:59.000000)
└─TableScan_26 50.00 cop[tikv] table:te, keep order:false, stats:pseudo
desc select 1 as a from dual order by a limit 1;
id count task operator info
Projection_6 1.00 root 1->Column#1
Expand Down
2 changes: 1 addition & 1 deletion planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -700,7 +700,7 @@ func (p *LogicalJoin) constructInnerTableScanTask(
StatsVersion: ds.stats.StatsVersion,
// Cardinality would not be used in cost computation of IndexJoin, set leave it as default nil.
}
rowSize := ds.TblColHists.GetTableAvgRowSize(ds.TblCols, ts.StoreType, true)
rowSize := ds.TblColHists.GetTableAvgRowSize(p.ctx, ds.TblCols, ts.StoreType, true)
sessVars := ds.ctx.GetSessionVars()
copTask := &copTask{
tablePlan: ts,
Expand Down
12 changes: 6 additions & 6 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,7 +546,7 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty,
rowCount float64,
isCovered bool) {
ts, partialCost, rowCount := ds.getOriginalPhysicalTableScan(prop, path, false)
rowSize := ds.TblColHists.GetAvgRowSize(ds.TblCols, false)
rowSize := ds.TblColHists.GetAvgRowSize(ds.ctx, ds.TblCols, false, false)
sessVars := ds.ctx.GetSessionVars()
if len(ts.filterCondition) > 0 {
selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition, nil)
Expand Down Expand Up @@ -584,7 +584,7 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty,
}
}
}
rowSize := ds.TblColHists.GetTableAvgRowSize(ds.TblCols, ts.StoreType, true)
rowSize := ds.TblColHists.GetTableAvgRowSize(ds.ctx, ds.TblCols, ts.StoreType, true)
partialCost += totalRowCount * rowSize * sessVars.ScanFactor
ts.stats = ds.tableStats.ScaleByExpectCnt(totalRowCount)
if ds.statisticTable.Pseudo {
Expand Down Expand Up @@ -710,9 +710,9 @@ func (is *PhysicalIndexScan) indexScanRowSize(idx *model.IndexInfo, ds *DataSour
scanCols = is.schema.Columns
}
if isForScan {
return ds.TblColHists.GetIndexAvgRowSize(scanCols, is.Index.Unique)
return ds.TblColHists.GetIndexAvgRowSize(is.ctx, scanCols, is.Index.Unique)
}
return ds.TblColHists.GetAvgRowSize(scanCols, true)
return ds.TblColHists.GetAvgRowSize(is.ctx, scanCols, true, false)
}

func (is *PhysicalIndexScan) initSchema(idx *model.IndexInfo, idxExprCols []*expression.Column, isDoubleRead bool) {
Expand Down Expand Up @@ -1109,11 +1109,11 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper
ts.stats = ds.tableStats.ScaleByExpectCnt(rowCount)
var rowSize float64
if ts.StoreType == kv.TiKV {
rowSize = ds.TblColHists.GetTableAvgRowSize(ds.TblCols, ts.StoreType, true)
rowSize = ds.TblColHists.GetTableAvgRowSize(ds.ctx, ds.TblCols, ts.StoreType, true)
} else {
// If `ds.handleCol` is nil, then the schema of tableScan doesn't have handle column.
// This logic can be ensured in column pruning.
rowSize = ds.TblColHists.GetTableAvgRowSize(ts.Schema().Columns, ts.StoreType, ds.handleCol != nil)
rowSize = ds.TblColHists.GetTableAvgRowSize(ds.ctx, ts.Schema().Columns, ts.StoreType, ds.handleCol != nil)
}
sessVars := ds.ctx.GetSessionVars()
cost := rowCount * rowSize * sessVars.ScanFactor
Expand Down
8 changes: 5 additions & 3 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,8 @@ func (t *copTask) finishIndexPlan() {
t.indexPlanFinished = true
sessVars := t.indexPlan.SCtx().GetSessionVars()
// Network cost of transferring rows of index scan to TiDB.
t.cst += cnt * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(t.indexPlan.Schema().Columns, true)
t.cst += cnt * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(t.indexPlan.SCtx(), t.indexPlan.Schema().Columns, true, false)

if t.tablePlan == nil {
return
}
Expand All @@ -138,7 +139,8 @@ func (t *copTask) finishIndexPlan() {
var p PhysicalPlan
for p = t.indexPlan; len(p.Children()) > 0; p = p.Children()[0] {
}
rowSize := t.tblColHists.GetIndexAvgRowSize(t.tblCols, p.(*PhysicalIndexScan).Index.Unique)
rowSize := t.tblColHists.GetIndexAvgRowSize(t.indexPlan.SCtx(), t.tblCols, p.(*PhysicalIndexScan).Index.Unique)

t.cst += cnt * rowSize * sessVars.ScanFactor
}

Expand Down Expand Up @@ -616,7 +618,7 @@ func finishCopTask(ctx sessionctx.Context, task task) task {
t.finishIndexPlan()
// Network cost of transferring rows of table scan to TiDB.
if t.tablePlan != nil {
t.cst += t.count() * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(t.tablePlan.Schema().Columns, false)
t.cst += t.count() * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(ctx, t.tablePlan.Schema().Columns, false, false)
}
t.cst /= copIterWorkers
newTask := &rootTask{
Expand Down
2 changes: 1 addition & 1 deletion planner/core/testdata/analyze_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,7 @@
"IndexReader(Index(t.e)[[NULL,+inf]])->HashAgg",
"IndexReader(Index(t.e)[[-inf,10]]->StreamAgg)->StreamAgg",
"IndexReader(Index(t.e)[[-inf,50]]->StreamAgg)->StreamAgg",
"IndexReader(Index(t.b_c)[[NULL,+inf]]->Sel([gt(test.t.c, 1)]))->HashAgg",
"IndexReader(Index(t.b_c)[[NULL,+inf]]->Sel([gt(test.t.c, 1)])->HashAgg)->HashAgg",
"IndexLookUp(Index(t.e)[[1,1]], Table(t))->HashAgg",
"TableReader(Table(t)->Sel([gt(test.t.e, 1)])->HashAgg)->HashAgg",
"IndexLookUp(Index(t.b)[[-inf,20]], Table(t)->HashAgg)->HashAgg",
Expand Down
8 changes: 4 additions & 4 deletions planner/implementation/datasource.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func NewTableReaderImpl(reader *plannercore.PhysicalTableReader, hists *statisti
// CalcCost calculates the cost of the table reader Implementation.
func (impl *TableReaderImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 {
reader := impl.plan.(*plannercore.PhysicalTableReader)
width := impl.tblColHists.GetAvgRowSize(reader.Schema().Columns, false)
width := impl.tblColHists.GetAvgRowSize(impl.plan.SCtx(), reader.Schema().Columns, false, false)
sessVars := reader.SCtx().GetSessionVars()
networkCost := outCount * sessVars.NetworkFactor * width
// copTasks are run in parallel, to make the estimated cost closer to execution time, we amortize
Expand Down Expand Up @@ -100,7 +100,7 @@ func NewTableScanImpl(ts *plannercore.PhysicalTableScan, cols []*expression.Colu
// CalcCost calculates the cost of the table scan Implementation.
func (impl *TableScanImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 {
ts := impl.plan.(*plannercore.PhysicalTableScan)
width := impl.tblColHists.GetTableAvgRowSize(impl.tblCols, kv.TiKV, true)
width := impl.tblColHists.GetTableAvgRowSize(impl.plan.SCtx(), impl.tblCols, kv.TiKV, true)
sessVars := ts.SCtx().GetSessionVars()
impl.cost = outCount * sessVars.ScanFactor * width
if ts.Desc {
Expand Down Expand Up @@ -130,7 +130,7 @@ func (impl *IndexReaderImpl) ScaleCostLimit(costLimit float64) float64 {
func (impl *IndexReaderImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 {
reader := impl.plan.(*plannercore.PhysicalIndexReader)
sessVars := reader.SCtx().GetSessionVars()
networkCost := outCount * sessVars.NetworkFactor * impl.tblColHists.GetAvgRowSize(children[0].GetPlan().Schema().Columns, true)
networkCost := outCount * sessVars.NetworkFactor * impl.tblColHists.GetAvgRowSize(reader.SCtx(), children[0].GetPlan().Schema().Columns, true, false)
copIterWorkers := float64(sessVars.DistSQLScanConcurrency)
impl.cost = (networkCost + children[0].GetCost()) / copIterWorkers
return impl.cost
Expand All @@ -154,7 +154,7 @@ type IndexScanImpl struct {
func (impl *IndexScanImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 {
is := impl.plan.(*plannercore.PhysicalIndexScan)
sessVars := is.SCtx().GetSessionVars()
rowSize := impl.tblColHists.GetIndexAvgRowSize(is.Schema().Columns, is.Index.Unique)
rowSize := impl.tblColHists.GetIndexAvgRowSize(is.SCtx(), is.Schema().Columns, is.Index.Unique)
cost := outCount * rowSize * sessVars.ScanFactor
if is.Desc {
cost = outCount * rowSize * sessVars.DescScanFactor
Expand Down
13 changes: 11 additions & 2 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package handle_test

import (
"fmt"
"math"
"testing"
"time"
"unsafe"
Expand Down Expand Up @@ -211,14 +212,18 @@ func (s *testStatsSuite) TestAvgColLen(c *C) {
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0)

// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0-1)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0-3)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3))))
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.Time{})))
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0-3+8)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3))))
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.Time{})))
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29')")
testKit.MustExec("analyze table t")
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
Expand All @@ -227,9 +232,13 @@ func (s *testStatsSuite) TestAvgColLen(c *C) {
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 10.5-1)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, math.Round((10.5-math.Log2(10.5))*100)/100)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3))))
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.Time{})))
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, math.Round((10.5-math.Log2(10.5))*100)/100+8)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3))))
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.Time{})))
}

func (s *testStatsSuite) TestDurationToTS(c *C) {
Expand Down
23 changes: 21 additions & 2 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ type Histogram struct {
// For some types like `Int`, we do not build it because we can get them directly from `Bounds`.
scalars []scalar
// TotColSize is the total column size for the histogram.
// For unfixed-len types, it includes LEN and BYTE.
TotColSize int64

// Correlation is the statistical correlation between physical row ordering and logical ordering of
Expand Down Expand Up @@ -140,6 +141,23 @@ func (c *Column) AvgColSize(count int64, isKey bool) float64 {
return math.Round(float64(c.TotColSize)/float64(count)*100) / 100
}

// AvgColSizeChunkFormat is the average column size of the histogram. These sizes are derived from function `Encode`
// and `DecodeToChunk`, so we need to update them if those 2 functions are changed.
func (c *Column) AvgColSizeChunkFormat(count int64) float64 {
if count == 0 {
return 0
}
fixedLen := chunk.GetFixedLen(c.Histogram.Tp)
if fixedLen != -1 {
return float64(fixedLen)
}
// Keep two decimal place.
// Add 8 bytes for unfixed-len type's offsets.
// Minus Log2(avgSize) for unfixed-len type LEN.
avgSize := float64(c.TotColSize) / float64(count)
return math.Round((avgSize-math.Log2(avgSize))*100)/100 + 8
}

// AvgColSizeListInDisk is the average column size of the histogram. These sizes are derived
// from `chunk.ListInDisk` so we need to update them if those 2 functions are changed.
func (c *Column) AvgColSizeListInDisk(count int64) float64 {
Expand All @@ -156,8 +174,9 @@ func (c *Column) AvgColSizeListInDisk(count int64) float64 {
return float64(size) * notNullRatio
}
// Keep two decimal place.
// size of varchar type is LEN + BYTE, so we minus 1 here.
return math.Round(float64(c.TotColSize)/float64(count)*100)/100 - 1
// Minus Log2(avgSize) for unfixed-len type LEN.
avgSize := float64(c.TotColSize) / float64(count)
return math.Round((avgSize-math.Log2(avgSize))*100) / 100
}

// AppendBucket appends a bucket into `hg`.
Expand Down
22 changes: 16 additions & 6 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -674,7 +675,8 @@ func getPseudoRowCountByUnsignedIntRanges(intRanges []*ranger.Range, tableRowCou
}

// GetAvgRowSize computes average row size for given columns.
func (coll *HistColl) GetAvgRowSize(cols []*expression.Column, isEncodedKey bool) (size float64) {
func (coll *HistColl) GetAvgRowSize(ctx sessionctx.Context, cols []*expression.Column, isEncodedKey bool, isForScan bool) (size float64) {
sessionVars := ctx.GetSessionVars()
if coll.Pseudo || len(coll.Columns) == 0 || coll.Count == 0 {
size = pseudoColSize * float64(len(cols))
} else {
Expand All @@ -688,9 +690,17 @@ func (coll *HistColl) GetAvgRowSize(cols []*expression.Column, isEncodedKey bool
}
// We differentiate if the column is encoded as key or value, because the resulted size
// is different.
size += colHist.AvgColSize(coll.Count, isEncodedKey)
if sessionVars.EnableChunkRPC && !isForScan {
size += colHist.AvgColSizeChunkFormat(coll.Count)
} else {
size += colHist.AvgColSize(coll.Count, isEncodedKey)
}
}
}
if sessionVars.EnableChunkRPC && !isForScan {
// Add 1/8 byte for each column's nullBitMap byte.
return size + float64(len(cols))/8
}
// Add 1 byte for each column's flag byte. See `encode` for details.
return size + float64(len(cols))
}
Expand Down Expand Up @@ -718,8 +728,8 @@ func (coll *HistColl) GetAvgRowSizeListInDisk(cols []*expression.Column) (size f
}

// GetTableAvgRowSize computes average row size for a table scan, exclude the index key-value pairs.
func (coll *HistColl) GetTableAvgRowSize(cols []*expression.Column, storeType kv.StoreType, handleInCols bool) (size float64) {
size = coll.GetAvgRowSize(cols, false)
func (coll *HistColl) GetTableAvgRowSize(ctx sessionctx.Context, cols []*expression.Column, storeType kv.StoreType, handleInCols bool) (size float64) {
size = coll.GetAvgRowSize(ctx, cols, false, true)
switch storeType {
case kv.TiKV:
size += tablecodec.RecordRowKeyLen
Expand All @@ -734,8 +744,8 @@ func (coll *HistColl) GetTableAvgRowSize(cols []*expression.Column, storeType kv
}

// GetIndexAvgRowSize computes average row size for a index scan.
func (coll *HistColl) GetIndexAvgRowSize(cols []*expression.Column, isUnique bool) (size float64) {
size = coll.GetAvgRowSize(cols, true)
func (coll *HistColl) GetIndexAvgRowSize(ctx sessionctx.Context, cols []*expression.Column, isUnique bool) (size float64) {
size = coll.GetAvgRowSize(ctx, cols, true, true)
// tablePrefix(1) + tableID(8) + indexPrefix(2) + indexID(8)
// Because the cols for index scan always contain the handle, so we don't add the rowID here.
size += 19
Expand Down

0 comments on commit 02408db

Please sign in to comment.