statistics

package
v1.1.0-beta.0...-92d45d5 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Sep 20, 2024 License: Apache-2.0 Imports: 56 Imported by: 0

Documentation

Index

Constants

View Source
const (
	// AnalyzePending means the analyze job is pending
	AnalyzePending = "pending"
	// AnalyzeRunning means the analyze job is running
	AnalyzeRunning = "running"
	// AnalyzeFinished means the analyze job has finished
	AnalyzeFinished = "finished"
	// AnalyzeFailed means the analyze job has failed
	AnalyzeFailed = "failed"
)
View Source
const (
	// Version0 is the state that no statistics is actually collected, only the meta info.(the total count and the average col size)
	Version0 = 0
	// Version1 maintains the statistics in the following way.
	// Column stats: CM Sketch is built in TiKV using full data. Histogram is built from samples. TopN is extracted from CM Sketch.
	//    TopN + CM Sketch represent all data. Histogram also represents all data.
	// Index stats: CM Sketch and Histogram is built in TiKV using full data. TopN is extracted from histogram. Then values covered by TopN is removed from CM Sketch.
	//    TopN + CM Sketch represent all data. Histogram also represents all data.
	// Int PK column stats is always Version1 because it only has histogram built from full data.
	// Fast analyze is always Version1 currently.
	Version1 = 1
	// Version2 maintains the statistics in the following way.
	// Column stats: CM Sketch is not used. TopN and Histogram are built from samples. TopN + Histogram represent all data.(The values covered by TopN is removed from Histogram.)
	// Index stats: CM SKetch is not used. TopN and Histograms are built from samples. TopN + Histogram represent all data.(The values covered by TopN is removed from Histogram.)
	// Both Column and Index's NDVs are collected by full scan.
	Version2 = 2
)

constants for stats version. These const can be used for solving compatibility issue.

View Source
const (
	// AllLoaded indicates all statistics are loaded
	AllLoaded = iota
	// AllEvicted indicates all statistics are evicted
	AllEvicted
)
View Source
const (
	// PseudoVersion means the pseudo statistics version is 0.
	PseudoVersion uint64 = 0

	// PseudoRowCount export for other pkg to use.
	// When we haven't analyzed a table, we use pseudo statistics to estimate costs.
	// It has row count 10000, equal condition selects 1/1000 of total rows, less condition selects 1/3 of total rows,
	// between condition selects 1/40 of total rows.
	PseudoRowCount = 10000
)
View Source
const (
	// ExtendedStatsInited is the status for extended stats which are just registered but have not been analyzed yet.
	ExtendedStatsInited uint8 = iota
	// ExtendedStatsAnalyzed is the status for extended stats which have been collected in analyze.
	ExtendedStatsAnalyzed
	// ExtendedStatsDeleted is the status for extended stats which were dropped. These "deleted" records would be removed from storage by GCStats().
	ExtendedStatsDeleted
)
View Source
const AnalyzeFlag = 1

AnalyzeFlag is set when the statistics comes from analyze.

View Source
const EmptyBucketSize = int64(unsafe.Sizeof(Bucket{}))

EmptyBucketSize is the size of empty bucket, 3*8=24 now.

View Source
const EmptyHistogramSize = int64(unsafe.Sizeof(Histogram{}))

EmptyHistogramSize is the size of empty histogram, about 112 = 8*6 for int64 & float64, 24*2 for arrays, 8*2 for references.

View Source
const EmptyReservoirSampleItemSize = int64(unsafe.Sizeof(ReservoirRowSampleItem{}))

EmptyReservoirSampleItemSize = (24 + 16 + 8) now.

View Source
const EmptySampleItemSize = int64(unsafe.Sizeof(SampleItem{}))

EmptySampleItemSize is the size of empty SampleItem, 96 = 72 (datum) + 8 (int) + 16.

View Source
const EmptyScalarSize = int64(unsafe.Sizeof(scalar{}))

EmptyScalarSize is the size of empty scalar.

View Source
const MaxSampleValueLength = mysql.MaxFieldVarCharLength / 2

MaxSampleValueLength defines the max length of the useful samples. If one sample value exceeds the max length, we drop it before building the stats.

View Source
const MaxSketchSize = 10000

MaxSketchSize is the maximum size of the hashset in the FM sketch. TODO: add this attribute to PB and persist it instead of using a fixed number(executor.maxSketchSize)

View Source
const NonPartitionTableID = -1

NonPartitionTableID is the partition id for non-partition table.

Variables

View Source
var (

	// GetRowCountByIndexRanges is a function type to get row count by index ranges.
	GetRowCountByIndexRanges func(sctx planctx.PlanContext, coll *HistColl, idxID int64, indexRanges []*ranger.Range) (result float64, err error)

	// GetRowCountByIntColumnRanges is a function type to get row count by int column ranges.
	GetRowCountByIntColumnRanges func(sctx planctx.PlanContext, coll *HistColl, colID int64, intRanges []*ranger.Range) (result float64, err error)

	// GetRowCountByColumnRanges is a function type to get row count by column ranges.
	GetRowCountByColumnRanges func(sctx planctx.PlanContext, coll *HistColl, colID int64, colRanges []*ranger.Range) (result float64, err error)
)
View Source
var AutoAnalyzeMinCnt int64 = 1000

AutoAnalyzeMinCnt means if the count of table is less than this value, we don't need to do auto analyze. Exported for testing.

View Source
var (
	// ErrQueryInterrupted indicates interrupted
	ErrQueryInterrupted = dbterror.ClassExecutor.NewStd(mysql.ErrQueryInterrupted)
)
View Source
var PrepareCols4MVIndex func(
	tableInfo *model.TableInfo,
	mvIndex *model.IndexInfo,
	tblCols []*expression.Column,
	checkOnly1ArrayTypeCol bool,
) (idxCols []*expression.Column, ok bool)

PrepareCols4MVIndex helps to identify the columns of an MV index. We need this information for estimation. This logic is shared between the estimation logic and the access path generation logic. We'd like to put the mv index related functions together in the planner/core package. So we use this trick here to avoid the import cycle.

View Source
var RatioOfPseudoEstimate = atomic.NewFloat64(0.7)

RatioOfPseudoEstimate means if modifyCount / statsTblCount is greater than this ratio, we think the stats is invalid and use pseudo estimation.

UTCWithAllowInvalidDateCtx is introduced for the following reason:

Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics.
Hence, when reading statistics, we should skip invalid date check. See #39336.

Functions

func BuildHistAndTopN

func BuildHistAndTopN(
	ctx sessionctx.Context,
	numBuckets, numTopN int,
	id int64,
	collector *SampleCollector,
	tp *types.FieldType,
	isColumn bool,
	memTracker *memory.Tracker,
	needExtStats bool,
) (*Histogram, *TopN, error)

BuildHistAndTopN build a histogram and TopN for a column or an index from samples.

func CMSketchAndTopNFromProto

func CMSketchAndTopNFromProto(protoSketch *tipb.CMSketch) (*CMSketch, *TopN)

CMSketchAndTopNFromProto converts CMSketch and TopN from its protobuf representation.

func CMSketchToProto

func CMSketchToProto(c *CMSketch, topn *TopN) *tipb.CMSketch

CMSketchToProto converts CMSketch to its protobuf representation.

func CheckAnalyzeVerOnTable

func CheckAnalyzeVerOnTable(tbl *Table, version *int) bool

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.

func CheckEmptyTopNs

func CheckEmptyTopNs(topNs []*TopN) bool

CheckEmptyTopNs checks whether all TopNs are empty.

func ColAndIdxExistenceMapIsEqual

func ColAndIdxExistenceMapIsEqual(m1, m2 *ColAndIdxExistenceMap) bool

ColAndIdxExistenceMapIsEqual is used in testing, checking whether the two are equal.

func ColumnStatsIsInvalid

func ColumnStatsIsInvalid(colStats *Column, sctx planctx.PlanContext, histColl *HistColl, cid int64) (res bool)

ColumnStatsIsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it as need histogram.

func DebugTraceBuckets

func DebugTraceBuckets(s planctx.PlanContext, hg *Histogram, bucketIdxs []int)

DebugTraceBuckets is used to trace the buckets used in the histogram.

func DecodeCMSketchAndTopN

func DecodeCMSketchAndTopN(data []byte, topNRows []chunk.Row) (*CMSketch, *TopN, error)

DecodeCMSketchAndTopN decode a CMSketch from the given byte slice.

func EncodeCMSketchWithoutTopN

func EncodeCMSketchWithoutTopN(c *CMSketch) ([]byte, error)

EncodeCMSketchWithoutTopN encodes the given CMSketch to byte slice. Note that it does not include the topN.

func EncodeFMSketch

func EncodeFMSketch(c *FMSketch) ([]byte, error)

EncodeFMSketch encodes the given FMSketch to byte slice.

func EnumRangeValues

func EnumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []types.Datum

EnumRangeValues generates enum values between low and high.

func FMSketchToProto

func FMSketchToProto(s *FMSketch) *tipb.FMSketch

FMSketchToProto converts FMSketch to its protobuf representation.

func GetIndexPrefixLens

func GetIndexPrefixLens(data []byte, numCols int) (prefixLens []int, err error)

GetIndexPrefixLens returns an array representing

func GetMergedTopNFromSortedSlice

func GetMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta)

GetMergedTopNFromSortedSlice returns merged topn

func HistogramEqual

func HistogramEqual(a, b *Histogram, ignoreID bool) bool

HistogramEqual tests if two histograms are equal.

func HistogramToProto

func HistogramToProto(hg *Histogram) *tipb.Histogram

HistogramToProto converts Histogram to its protobuf representation. Note that when this is used, the lower/upper bound in the bucket must be BytesDatum.

func IndexStatsIsInvalid

func IndexStatsIsInvalid(sctx planctx.PlanContext, idxStats *Index, coll *HistColl, cid int64) (res bool)

IndexStatsIsInvalid checks whether the index has valid stats or not.

func MergeTopN

func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta)

MergeTopN is used to merge more TopN structures to generate a new TopN struct by the given size. The input parameters are multiple TopN structures to be merged and the size of the new TopN that will be generated. The output parameters are the newly generated TopN structure and the remaining numbers. Notice: The n can be 0. So n has no default value, we must explicitly specify this value.

func NewCMSketchAndTopN

func NewCMSketchAndTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uint64) (*CMSketch, *TopN, uint64, uint64)

NewCMSketchAndTopN returns a new CM sketch with TopN elements, the estimate NDV and the scale ratio.

func QueryValue

func QueryValue(sctx planctx.PlanContext, c *CMSketch, t *TopN, val types.Datum) (uint64, error)

QueryValue is used to query the count of specified value.

func RowSamplesToProto

func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample

RowSamplesToProto converts the samp slice to the pb struct.

func RowToDatums

func RowToDatums(row chunk.Row, fields []*resolve.ResultField) []types.Datum

RowToDatums converts row to datum slice.

func SampleCollectorToProto

func SampleCollectorToProto(c *SampleCollector) *tipb.SampleCollector

SampleCollectorToProto converts SampleCollector to its protobuf representation.

func SortTopnMeta

func SortTopnMeta(topnMetas []TopNMeta)

SortTopnMeta sort topnMeta

func TopnMetaCompare

func TopnMetaCompare(i, j TopNMeta) int

TopnMetaCompare compare topnMeta

func ValueToString

func ValueToString(vars *variable.SessionVars, value *types.Datum, idxCols int, idxColumnTypes []byte) (string, error)

ValueToString converts a possible encoded value to a formatted string. If the value is encoded, then idxCols equals to number of origin values, else idxCols is 0.

Types

type AnalyzeJob

type AnalyzeJob struct {
	StartTime     time.Time
	EndTime       time.Time
	ID            *uint64
	DBName        string
	TableName     string
	PartitionName string
	JobInfo       string

	SampleRateReason string // why this sample-rate is chosen
	Progress         AnalyzeProgress
}

AnalyzeJob is used to represent the status of one analyze job.

type AnalyzeProgress

type AnalyzeProgress struct {
	// contains filtered or unexported fields
}

AnalyzeProgress represents the process of one analyze job.

func (*AnalyzeProgress) GetDeltaCount

func (p *AnalyzeProgress) GetDeltaCount() int64

GetDeltaCount returns the delta count which hasn't been dumped into mysql.analyze_jobs.

func (*AnalyzeProgress) GetLastDumpTime

func (p *AnalyzeProgress) GetLastDumpTime() time.Time

GetLastDumpTime returns the last dump time.

func (*AnalyzeProgress) SetLastDumpTime

func (p *AnalyzeProgress) SetLastDumpTime(t time.Time)

SetLastDumpTime sets the last dump time.

func (*AnalyzeProgress) Update

func (p *AnalyzeProgress) Update(rowCount int64) int64

Update adds rowCount to the delta count. If the updated delta count reaches threshold, it returns the delta count for dumping it into mysql.analyze_jobs and resets the delta count to 0. Otherwise, it returns 0.

type AnalyzeResult

type AnalyzeResult struct {
	Hist    []*Histogram
	Cms     []*CMSketch
	TopNs   []*TopN
	Fms     []*FMSketch
	IsIndex int
}

AnalyzeResult is used to represent analyze result.

func (*AnalyzeResult) DestroyAndPutToPool

func (a *AnalyzeResult) DestroyAndPutToPool()

DestroyAndPutToPool destroys the result and put it to the pool.

type AnalyzeResults

type AnalyzeResults struct {
	Err      error
	ExtStats *ExtendedStatsColl
	Job      *AnalyzeJob
	Ars      []*AnalyzeResult
	TableID  AnalyzeTableID
	Count    int64
	StatsVer int
	// Snapshot is the snapshot timestamp when we start the analysis job.
	Snapshot uint64
	// BaseCount is the original count in mysql.stats_meta at the beginning of analyze.
	BaseCount int64
	// BaseModifyCnt is the original modify_count in mysql.stats_meta at the beginning of analyze.
	BaseModifyCnt int64
	// For multi-valued index analyze, there are some very different behaviors, so we add this field to indicate it.
	//
	// Analyze result of multi-valued index come from an independent v2 analyze index task (AnalyzeIndexExec), and it's
	// done by a scan on the index data and building stats. According to the original design rational of v2 stats, we
	// should use the same samples to build stats for all columns/indexes. We created an exceptional case here to avoid
	// loading the samples of JSON columns to tidb, which may cost too much memory, and we can't handle such case very
	// well now.
	//
	// As the definition of multi-valued index, the row count and NDV of this index may be higher than the table row
	// count. So we can't use this result to update the table-level row count.
	// The snapshot field is used by v2 analyze to check if there are concurrent analyze, so we also can't update it.
	// The multi-valued index analyze task is always together with another normal v2 analyze table task, which will
	// take care of those table-level fields.
	// In conclusion, when saving the analyze result for mv index, we need to store the index stats, as for the
	// table-level fields, we only need to update the version.
	ForMVIndex bool
}

AnalyzeResults represents the analyze results of a task.

func (*AnalyzeResults) DestroyAndPutToPool

func (a *AnalyzeResults) DestroyAndPutToPool()

DestroyAndPutToPool destroys the result and put it to the pool.

type AnalyzeTableID

type AnalyzeTableID struct {
	TableID int64
	// PartitionID is used for the construction of partition table statistics. It indicate the ID of the partition.
	// If the table is not the partition table, the PartitionID will be equal to NonPartitionTableID.
	PartitionID int64
}

AnalyzeTableID is hybrid table id used to analyze table.

func (*AnalyzeTableID) Equals

func (h *AnalyzeTableID) Equals(t *AnalyzeTableID) bool

Equals indicates whether two table id is equal.

func (*AnalyzeTableID) GetStatisticsID

func (h *AnalyzeTableID) GetStatisticsID() int64

GetStatisticsID is used to obtain the table ID to build statistics. If the 'PartitionID == NonPartitionTableID', we use the TableID to build the statistics for non-partition tables. Otherwise, we use the PartitionID to build the statistics of the partitions in the partition tables.

func (*AnalyzeTableID) IsPartitionTable

func (h *AnalyzeTableID) IsPartitionTable() bool

IsPartitionTable indicates whether the table is partition table.

func (*AnalyzeTableID) String

func (h *AnalyzeTableID) String() string

type BernoulliRowSampleCollector

type BernoulliRowSampleCollector struct {
	SampleRate float64
	// contains filtered or unexported fields
}

BernoulliRowSampleCollector collects the samples from the source and organize the sample by row. It will maintain the following things:

Row samples.
FM sketches(To calculate the NDV).
Null counts.
The data sizes.
The number of rows.

It uses the bernoulli sampling to collect the data.

func NewBernoulliRowSampleCollector

func NewBernoulliRowSampleCollector(sampleRate float64, totalLen int) *BernoulliRowSampleCollector

NewBernoulliRowSampleCollector creates the new collector by the given inputs.

func (*BernoulliRowSampleCollector) Base

func (s *BernoulliRowSampleCollector) Base() *baseCollector

Base implements the interface RowSampleCollector.

func (*BernoulliRowSampleCollector) DestroyAndPutToPool

func (s *BernoulliRowSampleCollector) DestroyAndPutToPool()

DestroyAndPutToPool implements the interface RowSampleCollector.

func (BernoulliRowSampleCollector) FromProto

func (s BernoulliRowSampleCollector) FromProto(pbCollector *tipb.RowSampleCollector, memTracker *memory.Tracker)

func (*BernoulliRowSampleCollector) MergeCollector

func (s *BernoulliRowSampleCollector) MergeCollector(subCollector RowSampleCollector)

MergeCollector merges the collectors to a final one.

func (BernoulliRowSampleCollector) ToProto

func (s BernoulliRowSampleCollector) ToProto() *tipb.RowSampleCollector

ToProto converts the collector to pb struct.

type Bucket

type Bucket struct {
	// Count is the number of items till this bucket.
	Count int64
	// Repeat is the number of times the upper-bound value of the bucket appears in the data.
	// For example, in the range [x, y], Repeat indicates how many times y appears.
	// It is used to estimate the row count of values equal to the upper bound of the bucket, similar to TopN.
	Repeat int64
	// NDV is the number of distinct values in the bucket.
	NDV int64
}

Bucket store the bucket count and repeat.

type CMSketch

type CMSketch struct {
	// contains filtered or unexported fields
}

CMSketch is used to estimate point queries. Refer: https://en.wikipedia.org/wiki/Count-min_sketch

func DecodeCMSketch

func DecodeCMSketch(data []byte) (*CMSketch, error)

DecodeCMSketch encodes the given CMSketch to byte slice.

func NewCMSketch

func NewCMSketch(d, w int32) *CMSketch

NewCMSketch returns a new CM sketch.

func (*CMSketch) CalcDefaultValForAnalyze

func (c *CMSketch) CalcDefaultValForAnalyze(ndv uint64)

CalcDefaultValForAnalyze calculate the default value for Analyze. The value of it is count / NDV in CMSketch. This means count and NDV are not include topN.

func (*CMSketch) Copy

func (c *CMSketch) Copy() *CMSketch

Copy makes a copy for current CMSketch.

func (*CMSketch) Equal

func (c *CMSketch) Equal(rc *CMSketch) bool

Equal tests if two CM Sketch equal, it is only used for test.

func (*CMSketch) GetWidthAndDepth

func (c *CMSketch) GetWidthAndDepth() (width, depth int32)

GetWidthAndDepth returns the width and depth of CM Sketch.

func (*CMSketch) InsertBytes

func (c *CMSketch) InsertBytes(bytes []byte)

InsertBytes inserts the bytes value into the CM Sketch.

func (*CMSketch) InsertBytesByCount

func (c *CMSketch) InsertBytesByCount(bytes []byte, count uint64)

InsertBytesByCount adds the bytes value into the TopN (if value already in TopN) or CM Sketch by delta, this does not updates c.defaultValue.

func (*CMSketch) MemoryUsage

func (c *CMSketch) MemoryUsage() (sum int64)

MemoryUsage returns the total memory usage of a CMSketch. only calc the hashtable size(CMSketch.table) and the CMSketch.topN data are not tracked because size of CMSketch.topN take little influence We ignore the size of other metadata in CMSketch.

func (*CMSketch) MergeCMSketch

func (c *CMSketch) MergeCMSketch(rc *CMSketch) error

MergeCMSketch merges two CM Sketch.

func (*CMSketch) QueryBytes

func (c *CMSketch) QueryBytes(d []byte) uint64

QueryBytes is used to query the count of specified bytes.

func (*CMSketch) SubValue

func (c *CMSketch) SubValue(h1, h2 uint64, count uint64)

SubValue remove a value from the CMSketch.

func (*CMSketch) TotalCount

func (c *CMSketch) TotalCount() uint64

TotalCount returns the total count in the sketch, it is only used for test.

type CacheItemMemoryUsage

type CacheItemMemoryUsage interface {
	ItemID() int64
	TotalMemoryUsage() int64
	TrackingMemUsage() int64
	HistMemUsage() int64
	TopnMemUsage() int64
	CMSMemUsage() int64
}

CacheItemMemoryUsage indicates the memory usage of TableCacheItem

type ColAndIdxExistenceMap

type ColAndIdxExistenceMap struct {
	// contains filtered or unexported fields
}

ColAndIdxExistenceMap is the meta map for statistics.Table. It can tell whether a column/index really has its statistics. So we won't send useless kv request when we do online stats loading.

func NewColAndIndexExistenceMap

func NewColAndIndexExistenceMap(colCap, idxCap int) *ColAndIdxExistenceMap

NewColAndIndexExistenceMap return a new object with the given capcity.

func (*ColAndIdxExistenceMap) Clone

Clone deeply copies the map.

func (*ColAndIdxExistenceMap) ColNum

func (m *ColAndIdxExistenceMap) ColNum() int

ColNum returns the number of columns in the map.

func (*ColAndIdxExistenceMap) GetCol

GetCol gets the meta data of the given column.

func (*ColAndIdxExistenceMap) GetIndex

func (m *ColAndIdxExistenceMap) GetIndex(id int64) *model.IndexInfo

GetIndex gets the meta data of the given index.

func (*ColAndIdxExistenceMap) Has

func (m *ColAndIdxExistenceMap) Has(id int64, isIndex bool) bool

Has checks whether a column/index stats exists. This method only checks whether the given item exists or not. Don't check whether it has statistics or not.

func (*ColAndIdxExistenceMap) HasAnalyzed

func (m *ColAndIdxExistenceMap) HasAnalyzed(id int64, isIndex bool) bool

HasAnalyzed checks whether a column/index stats exists and it has stats. TODO: the map should only keep the analyzed cols. There's three possible status of column/index's statistics:

  1. We don't have this column/index.
  2. We have it, but it hasn't been analyzed yet.
  3. We have it and its statistics.

To figure out three status, we use HasAnalyzed's TRUE value to represents the status 3. The Has's FALSE to represents the status 1.

func (*ColAndIdxExistenceMap) InsertCol

func (m *ColAndIdxExistenceMap) InsertCol(id int64, info *model.ColumnInfo, analyzed bool)

InsertCol inserts a column with its meta into the map.

func (*ColAndIdxExistenceMap) InsertIndex

func (m *ColAndIdxExistenceMap) InsertIndex(id int64, info *model.IndexInfo, analyzed bool)

InsertIndex inserts an index with its meta into the map.

func (*ColAndIdxExistenceMap) IsEmpty

func (m *ColAndIdxExistenceMap) IsEmpty() bool

IsEmpty checks whether the map is empty.

type Column

type Column struct {
	LastAnalyzePos types.Datum
	CMSketch       *CMSketch
	TopN           *TopN
	FMSketch       *FMSketch
	Info           *model.ColumnInfo
	Histogram

	// StatsLoadedStatus indicates the status of column statistics
	StatsLoadedStatus
	// PhysicalID is the physical table id,
	// or it could possibly be -1, which means "stats not available".
	// The -1 case could happen in a pseudo stats table, and in this case, this stats should not trigger stats loading.
	PhysicalID int64
	Flag       int64
	StatsVer   int64 // StatsVer is the version of the current stats, used to maintain compatibility

	IsHandle bool
}

Column represents a column histogram.

func (*Column) Copy

func (c *Column) Copy() *Column

Copy copies the column.

func (*Column) DropUnnecessaryData

func (c *Column) DropUnnecessaryData()

DropUnnecessaryData drops the unnecessary data for the column.

func (*Column) GetEvictedStatus

func (c *Column) GetEvictedStatus() int

GetEvictedStatus indicates the evicted status

func (*Column) GetIncreaseFactor

func (c *Column) GetIncreaseFactor(realtimeRowCount int64) float64

GetIncreaseFactor get the increase factor to adjust the final estimated count when the table is modified.

func (*Column) GetStatsVer

func (c *Column) GetStatsVer() int64

GetStatsVer indicates the stats version

func (*Column) IsAllEvicted

func (c *Column) IsAllEvicted() bool

IsAllEvicted indicates whether all stats evicted

func (*Column) IsAnalyzed

func (c *Column) IsAnalyzed() bool

IsAnalyzed indicates whether the column is analyzed. The set of IsAnalyzed columns is a subset of the set of StatsAvailable columns.

func (*Column) IsCMSExist

func (c *Column) IsCMSExist() bool

IsCMSExist indicates whether CMSketch exists

func (*Column) IsStatsInitialized

func (c *Column) IsStatsInitialized() bool

IsStatsInitialized indicates whether stats is initialized

func (*Column) ItemID

func (c *Column) ItemID() int64

ItemID implements TableCacheItem

func (*Column) MemoryUsage

func (c *Column) MemoryUsage() CacheItemMemoryUsage

MemoryUsage returns the total memory usage of Histogram, CMSketch, FMSketch in Column. We ignore the size of other metadata in Column

func (*Column) NotNullCount

func (c *Column) NotNullCount() float64

NotNullCount returns the count of this column which is not null.

func (*Column) StatsAvailable

func (c *Column) StatsAvailable() bool

StatsAvailable indicates whether the column stats are collected. Note:

  1. The function merely talks about whether the stats are collected, regardless of the stats loaded status.
  2. The function is used to decide StatsLoadedStatus.statsInitialized when reading the column stats from storage.
  3. There are two cases that StatsAvailable is true: a. IsAnalyzed is true. b. The column is newly-added/modified and its stats are generated according to the default value.

func (*Column) String

func (c *Column) String() string

func (*Column) TotalRowCount

func (c *Column) TotalRowCount() float64

TotalRowCount returns the total count of this column.

type ColumnMemUsage

type ColumnMemUsage struct {
	ColumnID          int64
	HistogramMemUsage int64
	CMSketchMemUsage  int64
	FMSketchMemUsage  int64
	TopNMemUsage      int64
	TotalMemUsage     int64
}

ColumnMemUsage records column memory usage

func (*ColumnMemUsage) CMSMemUsage

func (c *ColumnMemUsage) CMSMemUsage() int64

CMSMemUsage implements CacheItemMemoryUsage

func (*ColumnMemUsage) HistMemUsage

func (c *ColumnMemUsage) HistMemUsage() int64

HistMemUsage implements CacheItemMemoryUsage

func (*ColumnMemUsage) ItemID

func (c *ColumnMemUsage) ItemID() int64

ItemID implements CacheItemMemoryUsage

func (*ColumnMemUsage) TopnMemUsage

func (c *ColumnMemUsage) TopnMemUsage() int64

TopnMemUsage implements CacheItemMemoryUsage

func (*ColumnMemUsage) TotalMemoryUsage

func (c *ColumnMemUsage) TotalMemoryUsage() int64

TotalMemoryUsage implements CacheItemMemoryUsage

func (*ColumnMemUsage) TrackingMemUsage

func (c *ColumnMemUsage) TrackingMemUsage() int64

TrackingMemUsage implements CacheItemMemoryUsage

type DatumMapCache

type DatumMapCache struct {
	// contains filtered or unexported fields
}

DatumMapCache is used to store the mapping from the string type to datum type. The datum is used to find the value in the histogram.

func NewDatumMapCache

func NewDatumMapCache() *DatumMapCache

NewDatumMapCache creates a new DatumMapCache.

func (*DatumMapCache) Get

func (d *DatumMapCache) Get(key hack.MutableString) (val types.Datum, ok bool)

Get gets the datum from the cache.

func (*DatumMapCache) Put

func (d *DatumMapCache) Put(val TopNMeta, encodedVal hack.MutableString,
	tp byte, isIndex bool, loc *time.Location) (dat types.Datum, err error)

Put puts the datum into the cache.

type ExtendedStatsColl

type ExtendedStatsColl struct {
	Stats             map[string]*ExtendedStatsItem
	LastUpdateVersion uint64
}

ExtendedStatsColl is a collection of cached items for mysql.stats_extended records.

func BuildExtendedStats

func BuildExtendedStats(sctx sessionctx.Context,
	tableID int64, cols []*model.ColumnInfo, collectors []*SampleCollector) (*ExtendedStatsColl, error)

BuildExtendedStats build extended stats for column groups if needed based on the column samples.

func NewExtendedStatsColl

func NewExtendedStatsColl() *ExtendedStatsColl

NewExtendedStatsColl allocate an ExtendedStatsColl struct.

type ExtendedStatsItem

type ExtendedStatsItem struct {
	StringVals string
	ColIDs     []int64
	ScalarVals float64
	Tp         uint8
}

ExtendedStatsItem is the cached item of a mysql.stats_extended record.

type FMSketch

type FMSketch struct {
	// contains filtered or unexported fields
}

FMSketch (Flajolet-Martin Sketch) is a probabilistic data structure that estimates the count of unique elements in a stream. It employs a hash function to convert each element into a binary number and then counts the trailing zeroes in each hashed value. **This variant of the FM sketch uses a set to store unique hashed values and a binary mask to track the maximum number of trailing zeroes.** The estimated count of distinct values is calculated as 2^r * count, where 'r' is the maximum number of trailing zeroes observed and 'count' is the number of unique hashed values. The fundamental idea is that our hash function maps the input domain onto a logarithmic scale. This is achieved by hashing the input value and counting the number of trailing zeroes in the binary representation of the hash value. Each distinct value is mapped to 'i' with a probability of 2^-(i+1). For example, a value is mapped to 0 with a probability of 1/2, to 1 with a probability of 1/4, to 2 with a probability of 1/8, and so on. This is achieved by hashing the input value and counting the trailing zeroes in the hash value. If we have a set of 'n' distinct values, the count of distinct values with 'r' trailing zeroes is n / 2^r. Therefore, the estimated count of distinct values is 2^r * count = n. The level-by-level approach increases the accuracy of the estimation by ensuring a minimum count of distinct values at each level. This way, the final estimation is less likely to be skewed by outliers. For more details, refer to the following papers:

  1. https://www.vldb.org/conf/2001/P541.pdf
  2. https://algo.inria.fr/flajolet/Publications/FlMa85.pdf

func DecodeFMSketch

func DecodeFMSketch(data []byte) (*FMSketch, error)

DecodeFMSketch decode a FMSketch from the given byte slice.

func FMSketchFromProto

func FMSketchFromProto(protoSketch *tipb.FMSketch) *FMSketch

FMSketchFromProto converts FMSketch from its protobuf representation.

func NewFMSketch

func NewFMSketch(maxSize int) *FMSketch

NewFMSketch returns a new FM sketch.

func (*FMSketch) Copy

func (s *FMSketch) Copy() *FMSketch

Copy makes a copy for current FMSketch.

func (*FMSketch) DestroyAndPutToPool

func (s *FMSketch) DestroyAndPutToPool()

DestroyAndPutToPool resets the FMSketch and puts it to the pool.

func (*FMSketch) InsertRowValue

func (s *FMSketch) InsertRowValue(sc *stmtctx.StatementContext, values []types.Datum) error

InsertRowValue inserts multi-column values to the sketch.

func (*FMSketch) InsertValue

func (s *FMSketch) InsertValue(sc *stmtctx.StatementContext, value types.Datum) error

InsertValue inserts a value into the FM sketch.

func (*FMSketch) MemoryUsage

func (s *FMSketch) MemoryUsage() (sum int64)

MemoryUsage returns the total memory usage of a FMSketch.

func (*FMSketch) MergeFMSketch

func (s *FMSketch) MergeFMSketch(rs *FMSketch)

MergeFMSketch merges two FM Sketch.

func (*FMSketch) NDV

func (s *FMSketch) NDV() int64

NDV returns the estimated number of distinct values (NDV) in the sketch.

type HistColl

type HistColl struct {
	PhysicalID int64
	// TODO: add AnalyzeCount here
	RealtimeCount int64 // RealtimeCount is the current table row count, maintained by applying stats delta based on AnalyzeCount.
	ModifyCount   int64 // Total modify count in a table.

	// The version of the statistics, refer to Version0, Version1, Version2 and so on.
	StatsVer int
	// HavePhysicalID is true means this HistColl is from single table and have its ID's information.
	// The physical id is used when try to load column stats from storage.
	HavePhysicalID bool
	Pseudo         bool

	CanNotTriggerLoad bool
	// Idx2ColUniqueIDs maps the index id to its column UniqueIDs. It's used to calculate the selectivity in planner.
	Idx2ColUniqueIDs map[int64][]int64
	// ColUniqueID2IdxIDs maps the column UniqueID to a list index ids whose first column is it.
	// It's used to calculate the selectivity in planner.
	ColUniqueID2IdxIDs map[int64][]int64
	// UniqueID2colInfoID maps the column UniqueID to its ID in the metadata.
	UniqueID2colInfoID map[int64]int64
	// MVIdx2Columns maps the index id to its columns by expression.Column.
	// For normal index, the column id is enough, as we already have in Idx2ColUniqueIDs. But currently, mv index needs more
	// information to match the filter against the mv index columns, and we need this map to provide this information.
	MVIdx2Columns map[int64][]*expression.Column
	// contains filtered or unexported fields
}

HistColl is a collection of histograms. It collects enough information for plan to calculate the selectivity.

func NewHistColl

func NewHistColl(id int64, havePhysicalID bool, realtimeCnt, modifyCnt int64, colNum, idxNum int) *HistColl

NewHistColl creates a new HistColl.

func NewHistCollWithColsAndIdxs

func NewHistCollWithColsAndIdxs(id int64, havePhysicalID bool, realtimeCnt, modifyCnt int64, cols map[int64]*Column, idxs map[int64]*Index) *HistColl

NewHistCollWithColsAndIdxs creates a new HistColl with given columns and indices.

func (*HistColl) CalcPreScalar

func (coll *HistColl) CalcPreScalar()

CalcPreScalar calculates the pre-calculated scalar for all columns and indices.

func (*HistColl) ColNum

func (coll *HistColl) ColNum() int

ColNum returns the number of columns in the HistColl.

func (*HistColl) DelCol

func (coll *HistColl) DelCol(id int64)

DelCol deletes the column with the given id.

func (*HistColl) DelIdx

func (coll *HistColl) DelIdx(id int64)

DelIdx deletes the index with the given id.

func (*HistColl) DropEvicted

func (coll *HistColl) DropEvicted()

DropEvicted will drop the unnecessary data for all columns and indices. It's triggerred by stats cache.

func (*HistColl) ForEachColumnImmutable

func (coll *HistColl) ForEachColumnImmutable(f func(int64, *Column) bool)

ForEachColumnImmutable iterates all columns in the HistColl. The bool return value of f is used to control the iteration. If f returns true, the iteration will be stopped. Warning: Don't change the content when calling this function.

func (*HistColl) ForEachIndexImmutable

func (coll *HistColl) ForEachIndexImmutable(f func(int64, *Index) bool)

ForEachIndexImmutable iterates all columns in the HistColl. The bool return value of f is used to control the iteration. If f returns true, the iteration will be stopped. WARNING: Don't change the content when calling this function.

func (*HistColl) GenerateHistCollFromColumnInfo

func (coll *HistColl) GenerateHistCollFromColumnInfo(tblInfo *model.TableInfo, columns []*expression.Column) *HistColl

GenerateHistCollFromColumnInfo generates a new HistColl whose ColUniqueID2IdxIDs and Idx2ColUniqueIDs is built from the given parameter.

func (*HistColl) GetAnalyzeRowCount

func (coll *HistColl) GetAnalyzeRowCount() float64

GetAnalyzeRowCount tries to get the row count of a column or an index if possible. This method is useful because this row count doesn't consider the modify count.

func (*HistColl) GetCol

func (coll *HistColl) GetCol(id int64) *Column

GetCol gets the column with the given id.

func (*HistColl) GetIdx

func (coll *HistColl) GetIdx(id int64) *Index

GetIdx gets the index with the given id.

func (*HistColl) GetScaledRealtimeAndModifyCnt

func (coll *HistColl) GetScaledRealtimeAndModifyCnt(idxStats *Index) (realtimeCnt, modifyCnt int64)

GetScaledRealtimeAndModifyCnt scale the RealtimeCount and ModifyCount for some special indexes where the total row count is different from the total row count of the table. Currently, only the mv index is this case. Because we will use the RealtimeCount and ModifyCount during the estimation for ranges on this index (like the upper bound for the out-of-range estimation logic and the IncreaseFactor logic), we can't directly use the RealtimeCount and ModifyCount of the table. Instead, we should scale them before using. For example, if the table analyze row count is 1000 and realtime row count is 1500, and the mv index total count is 5000, when calculating the IncreaseFactor, it should be 1500/1000 = 1.5 for normal columns/indexes, and we should use the same 1.5 for mv index. But obviously, use 1500/5000 would be wrong, the correct calculation should be 7500/5000 = 1.5. So we add this function to get this 7500.

func (*HistColl) ID2UniqueID

func (coll *HistColl) ID2UniqueID(columns []*expression.Column) *HistColl

ID2UniqueID generates a new HistColl whose `Columns` is built from UniqueID of given columns.

func (*HistColl) IdxNum

func (coll *HistColl) IdxNum() int

IdxNum returns the number of indices in the HistColl.

func (*HistColl) SetAllIndexFullLoadForBootstrap

func (coll *HistColl) SetAllIndexFullLoadForBootstrap()

SetAllIndexFullLoadForBootstrap sets all indices' stats loaded status to full load for bootstrap.

func (*HistColl) SetCol

func (coll *HistColl) SetCol(id int64, col *Column)

SetCol sets the column with the given id.

func (*HistColl) SetIdx

func (coll *HistColl) SetIdx(id int64, idx *Index)

SetIdx sets the index with the given id.

func (*HistColl) StableOrderColSlice

func (coll *HistColl) StableOrderColSlice() []*Column

StableOrderColSlice returns a slice of columns in stable order.

func (*HistColl) StableOrderIdxSlice

func (coll *HistColl) StableOrderIdxSlice() []*Index

StableOrderIdxSlice returns a slice of indices in stable order.

type Histogram

type Histogram struct {
	Tp *types.FieldType

	// Histogram elements.
	//
	// A bucket bound is the smallest and greatest values stored in the bucket. The lower and upper bound
	// are stored in one column.
	//
	// A bucket count is the number of items stored in all previous buckets and the current bucket.
	// Bucket counts are always in increasing order.
	//
	// A bucket repeat is the number of repeats of the bucket value, it can be used to find popular values.
	Bounds  *chunk.Chunk
	Buckets []Bucket

	// Used for estimating fraction of the interval [lower, upper] that lies within the [lower, value].
	// For some types like `Int`, we do not build it because we can get them directly from `Bounds`.
	Scalars   []scalar
	ID        int64 // Column ID.
	NDV       int64 // Number of distinct values.
	NullCount int64 // Number of null values.
	// LastUpdateVersion is the version that this histogram updated last time.
	LastUpdateVersion uint64

	// 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
	// the column values. This ranges from -1 to +1, and it is only valid for Column histogram, not for
	// Index histogram.
	Correlation float64
}

Histogram represents statistics for a column or index.

func BuildColumn

func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error)

BuildColumn builds histogram from samples for column.

func BuildColumnHist

func BuildColumnHist(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType, count int64, ndv int64, nullCount int64) (*Histogram, error)

BuildColumnHist build a histogram for a column. numBuckets: number of buckets for the histogram. id: the id of the table. collector: the collector of samples. tp: the FieldType for the column. count: represents the row count for the column. ndv: represents the number of distinct values for the column. nullCount: represents the number of null values for the column.

func HistogramFromProto

func HistogramFromProto(protoHg *tipb.Histogram) *Histogram

HistogramFromProto converts Histogram from its protobuf representation. Note that we will set BytesDatum for the lower/upper bound in the bucket, the decode will be after all histograms merged.

func MergeHistograms

func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, bucketSize int, statsVer int) (*Histogram, error)

MergeHistograms merges two histograms.

func MergePartitionHist2GlobalHist

func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histogram, popedTopN []TopNMeta, expBucketNumber int64, isIndex bool) (*Histogram, error)

MergePartitionHist2GlobalHist merges hists (partition-level Histogram) to a global-level Histogram

func NewHistogram

func NewHistogram(id, ndv, nullCount int64, version uint64, tp *types.FieldType, bucketSize int, totColSize int64) *Histogram

NewHistogram creates a new histogram.

func (*Histogram) AppendBucket

func (hg *Histogram) AppendBucket(lower *types.Datum, upper *types.Datum, count, repeat int64)

AppendBucket appends a bucket into `hg`.

func (*Histogram) AppendBucketWithNDV

func (hg *Histogram) AppendBucketWithNDV(lower *types.Datum, upper *types.Datum, count, repeat, ndv int64)

AppendBucketWithNDV appends a bucket into `hg` and set value for field `NDV`.

func (*Histogram) AvgCountPerNotNullValue

func (hg *Histogram) AvgCountPerNotNullValue(totalCount int64) float64

AvgCountPerNotNullValue gets the average row count per value by the data of histogram.

func (*Histogram) BetweenRowCount

func (hg *Histogram) BetweenRowCount(sctx planctx.PlanContext, a, b types.Datum) float64

BetweenRowCount estimates the row count where column greater or equal to a and less than b. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*Histogram) BinarySearchRemoveVal

func (hg *Histogram) BinarySearchRemoveVal(valCntPairs TopNMeta)

BinarySearchRemoveVal removes the value from the TopN using binary search.

func (*Histogram) BucketCount

func (hg *Histogram) BucketCount(idx int) int64

BucketCount returns the count of the bucket with index idx.

func (*Histogram) BucketToString

func (hg *Histogram) BucketToString(bktID, idxCols int) string

BucketToString change the given bucket to string format.

func (*Histogram) ConvertTo

func (hg *Histogram) ConvertTo(tctx types.Context, tp *types.FieldType) (*Histogram, error)

ConvertTo converts the histogram bucket values into `tp`.

func (*Histogram) Copy

func (hg *Histogram) Copy() *Histogram

Copy deep copies the histogram.

func (*Histogram) DecodeTo

func (hg *Histogram) DecodeTo(tp *types.FieldType, timeZone *time.Location) error

DecodeTo decodes the histogram bucket values into `tp`.

func (*Histogram) DestroyAndPutToPool

func (hg *Histogram) DestroyAndPutToPool()

DestroyAndPutToPool resets the FMSketch and puts it to the pool.

func (*Histogram) EqualRowCount

func (hg *Histogram) EqualRowCount(sctx planctx.PlanContext, value types.Datum, hasBucketNDV bool) (count float64, matched bool)

EqualRowCount estimates the row count where the column equals to value. matched: return true if this returned row count is from Bucket.Repeat or bucket NDV, which is more accurate than if not. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*Histogram) ExtractTopN

func (hg *Histogram) ExtractTopN(cms *CMSketch, topN *TopN, numCols int, numTopN uint32) error

ExtractTopN extracts topn from histogram.

func (*Histogram) GetIncreaseFactor

func (hg *Histogram) GetIncreaseFactor(totalCount int64) float64

GetIncreaseFactor will return a factor of data increasing after the last analysis.

func (*Histogram) GetLower

func (hg *Histogram) GetLower(idx int) *types.Datum

GetLower gets the lower bound of bucket `idx`.

func (*Histogram) GetUpper

func (hg *Histogram) GetUpper(idx int) *types.Datum

GetUpper gets the upper bound of bucket `idx`.

func (*Histogram) GreaterRowCount

func (hg *Histogram) GreaterRowCount(value types.Datum) float64

GreaterRowCount estimates the row count where the column greater than value. It's deprecated. Only used for test.

func (*Histogram) IsIndexHist

func (hg *Histogram) IsIndexHist() bool

IsIndexHist checks whether current histogram is one for index.

func (*Histogram) Len

func (hg *Histogram) Len() int

Len is the number of buckets in the histogram.

func (*Histogram) LessRowCount

func (hg *Histogram) LessRowCount(sctx planctx.PlanContext, value types.Datum) float64

LessRowCount estimates the row count where the column less than value. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*Histogram) LessRowCountWithBktIdx

func (hg *Histogram) LessRowCountWithBktIdx(sctx planctx.PlanContext, value types.Datum) (result float64, bucketIdx int)

LessRowCountWithBktIdx estimates the row count where the column less than value. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*Histogram) LocateBucket

func (hg *Histogram) LocateBucket(sctx planctx.PlanContext, value types.Datum) (exceed bool, bucketIdx int, inBucket, matchLastValue bool)

LocateBucket locates where a value falls in the range of the Histogram. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

Return value: exceed: if the value is larger than the upper bound of the last Bucket of the Histogram. bucketIdx: assuming exceed if false, which Bucket does this value fall in (note: the range before a Bucket is also considered belong to this Bucket). inBucket: assuming exceed if false, whether this value falls in this Bucket, instead of falls between this Bucket and the previous Bucket. matchLastValue: assuming inBucket is true, if this value is the last value in this Bucket, which has a counter (Bucket.Repeat).

Examples: val0 |<-[bkt0]->| |<-[bkt1]->val1(last value)| val2 |<--val3--[bkt2]->| |<-[bkt3]->| val4 locateBucket(val0): false, 0, false, false locateBucket(val1): false, 1, true, true locateBucket(val2): false, 2, false, false locateBucket(val3): false, 2, true, false locateBucket(val4): true, 3, false, false

func (*Histogram) LowerToDatum

func (hg *Histogram) LowerToDatum(idx int, d *types.Datum)

LowerToDatum gets the lower bound of bucket `idx` to datum.

func (*Histogram) MemoryUsage

func (hg *Histogram) MemoryUsage() (sum int64)

MemoryUsage returns the total memory usage of this Histogram.

func (*Histogram) NotNullCount

func (hg *Histogram) NotNullCount() float64

NotNullCount indicates the count of non-null values in column histogram and single-column index histogram, for multi-column index histogram, since we cannot define null for the row, we treat all rows as non-null, that means, notNullCount would return same value as TotalRowCount for multi-column index histograms.

func (*Histogram) OutOfRange

func (hg *Histogram) OutOfRange(val types.Datum) bool

OutOfRange checks if the datum is out of range.

func (*Histogram) OutOfRangeRowCount

func (hg *Histogram) OutOfRangeRowCount(
	sctx planctx.PlanContext,
	lDatum, rDatum *types.Datum,
	modifyCount, histNDV int64, increaseFactor float64,
) (result float64)

OutOfRangeRowCount estimate the row count of part of [lDatum, rDatum] which is out of range of the histogram. Here we assume the density of data is decreasing from the lower/upper bound of the histogram toward outside. The maximum row count it can get is the modifyCount. It reaches the maximum when out-of-range width reaches histogram range width. As it shows below. To calculate the out-of-range row count, we need to calculate the percentage of the shaded area. Note that we assume histL-boundL == histR-histL == boundR-histR here.

          /│             │\
        /  │             │  \
      /x│  │◄─histogram─►│    \
    / xx│  │    range    │      \
  / │xxx│  │             │        \
/   │xxx│  │             │          \

────┴────┴───┴──┴─────────────┴───────────┴─────

   ▲    ▲   ▲  ▲             ▲           ▲
   │    │   │  │             │           │
boundL  │   │histL         histR       boundR
        │   │
   lDatum  rDatum

The percentage of shaded area on the left side calculation formula is: leftPercent = (math.Pow(actualR-boundL, 2) - math.Pow(actualL-boundL, 2)) / math.Pow(histWidth, 2) You can find more details at https://github.com/pingcap/tidb/pull/47966#issuecomment-1778866876

func (*Histogram) PreCalculateScalar

func (hg *Histogram) PreCalculateScalar()

PreCalculateScalar converts the lower and upper to scalar. When the datum type is KindString or KindBytes, we also calculate their common prefix length, because when a value falls between lower and upper, the common prefix of lower and upper equals to the common prefix of the lower, upper and the value. For some simple types like `Int64`, we do not convert it because we can directly infer the scalar value.

func (*Histogram) RemoveVals

func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta)

RemoveVals remove the given values from the histogram. This function contains an **ASSUMPTION**: valCntPairs is sorted in ascending order.

func (*Histogram) SplitRange

func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, oldRanges []*ranger.Range, encoded bool) ([]*ranger.Range, bool)

SplitRange splits the range according to the histogram lower bound. Note that we treat first bucket's lower bound as -inf and last bucket's upper bound as +inf, so all the split ranges will totally fall in one of the (-inf, l(1)), [l(1), l(2)),...[l(n-2), l(n-1)), [l(n-1), +inf), where n is the number of buckets, l(i) is the i-th bucket's lower bound.

func (*Histogram) StandardizeForV2AnalyzeIndex

func (hg *Histogram) StandardizeForV2AnalyzeIndex()

StandardizeForV2AnalyzeIndex fixes some "irregular" places in the Histogram, which come from current implementation of analyze index task in v2. For now, it does two things: 1. Remove empty buckets. 2. Reset Bucket.NDV to 0.

func (*Histogram) ToString

func (hg *Histogram) ToString(idxCols int) string

ToString gets the string representation for the histogram.

func (*Histogram) TotalRowCount

func (hg *Histogram) TotalRowCount() float64

TotalRowCount returns the total count of this histogram.

func (*Histogram) TruncateHistogram

func (hg *Histogram) TruncateHistogram(numBkt int) *Histogram

TruncateHistogram truncates the histogram to `numBkt` buckets.

func (*Histogram) UpperToDatum

func (hg *Histogram) UpperToDatum(idx int, d *types.Datum)

UpperToDatum gets the upper bound of bucket `idx` to datum.

type Index

type Index struct {
	LastAnalyzePos types.Datum
	CMSketch       *CMSketch
	TopN           *TopN
	FMSketch       *FMSketch
	Info           *model.IndexInfo
	Histogram
	StatsLoadedStatus
	StatsVer int64 // StatsVer is the version of the current stats, used to maintain compatibility
	Flag     int64
	// PhysicalID is the physical table id,
	// or it could possibly be -1, which means "stats not available".
	// The -1 case could happen in a pseudo stats table, and in this case, this stats should not trigger stats loading.
	PhysicalID int64
}

Index represents an index histogram.

func (*Index) Copy

func (idx *Index) Copy() *Index

Copy copies the index.

func (*Index) DropUnnecessaryData

func (idx *Index) DropUnnecessaryData()

DropUnnecessaryData drops unnecessary data for index.

func (*Index) EvictAllStats

func (idx *Index) EvictAllStats()

EvictAllStats evicts all stats Note that this function is only used for test

func (*Index) GetEvictedStatus

func (idx *Index) GetEvictedStatus() int

GetEvictedStatus returns the evicted status

func (*Index) GetIncreaseFactor

func (idx *Index) GetIncreaseFactor(realtimeRowCount int64) float64

GetIncreaseFactor get the increase factor to adjust the final estimated count when the table is modified.

func (*Index) GetStatsVer

func (idx *Index) GetStatsVer() int64

GetStatsVer returns the version of the current stats

func (*Index) IsAllEvicted

func (idx *Index) IsAllEvicted() bool

IsAllEvicted indicates whether all stats evicted

func (*Index) IsAnalyzed

func (idx *Index) IsAnalyzed() bool

IsAnalyzed indicates whether the index is analyzed.

func (*Index) IsCMSExist

func (idx *Index) IsCMSExist() bool

IsCMSExist returns whether CMSketch exists.

func (*Index) IsEvicted

func (idx *Index) IsEvicted() bool

IsEvicted returns whether index statistics got evicted

func (*Index) ItemID

func (idx *Index) ItemID() int64

ItemID implements TableCacheItem

func (*Index) MemoryUsage

func (idx *Index) MemoryUsage() CacheItemMemoryUsage

MemoryUsage returns the total memory usage of a Histogram and CMSketch in Index. We ignore the size of other metadata in Index.

func (*Index) QueryBytes

func (idx *Index) QueryBytes(sctx planctx.PlanContext, d []byte) (result uint64)

QueryBytes is used to query the count of specified bytes. The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*Index) String

func (idx *Index) String() string

func (*Index) TotalRowCount

func (idx *Index) TotalRowCount() float64

TotalRowCount returns the total count of this index.

type IndexMemUsage

type IndexMemUsage struct {
	IndexID           int64
	HistogramMemUsage int64
	CMSketchMemUsage  int64
	TopNMemUsage      int64
	TotalMemUsage     int64
}

IndexMemUsage records index memory usage

func (*IndexMemUsage) CMSMemUsage

func (c *IndexMemUsage) CMSMemUsage() int64

CMSMemUsage implements CacheItemMemoryUsage

func (*IndexMemUsage) HistMemUsage

func (c *IndexMemUsage) HistMemUsage() int64

HistMemUsage implements CacheItemMemoryUsage

func (*IndexMemUsage) ItemID

func (c *IndexMemUsage) ItemID() int64

ItemID implements CacheItemMemoryUsage

func (*IndexMemUsage) TopnMemUsage

func (c *IndexMemUsage) TopnMemUsage() int64

TopnMemUsage implements CacheItemMemoryUsage

func (*IndexMemUsage) TotalMemoryUsage

func (c *IndexMemUsage) TotalMemoryUsage() int64

TotalMemoryUsage implements CacheItemMemoryUsage

func (*IndexMemUsage) TrackingMemUsage

func (c *IndexMemUsage) TrackingMemUsage() int64

TrackingMemUsage implements CacheItemMemoryUsage

type JobType

type JobType int

JobType is the type of the analyze job.

const (
	// TableAnalysisJob means the job is to analyze a table or partition.
	TableAnalysisJob JobType = iota + 1
	// GlobalStatsMergeJob means the job is to merge the global-level stats.
	GlobalStatsMergeJob
)

type ReservoirRowSampleCollector

type ReservoirRowSampleCollector struct {
	MaxSampleSize int
	// contains filtered or unexported fields
}

ReservoirRowSampleCollector collects the samples from the source and organize the samples by row. It will maintain the following things:

Row samples.
FM sketches(To calculate the NDV).
Null counts.
The data sizes.
The number of rows.

It uses weighted reservoir sampling(A-Res) to do the sampling.

func NewReservoirRowSampleCollector

func NewReservoirRowSampleCollector(maxSampleSize int, totalLen int) *ReservoirRowSampleCollector

NewReservoirRowSampleCollector creates the new collector by the given inputs.

func (*ReservoirRowSampleCollector) Base

func (s *ReservoirRowSampleCollector) Base() *baseCollector

Base implements the RowSampleCollector interface.

func (*ReservoirRowSampleCollector) DestroyAndPutToPool

func (s *ReservoirRowSampleCollector) DestroyAndPutToPool()

DestroyAndPutToPool implements the interface RowSampleCollector.

func (ReservoirRowSampleCollector) FromProto

func (s ReservoirRowSampleCollector) FromProto(pbCollector *tipb.RowSampleCollector, memTracker *memory.Tracker)

func (*ReservoirRowSampleCollector) MergeCollector

func (s *ReservoirRowSampleCollector) MergeCollector(subCollector RowSampleCollector)

MergeCollector merges the collectors to a final one.

func (ReservoirRowSampleCollector) ToProto

func (s ReservoirRowSampleCollector) ToProto() *tipb.RowSampleCollector

ToProto converts the collector to pb struct.

type ReservoirRowSampleItem

type ReservoirRowSampleItem struct {
	Handle  kv.Handle
	Columns []types.Datum
	Weight  int64
}

ReservoirRowSampleItem is the item for the ReservoirRowSampleCollector. The weight is needed for the sampling algorithm.

func (ReservoirRowSampleItem) MemUsage

func (i ReservoirRowSampleItem) MemUsage() (sum int64)

MemUsage returns the memory usage of sample item.

type RowSampleBuilder

type RowSampleBuilder struct {
	RecordSet       sqlexec.RecordSet
	Sc              *stmtctx.StatementContext
	Rng             *rand.Rand
	ColsFieldType   []*types.FieldType
	Collators       []collate.Collator
	ColGroups       [][]int64
	MaxSampleSize   int
	SampleRate      float64
	MaxFMSketchSize int
}

RowSampleBuilder is used to construct the ReservoirRowSampleCollector to get the samples.

func (*RowSampleBuilder) Collect

func (s *RowSampleBuilder) Collect() (RowSampleCollector, error)

Collect first builds the collector. Then maintain the null count, FM sketch and the data size for each column and column group. Then use the weighted reservoir sampling to collect the samples.

type RowSampleCollector

type RowSampleCollector interface {
	MergeCollector(collector RowSampleCollector)

	Base() *baseCollector
	DestroyAndPutToPool()
	// contains filtered or unexported methods
}

RowSampleCollector implements the needed interface for a row-based sample collector.

func NewRowSampleCollector

func NewRowSampleCollector(maxSampleSize int, sampleRate float64, totalLen int) RowSampleCollector

NewRowSampleCollector creates a collector from the given inputs.

type SampleBuilder

type SampleBuilder struct {
	RecordSet       sqlexec.RecordSet
	Sc              *stmtctx.StatementContext
	PkBuilder       *SortedBuilder
	Collators       []collate.Collator
	ColsFieldType   []*types.FieldType
	ColLen          int // ColLen is the number of columns need to be sampled.
	MaxBucketSize   int64
	MaxSampleSize   int64
	MaxFMSketchSize int64
	CMSketchDepth   int32
	CMSketchWidth   int32
}

SampleBuilder is used to build samples for columns. Also, if primary key is handle, it will directly build histogram for it.

func (SampleBuilder) CollectColumnStats

func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, error)

CollectColumnStats collects sample from the result set using Reservoir Sampling algorithm, and estimates NDVs using FM Sketch during the collecting process. It returns the sample collectors which contain total count, null count, distinct values count and CM Sketch. It also returns the statistic builder for PK which contains the histogram. See https://en.wikipedia.org/wiki/Reservoir_sampling

type SampleCollector

type SampleCollector struct {
	FMSketch *FMSketch
	CMSketch *CMSketch
	TopN     *TopN
	Samples  []*SampleItem

	NullCount     int64
	Count         int64 // Count is the number of non-null rows.
	MaxSampleSize int64
	TotalSize     int64 // TotalSize is the total size of column.
	MemSize       int64 // major memory size of this sample collector.
	IsMerger      bool
	// contains filtered or unexported fields
}

SampleCollector will collect Samples and calculate the count and ndv of an attribute.

func SampleCollectorFromProto

func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector

SampleCollectorFromProto converts SampleCollector from its protobuf representation.

func (*SampleCollector) CalcTotalSize

func (c *SampleCollector) CalcTotalSize()

CalcTotalSize is to calculate total size based on samples.

func (*SampleCollector) ExtractTopN

func (c *SampleCollector) ExtractTopN(numTop uint32, sc *stmtctx.StatementContext, tp *types.FieldType, timeZone *time.Location) error

ExtractTopN extracts the topn from the CM Sketch.

func (*SampleCollector) MergeSampleCollector

func (c *SampleCollector) MergeSampleCollector(sc *stmtctx.StatementContext, rc *SampleCollector)

MergeSampleCollector merges two sample collectors.

type SampleItem

type SampleItem struct {
	// Value is the sampled column value.
	Value types.Datum
	// Handle is the handle of the sample in its key.
	// This property is used to calculate Ordinal in fast analyze.
	Handle kv.Handle
	// Ordinal is original position of this item in SampleCollector before sorting. This
	// is used for computing correlation.
	Ordinal int
}

SampleItem is an item of sampled column value.

func CopySampleItems

func CopySampleItems(items []*SampleItem) []*SampleItem

CopySampleItems returns a deep copy of SampleItem slice.

type SortedBuilder

type SortedBuilder struct {
	Count int64
	// contains filtered or unexported fields
}

SortedBuilder is used to build histograms for PK and index.

func NewSortedBuilder

func NewSortedBuilder(sc *stmtctx.StatementContext, numBuckets, id int64, tp *types.FieldType, statsVer int) *SortedBuilder

NewSortedBuilder creates a new SortedBuilder.

func (*SortedBuilder) Hist

func (b *SortedBuilder) Hist() *Histogram

Hist returns the histogram built by SortedBuilder.

func (*SortedBuilder) Iterate

func (b *SortedBuilder) Iterate(data types.Datum) error

Iterate updates the histogram incrementally.

type StatsLoadedStatus

type StatsLoadedStatus struct {
	// contains filtered or unexported fields
}

StatsLoadedStatus indicates the status of statistics

func NewStatsAllEvictedStatus

func NewStatsAllEvictedStatus() StatsLoadedStatus

NewStatsAllEvictedStatus returns the status that only loads count/nullCount/NDV and doesn't load CMSketch/TopN/Histogram. When we load table stats, column stats is in AllEvicted status by default. CMSketch/TopN/Histogram of column is only loaded when we really need column stats.

func NewStatsFullLoadStatus

func NewStatsFullLoadStatus() StatsLoadedStatus

NewStatsFullLoadStatus returns the status that the column/index fully loaded

func (*StatsLoadedStatus) Copy

Copy copies the status

func (StatsLoadedStatus) IsAllEvicted

func (s StatsLoadedStatus) IsAllEvicted() bool

IsAllEvicted indicates whether all the stats got evicted or not.

func (StatsLoadedStatus) IsEssentialStatsLoaded

func (s StatsLoadedStatus) IsEssentialStatsLoaded() bool

IsEssentialStatsLoaded indicates whether the essential statistics is loaded. If the column/index was loaded, and at least histogram and topN still exists, the necessary statistics is still loaded.

func (StatsLoadedStatus) IsFullLoad

func (s StatsLoadedStatus) IsFullLoad() bool

IsFullLoad indicates whether the stats are full loaded

func (StatsLoadedStatus) IsLoadNeeded

func (s StatsLoadedStatus) IsLoadNeeded() bool

IsLoadNeeded indicates whether it needs load statistics during LoadNeededHistograms or sync stats If the column/index was loaded and any statistics of it is evicting, it also needs re-load statistics.

func (StatsLoadedStatus) IsStatsInitialized

func (s StatsLoadedStatus) IsStatsInitialized() bool

IsStatsInitialized indicates whether the column/index's statistics was loaded from storage before. Note that `IsStatsInitialized` only can be set in initializing

func (StatsLoadedStatus) StatusToString

func (s StatsLoadedStatus) StatusToString() string

StatusToString gets the string info of StatsLoadedStatus

type StatsTblTraceInfo

type StatsTblTraceInfo struct {
	Columns     []*statsTblColOrIdxInfo
	Indexes     []*statsTblColOrIdxInfo
	PhysicalID  int64
	Version     uint64
	Count       int64
	ModifyCount int64
}

StatsTblTraceInfo is simplified from Table and used for debug trace.

func TraceStatsTbl

func TraceStatsTbl(statsTbl *Table) *StatsTblTraceInfo

TraceStatsTbl converts a Table to StatsTblTraceInfo, which is suitable for the debug trace.

type Table

type Table struct {
	ExtendedStats *ExtendedStatsColl

	ColAndIdxExistenceMap *ColAndIdxExistenceMap
	HistColl
	Version uint64
	// It's the timestamp of the last analyze time.
	// We used it in auto-analyze to determine if this table has been analyzed.
	// The source of this field comes from two parts:
	// 1. Initialized by snapshot when loading stats_meta.
	// 2. Updated by the analysis time of a specific column or index when loading the histogram of the column or index.
	LastAnalyzeVersion uint64
	// TblInfoUpdateTS is the UpdateTS of the TableInfo used when filling this struct.
	// It is the schema version of the corresponding table. It is used to skip redundant
	// loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables,
	// and the schema of the table does not change, we don't need to load the stats for this
	// table again.
	TblInfoUpdateTS uint64

	IsPkIsHandle bool
}

Table represents statistics for a table.

func PseudoTable

func PseudoTable(tblInfo *model.TableInfo, allowTriggerLoading bool, allowFillHistMeta bool) *Table

PseudoTable creates a pseudo table statistics. Usually, we don't want to trigger stats loading for pseudo table. But there are exceptional cases. In such cases, we should pass allowTriggerLoading as true. Such case could possibly happen in getStatsTable().

func (*Table) ColumnByName

func (t *Table) ColumnByName(colName string) *Column

ColumnByName finds the statistics.Column for the given column.

func (*Table) ColumnIsLoadNeeded

func (t *Table) ColumnIsLoadNeeded(id int64, fullLoad bool) (*Column, bool, bool)

ColumnIsLoadNeeded checks whether the column needs trigger the async/sync load. The Column should be visible in the table and really has analyzed statistics in the stroage. Also, if the stats has been loaded into the memory, we also don't need to load it. We return the Column together with the checking result, to avoid accessing the map multiple times. The first bool is whether we have it in memory. The second bool is whether this column has stats in the system table or not.

func (*Table) Copy

func (t *Table) Copy() *Table

Copy copies the current table.

func (*Table) GetStatsHealthy

func (t *Table) GetStatsHealthy() (int64, bool)

GetStatsHealthy calculates stats healthy if the table stats is not pseudo. If the table stats is pseudo, it returns 0, false, otherwise it returns stats healthy, true.

func (*Table) GetStatsInfo

func (t *Table) GetStatsInfo(id int64, isIndex bool, needCopy bool) (*Histogram, *CMSketch, *TopN, *FMSketch, bool)

GetStatsInfo returns their statistics according to the ID of the column or index, including histogram, CMSketch, TopN and FMSketch.

needCopy: In order to protect the item in the cache from being damaged, we need to copy the item.

func (*Table) IndexIsLoadNeeded

func (t *Table) IndexIsLoadNeeded(id int64) (*Index, bool)

IndexIsLoadNeeded checks whether the index needs trigger the async/sync load. The Index should be visible in the table and really has analyzed statistics in the stroage. Also, if the stats has been loaded into the memory, we also don't need to load it. We return the Index together with the checking result, to avoid accessing the map multiple times.

func (*Table) IndexStartWithColumn

func (t *Table) IndexStartWithColumn(colName string) *Index

IndexStartWithColumn finds the first index whose first column is the given column.

func (*Table) IsAnalyzed

func (t *Table) IsAnalyzed() bool

IsAnalyzed checks whether the table is analyzed or not by checking its last analyze's timestamp value. A valid timestamp must be greater than 0.

func (*Table) IsEligibleForAnalysis

func (t *Table) IsEligibleForAnalysis() bool

IsEligibleForAnalysis checks whether the table is eligible for analysis.

func (*Table) IsInitialized

func (t *Table) IsInitialized() bool

IsInitialized returns true if any column/index stats of the table is initialized.

func (*Table) IsOutdated

func (t *Table) IsOutdated() bool

IsOutdated returns true if the table stats is outdated.

func (*Table) MemoryUsage

func (t *Table) MemoryUsage() *TableMemoryUsage

MemoryUsage returns the total memory usage of this Table. it will only calc the size of Columns and Indices stats data of table. We ignore the size of other metadata in Table

func (*Table) ReleaseAndPutToPool

func (t *Table) ReleaseAndPutToPool()

ReleaseAndPutToPool releases data structures of Table and put itself back to pool.

func (*Table) ShallowCopy

func (t *Table) ShallowCopy() *Table

ShallowCopy copies the current table. It's different from Copy(). Only the struct Table (and also the embedded HistColl) is copied here. The internal containers, like t.Columns and t.Indices, and the stats, like TopN and Histogram are not copied.

func (*Table) String

func (t *Table) String() string

String implements Stringer interface.

type TableCacheItem

type TableCacheItem interface {
	ItemID() int64
	MemoryUsage() CacheItemMemoryUsage
	IsAllEvicted() bool
	GetEvictedStatus() int

	DropUnnecessaryData()
	IsStatsInitialized() bool
	GetStatsVer() int64
}

TableCacheItem indicates the unit item stored in statsCache, eg: Column/Index

type TableMemoryUsage

type TableMemoryUsage struct {
	ColumnsMemUsage map[int64]CacheItemMemoryUsage
	IndicesMemUsage map[int64]CacheItemMemoryUsage
	TableID         int64
	TotalMemUsage   int64
}

TableMemoryUsage records tbl memory usage

func (*TableMemoryUsage) TotalColTrackingMemUsage

func (t *TableMemoryUsage) TotalColTrackingMemUsage() (sum int64)

TotalColTrackingMemUsage returns total columns' tracking memory usage

func (*TableMemoryUsage) TotalIdxTrackingMemUsage

func (t *TableMemoryUsage) TotalIdxTrackingMemUsage() (sum int64)

TotalIdxTrackingMemUsage returns total indices' tracking memory usage

func (*TableMemoryUsage) TotalTrackingMemUsage

func (t *TableMemoryUsage) TotalTrackingMemUsage() int64

TotalTrackingMemUsage return total tracking memory usage

type TopN

type TopN struct {
	TopN []TopNMeta
}

TopN stores most-common values, which is used to estimate point queries.

func DecodeTopN

func DecodeTopN(topNRows []chunk.Row) *TopN

DecodeTopN decodes a TopN from the given byte slice.

func NewTopN

func NewTopN(n int) *TopN

NewTopN creates the new TopN struct by the given size.

func TopNFromProto

func TopNFromProto(protoTopN []*tipb.CMSketchTopN) *TopN

TopNFromProto converts TopN from its protobuf representation.

func (*TopN) AppendTopN

func (c *TopN) AppendTopN(data []byte, count uint64)

AppendTopN appends a topn into the TopN struct.

func (*TopN) BetweenCount

func (c *TopN) BetweenCount(sctx planctx.PlanContext, l, r []byte) (result uint64)

BetweenCount estimates the row count for interval [l, r). The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*TopN) Copy

func (c *TopN) Copy() *TopN

Copy makes a copy for current TopN.

func (*TopN) DecodedString

func (c *TopN) DecodedString(ctx sessionctx.Context, colTypes []byte) (string, error)

DecodedString returns the value with decoded result.

func (*TopN) Equal

func (c *TopN) Equal(cc *TopN) bool

Equal checks whether the two TopN are equal.

func (*TopN) FindTopN

func (c *TopN) FindTopN(d []byte) int

FindTopN finds the index of the given value in the TopN.

func (*TopN) LowerBound

func (c *TopN) LowerBound(d []byte) (idx int, match bool)

LowerBound searches on the sorted top-n items, returns the smallest index i such that the value at element i is not less than `d`.

func (*TopN) MemoryUsage

func (c *TopN) MemoryUsage() (sum int64)

MemoryUsage returns the total memory usage of a topn.

func (*TopN) Num

func (c *TopN) Num() int

Num returns the ndv of the TopN.

TopN is declared directly in Histogram. So the Len is occupied by the Histogram. We use Num instead.

func (*TopN) QueryTopN

func (c *TopN) QueryTopN(sctx planctx.PlanContext, d []byte) (result uint64, found bool)

QueryTopN returns the results for (h1, h2) in murmur3.Sum128(), if not exists, return (0, false). The input sctx is just for debug trace, you can pass nil safely if that's not needed.

func (*TopN) RemoveVal

func (c *TopN) RemoveVal(val []byte)

RemoveVal remove the val from TopN if it exists.

func (*TopN) Scale

func (c *TopN) Scale(scaleFactor float64)

Scale scales the TopN by the given factor.

func (*TopN) Sort

func (c *TopN) Sort()

Sort sorts the topn items.

func (*TopN) String

func (c *TopN) String() string

func (*TopN) TotalCount

func (c *TopN) TotalCount() uint64

TotalCount returns how many data is stored in TopN.

type TopNMeta

type TopNMeta struct {
	Encoded []byte
	Count   uint64
}

TopNMeta stores the unit of the TopN.

func MergeTopNAndUpdateCMSketch

func MergeTopNAndUpdateCMSketch(dst, src *TopN, c *CMSketch, numTop uint32) []TopNMeta

MergeTopNAndUpdateCMSketch merges the src TopN into the dst, and spilled values will be inserted into the CMSketch.

type WeightedRowSampleHeap

type WeightedRowSampleHeap []*ReservoirRowSampleItem

WeightedRowSampleHeap implements the Heap interface.

func (WeightedRowSampleHeap) Len

func (h WeightedRowSampleHeap) Len() int

Len implements the Heap interface.

func (WeightedRowSampleHeap) Less

func (h WeightedRowSampleHeap) Less(i, j int) bool

Less implements the Heap interface.

func (*WeightedRowSampleHeap) Pop

func (h *WeightedRowSampleHeap) Pop() any

Pop implements the Heap interface.

func (*WeightedRowSampleHeap) Push

func (h *WeightedRowSampleHeap) Push(i any)

Push implements the Heap interface.

func (WeightedRowSampleHeap) Swap

func (h WeightedRowSampleHeap) Swap(i, j int)

Swap implements the Heap interface.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL