Skip to content

Commit

Permalink
statistics: set correlation when loading needed column histogram (pin…
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka committed Mar 14, 2019
1 parent fa00255 commit c5c3091
Show file tree
Hide file tree
Showing 4 changed files with 49 additions and 21 deletions.
2 changes: 1 addition & 1 deletion statistics/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ func (h *Handle) LoadNeededHistograms() error {
histogramNeededColumns.delete(col)
continue
}
hg, err := h.histogramFromStorage(col.tableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize)
hg, err := h.histogramFromStorage(col.tableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation)
if err != nil {
return errors.Trace(err)
}
Expand Down
23 changes: 22 additions & 1 deletion statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,14 +301,15 @@ func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error
return
}

func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64) (*Histogram, error) {
func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (*Histogram, error) {
selSQL := fmt.Sprintf("select count, repeats, lower_bound, upper_bound from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d order by bucket_id", tableID, isIndex, colID)
rows, fields, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL)
if err != nil {
return nil, errors.Trace(err)
}
bucketSize := len(rows)
hg := NewHistogram(colID, distinct, nullCount, ver, tp, bucketSize, totColSize)
hg.Correlation = corr
totalCount := int64(0)
for i := 0; i < bucketSize; i++ {
count := rows[i].GetInt64(0)
Expand Down Expand Up @@ -759,6 +760,21 @@ func (c *Column) String() string {
return c.Histogram.ToString(0)
}

var histogramNeededColumns = neededColumnMap{cols: map[tableColumnID]struct{}{}}

// IsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it
// as need histogram.
func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool {
if collPseudo && c.NotAccurate() {
return true
}
if c.NDV > 0 && c.Len() == 0 && sc != nil {
sc.SetHistogramsNotLoad()
histogramNeededColumns.insert(tableColumnID{tableID: c.PhysicalID, columnID: c.Info.ID})
}
return c.totalRowCount() == 0 || (c.NDV > 0 && c.Len() == 0)
}

func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, modifyCount int64) (float64, error) {
if val.IsNull() {
return float64(c.NullCount), nil
Expand Down Expand Up @@ -839,6 +855,11 @@ func (idx *Index) String() string {
return idx.Histogram.ToString(len(idx.Info.Columns))
}

// IsInvalid checks if this index is invalid.
func (idx *Index) IsInvalid(collPseudo bool) bool {
return (collPseudo && idx.NotAccurate()) || idx.totalRowCount() == 0
}

func (idx *Index) equalRowCount(sc *stmtctx.StatementContext, b []byte, modifyCount int64) float64 {
val := types.NewBytesDatum(b)
if idx.NDV > 0 && idx.outOfRange(val) {
Expand Down
22 changes: 3 additions & 19 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *Table, tableInfo *m
continue
}
if idx == nil || idx.LastUpdateVersion < histVer {
hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0)
hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -189,11 +189,10 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo *
break
}
if col == nil || col.LastUpdateVersion < histVer || loadAll {
hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize)
hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation)
if err != nil {
return errors.Trace(err)
}
hg.Correlation = correlation
cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -333,8 +332,6 @@ func (n *neededColumnMap) delete(col tableColumnID) {
n.m.Unlock()
}

var histogramNeededColumns = neededColumnMap{cols: map[tableColumnID]struct{}{}}

// RatioOfPseudoEstimate means if modifyCount / statsTblCount is greater than this ratio, we think the stats is invalid
// and use pseudo estimation.
var RatioOfPseudoEstimate = 0.7
Expand All @@ -347,19 +344,6 @@ func (t *Table) IsOutdated() bool {
return false
}

// IsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it
// as need histogram.
func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool {
if collPseudo && c.NotAccurate() {
return true
}
if c.NDV > 0 && c.Len() == 0 && sc != nil {
sc.SetHistogramsNotLoad()
histogramNeededColumns.insert(tableColumnID{tableID: c.PhysicalID, columnID: c.Info.ID})
}
return c.totalRowCount() == 0 || (c.NDV > 0 && c.Len() == 0)
}

// ColumnGreaterRowCount estimates the row count where the column greater than value.
func (t *Table) ColumnGreaterRowCount(sc *stmtctx.StatementContext, value types.Datum, colID int64) float64 {
c, ok := t.Columns[colID]
Expand Down Expand Up @@ -429,7 +413,7 @@ func (coll *HistColl) GetRowCountByColumnRanges(sc *stmtctx.StatementContext, co
// GetRowCountByIndexRanges estimates the row count by a slice of Range.
func (coll *HistColl) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idxID int64, indexRanges []*ranger.Range) (float64, error) {
idx := coll.Indices[idxID]
if idx == nil || coll.Pseudo && idx.NotAccurate() || idx.Len() == 0 {
if idx == nil || idx.IsInvalid(coll.Pseudo) {
colsLen := -1
if idx != nil && idx.Info.Unique {
colsLen = len(idx.Info.Columns)
Expand Down
23 changes: 23 additions & 0 deletions statistics/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1493,3 +1493,26 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) {
c.Assert(tbl.Columns[1].ToString(0), Equals, tests[i].hist)
}
}

func (s *testStatsSuite) TestLoadHistCorrelation(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)
h := s.do.StatsHandle()
origLease := h.Lease
h.Lease = time.Second
defer func() { h.Lease = origLease }()
testKit.MustExec("use test")
testKit.MustExec("create table t(c int)")
testKit.MustExec("insert into t values(1),(2),(3),(4),(5)")
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
testKit.MustExec("analyze table t")
h.Clear()
c.Assert(h.Update(s.do.InfoSchema()), IsNil)
result := testKit.MustQuery("show stats_histograms where Table_name = 't'")
c.Assert(len(result.Rows()), Equals, 0)
testKit.MustExec("explain select * from t where c = 1")
c.Assert(h.LoadNeededHistograms(), IsNil)
result = testKit.MustQuery("show stats_histograms where Table_name = 't'")
c.Assert(len(result.Rows()), Equals, 1)
c.Assert(result.Rows()[0][9], Equals, "1")
}

0 comments on commit c5c3091

Please sign in to comment.