From a9f7ade0c95555e934ee278d2bd8adba3d2e9096 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Oct 2018 23:10:15 +0800 Subject: [PATCH 01/23] planner, statistics: build new histogram using range information --- plan/logical_plans.go | 2 +- plan/property/stats_info.go | 2 +- plan/stats.go | 21 ++- statistics/feedback.go | 24 ++-- statistics/feedback_test.go | 4 +- statistics/histogram.go | 234 ++++++++++++++++++++++++++++++--- statistics/scalar.go | 41 ++++++ statistics/selectivity.go | 96 ++++++++------ statistics/selectivity_test.go | 8 +- statistics/table.go | 12 +- statistics/update_test.go | 2 +- util/ranger/types.go | 21 +++ 12 files changed, 379 insertions(+), 88 deletions(-) diff --git a/plan/logical_plans.go b/plan/logical_plans.go index e5bddebfc429c..583726253dbd2 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -448,7 +448,7 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { path.countAfterAccess = math.Min(ds.stats.RowCount/selectionFactor, float64(ds.statisticTable.Count)) } if path.indexFilters != nil { - selectivity, err := ds.stats.HistColl.Selectivity(ds.ctx, path.indexFilters) + selectivity, _, err := ds.stats.HistColl.Selectivity(ds.ctx, path.indexFilters) if err != nil { log.Warnf("An error happened: %v, we have to use the default selectivity", err.Error()) selectivity = selectionFactor diff --git a/plan/property/stats_info.go b/plan/property/stats_info.go index 67c112c20d256..8881f044d7ba5 100644 --- a/plan/property/stats_info.go +++ b/plan/property/stats_info.go @@ -24,7 +24,7 @@ type StatsInfo struct { RowCount float64 Cardinality []float64 - HistColl statistics.HistColl + HistColl *statistics.HistColl // UsePseudoStats indicates whether the StatsInfo is calculated using the // pseudo statistics on a table. UsePseudoStats bool diff --git a/plan/stats.go b/plan/stats.go index 3e270e02047d5..db9776b66e57b 100644 --- a/plan/stats.go +++ b/plan/stats.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/plan/property" + "github.com/pingcap/tidb/statistics" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) @@ -60,7 +61,7 @@ func (p *baseLogicalPlan) deriveStats() (*property.StatsInfo, error) { return profile, nil } -func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *property.StatsInfo { +func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) (*property.StatsInfo, *statistics.HistColl) { profile := &property.StatsInfo{ RowCount: float64(ds.statisticTable.Count), Cardinality: make([]float64, len(ds.Columns)), @@ -77,12 +78,20 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *property.Stat } } ds.stats = profile - selectivity, err := profile.HistColl.Selectivity(ds.ctx, conds) + selectivity, nodes, err := profile.HistColl.Selectivity(ds.ctx, conds) + log.Warnf("selectivity: %v", selectivity) if err != nil { log.Warnf("An error happened: %v, we have to use the default selectivity", err.Error()) selectivity = selectionFactor } - return profile.Scale(selectivity) + if ds.ctx.GetSessionVars().OptimizerSelectivityLevel >= 1 && ds.stats.HistColl != nil { + finalHist, err := ds.stats.HistColl.NewHistCollBySelectivity(ds.ctx.GetSessionVars().StmtCtx, nodes) + if err != nil { + log.Warnf("[stats-in-datasource]: An error happened: %v", err.Error()) + } + return profile, finalHist + } + return profile.Scale(selectivity), nil } func (ds *DataSource) deriveStats() (*property.StatsInfo, error) { @@ -90,7 +99,8 @@ func (ds *DataSource) deriveStats() (*property.StatsInfo, error) { for i, expr := range ds.pushedDownConds { ds.pushedDownConds[i] = expression.PushDownNot(nil, expr, false) } - ds.stats = ds.getStatsByFilter(ds.pushedDownConds) + var finalHist *statistics.HistColl + ds.stats, finalHist = ds.getStatsByFilter(ds.pushedDownConds) for _, path := range ds.possibleAccessPaths { if path.isTablePath { noIntervalRanges, err := ds.deriveTablePathStats(path) @@ -116,6 +126,9 @@ func (ds *DataSource) deriveStats() (*property.StatsInfo, error) { break } } + if ds.ctx.GetSessionVars().OptimizerSelectivityLevel >= 1 { + ds.stats.HistColl = finalHist + } return ds.stats, nil } diff --git a/statistics/feedback.go b/statistics/feedback.go index 6477aa57e42ad..bd9244c0e6949 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -45,7 +45,7 @@ type feedback struct { repeat int64 } -// QueryFeedback is used to represent the query feedback info. It contains the query's scan ranges and number of rows +// QueryFeedback is used to represent the query feedback info. It contains the query's scan Ranges and number of rows // in each range. type QueryFeedback struct { tableID int64 @@ -78,7 +78,7 @@ func NewQueryFeedback(tableID int64, hist *Histogram, expected int64, desc bool) } var ( - // MaxNumberOfRanges is the max number of ranges before split to collect feedback. + // MaxNumberOfRanges is the max number of Ranges before split to collect feedback. MaxNumberOfRanges = 20 // FeedbackProbability is the probability to collect the feedback. FeedbackProbability = 0.0 @@ -86,7 +86,7 @@ var ( // CollectFeedback decides whether to collect the feedback. It returns false when: // 1: the histogram is nil or has no buckets; -// 2: the number of scan ranges exceeds the limit because it may affect the performance; +// 2: the number of scan Ranges exceeds the limit because it may affect the performance; // 3: it does not pass the probabilistic sampler. func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { if q.hist == nil || q.hist.Len() == 0 { @@ -100,7 +100,7 @@ func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { return true } -// DecodeToRanges decode the feedback to ranges. +// DecodeToRanges decode the feedback to Ranges. func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { ranges := make([]*ranger.Range, 0, len(q.feedback)) for _, val := range q.feedback { @@ -158,7 +158,7 @@ func (q *QueryFeedback) decodeIntValues() *QueryFeedback { return nq } -// StoreRanges stores the ranges for update. +// StoreRanges stores the Ranges for update. func (q *QueryFeedback) StoreRanges(ranges []*ranger.Range) { q.feedback = make([]feedback, 0, len(ranges)) for _, ran := range ranges { @@ -545,7 +545,7 @@ func UpdateCMSketch(c *CMSketch, eqFeedbacks []feedback) *CMSketch { } func buildNewHistogram(h *Histogram, buckets []bucket) *Histogram { - hist := NewHistogram(h.ID, h.NDV, h.NullCount, h.LastUpdateVersion, h.tp, len(buckets), h.TotColSize) + hist := NewHistogram(h.ID, h.NDV, h.NullCount, h.LastUpdateVersion, h.Tp, len(buckets), h.TotColSize) preCount := int64(0) for _, bkt := range buckets { hist.AppendBucket(bkt.lower, bkt.upper, bkt.count+preCount, bkt.repeat) @@ -561,7 +561,7 @@ type queryFeedback struct { HashValues []uint64 IndexRanges [][]byte // Counts is the number of scan keys in each range. It first stores the count for `IntRanges`, `IndexRanges` or `ColumnRanges`. - // After that, it stores the ranges for `HashValues`. + // After that, it stores the Ranges for `HashValues`. Counts []int64 ColumnRanges [][]byte } @@ -746,7 +746,7 @@ func (q *QueryFeedback) recalculateExpectCount(h *Handle) error { if tablePseudo == false { return nil } - isIndex := q.hist.tp.Tp == mysql.TypeBlob + isIndex := q.hist.Tp.Tp == mysql.TypeBlob id := q.hist.ID if isIndex && (t.Indices[id] == nil || t.Indices[id].NotAccurate() == false) { return nil @@ -947,7 +947,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { } ranges, err := q.DecodeToRanges(true) if err != nil { - log.Debug("decode feedback ranges failed: ", err) + log.Debug("decode feedback Ranges failed: ", err) return nil } for i, ran := range ranges { @@ -981,7 +981,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { continue } if err != nil { - log.Debug("get row count by ranges failed: ", err) + log.Debug("get row count by Ranges failed: ", err) continue } @@ -1016,10 +1016,10 @@ func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCou return nil } if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = getMinValue(k, q.hist.tp) + ran.LowVal[0] = getMinValue(k, q.hist.Tp) } if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = getMaxValue(k, q.hist.tp) + ran.HighVal[0] = getMaxValue(k, q.hist.Tp) } } ranges := q.hist.SplitRange([]*ranger.Range{ran}) diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 9b0907a2b6a0e..33d7cbd6133a2 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -202,7 +202,7 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { } defaultBucketCount = t.bucketCount bkts = mergeBuckets(bkts, t.isNewBuckets, float64(totalCount)) - result := buildNewHistogram(&Histogram{tp: types.NewFieldType(mysql.TypeLong)}, bkts).ToString(0) + result := buildNewHistogram(&Histogram{Tp: types.NewFieldType(mysql.TypeLong)}, bkts).ToString(0) c.Assert(result, Equals, t.result) } } @@ -228,7 +228,7 @@ func (s *testFeedbackSuite) TestFeedbackEncoding(c *C) { } c.Assert(q.Equal(rq), IsTrue) - hist.tp = types.NewFieldType(mysql.TypeBlob) + hist.Tp = types.NewFieldType(mysql.TypeBlob) q = &QueryFeedback{hist: hist} q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(3), 1, 0}) q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(1), 1, 0}) diff --git a/statistics/histogram.go b/statistics/histogram.go index a7730d07d975d..c441f338ea8c4 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" + "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -44,7 +45,7 @@ type Histogram struct { // LastUpdateVersion is the version that this histogram updated last time. LastUpdateVersion uint64 - tp *types.FieldType + Tp *types.FieldType // Histogram elements. // @@ -84,7 +85,7 @@ func NewHistogram(id, ndv, nullCount int64, version uint64, tp *types.FieldType, NDV: ndv, NullCount: nullCount, LastUpdateVersion: version, - tp: tp, + Tp: tp, Bounds: chunk.NewChunkWithCapacity([]*types.FieldType{tp}, 2*bucketSize), Buckets: make([]Bucket, 0, bucketSize), TotColSize: totColSize, @@ -93,13 +94,13 @@ func NewHistogram(id, ndv, nullCount int64, version uint64, tp *types.FieldType, // GetLower gets the lower bound of bucket `idx`. func (hg *Histogram) GetLower(idx int) *types.Datum { - d := hg.Bounds.GetRow(2*idx).GetDatum(0, hg.tp) + d := hg.Bounds.GetRow(2*idx).GetDatum(0, hg.Tp) return &d } // GetUpper gets the upper bound of bucket `idx`. func (hg *Histogram) GetUpper(idx int) *types.Datum { - d := hg.Bounds.GetRow(2*idx+1).GetDatum(0, hg.tp) + d := hg.Bounds.GetRow(2*idx+1).GetDatum(0, hg.Tp) return &d } @@ -108,7 +109,7 @@ func (c *Column) AvgColSize(count int64) float64 { if count == 0 { return 0 } - switch c.Histogram.tp.Tp { + switch c.Histogram.Tp.Tp { case mysql.TypeFloat: return 4 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, @@ -138,11 +139,11 @@ func (hg *Histogram) updateLastBucket(upper *types.Datum, count, repeat int64) { hg.Buckets[len-1] = Bucket{Count: count, Repeat: repeat} } -// DecodeTo decodes the histogram bucket values into `tp`. +// DecodeTo decodes the histogram bucket values into `Tp`. func (hg *Histogram) DecodeTo(tp *types.FieldType, timeZone *time.Location) error { oldIter := chunk.NewIterator4Chunk(hg.Bounds) hg.Bounds = chunk.NewChunkWithCapacity([]*types.FieldType{tp}, oldIter.Len()) - hg.tp = tp + hg.Tp = tp for row := oldIter.Begin(); row != oldIter.End(); row = oldIter.Next() { datum, err := tablecodec.DecodeColumnValue(row.GetBytes(0), tp, timeZone) if err != nil { @@ -153,12 +154,12 @@ func (hg *Histogram) DecodeTo(tp *types.FieldType, timeZone *time.Location) erro return nil } -// ConvertTo converts the histogram bucket values into `tp`. +// ConvertTo converts the histogram bucket values into `Tp`. func (hg *Histogram) ConvertTo(sc *stmtctx.StatementContext, tp *types.FieldType) (*Histogram, error) { hist := NewHistogram(hg.ID, hg.NDV, hg.NullCount, hg.LastUpdateVersion, tp, hg.Len(), hg.TotColSize) iter := chunk.NewIterator4Chunk(hg.Bounds) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - d := row.GetDatum(0, hg.tp) + d := row.GetDatum(0, hg.Tp) d, err := d.ConvertTo(sc, tp) if err != nil { return nil, errors.Trace(err) @@ -385,7 +386,7 @@ func (hg *Histogram) equalRowCount(value types.Datum) float64 { return hg.totalRowCount() / float64(hg.NDV) } if match { - cmp := chunk.GetCompareFunc(hg.tp) + cmp := chunk.GetCompareFunc(hg.Tp) if cmp(hg.Bounds.GetRow(index), 0, hg.Bounds.GetRow(index+1), 0) == 0 { return float64(hg.Buckets[index/2].Repeat) } @@ -468,7 +469,7 @@ func (hg *Histogram) totalRowCount() float64 { // mergeBuckets is used to merge every two neighbor buckets. func (hg *Histogram) mergeBuckets(bucketIdx int) { curBuck := 0 - c := chunk.NewChunkWithCapacity([]*types.FieldType{hg.tp}, bucketIdx) + c := chunk.NewChunkWithCapacity([]*types.FieldType{hg.Tp}, bucketIdx) for i := 0; i+1 <= bucketIdx; i += 2 { hg.Buckets[curBuck] = hg.Buckets[i+1] c.AppendDatum(0, hg.GetLower(i)) @@ -515,7 +516,7 @@ func validRange(ran *ranger.Range) bool { } // SplitRange splits the range according to the histogram upper bound. Note that we treat last bucket's upper bound -// as inf, so all the split ranges will totally fall in one of the (-inf, u(0)], (u(0), u(1)],...(u(n-3), u(n-2)], +// as inf, so all the split Ranges will totally fall in one of the (-inf, u(0)], (u(0), u(1)],...(u(n-3), u(n-2)], // (u(n-2), +inf), where n is the number of buckets, u(i) is the i-th bucket's upper bound. func (hg *Histogram) SplitRange(ranges []*ranger.Range) []*ranger.Range { split := make([]*ranger.Range, 0, len(ranges)) @@ -553,7 +554,7 @@ func (hg *Histogram) SplitRange(ranges []*ranger.Range) []*ranger.Range { // Split according to the upper bound. cmp := chunk.Compare(upperBound, 0, &ranges[0].LowVal[0]) if cmp > 0 || (cmp == 0 && !ranges[0].LowExclude) { - upper := upperBound.GetDatum(0, hg.tp) + upper := upperBound.GetDatum(0, hg.Tp) split = append(split, &ranger.Range{ LowExclude: ranges[0].LowExclude, LowVal: []types.Datum{ranges[0].LowVal[0]}, @@ -609,13 +610,13 @@ func HistogramFromProto(protoHg *tipb.Histogram) *Histogram { func (hg *Histogram) popFirstBucket() { hg.Buckets = hg.Buckets[1:] - c := chunk.NewChunkWithCapacity([]*types.FieldType{hg.tp, hg.tp}, hg.Bounds.NumRows()-2) + c := chunk.NewChunkWithCapacity([]*types.FieldType{hg.Tp, hg.Tp}, hg.Bounds.NumRows()-2) c.Append(hg.Bounds, 2, hg.Bounds.NumRows()) hg.Bounds = c } func (hg *Histogram) isIndexHist() bool { - return hg.tp.Tp == mysql.TypeBlob + return hg.Tp.Tp == mysql.TypeBlob } // MergeHistograms merges two histograms. @@ -857,3 +858,206 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range } return totalCount, nil } + +func (c *Column) newNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { + newColHist := &Column{Info: c.Info} + if len(statsNode.Ranges) == 0 { + newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) + return newColHist, nil + } + newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, c.Tp, chunk.InitialCapacity, 0) + ranLowIdx := 0 + var totCnt int64 = 0 + // Process each bucket. + for i := 0; i < c.Bounds.NumRows() && ranLowIdx < len(statsNode.Ranges); i += 2 { + newBkt := Bucket{Repeat: c.Buckets[i/2].Repeat} + // Find the bucket which is the first one whose have intersection with the bucket. + for ; ranLowIdx < len(statsNode.Ranges); ranLowIdx++ { + if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranLowIdx].HighVal[0]) <= 0 { + break + } + } + ranHighIdx := ranLowIdx + // Find the bucket which is the first one whose lowVal is bigger than bucket's high bound, i.e. out of the bucket. + for ; ranHighIdx < len(statsNode.Ranges); ranHighIdx++ { + if chunk.Compare(c.Bounds.GetRow(i+1), 0, &statsNode.Ranges[ranHighIdx].LowVal[0]) < 0 { + break + } + } + if ranLowIdx == ranHighIdx { + continue + } + logrus.Warnf("ran low: %v, ran high: %v, bucket low: %v, bucketHigh: %v", statsNode.Ranges[ranLowIdx], statsNode.Ranges[ranHighIdx-1], c.Bounds.GetRow(i).GetInt64(0), c.Bounds.GetRow(i+1).GetInt64(0)) + overlapped := 0.0 + // Compute the overlap ratio. + for ranIdx := ranLowIdx; ranIdx < ranHighIdx; ranIdx++ { + overlapped += c.CalcRangeFraction(i/2, &statsNode.Ranges[ranIdx].LowVal[0], &statsNode.Ranges[ranIdx].HighVal[0]) + } + // If there's no overlap, skip it. + if overlapped == 0 { + ranLowIdx = ranHighIdx - 1 + continue + } + // Update the bound of the bucket. + if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranLowIdx].LowVal[0]) < 0 { + newColHist.Bounds.AppendDatum(0, &statsNode.Ranges[ranLowIdx].LowVal[0]) + } else { + newColHist.Bounds.AppendRow(c.Bounds.GetRow(i)) + } + if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranHighIdx-1].HighVal[0]) > 0 { + newColHist.Bounds.AppendDatum(0, &statsNode.Ranges[ranHighIdx-1].HighVal[0]) + // Update the repeat val. + newBkt.Repeat = 0 + } else { + newColHist.Bounds.AppendRow(c.Bounds.GetRow(i + 1)) + } + totCnt += int64(overlapped * float64(c.bucketCount(i/2))) + newBkt.Count = totCnt + newColHist.Buckets = append(newColHist.Buckets, newBkt) + ranLowIdx = ranHighIdx - 1 + } + return newColHist, nil +} + +func (c *Column) newNonNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { + newColHist := &Column{Info: c.Info} + if len(statsNode.Ranges) == 0 { + newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) + return newColHist, nil + } + newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) + + lowBucketIdx, highBucketIdx := 0, 0 + var totCnt int64 = 0 + + // Bucket bound of index is encoded one, so we need to decode it if we want to calculate the fraction accurately. + // TODO: enhance its calculation. + // Now just remove the bucket that no range fell in. + for _, ran := range statsNode.Ranges { + lowBucketIdx = highBucketIdx + for ; highBucketIdx*2 < c.Bounds.NumRows(); highBucketIdx++ { + if chunk.Compare(c.Bounds.GetRow(highBucketIdx*2), 0, &ran.HighVal[0]) >= 0 { + break + } + } + for ; lowBucketIdx < highBucketIdx; lowBucketIdx++ { + if chunk.Compare(c.Bounds.GetRow(lowBucketIdx*2+1), 0, &ran.LowVal[0]) > 0 { + break + } + } + if lowBucketIdx*2 >= c.Bounds.NumRows() { + break + } + for i := lowBucketIdx; i < highBucketIdx; i++ { + newColHist.Bounds.AppendBytes(0, c.Bounds.GetRow(i*2).GetBytes(0)) + newColHist.Bounds.AppendBytes(0, c.Bounds.GetRow(i*2+1).GetBytes(0)) + totCnt += c.bucketCount(i) + newColHist.Buckets = append(newColHist.Buckets, Bucket{Repeat: c.Buckets[i].Repeat, Count: totCnt}) + } + } + return newColHist, nil +} + +func (c *Column) newColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { + if len(statsNode.Ranges) == 0 { + + } + switch statsNode.Ranges[0].LowVal[0].Kind() { + case types.KindInt64, types.KindUint64, types.KindFloat32, types.KindFloat64: + return c.newNumericColumnBySelectivity(sc, statsNode) + default: + return c.newNonNumericColumnBySelectivity(sc, statsNode) + } + return nil, nil +} + +func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Index, error) { + var ( + ranLowEncode, ranHighEncode []byte + err error + ) + newIndexHist := &Index{Info: idx.Info} + newIndexHist.Histogram = *NewHistogram(idx.ID, int64(float64(idx.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) + + lowBucketIdx, highBucketIdx := 0, 0 + var totCnt int64 = 0 + + // Bucket bound of index is encoded one, so we need to decode it if we want to calculate the fraction accurately. + // TODO: enhance its calculation. + // Now just remove the bucket that no range fell in. + for _, ran := range statsNode.Ranges { + lowBucketIdx = highBucketIdx + ranLowEncode, ranHighEncode, err = ran.Encode(sc, ranLowEncode, ranHighEncode) + if err != nil { + return nil, err + } + for ; highBucketIdx*2 < idx.Bounds.NumRows(); highBucketIdx++ { + // Encoded value can only go to its next quickly. So ranHighEncode is actually range.HighVal's PrefixNext value. + // So the Bound should also go to its PrefixNext. + if bytes.Compare(ranHighEncode, kv.Key(idx.Bounds.GetRow(highBucketIdx*2).GetBytes(0)).PrefixNext()) < 0 { + break + } + } + for ; lowBucketIdx < highBucketIdx; lowBucketIdx++ { + if bytes.Compare(ranLowEncode, idx.Bounds.GetRow(lowBucketIdx*2+1).GetBytes(0)) <= 0 { + break + } + } + if lowBucketIdx*2 >= idx.Bounds.NumRows() { + break + } + for i := lowBucketIdx; i < highBucketIdx; i++ { + newIndexHist.Bounds.AppendBytes(0, idx.Bounds.GetRow(i*2).GetBytes(0)) + newIndexHist.Bounds.AppendBytes(0, idx.Bounds.GetRow(i*2+1).GetBytes(0)) + totCnt += idx.bucketCount(i) + newIndexHist.Buckets = append(newIndexHist.Buckets, Bucket{Repeat: idx.Buckets[i].Repeat, Count: totCnt}) + } + } + return newIndexHist, nil +} + +func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, statsNodes []*StatsNode) (*HistColl, error) { + newColl := &HistColl{ + Columns: make(map[int64]*Column), + Indices: make(map[int64]*Index), + Idx2ColumnIDs: coll.Idx2ColumnIDs, + ColID2IdxID: coll.ColID2IdxID, + Count: coll.Count, + } + for _, node := range statsNodes { + if node.Tp == indexType { + idxHist, ok := coll.Indices[node.ID] + if !ok { + continue + } + newIdxHist, err := idxHist.newIndexBySelectivity(sc, node) + if err != nil { + return nil, err + } + newColl.Indices[node.ID] = newIdxHist + } else { + colHist, ok := coll.Columns[node.ID] + if !ok { + continue + } + newColHist, err := colHist.newColumnBySelectivity(sc, node) + if err != nil { + return nil, err + } + newColl.Columns[node.ID] = newColHist + } + } + for id, idx := range coll.Indices { + _, ok := newColl.Indices[id] + if !ok { + newColl.Indices[id] = idx + } + } + for id, col := range coll.Columns { + _, ok := newColl.Columns[id] + if !ok { + newColl.Columns[id] = col + } + } + return newColl, nil +} diff --git a/statistics/scalar.go b/statistics/scalar.go index 1ea756a6e6d0c..b914dd7670a5c 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -41,6 +41,32 @@ func calcFraction(lower, upper, value float64) float64 { return frac } +func calcFloatRangeFraction(lower, upper, valLow, valHigh float64) float64 { + if upper <= lower { + return 0.5 + } + if valHigh > upper { + valHigh = upper + } + if valLow < lower { + valLow = lower + } + return (valHigh - valLow) / (upper - lower) +} + +func calcIntRangeFraction(lower, upper, valLow, valHigh float64) float64 { + if upper <= lower { + return 0.5 + } + if valHigh > upper { + valHigh = upper + } + if valLow < lower { + valLow = lower + } + return (valHigh - valLow + 1) / (upper - lower + 1) +} + func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { switch value.Kind() { case types.KindMysqlDecimal: @@ -135,6 +161,21 @@ func (hg *Histogram) calcFraction(index int, value *types.Datum) float64 { return 0.5 } +func (hg *Histogram) CalcRangeFraction(index int, valLow, valHigh *types.Datum) float64 { + lower, upper := hg.Bounds.GetRow(2*index), hg.Bounds.GetRow(2*index+1) + switch valLow.Kind() { + case types.KindFloat32: + return calcFloatRangeFraction(float64(lower.GetFloat32(0)), float64(upper.GetFloat32(0)), float64(valLow.GetFloat32()), float64(valHigh.GetFloat32())) + case types.KindFloat64: + return calcFloatRangeFraction(lower.GetFloat64(0), upper.GetFloat64(0), valLow.GetFloat64(), valHigh.GetFloat64()) + case types.KindInt64: + return calcIntRangeFraction(float64(lower.GetInt64(0)), float64(upper.GetInt64(0)), float64(valLow.GetInt64()), float64(valHigh.GetInt64())) + case types.KindUint64: + return calcIntRangeFraction(float64(lower.GetUint64(0)), float64(upper.GetUint64(0)), float64(valLow.GetUint64()), float64(valHigh.GetUint64())) + } + return 0.5 +} + func commonPrefixLength(lower, upper []byte) int { minLen := len(lower) if minLen > len(upper) { diff --git a/statistics/selectivity.go b/statistics/selectivity.go index ef474455d98ea..d48438f0332ad 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -27,19 +27,21 @@ import ( // If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8. const selectionFactor = 0.8 -// exprSet is used for calculating selectivity. -type exprSet struct { - tp int +// StatsNode is used for calculating selectivity. +type StatsNode struct { + Tp int ID int64 // mask is a bit pattern whose ith bit will indicate whether the ith expression is covered by this index/column. mask int64 - // ranges contains all the ranges we got. - ranges []*ranger.Range + // Ranges contains all the Ranges we got. + Ranges []*ranger.Range + // Selectivity indicates the Selectivity of this column/index. + Selectivity float64 // numCols is the number of columns contained in the index or column(which is always 1). numCols int } -// The type of the exprSet. +// The type of the StatsNode. const ( indexType = iota pkType @@ -142,18 +144,18 @@ func isColEqCorCol(filter expression.Expression) *expression.Column { // And exprs must be CNF now, in other words, `exprs[0] and exprs[1] and ... and exprs[len - 1]` should be held when you call this. // TODO: support expressions that the top layer is a DNF. // Currently the time complexity is o(n^2). -func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Expression) (float64, error) { +func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Expression) (float64, []*StatsNode, error) { // If table's count is zero or conditions are empty, we should return 100% selectivity. if coll.Count == 0 || len(exprs) == 0 { - return 1, nil + return 1, nil, nil } // TODO: If len(exprs) is bigger than 63, we could use bitset structure to replace the int64. // This will simplify some code and speed up if we use this rather than a boolean slice. if len(exprs) > 63 || (len(coll.Columns) == 0 && len(coll.Indices) == 0) { - return pseudoSelectivity(coll, exprs), nil + return pseudoSelectivity(coll, exprs), nil, nil } ret := 1.0 - var sets []*exprSet + var nodes []*StatsNode sc := ctx.GetSessionVars().StmtCtx remainedExprs := make([]expression.Expression, 0, len(exprs)) @@ -177,12 +179,23 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp if col != nil { maskCovered, ranges, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, col) if err != nil { - return 0, errors.Trace(err) + return 0, nil, errors.Trace(err) } - sets = append(sets, &exprSet{tp: colType, ID: id, mask: maskCovered, ranges: ranges, numCols: 1}) + nodes = append(nodes, &StatsNode{Tp: colType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) if mysql.HasPriKeyFlag(colInfo.Info.Flag) { - sets[len(sets)-1].tp = pkType + nodes[len(nodes)-1].Tp = pkType + cnt, err := coll.GetRowCountByIntColumnRanges(sc, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) + } + nodes[len(nodes)-1].Selectivity = cnt / float64(coll.Count) + continue + } + cnt, err := coll.GetRowCountByColumnRanges(sc, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) } + nodes[len(nodes)-1].Selectivity = cnt / float64(coll.Count) } } for id, idxInfo := range coll.Indices { @@ -194,38 +207,35 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp } maskCovered, ranges, err := getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, idxCols...) if err != nil { - return 0, errors.Trace(err) + return 0, nil, errors.Trace(err) } - sets = append(sets, &exprSet{tp: indexType, ID: id, mask: maskCovered, ranges: ranges, numCols: len(idxInfo.Info.Columns)}) + cnt, err := coll.GetRowCountByIndexRanges(sc, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) + } + selectivity := cnt / float64(coll.Count) + nodes = append(nodes, &StatsNode{ + Tp: indexType, + ID: id, + mask: maskCovered, + Ranges: ranges, + numCols: len(idxInfo.Info.Columns), + Selectivity: selectivity, + }) } } - sets = getUsableSetsByGreedy(sets) + usedSets := getUsableSetsByGreedy(nodes) // Initialize the mask with the full set. mask := (int64(1) << uint(len(remainedExprs))) - 1 - for _, set := range sets { + for _, set := range usedSets { mask ^= set.mask - var ( - rowCount float64 - err error - ) - switch set.tp { - case pkType: - rowCount, err = coll.GetRowCountByIntColumnRanges(sc, set.ID, set.ranges) - case colType: - rowCount, err = coll.GetRowCountByColumnRanges(sc, set.ID, set.ranges) - case indexType: - rowCount, err = coll.GetRowCountByIndexRanges(sc, set.ID, set.ranges) - } - if err != nil { - return 0, errors.Trace(err) - } - ret *= rowCount / float64(coll.Count) + ret *= set.Selectivity } // If there's still conditions which cannot be calculated, we will multiply a selectionFactor. if mask > 0 { ret *= selectionFactor } - return ret, nil + return ret, nodes, nil } func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, @@ -256,12 +266,14 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran } // getUsableSetsByGreedy will select the indices and pk used for calculate selectivity by greedy algorithm. -func getUsableSetsByGreedy(sets []*exprSet) (newBlocks []*exprSet) { +func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { + tmpNodes := make([]*StatsNode, len(nodes)) + copy(tmpNodes, nodes) mask := int64(math.MaxInt64) for { // Choose the index that covers most. bestID, bestCount, bestTp, bestNumCols := -1, 0, colType, 0 - for i, set := range sets { + for i, set := range tmpNodes { set.mask &= mask bits := popCount(set.mask) // This set cannot cover any thing, just skip it. @@ -272,20 +284,20 @@ func getUsableSetsByGreedy(sets []*exprSet) (newBlocks []*exprSet) { // (1): The stats type, always prefer the primary key or index. // (2): The number of expression that it covers, the more the better. // (3): The number of columns that it contains, the less the better. - if (bestTp == colType && set.tp != colType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { - bestID, bestCount, bestTp, bestNumCols = i, bits, set.tp, set.numCols + if (bestTp == colType && set.Tp != colType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { + bestID, bestCount, bestTp, bestNumCols = i, bits, set.Tp, set.numCols } } if bestCount == 0 { break } - // update the mask, remove the bit that sets[bestID].mask has. - mask &^= sets[bestID].mask + // update the mask, remove the bit that nodes[bestID].mask has. + mask &^= tmpNodes[bestID].mask - newBlocks = append(newBlocks, sets[bestID]) + newBlocks = append(newBlocks, tmpNodes[bestID]) // remove the chosen one - sets = append(sets[:bestID], sets[bestID+1:]...) + tmpNodes = append(tmpNodes[:bestID], tmpNodes[bestID+1:]...) } return } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index f2ae8b6819f75..d0fb128204032 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -199,12 +199,12 @@ func (s *testSelectivitySuite) TestSelectivity(c *C) { histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) - ratio, err := histColl.Selectivity(ctx, sel.Conditions) + ratio, _, err := histColl.Selectivity(ctx, sel.Conditions) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) histColl.Count *= 10 - ratio, err = histColl.Selectivity(ctx, sel.Conditions) + ratio, _, err = histColl.Selectivity(ctx, sel.Conditions) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) } @@ -307,10 +307,10 @@ func BenchmarkSelectivity(b *testing.B) { defer file.Close() pprof.StartCPUProfile(file) - b.Run("selectivity", func(b *testing.B) { + b.Run("Selectivity", func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, err := statsTbl.Selectivity(ctx, p.(plan.LogicalPlan).Children()[0].(*plan.LogicalSelection).Conditions) + _, _, err := statsTbl.Selectivity(ctx, p.(plan.LogicalPlan).Children()[0].(*plan.LogicalSelection).Conditions) c.Assert(err, IsNil) } b.ReportAllocs() diff --git a/statistics/table.go b/statistics/table.go index 336dee0e231f0..cf9996d63e0e5 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -50,7 +50,7 @@ type Table struct { name string } -// HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. +// HistColl is a collection of histogram. It collects enough information for plan to calculate the Selectivity. type HistColl struct { PhysicalID int64 // HavePhysicalID is true means this HistColl is from single table and have its ID's information. @@ -58,9 +58,9 @@ type HistColl struct { HavePhysicalID bool Columns map[int64]*Column Indices map[int64]*Index - // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the selectivity in planner. + // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the Selectivity in planner. Idx2ColumnIDs map[int64][]int64 - // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the selectivity in planner. + // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the Selectivity in planner. ColID2IdxID map[int64]int64 Pseudo bool Count int64 @@ -181,7 +181,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * ID: histID, NDV: distinct, NullCount: nullCount, - tp: &colInfo.FieldType, + Tp: &colInfo.FieldType, LastUpdateVersion: histVer, TotColSize: totColSize, }, @@ -471,7 +471,7 @@ func getOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int } // GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxID and IdxID2ColIDs is built from the given parameter. -func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) HistColl { +func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) *HistColl { newColHistMap := make(map[int64]*Column) colInfoID2UniqueID := make(map[int64]int64) colNames2UniqueID := make(map[string]int64) @@ -511,7 +511,7 @@ func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, newIdxHistMap[idxHist.ID] = idxHist idx2Columns[idxHist.ID] = ids } - newColl := HistColl{ + newColl := &HistColl{ PhysicalID: coll.PhysicalID, HavePhysicalID: coll.HavePhysicalID, Pseudo: coll.Pseudo, diff --git a/statistics/update_test.go b/statistics/update_test.go index 0affb367df228..54a89c4de9766 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -641,7 +641,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) - // Test only collect for max number of ranges. + // Test only collect for max number of Ranges. statistics.MaxNumberOfRanges = 0 for _, t := range tests { testKit.MustQuery(t.sql) diff --git a/util/ranger/types.go b/util/ranger/types.go index c17964845d25d..1c0ed2f3c54ec 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -18,8 +18,10 @@ import ( "math" "strings" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" ) @@ -95,6 +97,25 @@ func (ran *Range) String() string { return l + strings.Join(lowStrs, " ") + "," + strings.Join(highStrs, " ") + r } +func (ran *Range) Encode(sc *stmtctx.StatementContext, lowBuffer, highBuffer []byte) ([]byte, []byte, error) { + var err error + lowBuffer, err = codec.EncodeKey(sc, lowBuffer[:0], ran.LowVal...) + if err != nil { + return nil, nil, err + } + if ran.LowExclude { + lowBuffer = kv.Key(lowBuffer).PrefixNext() + } + highBuffer, err = codec.EncodeKey(sc, highBuffer[:0], ran.HighVal...) + if err != nil { + return nil, nil, err + } + if !ran.HighExclude { + highBuffer = kv.Key(highBuffer).PrefixNext() + } + return lowBuffer, highBuffer, nil +} + // PrefixEqualLen tells you how long the prefix of the range is a point. // e.g. If this range is (1 2 3, 1 2 +inf), then the return value is 2. func (ran *Range) PrefixEqualLen(sc *stmtctx.StatementContext) (int, error) { From 14be00487f4c942d0743cd3d06648a99febd23f5 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Oct 2018 23:21:28 +0800 Subject: [PATCH 02/23] fix check --- statistics/histogram.go | 4 +--- statistics/scalar.go | 1 + statistics/selectivity.go | 3 ++- util/ranger/types.go | 1 + 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 375470d71324d..005e544754aa7 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" - "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -886,7 +885,6 @@ func (c *Column) newNumericColumnBySelectivity(sc *stmtctx.StatementContext, sta if ranLowIdx == ranHighIdx { continue } - logrus.Warnf("ran low: %v, ran high: %v, bucket low: %v, bucketHigh: %v", statsNode.Ranges[ranLowIdx], statsNode.Ranges[ranHighIdx-1], c.Bounds.GetRow(i).GetInt64(0), c.Bounds.GetRow(i+1).GetInt64(0)) overlapped := 0.0 // Compute the overlap ratio. for ranIdx := ranLowIdx; ranIdx < ranHighIdx; ranIdx++ { @@ -967,7 +965,6 @@ func (c *Column) newColumnBySelectivity(sc *stmtctx.StatementContext, statsNode default: return c.newNonNumericColumnBySelectivity(sc, statsNode) } - return nil, nil } func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Index, error) { @@ -1015,6 +1012,7 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode return newIndexHist, nil } +// NewHistCollBySelectivity creates new HistColl by the given statsNodes. func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, statsNodes []*StatsNode) (*HistColl, error) { newColl := &HistColl{ Columns: make(map[int64]*Column), diff --git a/statistics/scalar.go b/statistics/scalar.go index b914dd7670a5c..7b127718b0d81 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -161,6 +161,7 @@ func (hg *Histogram) calcFraction(index int, value *types.Datum) float64 { return 0.5 } +// CalcRangeFraction calculates the fraction between [valLow, valHigh] and hg's indexth Bucket. func (hg *Histogram) CalcRangeFraction(index int, valLow, valHigh *types.Datum) float64 { lower, upper := hg.Bounds.GetRow(2*index), hg.Bounds.GetRow(2*index+1) switch valLow.Kind() { diff --git a/statistics/selectivity.go b/statistics/selectivity.go index d48438f0332ad..a2b4f297c1c5f 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -184,7 +184,8 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp nodes = append(nodes, &StatsNode{Tp: colType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) if mysql.HasPriKeyFlag(colInfo.Info.Flag) { nodes[len(nodes)-1].Tp = pkType - cnt, err := coll.GetRowCountByIntColumnRanges(sc, id, ranges) + var cnt float64 + cnt, err = coll.GetRowCountByIntColumnRanges(sc, id, ranges) if err != nil { return 0, nil, errors.Trace(err) } diff --git a/util/ranger/types.go b/util/ranger/types.go index 1c0ed2f3c54ec..16cf886618900 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -97,6 +97,7 @@ func (ran *Range) String() string { return l + strings.Join(lowStrs, " ") + "," + strings.Join(highStrs, " ") + r } +// Encode encodes the range to its encoded value. func (ran *Range) Encode(sc *stmtctx.StatementContext, lowBuffer, highBuffer []byte) ([]byte, []byte, error) { var err error lowBuffer, err = codec.EncodeKey(sc, lowBuffer[:0], ran.LowVal...) From 763f5c1eb4e1806f459b36e3b8a62826e09d1296 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Oct 2018 23:39:26 +0800 Subject: [PATCH 03/23] add unit-test file. --- statistics/histogram_test.go | 105 +++++++++++++++++++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 statistics/histogram_test.go diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go new file mode 100644 index 0000000000000..bc4b2ccebcd37 --- /dev/null +++ b/statistics/histogram_test.go @@ -0,0 +1,105 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package statistics + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/ranger" +) + +var _ = Suite(&HistogramTestSuite{}) + +type HistogramTestSuite struct { +} + +func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { + ctx := mock.NewContext() + sc := ctx.GetSessionVars().StmtCtx + intCol := &Column{} + intCol.Histogram = *NewHistogram(1, 30, 0, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) + for i := 0; i < 10; i++ { + intCol.Bounds.AppendInt64(0, int64(i*3)) + intCol.Bounds.AppendInt64(0, int64(i*3+2)) + intCol.Buckets = append(intCol.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) + } + node := &StatsNode{ID: 1, Tp: pkType, Selectivity: 0.2} + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(5), HighVal: types.MakeDatums(6)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(8), HighVal: types.MakeDatums(10)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(13), HighVal: types.MakeDatums(13)}) + newIntCol, err := intCol.newNumericColumnBySelectivity(sc, node) + c.Assert(err, IsNil, Commentf("Test failed: %v", err)) + intColResult := `column:1 ndv:6 totColSize:0 +num: 10 lower_bound: 5 upper_bound: 5 repeats: 10 +num: 20 lower_bound: 6 upper_bound: 8 repeats: 10 +num: 20 lower_bound: 9 upper_bound: 11 repeats: 10 +num: 10 lower_bound: 13 upper_bound: 14 repeats: 10` + c.Assert(newIntCol.String(), Equals, intColResult) + + stringCol := &Column{} + stringCol.Histogram = *NewHistogram(2, 9, 0, 0, types.NewFieldType(mysql.TypeString), chunk.InitialCapacity, 0) + stringCol.Bounds.AppendString(0, "a") + stringCol.Bounds.AppendString(0, "aaaabbbb") + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 30}) + stringCol.Bounds.AppendString(0, "bbbb") + stringCol.Bounds.AppendString(0, "fdsfdsfds") + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 60}) + stringCol.Bounds.AppendString(0, "kkkkk") + stringCol.Bounds.AppendString(0, "yyyyy") + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 90}) + node.Tp = colType + node.ID = 2 + node.Ranges = node.Ranges[:0] + node.Selectivity = 0.4 + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(""), HighVal: types.MakeDatums("aaa")}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) + newStringCol, err := stringCol.newNonNumericColumnBySelectivity(sc, node) + c.Assert(err, IsNil, Commentf("Test failed: %v", err)) + stringColResult := `column:2 ndv:3 totColSize:0 +num: 30 lower_bound: a upper_bound: aaaabbbb repeats: 10 +num: 30 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 10` + c.Assert(newStringCol.String(), Equals, stringColResult) + idx := Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: model.NewCIStr("a"), Offset: 0}}}} + idx.Histogram = *NewHistogram(0, 15, 0, 0, types.NewFieldType(mysql.TypeBlob), 0, 0) + for i := 0; i < 5; i++ { + low, err1 := codec.EncodeKey(sc, nil, types.NewIntDatum(int64(i*3))) + c.Assert(err1, IsNil, Commentf("Test failed: %v", err)) + high, err2 := codec.EncodeKey(sc, nil, types.NewIntDatum(int64(i*3+2))) + c.Assert(err2, IsNil, Commentf("Test failed: %v", err)) + idx.Bounds.AppendBytes(0, low) + idx.Bounds.AppendBytes(0, high) + idx.Buckets = append(idx.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) + } + node.Tp = indexType + node.ID = 0 + node.Selectivity = 0.47 + node.Ranges = node.Ranges[:0] + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(7), HighVal: types.MakeDatums(11)}) + newIdx, err := idx.newIndexBySelectivity(sc, node) + c.Assert(err, IsNil, Commentf("Test failed: %v", err)) + idxResult := `index:0 ndv:7 +num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 +num: 30 lower_bound: 3 upper_bound: 5 repeats: 10 +num: 30 lower_bound: 6 upper_bound: 8 repeats: 10 +num: 30 lower_bound: 9 upper_bound: 11 repeats: 10` + c.Assert(newIdx.String(), Equals, idxResult) +} From 5b61c12d44fa1d22a5e9ec662ee95e1a58b5ccea Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 24 Oct 2018 20:24:05 +0800 Subject: [PATCH 04/23] address comments --- planner/core/stats.go | 1 - statistics/feedback.go | 14 +++++++------- statistics/histogram.go | 27 +++++++++++---------------- statistics/table.go | 6 +++--- 4 files changed, 21 insertions(+), 27 deletions(-) diff --git a/planner/core/stats.go b/planner/core/stats.go index 53c30f4969b42..34813b89db4ab 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -79,7 +79,6 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) (*property.Sta } ds.stats = profile selectivity, nodes, err := profile.HistColl.Selectivity(ds.ctx, conds) - log.Warnf("selectivity: %v", selectivity) if err != nil { log.Warnf("An error happened: %v, we have to use the default selectivity", err.Error()) selectivity = selectionFactor diff --git a/statistics/feedback.go b/statistics/feedback.go index bd9244c0e6949..913d25f784665 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -45,7 +45,7 @@ type feedback struct { repeat int64 } -// QueryFeedback is used to represent the query feedback info. It contains the query's scan Ranges and number of rows +// QueryFeedback is used to represent the query feedback info. It contains the query's scan ranges and number of rows // in each range. type QueryFeedback struct { tableID int64 @@ -78,7 +78,7 @@ func NewQueryFeedback(tableID int64, hist *Histogram, expected int64, desc bool) } var ( - // MaxNumberOfRanges is the max number of Ranges before split to collect feedback. + // MaxNumberOfRanges is the max number of ranges before split to collect feedback. MaxNumberOfRanges = 20 // FeedbackProbability is the probability to collect the feedback. FeedbackProbability = 0.0 @@ -86,7 +86,7 @@ var ( // CollectFeedback decides whether to collect the feedback. It returns false when: // 1: the histogram is nil or has no buckets; -// 2: the number of scan Ranges exceeds the limit because it may affect the performance; +// 2: the number of scan ranges exceeds the limit because it may affect the performance; // 3: it does not pass the probabilistic sampler. func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { if q.hist == nil || q.hist.Len() == 0 { @@ -100,7 +100,7 @@ func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { return true } -// DecodeToRanges decode the feedback to Ranges. +// DecodeToRanges decode the feedback to ranges. func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { ranges := make([]*ranger.Range, 0, len(q.feedback)) for _, val := range q.feedback { @@ -158,7 +158,7 @@ func (q *QueryFeedback) decodeIntValues() *QueryFeedback { return nq } -// StoreRanges stores the Ranges for update. +// StoreRanges stores the ranges for update. func (q *QueryFeedback) StoreRanges(ranges []*ranger.Range) { q.feedback = make([]feedback, 0, len(ranges)) for _, ran := range ranges { @@ -947,7 +947,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { } ranges, err := q.DecodeToRanges(true) if err != nil { - log.Debug("decode feedback Ranges failed: ", err) + log.Debug("decode feedback ranges failed: ", err) return nil } for i, ran := range ranges { @@ -981,7 +981,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { continue } if err != nil { - log.Debug("get row count by Ranges failed: ", err) + log.Debug("get row count by ranges failed: ", err) continue } diff --git a/statistics/histogram.go b/statistics/histogram.go index 005e544754aa7..e45f99ea4a09e 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -859,10 +859,6 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range func (c *Column) newNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { newColHist := &Column{Info: c.Info} - if len(statsNode.Ranges) == 0 { - newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) - return newColHist, nil - } newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, c.Tp, chunk.InitialCapacity, 0) ranLowIdx := 0 var totCnt int64 = 0 @@ -918,16 +914,12 @@ func (c *Column) newNumericColumnBySelectivity(sc *stmtctx.StatementContext, sta func (c *Column) newNonNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { newColHist := &Column{Info: c.Info} - if len(statsNode.Ranges) == 0 { - newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) - return newColHist, nil - } newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) lowBucketIdx, highBucketIdx := 0, 0 var totCnt int64 = 0 - // Bucket bound of index is encoded one, so we need to decode it if we want to calculate the fraction accurately. + // Bucket bound of non-numeric column is not easy to calculate. Now just use its original bound. // TODO: enhance its calculation. // Now just remove the bucket that no range fell in. for _, ran := range statsNode.Ranges { @@ -945,9 +937,12 @@ func (c *Column) newNonNumericColumnBySelectivity(sc *stmtctx.StatementContext, if lowBucketIdx*2 >= c.Bounds.NumRows() { break } + // TODO: GetDatum and AppendDatum is very inefficient, use a better way to this. for i := lowBucketIdx; i < highBucketIdx; i++ { - newColHist.Bounds.AppendBytes(0, c.Bounds.GetRow(i*2).GetBytes(0)) - newColHist.Bounds.AppendBytes(0, c.Bounds.GetRow(i*2+1).GetBytes(0)) + low := c.Bounds.GetRow(i*2).GetDatum(0, c.Tp) + high := c.Bounds.GetRow(i*2+1).GetDatum(0, c.Tp) + newColHist.Bounds.AppendDatum(0, &low) + newColHist.Bounds.AppendDatum(0, &high) totCnt += c.bucketCount(i) newColHist.Buckets = append(newColHist.Buckets, Bucket{Repeat: c.Buckets[i].Repeat, Count: totCnt}) } @@ -957,14 +952,14 @@ func (c *Column) newNonNumericColumnBySelectivity(sc *stmtctx.StatementContext, func (c *Column) newColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { if len(statsNode.Ranges) == 0 { - + newColHist := &Column{Info: c.Info} + newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) + return newColHist, nil } - switch statsNode.Ranges[0].LowVal[0].Kind() { - case types.KindInt64, types.KindUint64, types.KindFloat32, types.KindFloat64: + if c.Tp.EvalType() == types.ETInt || c.Tp.EvalType() == types.ETReal { return c.newNumericColumnBySelectivity(sc, statsNode) - default: - return c.newNonNumericColumnBySelectivity(sc, statsNode) } + return c.newNonNumericColumnBySelectivity(sc, statsNode) } func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Index, error) { diff --git a/statistics/table.go b/statistics/table.go index 3fe051b650c6d..f3c05815e939a 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -50,7 +50,7 @@ type Table struct { name string } -// HistColl is a collection of histogram. It collects enough information for plan to calculate the Selectivity. +// HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. type HistColl struct { PhysicalID int64 // HavePhysicalID is true means this HistColl is from single table and have its ID's information. @@ -58,9 +58,9 @@ type HistColl struct { HavePhysicalID bool Columns map[int64]*Column Indices map[int64]*Index - // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the Selectivity in planner. + // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the selectivity in planner. Idx2ColumnIDs map[int64][]int64 - // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the Selectivity in planner. + // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the selectivity in planner. ColID2IdxID map[int64]int64 Pseudo bool Count int64 From e9df1a34c22a180fcf0ff60e6e8b9b849085c795 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 25 Oct 2018 13:23:52 +0800 Subject: [PATCH 05/23] fix strange build failed error --- statistics/feedback.go | 2 +- statistics/histogram_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/statistics/feedback.go b/statistics/feedback.go index 07cd750d76a80..c3cb4ed472069 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -297,7 +297,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket total := 0 sc := &stmtctx.StatementContext{TimeZone: time.UTC} kind := feedback.feedback[0].lower.Kind() - min, max := getMinValue(kind, h.tp), getMaxValue(kind, h.tp) + min, max := getMinValue(kind, h.Tp), getMaxValue(kind, h.Tp) for _, fb := range feedback.feedback { skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) if err != nil { diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index bc4b2ccebcd37..e752ccdb58bfd 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -15,8 +15,8 @@ package statistics import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" From 37dbb1fca2e72b33a0e0069291af15e7f694b820 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 25 Oct 2018 16:56:11 +0800 Subject: [PATCH 06/23] fix and add test when there's null, -inf or +inf --- statistics/histogram_test.go | 24 ++++++++++----- statistics/scalar.go | 60 ++++++++++++++++++++++++++++++++---- 2 files changed, 71 insertions(+), 13 deletions(-) diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index e752ccdb58bfd..1a59abf6fddf3 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -39,17 +39,23 @@ func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { intCol.Bounds.AppendInt64(0, int64(i*3+2)) intCol.Buckets = append(intCol.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) } - node := &StatsNode{ID: 1, Tp: pkType, Selectivity: 0.2} + node := &StatsNode{ID: 1, Tp: pkType, Selectivity: 0.56} + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums(2)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(5), HighVal: types.MakeDatums(6)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(8), HighVal: types.MakeDatums(10)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(13), HighVal: types.MakeDatums(13)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(25), HighVal: []types.Datum{types.MaxValueDatum()}}) newIntCol, err := intCol.newNumericColumnBySelectivity(sc, node) c.Assert(err, IsNil, Commentf("Test failed: %v", err)) - intColResult := `column:1 ndv:6 totColSize:0 + intColResult := `column:1 ndv:16 totColSize:0 +num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 num: 10 lower_bound: 5 upper_bound: 5 repeats: 10 num: 20 lower_bound: 6 upper_bound: 8 repeats: 10 num: 20 lower_bound: 9 upper_bound: 11 repeats: 10 -num: 10 lower_bound: 13 upper_bound: 14 repeats: 10` +num: 10 lower_bound: 13 upper_bound: 14 repeats: 10 +num: 20 lower_bound: 25 upper_bound: 26 repeats: 10 +num: 30 lower_bound: 27 upper_bound: 29 repeats: 10` c.Assert(newIntCol.String(), Equals, intColResult) stringCol := &Column{} @@ -66,16 +72,19 @@ num: 10 lower_bound: 13 upper_bound: 14 repeats: 10` node.Tp = colType node.ID = 2 node.Ranges = node.Ranges[:0] - node.Selectivity = 0.4 - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(""), HighVal: types.MakeDatums("aaa")}) + node.Selectivity = 0.6 + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa")}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}}) newStringCol, err := stringCol.newNonNumericColumnBySelectivity(sc, node) c.Assert(err, IsNil, Commentf("Test failed: %v", err)) - stringColResult := `column:2 ndv:3 totColSize:0 + stringColResult := `column:2 ndv:5 totColSize:0 num: 30 lower_bound: a upper_bound: aaaabbbb repeats: 10 -num: 30 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 10` +num: 30 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 10 +num: 30 lower_bound: kkkkk upper_bound: yyyyy repeats: 10` c.Assert(newStringCol.String(), Equals, stringColResult) idx := Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: model.NewCIStr("a"), Offset: 0}}}} idx.Histogram = *NewHistogram(0, 15, 0, 0, types.NewFieldType(mysql.TypeBlob), 0, 0) @@ -102,4 +111,5 @@ num: 30 lower_bound: 3 upper_bound: 5 repeats: 10 num: 30 lower_bound: 6 upper_bound: 8 repeats: 10 num: 30 lower_bound: 9 upper_bound: 11 repeats: 10` c.Assert(newIdx.String(), Equals, idxResult) + } diff --git a/statistics/scalar.go b/statistics/scalar.go index e05ba557300bc..bf50797b1770d 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -162,17 +162,65 @@ func (hg *Histogram) calcFraction(index int, value *types.Datum) float64 { } // CalcRangeFraction calculates the fraction between [valLow, valHigh] and hg's indexth Bucket. -func (hg *Histogram) CalcRangeFraction(index int, valLow, valHigh *types.Datum) float64 { +func (hg *Histogram) CalcRangeFraction(index int, dLow, dHigh *types.Datum) float64 { lower, upper := hg.Bounds.GetRow(2*index), hg.Bounds.GetRow(2*index+1) - switch valLow.Kind() { + k := dLow.Kind() + if k == types.KindNull || k == types.KindMinNotNull { + k = dHigh.Kind() + } + switch k { case types.KindFloat32: - return calcFloatRangeFraction(float64(lower.GetFloat32(0)), float64(upper.GetFloat32(0)), float64(valLow.GetFloat32()), float64(valHigh.GetFloat32())) + var varLow, varHigh float64 + if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { + varLow = float64(lower.GetFloat32(0)) + } else { + varLow = float64(dLow.GetFloat32()) + } + if dHigh.Kind() == types.KindMaxValue { + varHigh = float64(upper.GetFloat32(0)) + } else { + varHigh = float64(dHigh.GetFloat32()) + } + return calcFloatRangeFraction(float64(lower.GetFloat32(0)), float64(upper.GetFloat32(0)), varLow, varHigh) case types.KindFloat64: - return calcFloatRangeFraction(lower.GetFloat64(0), upper.GetFloat64(0), valLow.GetFloat64(), valHigh.GetFloat64()) + var varLow, varHigh float64 + if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { + varLow = lower.GetFloat64(0) + } else { + varLow = dLow.GetFloat64() + } + if dHigh.Kind() == types.KindMaxValue { + varHigh = upper.GetFloat64(0) + } else { + varHigh = dHigh.GetFloat64() + } + return calcFloatRangeFraction(lower.GetFloat64(0), upper.GetFloat64(0), varLow, varHigh) case types.KindInt64: - return calcIntRangeFraction(float64(lower.GetInt64(0)), float64(upper.GetInt64(0)), float64(valLow.GetInt64()), float64(valHigh.GetInt64())) + var varLow, varHigh float64 + if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { + varLow = float64(lower.GetInt64(0)) + } else { + varLow = float64(dLow.GetInt64()) + } + if dHigh.Kind() == types.KindMaxValue { + varHigh = float64(upper.GetInt64(0)) + } else { + varHigh = float64(dHigh.GetInt64()) + } + return calcIntRangeFraction(float64(lower.GetInt64(0)), float64(upper.GetInt64(0)), varLow, varHigh) case types.KindUint64: - return calcIntRangeFraction(float64(lower.GetUint64(0)), float64(upper.GetUint64(0)), float64(valLow.GetUint64()), float64(valHigh.GetUint64())) + var varLow, varHigh float64 + if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { + varLow = float64(lower.GetUint64(0)) + } else { + varLow = float64(dLow.GetUint64()) + } + if dHigh.Kind() == types.KindMaxValue { + varHigh = float64(upper.GetUint64(0)) + } else { + varHigh = float64(dHigh.GetUint64()) + } + return calcIntRangeFraction(float64(lower.GetUint64(0)), float64(upper.GetUint64(0)), varLow, varHigh) } return 0.5 } From c26f85f545446e10cb28c1dfbda0e26f8be86afc Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 6 Nov 2018 17:18:40 +0800 Subject: [PATCH 07/23] fix bug during merging --- statistics/selectivity.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 0f0b1f9b8244a..5a5eb7ad1d263 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -183,7 +182,8 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp return 0, nil, errors.Trace(err) } nodes = append(nodes, &StatsNode{Tp: colType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) - if colInfo.isHandle && colInfo.Tp.EvalType() == types.ETInt { + if colInfo.isHandle { + nodes[len(nodes)-1].Tp = pkType var cnt float64 cnt, err = coll.GetRowCountByIntColumnRanges(sc, id, ranges) if err != nil { @@ -196,9 +196,6 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp if err != nil { return 0, nil, errors.Trace(err) } - if colInfo.isHandle { - nodes[len(nodes)-1].Tp = pkType - } nodes[len(nodes)-1].Selectivity = cnt / float64(coll.Count) } } From 63db3e35191e5ecfefd7cee16d004667f9905cfe Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 9 Nov 2018 20:54:28 +0800 Subject: [PATCH 08/23] address comments for column type --- distsql/request_builder.go | 4 +- statistics/feedback.go | 2 +- statistics/histogram.go | 158 +++++++++++++---------------------- statistics/histogram_test.go | 74 +++++++++------- statistics/update_test.go | 2 +- 5 files changed, 104 insertions(+), 136 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index f66a83a5721d3..e652c78a7c63e 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -173,7 +173,7 @@ func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.Que if fb == nil || fb.Hist() == nil { return tableRangesToKVRangesWithoutSplit(tid, ranges) } - ranges = fb.Hist().SplitRange(ranges) + ranges = fb.Hist().SplitRange(ranges, false) krs := make([]kv.KeyRange, 0, len(ranges)) feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { @@ -260,7 +260,7 @@ func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, range feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true}) } - feedbackRanges = fb.Hist().SplitRange(feedbackRanges) + feedbackRanges = fb.Hist().SplitRange(feedbackRanges, true) krs := make([]kv.KeyRange, 0, len(feedbackRanges)) for _, ran := range feedbackRanges { low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() diff --git a/statistics/feedback.go b/statistics/feedback.go index b2089a3abdc65..8f0c74441c044 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -1086,7 +1086,7 @@ func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCou ran.HighVal[0] = getMaxValue(k, q.hist.Tp) } } - ranges := q.hist.SplitRange([]*ranger.Range{ran}) + ranges := q.hist.SplitRange([]*ranger.Range{ran}, q.tp == indexType) counts := make([]float64, 0, len(ranges)) sum := 0.0 for _, r := range ranges { diff --git a/statistics/histogram.go b/statistics/histogram.go index 0a93a7cf0ac49..63e4b16b87893 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -498,12 +499,20 @@ func (hg *Histogram) getIncreaseFactor(totalCount int64) float64 { // validRange checks if the range is valid, it is used by `SplitRange` to remove the invalid range, // the possible types of range are index key range and handle key range. -func validRange(ran *ranger.Range) bool { +func validRange(ran *ranger.Range, encoded bool) bool { var low, high []byte - if ran.LowVal[0].Kind() == types.KindBytes { + if encoded { low, high = ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() } else { - low, high = codec.EncodeInt(nil, ran.LowVal[0].GetInt64()), codec.EncodeInt(nil, ran.HighVal[0].GetInt64()) + var err error + low, err = codec.EncodeKey(nil, nil, ran.LowVal[0]) + if err != nil { + return false + } + high, err = codec.EncodeKey(nil, nil, ran.HighVal[0]) + if err != nil { + return false + } } if ran.LowExclude { low = kv.Key(low).PrefixNext() @@ -517,7 +526,7 @@ func validRange(ran *ranger.Range) bool { // SplitRange splits the range according to the histogram upper bound. Note that we treat last bucket's upper bound // as inf, so all the split Ranges will totally fall in one of the (-inf, u(0)], (u(0), u(1)],...(u(n-3), u(n-2)], // (u(n-2), +inf), where n is the number of buckets, u(i) is the i-th bucket's upper bound. -func (hg *Histogram) SplitRange(ranges []*ranger.Range) []*ranger.Range { +func (hg *Histogram) SplitRange(ranges []*ranger.Range, encoded bool) []*ranger.Range { split := make([]*ranger.Range, 0, len(ranges)) for len(ranges) > 0 { // Find the last bound that greater or equal to the LowVal. @@ -561,7 +570,7 @@ func (hg *Histogram) SplitRange(ranges []*ranger.Range) []*ranger.Range { HighExclude: false}) ranges[0].LowVal[0] = upper ranges[0].LowExclude = true - if !validRange(ranges[0]) { + if !validRange(ranges[0], encoded) { ranges = ranges[1:] } } @@ -858,109 +867,48 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range return totalCount, nil } -func (c *Column) newNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { - newColHist := &Column{Info: c.Info} - newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, c.Tp, chunk.InitialCapacity, 0) - ranLowIdx := 0 +type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range) (float64, error) + +// newHistogramBySelectivity fulfills the content of new histogram by the given selectivity result. +// TODO: Datum is not efficient, try to avoid using it here. +// Also, there're redundant calculation with Selectivity(). We need to reduce it too. +func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist *Histogram, statsNode *StatsNode, cntByRangeFunc countByRangeFunc) *Histogram { + newHist := NewHistogram(oldHist.ID, int64(float64(oldHist.NDV)*statsNode.Selectivity), 0, 0, oldHist.Tp, chunk.InitialCapacity, 0) + splitRanges := oldHist.SplitRange(statsNode.Ranges, false) + cntPerVal := int64(oldHist.AvgCountPerValue(int64(oldHist.totalRowCount()))) var totCnt int64 = 0 - // Process each bucket. - for i := 0; i < c.Bounds.NumRows() && ranLowIdx < len(statsNode.Ranges); i += 2 { - newBkt := Bucket{Repeat: c.Buckets[i/2].Repeat} - // Find the bucket which is the first one whose have intersection with the bucket. - for ; ranLowIdx < len(statsNode.Ranges); ranLowIdx++ { - if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranLowIdx].HighVal[0]) <= 0 { - break - } + for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(splitRanges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { + for highRangeIdx < len(splitRanges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &splitRanges[highRangeIdx].HighVal[0]) >= 0 { + highRangeIdx++ } - ranHighIdx := ranLowIdx - // Find the bucket which is the first one whose lowVal is bigger than bucket's high bound, i.e. out of the bucket. - for ; ranHighIdx < len(statsNode.Ranges); ranHighIdx++ { - if chunk.Compare(c.Bounds.GetRow(i+1), 0, &statsNode.Ranges[ranHighIdx].LowVal[0]) < 0 { - break - } + if boundIdx+2 >= oldHist.Bounds.NumRows() && splitRanges[highRangeIdx].HighVal[0].Kind() == types.KindMaxValue { + highRangeIdx++ } - if ranLowIdx == ranHighIdx { + if ranIdx == highRangeIdx { continue } - overlapped := 0.0 - // Compute the overlap ratio. - for ranIdx := ranLowIdx; ranIdx < ranHighIdx; ranIdx++ { - overlapped += c.CalcRangeFraction(i/2, &statsNode.Ranges[ranIdx].LowVal[0], &statsNode.Ranges[ranIdx].HighVal[0]) + cnt, err := cntByRangeFunc(sc, histID, splitRanges[ranIdx:highRangeIdx]) + // This should not happen. + if err != nil { + log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) + return nil } - // If there's no overlap, skip it. - if overlapped == 0 { - ranLowIdx = ranHighIdx - 1 + if cnt == 0 { continue } - // Update the bound of the bucket. - if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranLowIdx].LowVal[0]) < 0 { - newColHist.Bounds.AppendDatum(0, &statsNode.Ranges[ranLowIdx].LowVal[0]) - } else { - newColHist.Bounds.AppendRow(c.Bounds.GetRow(i)) - } - if chunk.Compare(c.Bounds.GetRow(i), 0, &statsNode.Ranges[ranHighIdx-1].HighVal[0]) > 0 { - newColHist.Bounds.AppendDatum(0, &statsNode.Ranges[ranHighIdx-1].HighVal[0]) - // Update the repeat val. - newBkt.Repeat = 0 - } else { - newColHist.Bounds.AppendRow(c.Bounds.GetRow(i + 1)) - } - totCnt += int64(overlapped * float64(c.bucketCount(i/2))) - newBkt.Count = totCnt - newColHist.Buckets = append(newColHist.Buckets, newBkt) - ranLowIdx = ranHighIdx - 1 - } - return newColHist, nil -} - -func (c *Column) newNonNumericColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { - newColHist := &Column{Info: c.Info} - newColHist.Histogram = *NewHistogram(c.ID, int64(float64(c.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) - - lowBucketIdx, highBucketIdx := 0, 0 - var totCnt int64 = 0 - - // Bucket bound of non-numeric column is not easy to calculate. Now just use its original bound. - // TODO: enhance its calculation. - // Now just remove the bucket that no range fell in. - for _, ran := range statsNode.Ranges { - lowBucketIdx = highBucketIdx - for ; highBucketIdx*2 < c.Bounds.NumRows(); highBucketIdx++ { - if chunk.Compare(c.Bounds.GetRow(highBucketIdx*2), 0, &ran.HighVal[0]) >= 0 { - break - } - } - for ; lowBucketIdx < highBucketIdx; lowBucketIdx++ { - if chunk.Compare(c.Bounds.GetRow(lowBucketIdx*2+1), 0, &ran.LowVal[0]) > 0 { - break - } + if int64(cnt) > oldHist.bucketCount(boundIdx/2) { + cnt = float64(oldHist.bucketCount(boundIdx / 2)) } - if lowBucketIdx*2 >= c.Bounds.NumRows() { - break + newHist.Bounds.AppendRow(oldHist.Bounds.GetRow(boundIdx)) + newHist.Bounds.AppendRow(oldHist.Bounds.GetRow(boundIdx + 1)) + totCnt += int64(cnt) + bkt := Bucket{Count: totCnt} + if chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &splitRanges[highRangeIdx-1].HighVal[0]) == 0 && !splitRanges[highRangeIdx-1].HighExclude { + bkt.Repeat = cntPerVal } - // TODO: GetDatum and AppendDatum is very inefficient, use a better way to this. - for i := lowBucketIdx; i < highBucketIdx; i++ { - low := c.Bounds.GetRow(i*2).GetDatum(0, c.Tp) - high := c.Bounds.GetRow(i*2+1).GetDatum(0, c.Tp) - newColHist.Bounds.AppendDatum(0, &low) - newColHist.Bounds.AppendDatum(0, &high) - totCnt += c.bucketCount(i) - newColHist.Buckets = append(newColHist.Buckets, Bucket{Repeat: c.Buckets[i].Repeat, Count: totCnt}) - } - } - return newColHist, nil -} - -func (c *Column) newColumnBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Column, error) { - if len(statsNode.Ranges) == 0 { - newColHist := &Column{Info: c.Info} - newColHist.Histogram = *NewHistogram(c.ID, 0, 0, 0, c.Tp, 0, 0) - return newColHist, nil + newHist.Buckets = append(newHist.Buckets, bkt) } - if c.Tp.EvalType() == types.ETInt || c.Tp.EvalType() == types.ETReal { - return c.newNumericColumnBySelectivity(sc, statsNode) - } - return c.newNonNumericColumnBySelectivity(sc, statsNode) + return newHist } func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Index, error) { @@ -1029,15 +977,23 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } newColl.Indices[node.ID] = newIdxHist } else { + colHist, ok := coll.Columns[node.ID] if !ok { continue } - newColHist, err := colHist.newColumnBySelectivity(sc, node) - if err != nil { - return nil, err + newCol := &Column{Info: colHist.Info} + newCol.Histogram = *NewHistogram(newCol.ID, int64(float64(newCol.NDV)*node.Selectivity), 0, 0, colHist.Tp, chunk.InitialCapacity, 0) + var newHist *Histogram + if colHist.isHandle { + newHist = newHistogramBySelectivity(sc, node.ID, &colHist.Histogram, node, coll.GetRowCountByIntColumnRanges) + } else { + newHist = newHistogramBySelectivity(sc, node.ID, &colHist.Histogram, node, coll.GetRowCountByColumnRanges) + } + if newHist != nil { + newCol.Histogram = *newHist + newColl.Columns[node.ID] = newCol } - newColl.Columns[node.ID] = newColHist } } for id, idx := range coll.Indices { diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 1a59abf6fddf3..3ac49f999f746 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -30,15 +30,22 @@ type HistogramTestSuite struct { } func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { + coll := &HistColl{ + Count: 300, + Columns: make(map[int64]*Column), + Indices: make(map[int64]*Index), + } ctx := mock.NewContext() sc := ctx.GetSessionVars().StmtCtx intCol := &Column{} intCol.Histogram = *NewHistogram(1, 30, 0, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) + intCol.isHandle = true for i := 0; i < 10; i++ { intCol.Bounds.AppendInt64(0, int64(i*3)) intCol.Bounds.AppendInt64(0, int64(i*3+2)) intCol.Buckets = append(intCol.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) } + coll.Columns[1] = intCol node := &StatsNode{ID: 1, Tp: pkType, Selectivity: 0.56} node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums(2)}) @@ -46,53 +53,58 @@ func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(8), HighVal: types.MakeDatums(10)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(13), HighVal: types.MakeDatums(13)}) node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(25), HighVal: []types.Datum{types.MaxValueDatum()}}) - newIntCol, err := intCol.newNumericColumnBySelectivity(sc, node) - c.Assert(err, IsNil, Commentf("Test failed: %v", err)) intColResult := `column:1 ndv:16 totColSize:0 num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 -num: 10 lower_bound: 5 upper_bound: 5 repeats: 10 +num: 10 lower_bound: 3 upper_bound: 5 repeats: 10 num: 20 lower_bound: 6 upper_bound: 8 repeats: 10 -num: 20 lower_bound: 9 upper_bound: 11 repeats: 10 -num: 10 lower_bound: 13 upper_bound: 14 repeats: 10 -num: 20 lower_bound: 25 upper_bound: 26 repeats: 10 -num: 30 lower_bound: 27 upper_bound: 29 repeats: 10` - c.Assert(newIntCol.String(), Equals, intColResult) +num: 20 lower_bound: 9 upper_bound: 11 repeats: 0 +num: 10 lower_bound: 12 upper_bound: 14 repeats: 0 +num: 20 lower_bound: 24 upper_bound: 26 repeats: 10 +num: 30 lower_bound: 27 upper_bound: 29 repeats: 0` stringCol := &Column{} - stringCol.Histogram = *NewHistogram(2, 9, 0, 0, types.NewFieldType(mysql.TypeString), chunk.InitialCapacity, 0) + stringCol.Histogram = *NewHistogram(2, 15, 0, 0, types.NewFieldType(mysql.TypeString), chunk.InitialCapacity, 0) stringCol.Bounds.AppendString(0, "a") stringCol.Bounds.AppendString(0, "aaaabbbb") - stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 30}) + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 60}) stringCol.Bounds.AppendString(0, "bbbb") stringCol.Bounds.AppendString(0, "fdsfdsfds") - stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 60}) + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 120}) stringCol.Bounds.AppendString(0, "kkkkk") + stringCol.Bounds.AppendString(0, "ooooo") + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 180}) + stringCol.Bounds.AppendString(0, "oooooo") + stringCol.Bounds.AppendString(0, "sssss") + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 240}) + stringCol.Bounds.AppendString(0, "ssssssu") stringCol.Bounds.AppendString(0, "yyyyy") - stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 90}) - node.Tp = colType - node.ID = 2 - node.Ranges = node.Ranges[:0] - node.Selectivity = 0.6 - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa")}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}}) - newStringCol, err := stringCol.newNonNumericColumnBySelectivity(sc, node) - c.Assert(err, IsNil, Commentf("Test failed: %v", err)) - stringColResult := `column:2 ndv:5 totColSize:0 -num: 30 lower_bound: a upper_bound: aaaabbbb repeats: 10 -num: 30 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 10 -num: 30 lower_bound: kkkkk upper_bound: yyyyy repeats: 10` - c.Assert(newStringCol.String(), Equals, stringColResult) + stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 300}) + stringCol.PreCalculateScalar() + coll.Columns[2] = stringCol + node2 := &StatsNode{ID: 2, Tp: colType, Selectivity: 0.6} + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa")}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}}) + newColl, _ := coll.NewHistCollBySelectivity(sc, []*StatsNode{node, node2}) + stringColResult := `column:2 ndv:9 totColSize:0 +num: 60 lower_bound: a upper_bound: aaaabbbb repeats: 0 +num: 60 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 20 +num: 60 lower_bound: kkkkk upper_bound: ooooo repeats: 20 +num: 60 lower_bound: oooooo upper_bound: sssss repeats: 20 +num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` + + c.Assert(newColl.Columns[1].String(), Equals, intColResult) + c.Assert(newColl.Columns[2].String(), Equals, stringColResult) idx := Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: model.NewCIStr("a"), Offset: 0}}}} idx.Histogram = *NewHistogram(0, 15, 0, 0, types.NewFieldType(mysql.TypeBlob), 0, 0) for i := 0; i < 5; i++ { low, err1 := codec.EncodeKey(sc, nil, types.NewIntDatum(int64(i*3))) - c.Assert(err1, IsNil, Commentf("Test failed: %v", err)) + c.Assert(err1, IsNil, Commentf("Test failed: %v", err1)) high, err2 := codec.EncodeKey(sc, nil, types.NewIntDatum(int64(i*3+2))) - c.Assert(err2, IsNil, Commentf("Test failed: %v", err)) + c.Assert(err2, IsNil, Commentf("Test failed: %v", err2)) idx.Bounds.AppendBytes(0, low) idx.Bounds.AppendBytes(0, high) idx.Buckets = append(idx.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) diff --git a/statistics/update_test.go b/statistics/update_test.go index a28f48c8534d9..40a7d7f3c5c7f 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -701,7 +701,7 @@ func (s *testStatsUpdateSuite) TestSplitRange(c *C) { HighExclude: t.exclude[i+1], }) } - ranges = h.SplitRange(ranges) + ranges = h.SplitRange(ranges, false) var ranStrs []string for _, ran := range ranges { ranStrs = append(ranStrs, ran.String()) From 51a6a5c318efaa95caada36e2aebb2697eb0ef3f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 12 Nov 2018 15:30:42 +0800 Subject: [PATCH 09/23] clean code --- statistics/histogram.go | 46 ++++++++++-------- statistics/histogram_test.go | 22 ++++----- statistics/scalar.go | 90 ------------------------------------ 3 files changed, 37 insertions(+), 121 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 63e4b16b87893..839c3885d940e 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -872,16 +872,15 @@ type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range) // newHistogramBySelectivity fulfills the content of new histogram by the given selectivity result. // TODO: Datum is not efficient, try to avoid using it here. // Also, there're redundant calculation with Selectivity(). We need to reduce it too. -func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist *Histogram, statsNode *StatsNode, cntByRangeFunc countByRangeFunc) *Histogram { - newHist := NewHistogram(oldHist.ID, int64(float64(oldHist.NDV)*statsNode.Selectivity), 0, 0, oldHist.Tp, chunk.InitialCapacity, 0) - splitRanges := oldHist.SplitRange(statsNode.Ranges, false) +func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error { + splitRanges := oldHist.SplitRange(ranges, false) cntPerVal := int64(oldHist.AvgCountPerValue(int64(oldHist.totalRowCount()))) var totCnt int64 = 0 for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(splitRanges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { for highRangeIdx < len(splitRanges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &splitRanges[highRangeIdx].HighVal[0]) >= 0 { highRangeIdx++ } - if boundIdx+2 >= oldHist.Bounds.NumRows() && splitRanges[highRangeIdx].HighVal[0].Kind() == types.KindMaxValue { + if boundIdx+2 >= oldHist.Bounds.NumRows() && highRangeIdx < len(splitRanges) && splitRanges[highRangeIdx].HighVal[0].Kind() == types.KindMaxValue { highRangeIdx++ } if ranIdx == highRangeIdx { @@ -890,8 +889,7 @@ func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHi cnt, err := cntByRangeFunc(sc, histID, splitRanges[ranIdx:highRangeIdx]) // This should not happen. if err != nil { - log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) - return nil + return err } if cnt == 0 { continue @@ -907,8 +905,12 @@ func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHi bkt.Repeat = cntPerVal } newHist.Buckets = append(newHist.Buckets, bkt) + switch newHist.Tp.EvalType() { + case types.ETString, types.ETDecimal, types.ETDatetime, types.ETTimestamp: + newHist.scalars = append(newHist.scalars, oldHist.scalars[boundIdx/2]) + } } - return newHist + return nil } func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode *StatsNode) (*Index, error) { @@ -947,10 +949,11 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode break } for i := lowBucketIdx; i < highBucketIdx; i++ { - newIndexHist.Bounds.AppendBytes(0, idx.Bounds.GetRow(i*2).GetBytes(0)) - newIndexHist.Bounds.AppendBytes(0, idx.Bounds.GetRow(i*2+1).GetBytes(0)) + newIndexHist.Bounds.AppendRow(idx.Bounds.GetRow(i * 2)) + newIndexHist.Bounds.AppendRow(idx.Bounds.GetRow(i*2 + 1)) totCnt += idx.bucketCount(i) newIndexHist.Buckets = append(newIndexHist.Buckets, Bucket{Repeat: idx.Buckets[i].Repeat, Count: totCnt}) + newIndexHist.scalars = append(newIndexHist.scalars, idx.scalars[i]) } } return newIndexHist, nil @@ -973,27 +976,29 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } newIdxHist, err := idxHist.newIndexBySelectivity(sc, node) if err != nil { - return nil, err + log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) + continue } newColl.Indices[node.ID] = newIdxHist } else { - colHist, ok := coll.Columns[node.ID] + oldCol, ok := coll.Columns[node.ID] if !ok { continue } - newCol := &Column{Info: colHist.Info} - newCol.Histogram = *NewHistogram(newCol.ID, int64(float64(newCol.NDV)*node.Selectivity), 0, 0, colHist.Tp, chunk.InitialCapacity, 0) - var newHist *Histogram - if colHist.isHandle { - newHist = newHistogramBySelectivity(sc, node.ID, &colHist.Histogram, node, coll.GetRowCountByIntColumnRanges) + newCol := &Column{Info: oldCol.Info} + newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) + var err error + if oldCol.isHandle { + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByIntColumnRanges) } else { - newHist = newHistogramBySelectivity(sc, node.ID, &colHist.Histogram, node, coll.GetRowCountByColumnRanges) + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByColumnRanges) } - if newHist != nil { - newCol.Histogram = *newHist - newColl.Columns[node.ID] = newCol + if err != nil { + log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) + continue } + newColl.Columns[node.ID] = newCol } } for id, idx := range coll.Indices { @@ -1010,6 +1015,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } return newColl, nil } + func (idx *Index) outOfRange(val types.Datum) bool { if idx.Histogram.Len() == 0 { return true diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 3ac49f999f746..878f1eed7ed82 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -88,7 +88,6 @@ num: 30 lower_bound: 27 upper_bound: 29 repeats: 0` node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}}) - newColl, _ := coll.NewHistCollBySelectivity(sc, []*StatsNode{node, node2}) stringColResult := `column:2 ndv:9 totColSize:0 num: 60 lower_bound: a upper_bound: aaaabbbb repeats: 0 num: 60 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 20 @@ -96,9 +95,11 @@ num: 60 lower_bound: kkkkk upper_bound: ooooo repeats: 20 num: 60 lower_bound: oooooo upper_bound: sssss repeats: 20 num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` + newColl, _ := coll.NewHistCollBySelectivity(sc, []*StatsNode{node, node2}) c.Assert(newColl.Columns[1].String(), Equals, intColResult) c.Assert(newColl.Columns[2].String(), Equals, stringColResult) - idx := Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: model.NewCIStr("a"), Offset: 0}}}} + + idx := &Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: model.NewCIStr("a"), Offset: 0}}}} idx.Histogram = *NewHistogram(0, 15, 0, 0, types.NewFieldType(mysql.TypeBlob), 0, 0) for i := 0; i < 5; i++ { low, err1 := codec.EncodeKey(sc, nil, types.NewIntDatum(int64(i*3))) @@ -109,19 +110,18 @@ num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` idx.Bounds.AppendBytes(0, high) idx.Buckets = append(idx.Buckets, Bucket{Repeat: 10, Count: int64(30*i + 30)}) } - node.Tp = indexType - node.ID = 0 - node.Selectivity = 0.47 - node.Ranges = node.Ranges[:0] - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(7), HighVal: types.MakeDatums(11)}) - newIdx, err := idx.newIndexBySelectivity(sc, node) - c.Assert(err, IsNil, Commentf("Test failed: %v", err)) + idx.PreCalculateScalar() + node3 := &StatsNode{ID: 0, Tp: indexType, Selectivity: 0.47} + node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) + node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(7), HighVal: types.MakeDatums(11)}) + idxResult := `index:0 ndv:7 num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 num: 30 lower_bound: 3 upper_bound: 5 repeats: 10 num: 30 lower_bound: 6 upper_bound: 8 repeats: 10 num: 30 lower_bound: 9 upper_bound: 11 repeats: 10` - c.Assert(newIdx.String(), Equals, idxResult) + newIdx, err := idx.newIndexBySelectivity(sc, node3) + c.Assert(err, IsNil, Commentf("Test failed: %v", err)) + c.Assert(newIdx.String(), Equals, idxResult) } diff --git a/statistics/scalar.go b/statistics/scalar.go index bf50797b1770d..446a78a7383f6 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -41,32 +41,6 @@ func calcFraction(lower, upper, value float64) float64 { return frac } -func calcFloatRangeFraction(lower, upper, valLow, valHigh float64) float64 { - if upper <= lower { - return 0.5 - } - if valHigh > upper { - valHigh = upper - } - if valLow < lower { - valLow = lower - } - return (valHigh - valLow) / (upper - lower) -} - -func calcIntRangeFraction(lower, upper, valLow, valHigh float64) float64 { - if upper <= lower { - return 0.5 - } - if valHigh > upper { - valHigh = upper - } - if valLow < lower { - valLow = lower - } - return (valHigh - valLow + 1) / (upper - lower + 1) -} - func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { switch value.Kind() { case types.KindMysqlDecimal: @@ -161,70 +135,6 @@ func (hg *Histogram) calcFraction(index int, value *types.Datum) float64 { return 0.5 } -// CalcRangeFraction calculates the fraction between [valLow, valHigh] and hg's indexth Bucket. -func (hg *Histogram) CalcRangeFraction(index int, dLow, dHigh *types.Datum) float64 { - lower, upper := hg.Bounds.GetRow(2*index), hg.Bounds.GetRow(2*index+1) - k := dLow.Kind() - if k == types.KindNull || k == types.KindMinNotNull { - k = dHigh.Kind() - } - switch k { - case types.KindFloat32: - var varLow, varHigh float64 - if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { - varLow = float64(lower.GetFloat32(0)) - } else { - varLow = float64(dLow.GetFloat32()) - } - if dHigh.Kind() == types.KindMaxValue { - varHigh = float64(upper.GetFloat32(0)) - } else { - varHigh = float64(dHigh.GetFloat32()) - } - return calcFloatRangeFraction(float64(lower.GetFloat32(0)), float64(upper.GetFloat32(0)), varLow, varHigh) - case types.KindFloat64: - var varLow, varHigh float64 - if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { - varLow = lower.GetFloat64(0) - } else { - varLow = dLow.GetFloat64() - } - if dHigh.Kind() == types.KindMaxValue { - varHigh = upper.GetFloat64(0) - } else { - varHigh = dHigh.GetFloat64() - } - return calcFloatRangeFraction(lower.GetFloat64(0), upper.GetFloat64(0), varLow, varHigh) - case types.KindInt64: - var varLow, varHigh float64 - if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { - varLow = float64(lower.GetInt64(0)) - } else { - varLow = float64(dLow.GetInt64()) - } - if dHigh.Kind() == types.KindMaxValue { - varHigh = float64(upper.GetInt64(0)) - } else { - varHigh = float64(dHigh.GetInt64()) - } - return calcIntRangeFraction(float64(lower.GetInt64(0)), float64(upper.GetInt64(0)), varLow, varHigh) - case types.KindUint64: - var varLow, varHigh float64 - if dLow.IsNull() || dLow.Kind() == types.KindMinNotNull { - varLow = float64(lower.GetUint64(0)) - } else { - varLow = float64(dLow.GetUint64()) - } - if dHigh.Kind() == types.KindMaxValue { - varHigh = float64(upper.GetUint64(0)) - } else { - varHigh = float64(dHigh.GetUint64()) - } - return calcIntRangeFraction(float64(lower.GetUint64(0)), float64(upper.GetUint64(0)), varLow, varHigh) - } - return 0.5 -} - func commonPrefixLength(lower, upper []byte) int { minLen := len(lower) if minLen > len(upper) { From 27886bbacc29512d3da97b9e0a5348b1efc25baf Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 15 Nov 2018 14:54:23 +0800 Subject: [PATCH 10/23] address comments --- statistics/histogram.go | 45 ++++++++++++++++++------------------ statistics/histogram_test.go | 6 ++--- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 839c3885d940e..270d599ee0309 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -933,19 +933,21 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode if err != nil { return nil, err } - for ; highBucketIdx*2 < idx.Bounds.NumRows(); highBucketIdx++ { + for ; highBucketIdx < idx.Len(); highBucketIdx++ { // Encoded value can only go to its next quickly. So ranHighEncode is actually range.HighVal's PrefixNext value. // So the Bound should also go to its PrefixNext. - if bytes.Compare(ranHighEncode, kv.Key(idx.Bounds.GetRow(highBucketIdx*2).GetBytes(0)).PrefixNext()) < 0 { + bucketLowerEncoded := idx.Bounds.GetRow(highBucketIdx * 2).GetBytes(0) + if bytes.Compare(ranHighEncode, kv.Key(bucketLowerEncoded).PrefixNext()) < 0 { break } } for ; lowBucketIdx < highBucketIdx; lowBucketIdx++ { - if bytes.Compare(ranLowEncode, idx.Bounds.GetRow(lowBucketIdx*2+1).GetBytes(0)) <= 0 { + bucketUpperEncoded := idx.Bounds.GetRow(lowBucketIdx*2 + 1).GetBytes(0) + if bytes.Compare(ranLowEncode, bucketUpperEncoded) <= 0 { break } } - if lowBucketIdx*2 >= idx.Bounds.NumRows() { + if lowBucketIdx >= idx.Len() { break } for i := lowBucketIdx; i < highBucketIdx; i++ { @@ -980,26 +982,25 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta continue } newColl.Indices[node.ID] = newIdxHist + continue + } + oldCol, ok := coll.Columns[node.ID] + if !ok { + continue + } + newCol := &Column{Info: oldCol.Info} + newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) + var err error + if oldCol.isHandle { + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByIntColumnRanges) } else { - - oldCol, ok := coll.Columns[node.ID] - if !ok { - continue - } - newCol := &Column{Info: oldCol.Info} - newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) - var err error - if oldCol.isHandle { - err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByIntColumnRanges) - } else { - err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByColumnRanges) - } - if err != nil { - log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) - continue - } - newColl.Columns[node.ID] = newCol + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByColumnRanges) + } + if err != nil { + log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) + continue } + newColl.Columns[node.ID] = newCol } for id, idx := range coll.Indices { _, ok := newColl.Indices[id] diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 878f1eed7ed82..42a13f5d359fa 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -113,13 +113,13 @@ num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` idx.PreCalculateScalar() node3 := &StatsNode{ID: 0, Tp: indexType, Selectivity: 0.47} node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) - node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(7), HighVal: types.MakeDatums(11)}) + node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(10), HighVal: types.MakeDatums(13)}) idxResult := `index:0 ndv:7 num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 num: 30 lower_bound: 3 upper_bound: 5 repeats: 10 -num: 30 lower_bound: 6 upper_bound: 8 repeats: 10 -num: 30 lower_bound: 9 upper_bound: 11 repeats: 10` +num: 30 lower_bound: 9 upper_bound: 11 repeats: 10 +num: 30 lower_bound: 12 upper_bound: 14 repeats: 10` newIdx, err := idx.newIndexBySelectivity(sc, node3) c.Assert(err, IsNil, Commentf("Test failed: %v", err)) From c53f28956b43c97f04bee82e8f47c08eac1c9a0a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 15 Nov 2018 20:22:02 +0800 Subject: [PATCH 11/23] deal with null --- planner/core/stats.go | 2 +- statistics/histogram.go | 32 ++++++++++++++++++++++---------- statistics/histogram_test.go | 2 +- types/datum.go | 5 +++++ 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/planner/core/stats.go b/planner/core/stats.go index a92ab12dfdbf4..03d5b8faa511c 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -84,7 +84,7 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) (*property.Sta selectivity = selectionFactor } if ds.ctx.GetSessionVars().OptimizerSelectivityLevel >= 1 && ds.stats.HistColl != nil { - finalHist, err := ds.stats.HistColl.NewHistCollBySelectivity(ds.ctx.GetSessionVars().StmtCtx, nodes) + finalHist := ds.stats.HistColl.NewHistCollBySelectivity(ds.ctx.GetSessionVars().StmtCtx, nodes) if err != nil { log.Warnf("[stats-in-datasource]: An error happened: %v", err.Error()) } diff --git a/statistics/histogram.go b/statistics/histogram.go index 270d599ee0309..632b9f48207c5 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -873,20 +873,19 @@ type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range) // TODO: Datum is not efficient, try to avoid using it here. // Also, there're redundant calculation with Selectivity(). We need to reduce it too. func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error { - splitRanges := oldHist.SplitRange(ranges, false) cntPerVal := int64(oldHist.AvgCountPerValue(int64(oldHist.totalRowCount()))) var totCnt int64 = 0 - for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(splitRanges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { - for highRangeIdx < len(splitRanges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &splitRanges[highRangeIdx].HighVal[0]) >= 0 { + for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(ranges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { + for highRangeIdx < len(ranges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &ranges[highRangeIdx].HighVal[0]) >= 0 { highRangeIdx++ } - if boundIdx+2 >= oldHist.Bounds.NumRows() && highRangeIdx < len(splitRanges) && splitRanges[highRangeIdx].HighVal[0].Kind() == types.KindMaxValue { + if boundIdx+2 >= oldHist.Bounds.NumRows() && highRangeIdx < len(ranges) && ranges[highRangeIdx].HighVal[0].Kind() == types.KindMaxValue { highRangeIdx++ } if ranIdx == highRangeIdx { continue } - cnt, err := cntByRangeFunc(sc, histID, splitRanges[ranIdx:highRangeIdx]) + cnt, err := cntByRangeFunc(sc, histID, ranges[ranIdx:highRangeIdx]) // This should not happen. if err != nil { return err @@ -901,7 +900,7 @@ func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHi newHist.Bounds.AppendRow(oldHist.Bounds.GetRow(boundIdx + 1)) totCnt += int64(cnt) bkt := Bucket{Count: totCnt} - if chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &splitRanges[highRangeIdx-1].HighVal[0]) == 0 && !splitRanges[highRangeIdx-1].HighExclude { + if chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &ranges[highRangeIdx-1].HighVal[0]) == 0 && !ranges[highRangeIdx-1].HighExclude { bkt.Repeat = cntPerVal } newHist.Buckets = append(newHist.Buckets, bkt) @@ -962,7 +961,7 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode } // NewHistCollBySelectivity creates new HistColl by the given statsNodes. -func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, statsNodes []*StatsNode) (*HistColl, error) { +func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, statsNodes []*StatsNode) *HistColl { newColl := &HistColl{ Columns: make(map[int64]*Column), Indices: make(map[int64]*Index), @@ -991,10 +990,23 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta newCol := &Column{Info: oldCol.Info} newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) var err error + splitRanges := oldCol.Histogram.SplitRange(node.Ranges, false) + // Deal with some corner case. + if len(splitRanges) > 0 { + // Deal with NULL values. + if splitRanges[0].LowVal[0].IsNull() { + newCol.NullCount = oldCol.NullCount + if splitRanges[0].HighVal[0].IsNull() { + splitRanges = splitRanges[1:] + } else { + splitRanges[0].LowVal[0].SetMinNotNull() + } + } + } if oldCol.isHandle { - err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByIntColumnRanges) + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, splitRanges, coll.GetRowCountByIntColumnRanges) } else { - err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, node.Ranges, coll.GetRowCountByColumnRanges) + err = newHistogramBySelectivity(sc, node.ID, &oldCol.Histogram, &newCol.Histogram, splitRanges, coll.GetRowCountByColumnRanges) } if err != nil { log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) @@ -1014,7 +1026,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta newColl.Columns[id] = col } } - return newColl, nil + return newColl } func (idx *Index) outOfRange(val types.Datum) bool { diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 42a13f5d359fa..f7e36ab43f41f 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -95,7 +95,7 @@ num: 60 lower_bound: kkkkk upper_bound: ooooo repeats: 20 num: 60 lower_bound: oooooo upper_bound: sssss repeats: 20 num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0` - newColl, _ := coll.NewHistCollBySelectivity(sc, []*StatsNode{node, node2}) + newColl := coll.NewHistCollBySelectivity(sc, []*StatsNode{node, node2}) c.Assert(newColl.Columns[1].String(), Equals, intColResult) c.Assert(newColl.Columns[2].String(), Equals, stringColResult) diff --git a/types/datum.go b/types/datum.go index 09607c474d9b4..4b290c317cce1 100644 --- a/types/datum.go +++ b/types/datum.go @@ -218,6 +218,11 @@ func (d *Datum) SetNull() { d.x = nil } +func (d *Datum) SetMinNotNull() { + d.k = KindMinNotNull + d.x = nil +} + // GetBinaryLiteral gets Bit value func (d *Datum) GetBinaryLiteral() BinaryLiteral { return d.b From bd9944a324ca3beda81627c240df3b50e2cee17c Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 29 Nov 2018 18:16:08 +0800 Subject: [PATCH 12/23] modify unit test --- statistics/histogram.go | 17 +++++++++++++---- statistics/histogram_test.go | 11 +++-------- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 632b9f48207c5..10443968a1c3c 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -383,14 +383,14 @@ func (hg *Histogram) equalRowCount(value types.Datum) float64 { if match { return float64(hg.Buckets[index/2].Repeat) } - return hg.totalRowCount() / float64(hg.NDV) + return hg.notNullCount() / float64(hg.NDV) } if match { cmp := chunk.GetCompareFunc(hg.Tp) if cmp(hg.Bounds.GetRow(index), 0, hg.Bounds.GetRow(index+1), 0) == 0 { return float64(hg.Buckets[index/2].Repeat) } - return hg.totalRowCount() / float64(hg.NDV) + return hg.notNullCount() / float64(hg.NDV) } return 0 } @@ -466,6 +466,13 @@ func (hg *Histogram) totalRowCount() float64 { return float64(hg.Buckets[hg.Len()-1].Count + hg.NullCount) } +func (hg *Histogram) notNullCount() float64 { + if hg.Len() == 0 { + return 0 + } + return float64(hg.Buckets[hg.Len()-1].Count) +} + // mergeBuckets is used to merge every two neighbor buckets. func (hg *Histogram) mergeBuckets(bucketIdx int) { curBuck := 0 @@ -680,11 +687,13 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, // AvgCountPerValue gets the average row count per value by the data of histogram. func (hg *Histogram) AvgCountPerValue(totalCount int64) float64 { + factor := hg.getIncreaseFactor(totalCount) + totalNotNull := hg.notNullCount() * factor curNDV := float64(hg.NDV) * hg.getIncreaseFactor(totalCount) if curNDV == 0 { curNDV = 1 } - return float64(totalCount) / curNDV + return totalNotNull / curNDV } func (hg *Histogram) outOfRange(val types.Datum) bool { @@ -987,7 +996,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta if !ok { continue } - newCol := &Column{Info: oldCol.Info} + newCol := &Column{Info: oldCol.Info, isHandle: oldCol.isHandle} newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) var err error splitRanges := oldCol.Histogram.SplitRange(node.Ranges, false) diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index f7e36ab43f41f..73b7daaa64ecc 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -24,21 +24,16 @@ import ( "github.com/pingcap/tidb/util/ranger" ) -var _ = Suite(&HistogramTestSuite{}) - -type HistogramTestSuite struct { -} - func (s *testStatisticsSuite) TestNewHistogramBySelectivity(c *C) { coll := &HistColl{ - Count: 300, + Count: 330, Columns: make(map[int64]*Column), Indices: make(map[int64]*Index), } ctx := mock.NewContext() sc := ctx.GetSessionVars().StmtCtx intCol := &Column{} - intCol.Histogram = *NewHistogram(1, 30, 0, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) + intCol.Histogram = *NewHistogram(1, 30, 30, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) intCol.isHandle = true for i := 0; i < 10; i++ { intCol.Bounds.AppendInt64(0, int64(i*3)) @@ -63,7 +58,7 @@ num: 20 lower_bound: 24 upper_bound: 26 repeats: 10 num: 30 lower_bound: 27 upper_bound: 29 repeats: 0` stringCol := &Column{} - stringCol.Histogram = *NewHistogram(2, 15, 0, 0, types.NewFieldType(mysql.TypeString), chunk.InitialCapacity, 0) + stringCol.Histogram = *NewHistogram(2, 15, 30, 0, types.NewFieldType(mysql.TypeString), chunk.InitialCapacity, 0) stringCol.Bounds.AppendString(0, "a") stringCol.Bounds.AppendString(0, "aaaabbbb") stringCol.Buckets = append(stringCol.Buckets, Bucket{Repeat: 10, Count: 60}) From 6ba114843d5f126779e033a62f4683a2156a95d6 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 29 Nov 2018 19:43:43 +0800 Subject: [PATCH 13/23] address comments --- statistics/histogram.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 10443968a1c3c..bd6215608d72e 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -926,7 +926,7 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode ranLowEncode, ranHighEncode []byte err error ) - newIndexHist := &Index{Info: idx.Info} + newIndexHist := &Index{Info: idx.Info, statsVer: idx.statsVer, CMSketch: idx.CMSketch} newIndexHist.Histogram = *NewHistogram(idx.ID, int64(float64(idx.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0) lowBucketIdx, highBucketIdx := 0, 0 @@ -996,7 +996,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta if !ok { continue } - newCol := &Column{Info: oldCol.Info, isHandle: oldCol.isHandle} + newCol := &Column{Info: oldCol.Info, isHandle: oldCol.isHandle, CMSketch: oldCol.CMSketch} newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) var err error splitRanges := oldCol.Histogram.SplitRange(node.Ranges, false) From 28970e1ca705b669b944280d4719e00da7e86eb4 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 10 Dec 2018 18:51:05 +0800 Subject: [PATCH 14/23] Update statistics/histogram.go Co-Authored-By: winoros --- statistics/histogram.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index bd6215608d72e..a7751a49a8969 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -689,7 +689,7 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, func (hg *Histogram) AvgCountPerValue(totalCount int64) float64 { factor := hg.getIncreaseFactor(totalCount) totalNotNull := hg.notNullCount() * factor - curNDV := float64(hg.NDV) * hg.getIncreaseFactor(totalCount) + curNDV := float64(hg.NDV) * factor if curNDV == 0 { curNDV = 1 } From d55c26e59bcf073bb68502cb3a33dbf2f6b1478c Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 12 Dec 2018 19:52:55 +0800 Subject: [PATCH 15/23] address comments --- distsql/request_builder.go | 5 +++-- planner/core/exhaust_physical_plans.go | 4 ++-- planner/core/logical_plans.go | 2 +- statistics/feedback.go | 6 +++--- statistics/histogram.go | 18 +++++++++--------- statistics/selectivity.go | 15 ++++++++------- statistics/update_test.go | 2 +- 7 files changed, 27 insertions(+), 25 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index e652c78a7c63e..dcf8db67c3c8d 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -173,7 +173,8 @@ func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.Que if fb == nil || fb.Hist() == nil { return tableRangesToKVRangesWithoutSplit(tid, ranges) } - ranges = fb.Hist().SplitRange(ranges, false) + // EncodeInt don't need *statement.Context. + ranges = fb.Hist().SplitRange(nil, ranges, false) krs := make([]kv.KeyRange, 0, len(ranges)) feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { @@ -260,7 +261,7 @@ func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, range feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true}) } - feedbackRanges = fb.Hist().SplitRange(feedbackRanges, true) + feedbackRanges = fb.Hist().SplitRange(sc, feedbackRanges, true) krs := make([]kv.KeyRange, 0, len(feedbackRanges)) for _, ran := range feedbackRanges { low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index bd4937251de56..cd04402facaad 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -448,7 +448,7 @@ func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Col var rowCount float64 pkHist, ok := ds.statisticTable.Columns[pk.ID] if ok && !ds.statisticTable.Pseudo { - rowCount = pkHist.AvgCountPerValue(ds.statisticTable.Count) + rowCount = pkHist.AvgCountPerNotNullValue(ds.statisticTable.Count) } else { rowCount = ds.statisticTable.PseudoAvgCountPerValue() } @@ -495,7 +495,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn var rowCount float64 idxHist, ok := ds.statisticTable.Indices[idx.ID] if ok && !ds.statisticTable.Pseudo { - rowCount = idxHist.AvgCountPerValue(ds.statisticTable.Count) + rowCount = idxHist.AvgCountPerNotNullValue(ds.statisticTable.Count) } else { rowCount = ds.statisticTable.PseudoAvgCountPerValue() } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 9c16288ffe726..d52db2b9b0024 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -448,7 +448,7 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { if corColInAccessConds { idxHist, ok := ds.stats.HistColl.Indices[path.index.ID] if ok && !ds.stats.HistColl.Pseudo { - path.countAfterAccess = idxHist.AvgCountPerValue(ds.statisticTable.Count) + path.countAfterAccess = idxHist.AvgCountPerNotNullValue(ds.statisticTable.Count) } else { path.countAfterAccess = ds.statisticTable.PseudoAvgCountPerValue() } diff --git a/statistics/feedback.go b/statistics/feedback.go index 8f0c74441c044..e00848d85129e 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -1052,7 +1052,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { equalityCount, rangeCount = getNewCountForIndex(equalityCount, rangeCount, float64(t.Count), float64(q.feedback[i].count)) value := types.NewBytesDatum(bytes) q.feedback[i] = feedback{lower: &value, upper: &value, count: int64(equalityCount)} - err = rangeFB.dumpRangeFeedback(h, &rang, rangeCount) + err = rangeFB.dumpRangeFeedback(sc, h, &rang, rangeCount) if err != nil { log.Debug("dump range feedback failed:", err) continue @@ -1061,7 +1061,7 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { return errors.Trace(h.dumpFeedbackToKV(q)) } -func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCount float64) error { +func (q *QueryFeedback) dumpRangeFeedback(sc *stmtctx.StatementContext, h *Handle, ran *ranger.Range, rangeCount float64) error { if q.tp == indexType { sc := &stmtctx.StatementContext{TimeZone: time.UTC} lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) @@ -1086,7 +1086,7 @@ func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCou ran.HighVal[0] = getMaxValue(k, q.hist.Tp) } } - ranges := q.hist.SplitRange([]*ranger.Range{ran}, q.tp == indexType) + ranges := q.hist.SplitRange(sc, []*ranger.Range{ran}, q.tp == indexType) counts := make([]float64, 0, len(ranges)) sum := 0.0 for _, r := range ranges { diff --git a/statistics/histogram.go b/statistics/histogram.go index a7751a49a8969..af0d24fefd696 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -506,17 +506,17 @@ func (hg *Histogram) getIncreaseFactor(totalCount int64) float64 { // validRange checks if the range is valid, it is used by `SplitRange` to remove the invalid range, // the possible types of range are index key range and handle key range. -func validRange(ran *ranger.Range, encoded bool) bool { +func validRange(sc *stmtctx.StatementContext, ran *ranger.Range, encoded bool) bool { var low, high []byte if encoded { low, high = ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() } else { var err error - low, err = codec.EncodeKey(nil, nil, ran.LowVal[0]) + low, err = codec.EncodeKey(sc, nil, ran.LowVal[0]) if err != nil { return false } - high, err = codec.EncodeKey(nil, nil, ran.HighVal[0]) + high, err = codec.EncodeKey(sc, nil, ran.HighVal[0]) if err != nil { return false } @@ -533,7 +533,7 @@ func validRange(ran *ranger.Range, encoded bool) bool { // SplitRange splits the range according to the histogram upper bound. Note that we treat last bucket's upper bound // as inf, so all the split Ranges will totally fall in one of the (-inf, u(0)], (u(0), u(1)],...(u(n-3), u(n-2)], // (u(n-2), +inf), where n is the number of buckets, u(i) is the i-th bucket's upper bound. -func (hg *Histogram) SplitRange(ranges []*ranger.Range, encoded bool) []*ranger.Range { +func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, ranges []*ranger.Range, encoded bool) []*ranger.Range { split := make([]*ranger.Range, 0, len(ranges)) for len(ranges) > 0 { // Find the last bound that greater or equal to the LowVal. @@ -577,7 +577,7 @@ func (hg *Histogram) SplitRange(ranges []*ranger.Range, encoded bool) []*ranger. HighExclude: false}) ranges[0].LowVal[0] = upper ranges[0].LowExclude = true - if !validRange(ranges[0], encoded) { + if !validRange(sc, ranges[0], encoded) { ranges = ranges[1:] } } @@ -685,8 +685,8 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, return lh, nil } -// AvgCountPerValue gets the average row count per value by the data of histogram. -func (hg *Histogram) AvgCountPerValue(totalCount int64) float64 { +// AvgCountPerNotNullValue gets the average row count per value by the data of histogram. +func (hg *Histogram) AvgCountPerNotNullValue(totalCount int64) float64 { factor := hg.getIncreaseFactor(totalCount) totalNotNull := hg.notNullCount() * factor curNDV := float64(hg.NDV) * factor @@ -882,7 +882,7 @@ type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range) // TODO: Datum is not efficient, try to avoid using it here. // Also, there're redundant calculation with Selectivity(). We need to reduce it too. func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error { - cntPerVal := int64(oldHist.AvgCountPerValue(int64(oldHist.totalRowCount()))) + cntPerVal := int64(oldHist.AvgCountPerNotNullValue(int64(oldHist.totalRowCount()))) var totCnt int64 = 0 for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(ranges); boundIdx, ranIdx = boundIdx+2, highRangeIdx { for highRangeIdx < len(ranges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &ranges[highRangeIdx].HighVal[0]) >= 0 { @@ -999,7 +999,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta newCol := &Column{Info: oldCol.Info, isHandle: oldCol.isHandle, CMSketch: oldCol.CMSketch} newCol.Histogram = *NewHistogram(oldCol.ID, int64(float64(oldCol.NDV)*node.Selectivity), 0, 0, oldCol.Tp, chunk.InitialCapacity, 0) var err error - splitRanges := oldCol.Histogram.SplitRange(node.Ranges, false) + splitRanges := oldCol.Histogram.SplitRange(sc, node.Ranges, false) // Deal with some corner case. if len(splitRanges) > 0 { // Deal with NULL values. diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 5a5eb7ad1d263..1f8c2be3b4fbe 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -268,13 +268,15 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran // getUsableSetsByGreedy will select the indices and pk used for calculate selectivity by greedy algorithm. func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { - tmpNodes := make([]*StatsNode, len(nodes)) - copy(tmpNodes, nodes) + marked := make([]bool, len(nodes)) mask := int64(math.MaxInt64) for { // Choose the index that covers most. bestID, bestCount, bestTp, bestNumCols := -1, 0, colType, 0 - for i, set := range tmpNodes { + for i, set := range nodes { + if marked[i] { + continue + } set.mask &= mask bits := popCount(set.mask) // This set cannot cover any thing, just skip it. @@ -294,11 +296,10 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { } // update the mask, remove the bit that nodes[bestID].mask has. - mask &^= tmpNodes[bestID].mask + mask &^= nodes[bestID].mask - newBlocks = append(newBlocks, tmpNodes[bestID]) - // remove the chosen one - tmpNodes = append(tmpNodes[:bestID], tmpNodes[bestID+1:]...) + newBlocks = append(newBlocks, nodes[bestID]) + marked[bestID] = true } return } diff --git a/statistics/update_test.go b/statistics/update_test.go index 40a7d7f3c5c7f..de3da77adbadb 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -701,7 +701,7 @@ func (s *testStatsUpdateSuite) TestSplitRange(c *C) { HighExclude: t.exclude[i+1], }) } - ranges = h.SplitRange(ranges, false) + ranges = h.SplitRange(nil, ranges, false) var ranStrs []string for _, ran := range ranges { ranStrs = append(ranStrs, ran.String()) From 8de9cc73d785529fb32b2699ee38f6a1fa103a25 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 13:32:00 +0800 Subject: [PATCH 16/23] fix lint --- statistics/feedback.go | 1 - types/datum.go | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/feedback.go b/statistics/feedback.go index e00848d85129e..82a86c22fb2d5 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -1063,7 +1063,6 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error { func (q *QueryFeedback) dumpRangeFeedback(sc *stmtctx.StatementContext, h *Handle, ran *ranger.Range, rangeCount float64) error { if q.tp == indexType { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) if err != nil { return errors.Trace(err) diff --git a/types/datum.go b/types/datum.go index 4b290c317cce1..83aa0f043e2ec 100644 --- a/types/datum.go +++ b/types/datum.go @@ -218,6 +218,7 @@ func (d *Datum) SetNull() { d.x = nil } +// SetMinNotNull sets datum to minNotNull value. func (d *Datum) SetMinNotNull() { d.k = KindMinNotNull d.x = nil From e679269e9e8da243e1c2660084d487ddd153b828 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 13:38:17 +0800 Subject: [PATCH 17/23] tiny change --- statistics/selectivity.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 1f8c2be3b4fbe..ccb2328c27108 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -229,7 +229,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp // Initialize the mask with the full set. mask := (int64(1) << uint(len(remainedExprs))) - 1 for _, set := range usedSets { - mask ^= set.mask + mask &^= set.mask ret *= set.Selectivity } // If there's still conditions which cannot be calculated, we will multiply a selectionFactor. @@ -272,13 +272,13 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { mask := int64(math.MaxInt64) for { // Choose the index that covers most. - bestID, bestCount, bestTp, bestNumCols := -1, 0, colType, 0 + bestID, bestCount, bestTp, bestNumCols, bestMask := -1, 0, colType, 0, int64(0) for i, set := range nodes { if marked[i] { continue } - set.mask &= mask - bits := popCount(set.mask) + curMask := set.mask & mask + bits := popCount(curMask) // This set cannot cover any thing, just skip it. if bits == 0 { continue @@ -288,7 +288,7 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { // (2): The number of expression that it covers, the more the better. // (3): The number of columns that it contains, the less the better. if (bestTp == colType && set.Tp != colType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { - bestID, bestCount, bestTp, bestNumCols = i, bits, set.Tp, set.numCols + bestID, bestCount, bestTp, bestNumCols, bestMask = i, bits, set.Tp, set.numCols, curMask } } if bestCount == 0 { @@ -296,7 +296,7 @@ func getUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { } // update the mask, remove the bit that nodes[bestID].mask has. - mask &^= nodes[bestID].mask + mask &^= bestMask newBlocks = append(newBlocks, nodes[bestID]) marked[bestID] = true From 4d3cc206132d9bf5233e9b65ebb0f60bff8cba83 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 7 Jan 2019 13:13:05 +0800 Subject: [PATCH 18/23] remove useless err check --- planner/core/stats.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/planner/core/stats.go b/planner/core/stats.go index 76ed2048532a4..c9dac5b58c045 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -99,9 +99,6 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) (*property.Sta } if ds.ctx.GetSessionVars().OptimizerSelectivityLevel >= 1 && ds.stats.HistColl != nil { finalHist := ds.stats.HistColl.NewHistCollBySelectivity(ds.ctx.GetSessionVars().StmtCtx, nodes) - if err != nil { - log.Warnf("[stats-in-datasource]: An error happened: %v", err.Error()) - } return profile, finalHist } return profile.Scale(selectivity), nil From 13a6d772050c10e3e616c3756503d5a7b11ff876 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 8 Jan 2019 13:46:58 +0800 Subject: [PATCH 19/23] address comment --- statistics/histogram.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index f0614bb086359..f359ab4718c70 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -994,7 +994,7 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } newIdxHist, err := idxHist.newIndexBySelectivity(sc, node) if err != nil { - log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v", err) + log.Warnf("[Histogram-in-plan]: error happened when calculating row count: %v, failed to build histogram for index %v of table %v", err, idxHist.Info.Name, idxHist.Info.Table) continue } newColl.Indices[node.ID] = newIdxHist From cefba9677df888faf9e6102002fbeabde948bedb Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 11 Jan 2019 15:46:08 +0800 Subject: [PATCH 20/23] address comment --- statistics/histogram.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index f359ab4718c70..860c8a4d476fd 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -468,10 +468,7 @@ func (hg *Histogram) betweenRowCount(a, b types.Datum) float64 { } func (hg *Histogram) totalRowCount() float64 { - if hg.Len() == 0 { - return float64(hg.NullCount) - } - return float64(hg.Buckets[hg.Len()-1].Count + hg.NullCount) + return hg.notNullCount() + float64(hg.NullCount) } func (hg *Histogram) notNullCount() float64 { From 3a9aa6cf6f48f82c562aab99fa7353e60d9175e7 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 11 Jan 2019 16:09:42 +0800 Subject: [PATCH 21/23] fix merge error --- statistics/feedback.go | 8 ++++---- statistics/update.go | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/statistics/feedback.go b/statistics/feedback.go index d960c8ac6f9d0..aedf265db3186 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -294,7 +294,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } total := 0 sc := &stmtctx.StatementContext{TimeZone: time.UTC} - min, max := getMinValue(h.tp), getMaxValue(h.tp) + min, max := getMinValue(h.Tp), getMaxValue(h.Tp) for _, fb := range feedback.feedback { skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) if err != nil { @@ -1078,14 +1078,14 @@ func (q *QueryFeedback) dumpRangeFeedback(sc *stmtctx.StatementContext, h *Handl ran.LowVal[0].SetBytes(lower) ran.HighVal[0].SetBytes(upper) } else { - if !supportColumnType(q.hist.tp) { + if !supportColumnType(q.hist.Tp) { return nil } if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = getMinValue(q.hist.tp) + ran.LowVal[0] = getMinValue(q.hist.Tp) } if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = getMaxValue(q.hist.tp) + ran.HighVal[0] = getMaxValue(q.hist.Tp) } } ranges := q.hist.SplitRange(sc, []*ranger.Range{ran}, q.tp == indexType) diff --git a/statistics/update.go b/statistics/update.go index b65c4cdbb7734..bc1e03e739830 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -562,7 +562,7 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch } q := &QueryFeedback{} for _, row := range rows { - err1 := decodeFeedback(row.GetBytes(3), q, cms, mysql.HasUnsignedFlag(hist.tp.Flag)) + err1 := decodeFeedback(row.GetBytes(3), q, cms, mysql.HasUnsignedFlag(hist.Tp.Flag)) if err1 != nil { log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err)) } From 8006aeee6a432c16f2f4e330815a3ecc3161a488 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 14 Jan 2019 16:21:34 +0800 Subject: [PATCH 22/23] fix feedback error --- distsql/request_builder.go | 2 -- executor/table_reader.go | 4 ++++ planner/core/rule_column_pruning.go | 17 +++++++++++++++-- statistics/histogram.go | 1 + 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index dcf8db67c3c8d..4144d3dc20056 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -173,8 +173,6 @@ func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.Que if fb == nil || fb.Hist() == nil { return tableRangesToKVRangesWithoutSplit(tid, ranges) } - // EncodeInt don't need *statement.Context. - ranges = fb.Hist().SplitRange(nil, ranges, false) krs := make([]kv.KeyRange, 0, len(ranges)) feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { diff --git a/executor/table_reader.go b/executor/table_reader.go index af2202919e511..3b1396acca9af 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -78,6 +78,10 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { } e.resultHandler = &tableResultHandler{} + if e.feedback != nil && e.feedback.Hist() != nil { + // EncodeInt don't need *statement.Context. + e.ranges = e.feedback.Hist().SplitRange(nil, e.ranges, false) + } firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder) firstResult, err := e.buildResp(ctx, firstPartRanges) if err != nil { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index bf2d86a916575..170a8b0241502 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -17,6 +17,7 @@ import ( "fmt" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" ) @@ -155,7 +156,15 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column) { // PruneColumns implements LogicalPlan interface. func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column) { used := getUsedList(parentUsedCols, ds.schema) + var ( + handleCol *expression.Column + handleColInfo *model.ColumnInfo + ) for i := len(used) - 1; i >= 0; i-- { + if ds.tableInfo.PKIsHandle && mysql.HasPriKeyFlag(ds.Columns[i].Flag) { + handleCol = ds.schema.Columns[i] + handleColInfo = ds.Columns[i] + } if !used[i] { ds.schema.Columns = append(ds.schema.Columns[:i], ds.schema.Columns[i+1:]...) ds.Columns = append(ds.Columns[:i], ds.Columns[i+1:]...) @@ -169,8 +178,12 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column) { // For SQL like `select 1 from t`, tikv's response will be empty if no column is in schema. // So we'll force to push one if schema doesn't have any column. if ds.schema.Len() == 0 && !infoschema.IsMemoryDB(ds.DBName.L) { - ds.Columns = append(ds.Columns, model.NewExtraHandleColInfo()) - ds.schema.Append(ds.newExtraHandleSchemaCol()) + if handleCol == nil { + handleCol = ds.newExtraHandleSchemaCol() + handleColInfo = model.NewExtraHandleColInfo() + } + ds.Columns = append(ds.Columns, handleColInfo) + ds.schema.Append(handleCol) } } diff --git a/statistics/histogram.go b/statistics/histogram.go index 860c8a4d476fd..cfcc727cd6eab 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -541,6 +541,7 @@ func validRange(sc *stmtctx.StatementContext, ran *ranger.Range, encoded bool) b func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, ranges []*ranger.Range, encoded bool) []*ranger.Range { split := make([]*ranger.Range, 0, len(ranges)) for len(ranges) > 0 { + log.Warnf("histogram tp: %v, unsigned: %v", hg.Tp.Tp, mysql.HasUnsignedFlag(hg.Tp.Flag)) // Find the last bound that greater or equal to the LowVal. idx := hg.Bounds.UpperBound(0, &ranges[0].LowVal[0]) if !ranges[0].LowExclude && idx > 0 { From b6f57cf129b6d9b322365be9f363d63473e68bcc Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 14 Jan 2019 16:27:33 +0800 Subject: [PATCH 23/23] remove debug log --- statistics/histogram.go | 1 - 1 file changed, 1 deletion(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index cfcc727cd6eab..860c8a4d476fd 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -541,7 +541,6 @@ func validRange(sc *stmtctx.StatementContext, ran *ranger.Range, encoded bool) b func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, ranges []*ranger.Range, encoded bool) []*ranger.Range { split := make([]*ranger.Range, 0, len(ranges)) for len(ranges) > 0 { - log.Warnf("histogram tp: %v, unsigned: %v", hg.Tp.Tp, mysql.HasUnsignedFlag(hg.Tp.Flag)) // Find the last bound that greater or equal to the LowVal. idx := hg.Bounds.UpperBound(0, &ranges[0].LowVal[0]) if !ranges[0].LowExclude && idx > 0 {