Skip to content

Commit

Permalink
*: build and use count-min sketch (pingcap#5042)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and zz-jason committed Nov 8, 2017
1 parent 49f2a90 commit 30a080d
Show file tree
Hide file tree
Showing 17 changed files with 393 additions and 158 deletions.
10 changes: 10 additions & 0 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,7 @@ const (
null_count bigint(64) NOT NULL DEFAULT 0,
modify_count bigint(64) NOT NULL DEFAULT 0,
version bigint(64) unsigned NOT NULL DEFAULT 0,
cm_sketch blob,
unique index tbl(table_id, is_index, hist_id)
);`

Expand Down Expand Up @@ -227,6 +228,7 @@ const (
version13 = 13
version14 = 14
version15 = 15
version16 = 16
)

func checkBootstrapped(s Session) (bool, error) {
Expand Down Expand Up @@ -339,6 +341,10 @@ func upgrade(s Session) {
upgradeToVer15(s)
}

if ver < version16 {
upgradeToVer16(s)
}

updateBootstrapVer(s)
_, err = s.Execute(goctx.Background(), "COMMIT")

Expand Down Expand Up @@ -540,6 +546,10 @@ func upgradeToVer15(s Session) {
}
}

func upgradeToVer16(s Session) {
doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `cm_sketch` blob", infoschema.ErrColumnExists)
}

// updateBootstrapVer updates bootstrap version variable in mysql.TiDB table.
func updateBootstrapVer(s Session) {
// Update bootstrap version.
Expand Down
4 changes: 2 additions & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -594,8 +594,8 @@ func (do *Domain) updateStatsWorker(ctx context.Context, lease time.Duration) {
log.Error("[stats] handle ddl event fail: ", errors.ErrorStack(err))
}
case t := <-statsHandle.AnalyzeResultCh():
for _, hg := range t.Hist {
err := hg.SaveToStorage(ctx, t.TableID, t.Count, t.IsIndex)
for i, hg := range t.Hist {
err := statistics.SaveStatsToStorage(ctx, t.TableID, t.Count, t.IsIndex, hg, t.Cms[i])
if err != nil {
log.Error("[stats] save histogram to storage fail: ", errors.ErrorStack(err))
}
Expand Down
2 changes: 1 addition & 1 deletion executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ func (s *testSuite) TestAggregation(c *C) {

result = tk.MustQuery("select count(*) from information_schema.columns")
// When adding new memory table in information_schema, please update this variable.
columnCountOfAllInformationSchemaTables := "742"
columnCountOfAllInformationSchemaTables := "743"
result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables))

tk.MustExec("drop table if exists t1")
Expand Down
66 changes: 41 additions & 25 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,12 @@ type AnalyzeExec struct {
}

const (
maxSampleSize = 10000
maxRegionSampleSize = 1000
maxSketchSize = 10000
maxBucketSize = 256
maxSampleSize = 10000
maxRegionSampleSize = 1000
maxSketchSize = 10000
maxBucketSize = 256
defaultCMSketchDepth = 8
defaultCMSketchWidth = 2048
)

// Schema implements the Executor Schema interface.
Expand Down Expand Up @@ -111,8 +113,8 @@ func (e *AnalyzeExec) Next() (Row, error) {
return nil, errors.Trace(err1)
}
for _, result := range results {
for _, hg := range result.Hist {
err = hg.SaveToStorage(e.ctx, result.TableID, result.Count, result.IsIndex)
for i, hg := range result.Hist {
err = statistics.SaveStatsToStorage(e.ctx, result.TableID, result.Count, result.IsIndex, hg, result.Cms[i])
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -160,13 +162,14 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<-
}

func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) statistics.AnalyzeResult {
hist, err := idxExec.buildHistogram()
hist, cms, err := idxExec.buildStats()
if err != nil {
return statistics.AnalyzeResult{Err: err}
}
result := statistics.AnalyzeResult{
TableID: idxExec.tblInfo.ID,
Hist: []*statistics.Histogram{hist},
Cms: []*statistics.CMSketch{cms},
IsIndex: 1,
}
if len(hist.Buckets) > 0 {
Expand Down Expand Up @@ -208,47 +211,56 @@ func (e *AnalyzeIndexExec) open() error {
return nil
}

func (e *AnalyzeIndexExec) buildHistogram() (hist *statistics.Histogram, err error) {
func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statistics.CMSketch, err error) {
if err = e.open(); err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
defer func() {
if err1 := e.result.Close(); err1 != nil {
hist = nil
cms = nil
err = errors.Trace(err1)
}
}()
hist = &statistics.Histogram{}
cms = statistics.NewCMSketch(defaultCMSketchDepth, defaultCMSketchWidth)
for {
data, err := e.result.NextRaw()
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
if data == nil {
break
}
resp := &tipb.AnalyzeIndexResp{}
err = resp.Unmarshal(data)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, statistics.HistogramFromProto(resp.Hist), maxBucketSize)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
if resp.Cms != nil {
err := cms.MergeCMSketch(statistics.CMSketchFromProto(resp.Cms))
if err != nil {
return nil, nil, errors.Trace(err)
}
}
}
hist.ID = e.idxInfo.ID
return hist, nil
return hist, cms, nil
}

func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) statistics.AnalyzeResult {
hists, err := colExec.buildHistograms()
hists, cms, err := colExec.buildStats()
if err != nil {
return statistics.AnalyzeResult{Err: err}
}
result := statistics.AnalyzeResult{
TableID: colExec.tblInfo.ID,
Hist: hists,
Cms: cms,
}
hist := hists[0]
result.Count = hist.NullCount
Expand Down Expand Up @@ -292,13 +304,14 @@ func (e *AnalyzeColumnsExec) open() error {
return nil
}

func (e *AnalyzeColumnsExec) buildHistograms() (hists []*statistics.Histogram, err error) {
func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []*statistics.CMSketch, err error) {
if err = e.open(); err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
defer func() {
if err1 := e.result.Close(); err1 != nil {
hists = nil
cms = nil
err = errors.Trace(err1)
}
}()
Expand All @@ -307,27 +320,28 @@ func (e *AnalyzeColumnsExec) buildHistograms() (hists []*statistics.Histogram, e
for i := range collectors {
collectors[i] = &statistics.SampleCollector{
IsMerger: true,
Sketch: statistics.NewFMSketch(maxSketchSize),
FMSketch: statistics.NewFMSketch(maxSketchSize),
MaxSampleSize: maxSampleSize,
CMSketch: statistics.NewCMSketch(defaultCMSketchDepth, defaultCMSketchWidth),
}
}
for {
data, err1 := e.result.NextRaw()
if err1 != nil {
return nil, errors.Trace(err1)
return nil, nil, errors.Trace(err1)
}
if data == nil {
break
}
resp := &tipb.AnalyzeColumnsResp{}
err = resp.Unmarshal(data)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
if e.pkInfo != nil {
pkHist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, pkHist, statistics.HistogramFromProto(resp.PkHist), maxBucketSize)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
}
for i, rc := range resp.Collectors {
Expand All @@ -340,27 +354,29 @@ func (e *AnalyzeColumnsExec) buildHistograms() (hists []*statistics.Histogram, e
for i, bkt := range pkHist.Buckets {
pkHist.Buckets[i].LowerBound, err = tablecodec.DecodeColumnValue(bkt.LowerBound.GetBytes(), &e.pkInfo.FieldType, timeZone)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
pkHist.Buckets[i].UpperBound, err = tablecodec.DecodeColumnValue(bkt.UpperBound.GetBytes(), &e.pkInfo.FieldType, timeZone)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
}
hists = append(hists, pkHist)
cms = append(cms, nil)
}
for i, col := range e.colsInfo {
for j, s := range collectors[i].Samples {
collectors[i].Samples[j], err = tablecodec.DecodeColumnValue(s.GetBytes(), &col.FieldType, timeZone)
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
}
hg, err := statistics.BuildColumn(e.ctx, maxBucketSize, col.ID, collectors[i])
if err != nil {
return nil, errors.Trace(err)
return nil, nil, errors.Trace(err)
}
hists = append(hists, hg)
cms = append(cms, collectors[i].CMSketch)
}
return hists, nil
return hists, cms, nil
}
18 changes: 14 additions & 4 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -834,6 +834,12 @@ func (b *executorBuilder) buildAnalyzeIndexPushdown(task plan.AnalyzeIndexTask)
BucketSize: maxBucketSize,
NumColumns: int32(len(task.IndexInfo.Columns)),
}
if !task.IndexInfo.Unique {
depth := int32(defaultCMSketchDepth)
width := int32(defaultCMSketchWidth)
e.analyzePB.IdxReq.CmsketchDepth = &depth
e.analyzePB.IdxReq.CmsketchWidth = &width
}
return e
}

Expand All @@ -859,11 +865,15 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plan.AnalyzeColumnsTa
TimeZoneOffset: timeZoneOffset(b.ctx),
},
}
depth := int32(defaultCMSketchDepth)
width := int32(defaultCMSketchWidth)
e.analyzePB.ColReq = &tipb.AnalyzeColumnsReq{
BucketSize: maxBucketSize,
SampleSize: maxRegionSampleSize,
SketchSize: maxSketchSize,
ColumnsInfo: distsql.ColumnsToProto(cols, task.TableInfo.PKIsHandle),
BucketSize: maxBucketSize,
SampleSize: maxRegionSampleSize,
SketchSize: maxSketchSize,
ColumnsInfo: distsql.ColumnsToProto(cols, task.TableInfo.PKIsHandle),
CmsketchDepth: &depth,
CmsketchWidth: &width,
}
b.err = setPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, cols)
return e
Expand Down
2 changes: 1 addition & 1 deletion session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1125,7 +1125,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er

const (
notBootstrapped = 0
currentBootstrapVersion = 15
currentBootstrapVersion = 16
)

func getStoreBootstrapVersion(store kv.Storage) int64 {
Expand Down
29 changes: 2 additions & 27 deletions statistics/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,9 @@ package statistics

import (
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
)

// SortedBuilder is used to build histograms for PK and index.
Expand Down Expand Up @@ -108,30 +106,6 @@ func (b *SortedBuilder) Iterate(data types.Datum) error {
return nil
}

// BuildIndex builds histogram for index.
func BuildIndex(ctx context.Context, numBuckets, id int64, records ast.RecordSet) (int64, *Histogram, error) {
b := NewSortedBuilder(ctx.GetSessionVars().StmtCtx, numBuckets, id)
for {
row, err := records.Next()
if err != nil {
return 0, nil, errors.Trace(err)
}
if row == nil {
break
}
bytes, err := codec.EncodeKey(nil, row.Data...)
if err != nil {
return 0, nil, errors.Trace(err)
}
data := types.NewBytesDatum(bytes)
err = b.Iterate(data)
if err != nil {
return 0, nil, errors.Trace(err)
}
}
return b.Count, b.Hist(), nil
}

// BuildColumn builds histogram from samples for column.
func BuildColumn(ctx context.Context, numBuckets, id int64, collector *SampleCollector) (*Histogram, error) {
count := collector.Count
Expand All @@ -144,7 +118,7 @@ func BuildColumn(ctx context.Context, numBuckets, id int64, collector *SampleCol
if err != nil {
return nil, errors.Trace(err)
}
ndv := collector.Sketch.NDV()
ndv := collector.FMSketch.NDV()
if ndv > count {
ndv = count
}
Expand Down Expand Up @@ -205,6 +179,7 @@ func BuildColumn(ctx context.Context, numBuckets, id int64, collector *SampleCol
type AnalyzeResult struct {
TableID int64
Hist []*Histogram
Cms []*CMSketch
Count int64
IsIndex int
Err error
Expand Down
Loading

0 comments on commit 30a080d

Please sign in to comment.