planner: move some estimation functions from the physical-optimization package into cardinality package (#46479)
ref pingcap/tidb#46358
This commit is contained in:
@ -148,6 +148,7 @@ go_library(
|
||||
"//parser/tidb",
|
||||
"//parser/types",
|
||||
"//planner",
|
||||
"//planner/cardinality",
|
||||
"//planner/core",
|
||||
"//planner/util",
|
||||
"//plugin",
|
||||
|
||||
@ -26,6 +26,7 @@ import (
|
||||
"github.com/pingcap/tidb/parser/ast"
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/parser/mysql"
|
||||
"github.com/pingcap/tidb/planner/cardinality"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/collate"
|
||||
@ -225,7 +226,7 @@ func (e *ShowExec) appendTableForStatsHistograms(dbName, tblName, partitionName
|
||||
if !col.IsStatsInitialized() {
|
||||
continue
|
||||
}
|
||||
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.RealtimeCount, false),
|
||||
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, cardinality.AvgColSize(col, statsTbl.RealtimeCount, false),
|
||||
col.StatsLoadedStatus.StatusToString(), col.MemoryUsage())
|
||||
}
|
||||
for _, idx := range stableIdxsStats(statsTbl.Indices) {
|
||||
|
||||
@ -10,6 +10,7 @@ go_library(
|
||||
"pseudo.go",
|
||||
"row_count_column.go",
|
||||
"row_count_index.go",
|
||||
"row_size.go",
|
||||
"selectivity.go",
|
||||
"trace.go",
|
||||
],
|
||||
@ -28,6 +29,7 @@ go_library(
|
||||
"//sessionctx",
|
||||
"//sessionctx/stmtctx",
|
||||
"//statistics",
|
||||
"//tablecodec",
|
||||
"//types",
|
||||
"//types/parser_driver",
|
||||
"//util/chunk",
|
||||
@ -51,13 +53,14 @@ go_test(
|
||||
srcs = [
|
||||
"main_test.go",
|
||||
"row_count_test.go",
|
||||
"row_size_test.go",
|
||||
"selectivity_test.go",
|
||||
"trace_test.go",
|
||||
],
|
||||
data = glob(["testdata/**"]),
|
||||
embed = [":cardinality"],
|
||||
flaky = True,
|
||||
shard_count = 31,
|
||||
shard_count = 32,
|
||||
deps = [
|
||||
"//config",
|
||||
"//domain",
|
||||
|
||||
@ -36,6 +36,11 @@ const (
|
||||
// If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8.
|
||||
const selectionFactor = 0.8
|
||||
|
||||
// PseudoAvgCountPerValue gets a pseudo average count if histogram not exists.
|
||||
func PseudoAvgCountPerValue(t *statistics.Table) float64 {
|
||||
return float64(t.RealtimeCount) / pseudoEqualRate
|
||||
}
|
||||
|
||||
func pseudoSelectivity(coll *statistics.HistColl, exprs []expression.Expression) float64 {
|
||||
minFactor := selectionFactor
|
||||
colExists := make(map[string]bool)
|
||||
|
||||
185
planner/cardinality/row_size.go
Normal file
185
planner/cardinality/row_size.go
Normal file
@ -0,0 +1,185 @@
|
||||
// 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 cardinality
|
||||
|
||||
import (
|
||||
"math"
|
||||
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/parser/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
)
|
||||
|
||||
const pseudoColSize = 8.0
|
||||
|
||||
// GetIndexAvgRowSize computes average row size for a index scan.
|
||||
func GetIndexAvgRowSize(ctx sessionctx.Context, coll *statistics.HistColl, cols []*expression.Column, isUnique bool) (size float64) {
|
||||
size = GetAvgRowSize(ctx, coll, 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
|
||||
if !isUnique {
|
||||
// add the len("_")
|
||||
size++
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// GetTableAvgRowSize computes average row size for a table scan, exclude the index key-value pairs.
|
||||
func GetTableAvgRowSize(ctx sessionctx.Context, coll *statistics.HistColl, cols []*expression.Column, storeType kv.StoreType, handleInCols bool) (size float64) {
|
||||
size = GetAvgRowSize(ctx, coll, cols, false, true)
|
||||
switch storeType {
|
||||
case kv.TiKV:
|
||||
size += tablecodec.RecordRowKeyLen
|
||||
// The `cols` for TiKV always contain the row_id, so prefix row size subtract its length.
|
||||
size -= 8
|
||||
case kv.TiFlash:
|
||||
if !handleInCols {
|
||||
size += 8 /* row_id length */
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// GetAvgRowSize computes average row size for given columns.
|
||||
func GetAvgRowSize(ctx sessionctx.Context, coll *statistics.HistColl, cols []*expression.Column, isEncodedKey bool, isForScan bool) (size float64) {
|
||||
sessionVars := ctx.GetSessionVars()
|
||||
if coll.Pseudo || len(coll.Columns) == 0 || coll.RealtimeCount == 0 {
|
||||
size = pseudoColSize * float64(len(cols))
|
||||
} else {
|
||||
for _, col := range cols {
|
||||
colHist, ok := coll.Columns[col.UniqueID]
|
||||
// Normally this would not happen, it is for compatibility with old version stats which
|
||||
// does not include TotColSize.
|
||||
if !ok || (!colHist.IsHandle && colHist.TotColSize == 0 && (colHist.NullCount != coll.RealtimeCount)) {
|
||||
size += pseudoColSize
|
||||
continue
|
||||
}
|
||||
// We differentiate if the column is encoded as key or value, because the resulted size
|
||||
// is different.
|
||||
if sessionVars.EnableChunkRPC && !isForScan {
|
||||
size += AvgColSizeChunkFormat(colHist, coll.RealtimeCount)
|
||||
} else {
|
||||
size += AvgColSize(colHist, coll.RealtimeCount, 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))
|
||||
}
|
||||
|
||||
// GetAvgRowSizeListInDisk computes average row size for given columns.
|
||||
func GetAvgRowSizeListInDisk(coll *statistics.HistColl, cols []*expression.Column) (size float64) {
|
||||
if coll.Pseudo || len(coll.Columns) == 0 || coll.RealtimeCount == 0 {
|
||||
for _, col := range cols {
|
||||
size += float64(chunk.EstimateTypeWidth(col.GetType()))
|
||||
}
|
||||
} else {
|
||||
for _, col := range cols {
|
||||
colHist, ok := coll.Columns[col.UniqueID]
|
||||
// Normally this would not happen, it is for compatibility with old version stats which
|
||||
// does not include TotColSize.
|
||||
if !ok || (!colHist.IsHandle && colHist.TotColSize == 0 && (colHist.NullCount != coll.RealtimeCount)) {
|
||||
size += float64(chunk.EstimateTypeWidth(col.GetType()))
|
||||
continue
|
||||
}
|
||||
size += AvgColSizeListInDisk(colHist, coll.RealtimeCount)
|
||||
}
|
||||
}
|
||||
// Add 8 byte for each column's size record. See `ListInDisk` for details.
|
||||
return size + float64(8*len(cols))
|
||||
}
|
||||
|
||||
// AvgColSize is the average column size of the histogram. These sizes are derived from function `encode`
|
||||
// and `Datum::ConvertTo`, so we need to update them if those 2 functions are changed.
|
||||
func AvgColSize(c *statistics.Column, count int64, isKey bool) float64 {
|
||||
if count == 0 {
|
||||
return 0
|
||||
}
|
||||
// Note that, if the handle column is encoded as value, instead of key, i.e,
|
||||
// when the handle column is in a unique index, the real column size may be
|
||||
// smaller than 8 because it is encoded using `EncodeVarint`. Since we don't
|
||||
// know the exact value size now, use 8 as approximation.
|
||||
if c.IsHandle {
|
||||
return 8
|
||||
}
|
||||
histCount := c.TotalRowCount()
|
||||
notNullRatio := 1.0
|
||||
if histCount > 0 {
|
||||
notNullRatio = 1.0 - float64(c.NullCount)/histCount
|
||||
}
|
||||
switch c.Histogram.Tp.GetType() {
|
||||
case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
|
||||
return 8 * notNullRatio
|
||||
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeYear, mysql.TypeEnum, mysql.TypeBit, mysql.TypeSet:
|
||||
if isKey {
|
||||
return 8 * notNullRatio
|
||||
}
|
||||
}
|
||||
// Keep two decimal place.
|
||||
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 AvgColSizeChunkFormat(c *statistics.Column, 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)
|
||||
if avgSize < 1 {
|
||||
return math.Round(avgSize*100)/100 + 8
|
||||
}
|
||||
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 AvgColSizeListInDisk(c *statistics.Column, count int64) float64 {
|
||||
if count == 0 {
|
||||
return 0
|
||||
}
|
||||
histCount := c.TotalRowCount()
|
||||
notNullRatio := 1.0
|
||||
if histCount > 0 {
|
||||
notNullRatio = 1.0 - float64(c.NullCount)/histCount
|
||||
}
|
||||
size := chunk.GetFixedLen(c.Histogram.Tp)
|
||||
if size != -1 {
|
||||
return float64(size) * notNullRatio
|
||||
}
|
||||
// Keep two decimal place.
|
||||
// Minus Log2(avgSize) for unfixed-len type LEN.
|
||||
avgSize := float64(c.TotColSize) / float64(count)
|
||||
if avgSize < 1 {
|
||||
return math.Round((avgSize)*100) / 100
|
||||
}
|
||||
return math.Round((avgSize-math.Log2(avgSize))*100) / 100
|
||||
}
|
||||
75
planner/cardinality/row_size_test.go
Normal file
75
planner/cardinality/row_size_test.go
Normal file
@ -0,0 +1,75 @@
|
||||
// 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 cardinality_test
|
||||
|
||||
import (
|
||||
"math"
|
||||
"testing"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/planner/cardinality"
|
||||
"github.com/pingcap/tidb/testkit"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestAvgColLen(t *testing.T) {
|
||||
store, dom := testkit.CreateMockStoreAndDomain(t)
|
||||
testKit := testkit.NewTestKit(t, store)
|
||||
testKit.MustExec("use test")
|
||||
testKit.MustExec("create table t (c1 int, c2 varchar(100), c3 float, c4 datetime, c5 varchar(100))")
|
||||
testKit.MustExec("insert into t values(1, '1234567', 12.3, '2018-03-07 19:00:57', NULL)")
|
||||
testKit.MustExec("analyze table t")
|
||||
do := dom
|
||||
is := do.InfoSchema()
|
||||
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
||||
require.NoError(t, err)
|
||||
tableInfo := tbl.Meta()
|
||||
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
|
||||
require.Equal(t, 1.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
|
||||
|
||||
// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
|
||||
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0-3, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0-3+8, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 0.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
|
||||
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29', NULL)")
|
||||
testKit.MustExec("analyze table t")
|
||||
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
||||
require.Equal(t, 1.5, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 10.5, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100+8, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
|
||||
require.Equal(t, 0.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
|
||||
}
|
||||
@ -2131,7 +2131,7 @@ func checkChildFitBC(p Plan) bool {
|
||||
if p.StatsInfo().HistColl == nil {
|
||||
return p.SCtx().GetSessionVars().BroadcastJoinThresholdCount == -1 || p.StatsInfo().Count() < p.SCtx().GetSessionVars().BroadcastJoinThresholdCount
|
||||
}
|
||||
avg := p.StatsInfo().HistColl.GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false)
|
||||
avg := cardinality.GetAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, p.Schema().Columns, false, false)
|
||||
sz := avg * float64(p.StatsInfo().Count())
|
||||
return p.SCtx().GetSessionVars().BroadcastJoinThresholdSize == -1 || sz < float64(p.SCtx().GetSessionVars().BroadcastJoinThresholdSize)
|
||||
}
|
||||
@ -2142,7 +2142,7 @@ func calcBroadcastExchangeSize(p Plan, mppStoreCnt int) (row float64, size float
|
||||
if s.HistColl == nil {
|
||||
return row, 0, false
|
||||
}
|
||||
avg := s.HistColl.GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false)
|
||||
avg := cardinality.GetAvgRowSize(p.SCtx(), s.HistColl, p.Schema().Columns, false, false)
|
||||
size = avg * row
|
||||
return row, size, true
|
||||
}
|
||||
@ -2169,7 +2169,7 @@ func calcHashExchangeSize(p Plan, mppStoreCnt int) (row float64, sz float64, has
|
||||
if s.HistColl == nil {
|
||||
return row, 0, false
|
||||
}
|
||||
avg := s.HistColl.GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false)
|
||||
avg := cardinality.GetAvgRowSize(p.SCtx(), s.HistColl, p.Schema().Columns, false, false)
|
||||
sz = avg * row
|
||||
return row, sz, true
|
||||
}
|
||||
|
||||
@ -1795,7 +1795,7 @@ func (is *PhysicalIndexScan) getScanRowSize() float64 {
|
||||
} else {
|
||||
scanCols = is.schema.Columns
|
||||
}
|
||||
return is.tblColHists.GetIndexAvgRowSize(is.SCtx(), scanCols, is.Index.Unique)
|
||||
return cardinality.GetIndexAvgRowSize(is.SCtx(), is.tblColHists, scanCols, is.Index.Unique)
|
||||
}
|
||||
|
||||
// initSchema is used to set the schema of PhysicalIndexScan. Before calling this,
|
||||
@ -2422,11 +2422,11 @@ func (ts *PhysicalTableScan) addPushedDownSelection(copTask *copTask, stats *pro
|
||||
|
||||
func (ts *PhysicalTableScan) getScanRowSize() float64 {
|
||||
if ts.StoreType == kv.TiKV {
|
||||
return ts.tblColHists.GetTableAvgRowSize(ts.SCtx(), ts.tblCols, ts.StoreType, true)
|
||||
return cardinality.GetTableAvgRowSize(ts.SCtx(), ts.tblColHists, ts.tblCols, ts.StoreType, true)
|
||||
}
|
||||
// If `ts.handleCol` is nil, then the schema of tableScan doesn't have handle column.
|
||||
// This logic can be ensured in column pruning.
|
||||
return ts.tblColHists.GetTableAvgRowSize(ts.SCtx(), ts.Schema().Columns, ts.StoreType, ts.HandleCols != nil)
|
||||
return cardinality.GetTableAvgRowSize(ts.SCtx(), ts.tblColHists, ts.Schema().Columns, ts.StoreType, ts.HandleCols != nil)
|
||||
}
|
||||
|
||||
func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool) (*PhysicalTableScan, float64) {
|
||||
|
||||
@ -1698,7 +1698,7 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co
|
||||
path.AccessConds = append(path.AccessConds, accesses...)
|
||||
path.TableFilters = remained
|
||||
if len(accesses) > 0 && ds.statisticTable.Pseudo {
|
||||
path.CountAfterAccess = ds.statisticTable.PseudoAvgCountPerValue()
|
||||
path.CountAfterAccess = cardinality.PseudoAvgCountPerValue(ds.statisticTable)
|
||||
} else {
|
||||
selectivity := path.CountAfterAccess / float64(ds.statisticTable.RealtimeCount)
|
||||
for i := range accesses {
|
||||
@ -1849,7 +1849,7 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression
|
||||
path.AccessConds = append(path.AccessConds, accesses...)
|
||||
path.TableFilters = remained
|
||||
if len(accesses) > 0 && ds.statisticTable.Pseudo {
|
||||
path.CountAfterAccess = ds.statisticTable.PseudoAvgCountPerValue()
|
||||
path.CountAfterAccess = cardinality.PseudoAvgCountPerValue(ds.statisticTable)
|
||||
} else {
|
||||
selectivity := path.CountAfterAccess / float64(ds.statisticTable.RealtimeCount)
|
||||
for i := range accesses {
|
||||
|
||||
@ -27,6 +27,7 @@ import (
|
||||
"github.com/pingcap/tidb/parser/ast"
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/parser/mysql"
|
||||
"github.com/pingcap/tidb/planner/cardinality"
|
||||
"github.com/pingcap/tidb/planner/property"
|
||||
"github.com/pingcap/tidb/planner/util"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
@ -200,7 +201,7 @@ func (p *PhysicalTableReader) GetTableScan() (*PhysicalTableScan, error) {
|
||||
|
||||
// GetAvgRowSize return the average row size of this plan.
|
||||
func (p *PhysicalTableReader) GetAvgRowSize() float64 {
|
||||
return getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
return cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of PhysicalTableReader
|
||||
@ -492,12 +493,12 @@ func (p *PhysicalIndexLookUpReader) ExtractCorrelatedCols() (corCols []*expressi
|
||||
|
||||
// GetIndexNetDataSize return the estimated total size in bytes via network transfer.
|
||||
func (p *PhysicalIndexLookUpReader) GetIndexNetDataSize() float64 {
|
||||
return getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) * p.indexPlan.StatsCount()
|
||||
return cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.indexPlan), p.indexPlan.Schema().Columns, true, false) * p.indexPlan.StatsCount()
|
||||
}
|
||||
|
||||
// GetAvgTableRowSize return the average row size of each final row.
|
||||
func (p *PhysicalIndexLookUpReader) GetAvgTableRowSize() float64 {
|
||||
return getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
return cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
}
|
||||
|
||||
// BuildPlanTrace implements PhysicalPlan interface.
|
||||
@ -584,7 +585,7 @@ type PhysicalIndexMergeReader struct {
|
||||
|
||||
// GetAvgTableRowSize return the average row size of table plan.
|
||||
func (p *PhysicalIndexMergeReader) GetAvgTableRowSize() float64 {
|
||||
return getTblStats(p.TablePlans[len(p.TablePlans)-1]).GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false)
|
||||
return cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.TablePlans[len(p.TablePlans)-1]), p.Schema().Columns, false, false)
|
||||
}
|
||||
|
||||
// ExtractCorrelatedCols implements PhysicalPlan interface.
|
||||
|
||||
@ -205,14 +205,14 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer1(_ property.TaskType, option
|
||||
|
||||
// index-side net I/O cost: rows * row-size * net-factor
|
||||
netFactor := getTableNetFactor(p.tablePlan)
|
||||
rowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false)
|
||||
rowSize := cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.indexPlan), p.indexPlan.Schema().Columns, true, false)
|
||||
p.planCost += getCardinality(p.indexPlan, costFlag) * rowSize * netFactor
|
||||
|
||||
// index-side net seek cost
|
||||
p.planCost += estimateNetSeekCost(p.indexPlan)
|
||||
|
||||
// table-side net I/O cost: rows * row-size * net-factor
|
||||
tblRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
tblRowSize := cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
p.planCost += getCardinality(p.tablePlan, costFlag) * tblRowSize * netFactor
|
||||
|
||||
// table-side seek cost
|
||||
@ -245,7 +245,7 @@ func (p *PhysicalIndexReader) getPlanCostVer1(_ property.TaskType, option *PlanC
|
||||
p.planCost = indexPlanCost
|
||||
// net I/O cost: rows * row-size * net-factor
|
||||
tblStats := getTblStats(p.indexPlan)
|
||||
rowSize = tblStats.GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false)
|
||||
rowSize = cardinality.GetAvgRowSize(p.SCtx(), tblStats, p.indexPlan.Schema().Columns, true, false)
|
||||
rowCount = getCardinality(p.indexPlan, costFlag)
|
||||
netFactor = getTableNetFactor(p.indexPlan)
|
||||
p.planCost += rowCount * rowSize * netFactor
|
||||
@ -265,7 +265,7 @@ func (p *PhysicalIndexReader) getPlanCostVer1(_ property.TaskType, option *PlanC
|
||||
// GetNetDataSize calculates the cost of the plan in network data transfer.
|
||||
func (p *PhysicalIndexReader) GetNetDataSize() float64 {
|
||||
tblStats := getTblStats(p.indexPlan)
|
||||
rowSize := tblStats.GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false)
|
||||
rowSize := cardinality.GetAvgRowSize(p.SCtx(), tblStats, p.indexPlan.Schema().Columns, true, false)
|
||||
return p.indexPlan.StatsCount() * rowSize
|
||||
}
|
||||
|
||||
@ -291,7 +291,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC
|
||||
tableCost = childCost
|
||||
p.planCost = childCost
|
||||
// net I/O cost: rows * row-size * net-factor
|
||||
rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
rowSize = cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
rowCount = getCardinality(p.tablePlan, costFlag)
|
||||
p.planCost += rowCount * rowSize * netFactor
|
||||
// net seek cost
|
||||
@ -315,7 +315,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC
|
||||
} else {
|
||||
// cop protocol
|
||||
concurrency = float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency())
|
||||
rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
rowSize = cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
seekCost = estimateNetSeekCost(p.tablePlan)
|
||||
tType := property.CopSingleReadTaskType
|
||||
childCost, err := p.tablePlan.getPlanCostVer1(tType, option)
|
||||
@ -348,7 +348,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC
|
||||
|
||||
// GetNetDataSize calculates the estimated total data size fetched from storage.
|
||||
func (p *PhysicalTableReader) GetNetDataSize() float64 {
|
||||
rowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
rowSize := cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
return p.tablePlan.StatsCount() * rowSize
|
||||
}
|
||||
|
||||
@ -368,7 +368,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option *
|
||||
netFactor := getTableNetFactor(tblScan)
|
||||
p.planCost += childCost // child's cost
|
||||
tblStats := getTblStats(tblScan)
|
||||
rowSize := tblStats.GetAvgRowSize(p.SCtx(), tblScan.Schema().Columns, false, false)
|
||||
rowSize := cardinality.GetAvgRowSize(p.SCtx(), tblStats, tblScan.Schema().Columns, false, false)
|
||||
p.planCost += getCardinality(tblScan, costFlag) * rowSize * netFactor // net I/O cost
|
||||
}
|
||||
for _, partialScan := range p.partialPlans {
|
||||
@ -387,7 +387,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option *
|
||||
netFactor := getTableNetFactor(partialScan)
|
||||
p.planCost += childCost // child's cost
|
||||
tblStats := getTblStats(partialScan)
|
||||
rowSize := tblStats.GetAvgRowSize(p.SCtx(), partialScan.Schema().Columns, isIdxScan, false)
|
||||
rowSize := cardinality.GetAvgRowSize(p.SCtx(), tblStats, partialScan.Schema().Columns, isIdxScan, false)
|
||||
p.planCost += getCardinality(partialScan, costFlag) * rowSize * netFactor // net I/O cost
|
||||
}
|
||||
|
||||
@ -403,7 +403,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option *
|
||||
// GetPartialReaderNetDataSize returns the estimated total response data size of a partial read.
|
||||
func (p *PhysicalIndexMergeReader) GetPartialReaderNetDataSize(plan PhysicalPlan) float64 {
|
||||
_, isIdxScan := plan.(*PhysicalIndexScan)
|
||||
return plan.StatsCount() * getTblStats(plan).GetAvgRowSize(p.SCtx(), plan.Schema().Columns, isIdxScan, false)
|
||||
return plan.StatsCount() * cardinality.GetAvgRowSize(p.SCtx(), getTblStats(plan), plan.Schema().Columns, isIdxScan, false)
|
||||
}
|
||||
|
||||
// getPlanCostVer1 calculates the cost of the plan if it has not been calculated yet and returns the cost.
|
||||
@ -1114,10 +1114,10 @@ func (p *BatchPointGetPlan) GetCost(opt *physicalOptimizeOp) float64 {
|
||||
cost := 0.0
|
||||
if p.IndexInfo == nil {
|
||||
rowCount = float64(len(p.Handles))
|
||||
rowSize = p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true)
|
||||
rowSize = cardinality.GetTableAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, kv.TiKV, true)
|
||||
} else {
|
||||
rowCount = float64(len(p.IndexValues))
|
||||
rowSize = p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique)
|
||||
rowSize = cardinality.GetIndexAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, p.IndexInfo.Unique)
|
||||
}
|
||||
networkFactor := sessVars.GetNetworkFactor(p.TblInfo)
|
||||
seekFactor := sessVars.GetSeekFactor(p.TblInfo)
|
||||
@ -1149,9 +1149,9 @@ func (p *BatchPointGetPlan) GetAvgRowSize() float64 {
|
||||
return 0 // the cost of BatchGet generated in fast plan optimization is always 0
|
||||
}
|
||||
if p.IndexInfo == nil {
|
||||
return p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true)
|
||||
return cardinality.GetTableAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, kv.TiKV, true)
|
||||
}
|
||||
return p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique)
|
||||
return cardinality.GetIndexAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, p.IndexInfo.Unique)
|
||||
}
|
||||
|
||||
// GetCost returns cost of the PointGetPlan.
|
||||
@ -1164,9 +1164,9 @@ func (p *PointGetPlan) GetCost(opt *physicalOptimizeOp) float64 {
|
||||
var rowSize float64
|
||||
cost := 0.0
|
||||
if p.IndexInfo == nil {
|
||||
rowSize = p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true)
|
||||
rowSize = cardinality.GetTableAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, kv.TiKV, true)
|
||||
} else {
|
||||
rowSize = p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique)
|
||||
rowSize = cardinality.GetIndexAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, p.IndexInfo.Unique)
|
||||
}
|
||||
networkFactor := sessVars.GetNetworkFactor(p.TblInfo)
|
||||
seekFactor := sessVars.GetSeekFactor(p.TblInfo)
|
||||
@ -1197,9 +1197,9 @@ func (p *PointGetPlan) GetAvgRowSize() float64 {
|
||||
return 0 // the cost of PointGet generated in fast plan optimization is always 0
|
||||
}
|
||||
if p.IndexInfo == nil {
|
||||
return p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true)
|
||||
return cardinality.GetTableAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, kv.TiKV, true)
|
||||
}
|
||||
return p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique)
|
||||
return cardinality.GetIndexAvgRowSize(p.SCtx(), p.StatsInfo().HistColl, cols, p.IndexInfo.Unique)
|
||||
}
|
||||
|
||||
// getPlanCostVer1 calculates the cost of the plan if it has not been calculated yet and returns the cost.
|
||||
|
||||
@ -23,6 +23,7 @@ import (
|
||||
"github.com/pingcap/tidb/expression/aggregation"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/planner/cardinality"
|
||||
"github.com/pingcap/tidb/planner/property"
|
||||
"github.com/pingcap/tidb/planner/util"
|
||||
"github.com/pingcap/tidb/sessionctx/variable"
|
||||
@ -241,8 +242,8 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType,
|
||||
|
||||
indexRows := getCardinality(p.indexPlan, option.CostFlag)
|
||||
tableRows := getCardinality(p.indexPlan, option.CostFlag)
|
||||
indexRowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false)
|
||||
tableRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false)
|
||||
indexRowSize := cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.indexPlan), p.indexPlan.Schema().Columns, true, false)
|
||||
tableRowSize := cardinality.GetAvgRowSize(p.SCtx(), getTblStats(p.tablePlan), p.tablePlan.Schema().Columns, false, false)
|
||||
cpuFactor := getTaskCPUFactorVer2(p, taskType)
|
||||
netFactor := getTaskNetFactorVer2(p, taskType)
|
||||
requestFactor := getTaskRequestFactorVer2(p, taskType)
|
||||
|
||||
@ -305,9 +305,10 @@ func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task {
|
||||
return t
|
||||
}
|
||||
|
||||
// RowSize for cost model ver2 is simplified, always use this function to calculate row size.
|
||||
func getAvgRowSize(stats *property.StatsInfo, cols []*expression.Column) (size float64) {
|
||||
if stats.HistColl != nil {
|
||||
size = stats.HistColl.GetAvgRowSizeListInDisk(cols)
|
||||
size = cardinality.GetAvgRowSizeListInDisk(stats.HistColl, cols)
|
||||
} else {
|
||||
// Estimate using just the type info.
|
||||
for _, col := range cols {
|
||||
@ -2548,7 +2549,7 @@ func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan PhysicalPl
|
||||
|
||||
func collectRowSizeFromMPPPlan(mppPlan PhysicalPlan) (rowSize float64) {
|
||||
if mppPlan != nil && mppPlan.StatsInfo() != nil && mppPlan.StatsInfo().HistColl != nil {
|
||||
return mppPlan.StatsInfo().HistColl.GetAvgRowSize(mppPlan.SCtx(), mppPlan.Schema().Columns, false, false)
|
||||
return cardinality.GetAvgRowSize(mppPlan.SCtx(), mppPlan.StatsInfo().HistColl, mppPlan.Schema().Columns, false, false)
|
||||
}
|
||||
return 1 // use 1 as lower-bound for safety
|
||||
}
|
||||
|
||||
@ -15,6 +15,7 @@ go_library(
|
||||
"//expression",
|
||||
"//kv",
|
||||
"//parser/model",
|
||||
"//planner/cardinality",
|
||||
"//planner/core",
|
||||
"//planner/memo",
|
||||
"//statistics",
|
||||
|
||||
@ -20,6 +20,7 @@ import (
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/planner/cardinality"
|
||||
plannercore "github.com/pingcap/tidb/planner/core"
|
||||
"github.com/pingcap/tidb/planner/memo"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
@ -76,7 +77,7 @@ func NewTableReaderImpl(reader *plannercore.PhysicalTableReader, source *planner
|
||||
// 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(impl.plan.SCtx(), reader.Schema().Columns, false, false)
|
||||
width := cardinality.GetAvgRowSize(impl.plan.SCtx(), impl.tblColHists, reader.Schema().Columns, false, false)
|
||||
sessVars := reader.SCtx().GetSessionVars()
|
||||
// TableReaderImpl don't have tableInfo property, so using nil to replace it.
|
||||
// Todo add the tableInfo property for the TableReaderImpl.
|
||||
@ -123,7 +124,7 @@ func NewTableScanImpl(ts *plannercore.PhysicalTableScan, cols []*expression.Colu
|
||||
// CalcCost calculates the cost of the table scan Implementation.
|
||||
func (impl *TableScanImpl) CalcCost(outCount float64, _ ...memo.Implementation) float64 {
|
||||
ts := impl.plan.(*plannercore.PhysicalTableScan)
|
||||
width := impl.tblColHists.GetTableAvgRowSize(impl.plan.SCtx(), impl.tblCols, kv.TiKV, true)
|
||||
width := cardinality.GetTableAvgRowSize(impl.plan.SCtx(), impl.tblColHists, impl.tblCols, kv.TiKV, true)
|
||||
sessVars := ts.SCtx().GetSessionVars()
|
||||
impl.cost = outCount * sessVars.GetScanFactor(ts.Table) * width
|
||||
if ts.Desc {
|
||||
@ -155,7 +156,7 @@ func (impl *IndexReaderImpl) CalcCost(outCount float64, children ...memo.Impleme
|
||||
reader := impl.plan.(*plannercore.PhysicalIndexReader)
|
||||
sessVars := reader.SCtx().GetSessionVars()
|
||||
networkCost := outCount * sessVars.GetNetworkFactor(impl.tblInfo) *
|
||||
impl.tblColHists.GetAvgRowSize(reader.SCtx(), children[0].GetPlan().Schema().Columns,
|
||||
cardinality.GetAvgRowSize(reader.SCtx(), impl.tblColHists, children[0].GetPlan().Schema().Columns,
|
||||
true, false)
|
||||
copIterWorkers := float64(sessVars.DistSQLScanConcurrency())
|
||||
impl.cost = (networkCost + children[0].GetCost()) / copIterWorkers
|
||||
@ -181,7 +182,7 @@ type IndexScanImpl struct {
|
||||
func (impl *IndexScanImpl) CalcCost(outCount float64, _ ...memo.Implementation) float64 {
|
||||
is := impl.plan.(*plannercore.PhysicalIndexScan)
|
||||
sessVars := is.SCtx().GetSessionVars()
|
||||
rowSize := impl.tblColHists.GetIndexAvgRowSize(is.SCtx(), is.Schema().Columns, is.Index.Unique)
|
||||
rowSize := cardinality.GetIndexAvgRowSize(is.SCtx(), impl.tblColHists, is.Schema().Columns, is.Index.Unique)
|
||||
cost := outCount * rowSize * sessVars.GetScanFactor(is.Table)
|
||||
if is.Desc {
|
||||
cost = outCount * rowSize * sessVars.GetDescScanFactor(is.Table)
|
||||
|
||||
@ -15,7 +15,6 @@
|
||||
package statistics
|
||||
|
||||
import (
|
||||
"math"
|
||||
"strconv"
|
||||
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
@ -229,80 +228,6 @@ func (c *Column) IsCMSExist() bool {
|
||||
return c.CMSketch != nil
|
||||
}
|
||||
|
||||
// AvgColSize is the average column size of the histogram. These sizes are derived from function `encode`
|
||||
// and `Datum::ConvertTo`, so we need to update them if those 2 functions are changed.
|
||||
func (c *Column) AvgColSize(count int64, isKey bool) float64 {
|
||||
if count == 0 {
|
||||
return 0
|
||||
}
|
||||
// Note that, if the handle column is encoded as value, instead of key, i.e,
|
||||
// when the handle column is in a unique index, the real column size may be
|
||||
// smaller than 8 because it is encoded using `EncodeVarint`. Since we don't
|
||||
// know the exact value size now, use 8 as approximation.
|
||||
if c.IsHandle {
|
||||
return 8
|
||||
}
|
||||
histCount := c.TotalRowCount()
|
||||
notNullRatio := 1.0
|
||||
if histCount > 0 {
|
||||
notNullRatio = 1.0 - float64(c.NullCount)/histCount
|
||||
}
|
||||
switch c.Histogram.Tp.GetType() {
|
||||
case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
|
||||
return 8 * notNullRatio
|
||||
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeYear, mysql.TypeEnum, mysql.TypeBit, mysql.TypeSet:
|
||||
if isKey {
|
||||
return 8 * notNullRatio
|
||||
}
|
||||
}
|
||||
// Keep two decimal place.
|
||||
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)
|
||||
if avgSize < 1 {
|
||||
return math.Round(avgSize*100)/100 + 8
|
||||
}
|
||||
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 {
|
||||
if count == 0 {
|
||||
return 0
|
||||
}
|
||||
histCount := c.TotalRowCount()
|
||||
notNullRatio := 1.0
|
||||
if histCount > 0 {
|
||||
notNullRatio = 1.0 - float64(c.NullCount)/histCount
|
||||
}
|
||||
size := chunk.GetFixedLen(c.Histogram.Tp)
|
||||
if size != -1 {
|
||||
return float64(size) * notNullRatio
|
||||
}
|
||||
// Keep two decimal place.
|
||||
// Minus Log2(avgSize) for unfixed-len type LEN.
|
||||
avgSize := float64(c.TotColSize) / float64(count)
|
||||
if avgSize < 1 {
|
||||
return math.Round((avgSize)*100) / 100
|
||||
}
|
||||
return math.Round((avgSize-math.Log2(avgSize))*100) / 100
|
||||
}
|
||||
|
||||
// StatusToString gets the string info of StatsLoadedStatus
|
||||
func (s StatsLoadedStatus) StatusToString() string {
|
||||
if !s.statsInitialized {
|
||||
|
||||
@ -164,7 +164,7 @@ func TestDDLHistogram(t *testing.T) {
|
||||
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
||||
require.False(t, statsTbl.Pseudo)
|
||||
require.True(t, statsTbl.Columns[tableInfo.Columns[5].ID].IsStatsInitialized())
|
||||
require.Equal(t, 3.0, statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 3.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[5].ID], statsTbl.RealtimeCount, false))
|
||||
|
||||
testKit.MustExec("alter table t add column c6 varchar(15) DEFAULT '123', add column c7 varchar(15) DEFAULT '123'")
|
||||
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
||||
@ -231,7 +231,7 @@ PARTITION BY RANGE ( a ) (
|
||||
for _, def := range pi.Definitions {
|
||||
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
|
||||
require.False(t, statsTbl.Pseudo)
|
||||
require.Equal(t, 3.0, statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 3.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount, false))
|
||||
}
|
||||
|
||||
addPartition := "alter table t add partition (partition p4 values less than (26))"
|
||||
|
||||
@ -9,7 +9,7 @@ go_test(
|
||||
],
|
||||
flaky = True,
|
||||
race = "on",
|
||||
shard_count = 45,
|
||||
shard_count = 44,
|
||||
deps = [
|
||||
"//config",
|
||||
"//domain",
|
||||
|
||||
@ -16,12 +16,10 @@ package handletest
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
"unsafe"
|
||||
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/pingcap/tidb/config"
|
||||
@ -100,54 +98,6 @@ func TestColumnIDs(t *testing.T) {
|
||||
require.Equal(t, 0.0, count)
|
||||
}
|
||||
|
||||
func TestAvgColLen(t *testing.T) {
|
||||
store, dom := testkit.CreateMockStoreAndDomain(t)
|
||||
testKit := testkit.NewTestKit(t, store)
|
||||
testKit.MustExec("use test")
|
||||
testKit.MustExec("create table t (c1 int, c2 varchar(100), c3 float, c4 datetime, c5 varchar(100))")
|
||||
testKit.MustExec("insert into t values(1, '1234567', 12.3, '2018-03-07 19:00:57', NULL)")
|
||||
testKit.MustExec("analyze table t")
|
||||
do := dom
|
||||
is := do.InfoSchema()
|
||||
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
||||
require.NoError(t, err)
|
||||
tableInfo := tbl.Meta()
|
||||
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
|
||||
require.Equal(t, 1.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
|
||||
// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0-3, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0-3+8, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 0.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29', NULL)")
|
||||
testKit.MustExec("analyze table t")
|
||||
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
||||
require.Equal(t, 1.5, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 10.5, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.RealtimeCount, false))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100+8, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.RealtimeCount))
|
||||
require.Equal(t, 0.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.RealtimeCount))
|
||||
}
|
||||
|
||||
func TestDurationToTS(t *testing.T) {
|
||||
tests := []time.Duration{time.Millisecond, time.Second, time.Minute, time.Hour}
|
||||
for _, test := range tests {
|
||||
|
||||
@ -22,24 +22,14 @@ import (
|
||||
"sync"
|
||||
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/parser/model"
|
||||
"github.com/pingcap/tidb/parser/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
"github.com/pingcap/tidb/util/ranger"
|
||||
"go.uber.org/atomic"
|
||||
)
|
||||
|
||||
const (
|
||||
pseudoEqualRate = 1000
|
||||
pseudoLessRate = 3
|
||||
pseudoBetweenRate = 40
|
||||
pseudoColSize = 8.0
|
||||
)
|
||||
|
||||
const (
|
||||
// PseudoVersion means the pseudo statistics version is 0.
|
||||
PseudoVersion uint64 = 0
|
||||
@ -480,11 +470,6 @@ func (t *Table) IsOutdated() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// PseudoAvgCountPerValue gets a pseudo average count if histogram not exists.
|
||||
func (t *Table) PseudoAvgCountPerValue() float64 {
|
||||
return float64(t.RealtimeCount) / pseudoEqualRate
|
||||
}
|
||||
|
||||
// ID2UniqueID generates a new HistColl whose `Columns` is built from UniqueID of given columns.
|
||||
func (coll *HistColl) ID2UniqueID(columns []*expression.Column) *HistColl {
|
||||
cols := make(map[int64]*Column)
|
||||
@ -603,88 +588,6 @@ func PseudoTable(tblInfo *model.TableInfo) *Table {
|
||||
return t
|
||||
}
|
||||
|
||||
// GetAvgRowSize computes average row size for given columns.
|
||||
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.RealtimeCount == 0 {
|
||||
size = pseudoColSize * float64(len(cols))
|
||||
} else {
|
||||
for _, col := range cols {
|
||||
colHist, ok := coll.Columns[col.UniqueID]
|
||||
// Normally this would not happen, it is for compatibility with old version stats which
|
||||
// does not include TotColSize.
|
||||
if !ok || (!colHist.IsHandle && colHist.TotColSize == 0 && (colHist.NullCount != coll.RealtimeCount)) {
|
||||
size += pseudoColSize
|
||||
continue
|
||||
}
|
||||
// We differentiate if the column is encoded as key or value, because the resulted size
|
||||
// is different.
|
||||
if sessionVars.EnableChunkRPC && !isForScan {
|
||||
size += colHist.AvgColSizeChunkFormat(coll.RealtimeCount)
|
||||
} else {
|
||||
size += colHist.AvgColSize(coll.RealtimeCount, 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))
|
||||
}
|
||||
|
||||
// GetAvgRowSizeListInDisk computes average row size for given columns.
|
||||
func (coll *HistColl) GetAvgRowSizeListInDisk(cols []*expression.Column) (size float64) {
|
||||
if coll.Pseudo || len(coll.Columns) == 0 || coll.RealtimeCount == 0 {
|
||||
for _, col := range cols {
|
||||
size += float64(chunk.EstimateTypeWidth(col.GetType()))
|
||||
}
|
||||
} else {
|
||||
for _, col := range cols {
|
||||
colHist, ok := coll.Columns[col.UniqueID]
|
||||
// Normally this would not happen, it is for compatibility with old version stats which
|
||||
// does not include TotColSize.
|
||||
if !ok || (!colHist.IsHandle && colHist.TotColSize == 0 && (colHist.NullCount != coll.RealtimeCount)) {
|
||||
size += float64(chunk.EstimateTypeWidth(col.GetType()))
|
||||
continue
|
||||
}
|
||||
size += colHist.AvgColSizeListInDisk(coll.RealtimeCount)
|
||||
}
|
||||
}
|
||||
// Add 8 byte for each column's size record. See `ListInDisk` for details.
|
||||
return size + float64(8*len(cols))
|
||||
}
|
||||
|
||||
// GetTableAvgRowSize computes average row size for a table scan, exclude the index key-value pairs.
|
||||
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
|
||||
// The `cols` for TiKV always contain the row_id, so prefix row size subtract its length.
|
||||
size -= 8
|
||||
case kv.TiFlash:
|
||||
if !handleInCols {
|
||||
size += 8 /* row_id length */
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// GetIndexAvgRowSize computes average row size for a index scan.
|
||||
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
|
||||
if !isUnique {
|
||||
// add the len("_")
|
||||
size++
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// CheckAnalyzeVerOnTable checks whether the given version is the one from the tbl.
|
||||
// If not, it will return false and set the version to the tbl's.
|
||||
// We use this check to make sure all the statistics of the table are in the same version.
|
||||
|
||||
Reference in New Issue
Block a user