Skip to content

Commit

Permalink
stats: improve code coverage (#10606)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and zz-jason committed Jun 12, 2019
1 parent fbf58fc commit 3f4f3d0
Show file tree
Hide file tree
Showing 16 changed files with 191 additions and 231 deletions.
2 changes: 2 additions & 0 deletions statistics/analyze_jobs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,4 +31,6 @@ func (s *testStatisticsSuite) TestMoveToHistory(c *C) {
MoveToHistory(jobs[i])
}
c.Assert(len(GetAllAnalyzeJobs()), Equals, numMaxHistoryJobs)
ClearHistoryJobs()
c.Assert(len(GetAllAnalyzeJobs()), Equals, 0)
}
30 changes: 2 additions & 28 deletions statistics/cmsketch.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"bytes"
"fmt"
"math"
"reflect"
"sort"

"github.com/cznic/mathutil"
Expand Down Expand Up @@ -418,34 +419,7 @@ func (c *CMSketch) TotalCount() uint64 {

// Equal tests if two CM Sketch equal, it is only used for test.
func (c *CMSketch) Equal(rc *CMSketch) bool {
if c == nil || rc == nil {
return c == nil && rc == nil
}
if c.width != rc.width || c.depth != rc.depth || c.count != rc.count || c.defaultValue != rc.defaultValue {
return false
}
for i := range c.table {
for j := range c.table[i] {
if c.table[i][j] != rc.table[i][j] {
return false
}
}
}
if len(c.topN) != len(rc.topN) {
return false
}
for h1, topNData := range c.topN {
if len(topNData) != len(rc.topN[h1]) {
return false
}
for _, val := range topNData {
meta := rc.findTopNMeta(h1, val.h2, val.Data)
if meta == nil || meta.Count != val.Count {
return false
}
}
}
return true
return reflect.DeepEqual(c, rc)
}

// Copy makes a copy for current CMSketch.
Expand Down
59 changes: 57 additions & 2 deletions statistics/cmsketch_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,12 +49,13 @@ func prepareCMSWithTopN(d, w int32, vals []*types.Datum, n uint32, total uint64)
return cms, nil
}

func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64) (*CMSketch, map[int64]uint32, error) {
// buildCMSketchAndMapWithOffset builds cm sketch using zipf and the generated values starts from `offset`.
func buildCMSketchAndMapWithOffset(d, w int32, seed int64, total, imax uint64, s float64, offset int64) (*CMSketch, map[int64]uint32, error) {
cms := NewCMSketch(d, w)
mp := make(map[int64]uint32)
zipf := rand.NewZipf(rand.New(rand.NewSource(seed)), s, 1, imax)
for i := uint64(0); i < total; i++ {
val := types.NewIntDatum(int64(zipf.Uint64()))
val := types.NewIntDatum(int64(zipf.Uint64()) + offset)
err := cms.insert(&val)
if err != nil {
return nil, nil, errors.Trace(err)
Expand All @@ -64,6 +65,10 @@ func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64)
return cms, mp, nil
}

func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64) (*CMSketch, map[int64]uint32, error) {
return buildCMSketchAndMapWithOffset(d, w, seed, total, imax, s, 0)
}

func buildCMSketchTopNAndMap(d, w, n, sample int32, seed int64, total, imax uint64, s float64) (*CMSketch, map[int64]uint32, error) {
mp := make(map[int64]uint32)
zipf := rand.NewZipf(rand.New(rand.NewSource(seed)), s, 1, imax)
Expand Down Expand Up @@ -195,6 +200,56 @@ func (s *testStatisticsSuite) TestCMSketchTopN(c *C) {
}
}

func (s *testStatisticsSuite) TestMergeCMSketch4IncrementalAnalyze(c *C) {
tests := []struct {
zipfFactor float64
avgError uint64
}{
{
zipfFactor: 1.0000001,
avgError: 48,
},
{
zipfFactor: 1.1,
avgError: 48,
},
{
zipfFactor: 2,
avgError: 128,
},
{
zipfFactor: 5,
avgError: 256,
},
}
d, w := int32(5), int32(2048)
total, imax := uint64(100000), uint64(1000000)
for _, t := range tests {
lSketch, lMap, err := buildCMSketchAndMap(d, w, 0, total, imax, t.zipfFactor)
c.Check(err, IsNil)
avg, err := averageAbsoluteError(lSketch, lMap)
c.Assert(err, IsNil)
c.Check(avg, LessEqual, t.avgError)

rSketch, rMap, err := buildCMSketchAndMapWithOffset(d, w, 1, total, imax, t.zipfFactor, int64(imax))
c.Check(err, IsNil)
avg, err = averageAbsoluteError(rSketch, rMap)
c.Assert(err, IsNil)
c.Check(avg, LessEqual, t.avgError)

for key, val := range rMap {
lMap[key] += val
}
lSketch.MergeCMSketch4IncrementalAnalyze(rSketch)
avg, err = averageAbsoluteError(lSketch, lMap)
c.Assert(err, IsNil)
c.Check(avg, LessEqual, t.avgError)
width, depth := lSketch.GetWidthAndDepth()
c.Assert(width, Equals, int32(2048))
c.Assert(depth, Equals, int32(5))
}
}

func (s *testStatisticsSuite) TestCMSketchTopNUniqueData(c *C) {
d, w := int32(5), int32(2048)
total := uint64(1000000)
Expand Down
11 changes: 4 additions & 7 deletions statistics/estimate.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@ package statistics

import (
"math"

"github.com/cznic/mathutil"
)

// calculateEstimateNDV calculates the estimate ndv of a sampled data from a multisize with size total.
Expand Down Expand Up @@ -42,12 +44,7 @@ func calculateEstimateNDV(h *topNHelper, rowCount uint64) (ndv uint64, scaleRati
d := float64(sampleNDV)

ndv = uint64(math.Sqrt(N/n)*f1 + d - f1 + 0.5)

if ndv < sampleNDV {
ndv = sampleNDV
}
if ndv > rowCount {
ndv = rowCount
}
ndv = mathutil.MaxUint64(ndv, sampleNDV)
ndv = mathutil.MinUint64(ndv, rowCount)
return ndv, scaleRatio
}
48 changes: 5 additions & 43 deletions statistics/feedback.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,9 +366,7 @@ func (b *BucketFeedback) getBoundaries(num int) []types.Datum {
err := types.SortDatums(nil, vals)
if err != nil {
logutil.Logger(context.Background()).Debug("sort datums failed", zap.Error(err))
vals = vals[:0]
vals = append(vals, *b.lower, *b.upper)
return vals
return []types.Datum{*b.lower, *b.upper}
}
total, interval := 0, len(vals)/num
// Pick values per `interval`.
Expand Down Expand Up @@ -772,10 +770,7 @@ func EncodeFeedback(q *QueryFeedback) ([]byte, error) {
var buf bytes.Buffer
enc := gob.NewEncoder(&buf)
err = enc.Encode(pb)
if err != nil {
return nil, errors.Trace(err)
}
return buf.Bytes(), nil
return buf.Bytes(), errors.Trace(err)
}

func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch) {
Expand Down Expand Up @@ -865,41 +860,9 @@ func DecodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, ft *types.FieldTy
} else if len(pb.IntRanges) > 0 {
decodeFeedbackForPK(q, pb, mysql.HasUnsignedFlag(ft.Flag))
} else {
err := decodeFeedbackForColumn(q, pb, ft)
if err != nil {
return errors.Trace(err)
}
}
return nil
}

// Equal tests if two query feedback equal, it is only used in test.
func (q *QueryFeedback) Equal(rq *QueryFeedback) bool {
if len(q.Feedback) != len(rq.Feedback) {
return false
err = decodeFeedbackForColumn(q, pb, ft)
}
for i, fb := range q.Feedback {
rfb := rq.Feedback[i]
if fb.Count != rfb.Count {
return false
}
if fb.Lower.Kind() == types.KindInt64 {
if fb.Lower.GetInt64() != rfb.Lower.GetInt64() {
return false
}
if fb.Upper.GetInt64() != rfb.Upper.GetInt64() {
return false
}
} else {
if !bytes.Equal(fb.Lower.GetBytes(), rfb.Lower.GetBytes()) {
return false
}
if !bytes.Equal(fb.Upper.GetBytes(), rfb.Upper.GetBytes()) {
return false
}
}
}
return true
return err
}

// SplitFeedbackByQueryType splits the feedbacks into equality feedbacks and range feedbacks.
Expand Down Expand Up @@ -947,9 +910,8 @@ func SupportColumnType(ft *types.FieldType) bool {
mysql.TypeDouble, mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob,
mysql.TypeNewDecimal, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
return true
default:
return false
}
return false
}

// GetMaxValue returns the max value datum for each type.
Expand Down
31 changes: 31 additions & 0 deletions statistics/feedback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@
package statistics

import (
"bytes"

. "github.com/pingcap/check"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -256,3 +258,32 @@ func (s *testFeedbackSuite) TestFeedbackEncoding(c *C) {
q.Feedback = q.Feedback[:1]
c.Assert(q.Equal(rq), IsTrue)
}

// Equal tests if two query feedback equal, it is only used in test.
func (q *QueryFeedback) Equal(rq *QueryFeedback) bool {
if len(q.Feedback) != len(rq.Feedback) {
return false
}
for i, fb := range q.Feedback {
rfb := rq.Feedback[i]
if fb.Count != rfb.Count {
return false
}
if fb.Lower.Kind() == types.KindInt64 {
if fb.Lower.GetInt64() != rfb.Lower.GetInt64() {
return false
}
if fb.Upper.GetInt64() != rfb.Upper.GetInt64() {
return false
}
} else {
if !bytes.Equal(fb.Lower.GetBytes(), rfb.Lower.GetBytes()) {
return false
}
if !bytes.Equal(fb.Upper.GetBytes(), rfb.Upper.GetBytes()) {
return false
}
}
}
return true
}
45 changes: 20 additions & 25 deletions statistics/handle/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,23 +82,15 @@ func (h *Handle) insertTableStats2KV(info *model.TableInfo, physicalID int64) (e
return errors.Trace(err)
}
startTS := txn.StartTS()
_, err = exec.Execute(context.Background(), fmt.Sprintf("insert into mysql.stats_meta (version, table_id) values(%d, %d)", startTS, physicalID))
if err != nil {
return
}
sqls := make([]string, 0, 1+len(info.Columns)+len(info.Indices))
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_meta (version, table_id) values(%d, %d)", startTS, physicalID))
for _, col := range info.Columns {
_, err = exec.Execute(context.Background(), fmt.Sprintf("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version) values(%d, 0, %d, 0, %d)", physicalID, col.ID, startTS))
if err != nil {
return
}
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version) values(%d, 0, %d, 0, %d)", physicalID, col.ID, startTS))
}
for _, idx := range info.Indices {
_, err = exec.Execute(context.Background(), fmt.Sprintf("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version) values(%d, 1, %d, 0, %d)", physicalID, idx.ID, startTS))
if err != nil {
return
}
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version) values(%d, 1, %d, 0, %d)", physicalID, idx.ID, startTS))
}
return
return execSQLs(context.Background(), exec, sqls)
}

// insertColStats2KV insert a record to stats_histograms with distinct_count 1 and insert a bucket to stats_buckets with default value.
Expand Down Expand Up @@ -148,28 +140,21 @@ func (h *Handle) insertColStats2KV(physicalID int64, colInfo *model.ColumnInfo)
if err != nil {
return
}
sqls := make([]string, 0, 1)
if value.IsNull() {
// If the adding column has default value null, all the existing rows have null value on the newly added column.
_, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, null_count) values (%d, %d, 0, %d, 0, %d)", startTS, physicalID, colInfo.ID, count))
if err != nil {
return
}
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, null_count) values (%d, %d, 0, %d, 0, %d)", startTS, physicalID, colInfo.ID, count))
} else {
// If this stats exists, we insert histogram meta first, the distinct_count will always be one.
_, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", startTS, physicalID, colInfo.ID, int64(len(value.GetBytes()))*count))
if err != nil {
return
}
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", startTS, physicalID, colInfo.ID, int64(len(value.GetBytes()))*count))
value, err = value.ConvertTo(h.mu.ctx.GetSessionVars().StmtCtx, types.NewFieldType(mysql.TypeBlob))
if err != nil {
return
}
// There must be only one bucket for this new column and the value is the default value.
_, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_buckets (table_id, is_index, hist_id, bucket_id, repeats, count, lower_bound, upper_bound) values (%d, 0, %d, 0, %d, %d, X'%X', X'%X')", physicalID, colInfo.ID, count, count, value.GetBytes(), value.GetBytes()))
if err != nil {
return
}
sqls = append(sqls, fmt.Sprintf("insert into mysql.stats_buckets (table_id, is_index, hist_id, bucket_id, repeats, count, lower_bound, upper_bound) values (%d, 0, %d, 0, %d, %d, X'%X', X'%X')", physicalID, colInfo.ID, count, count, value.GetBytes(), value.GetBytes()))
}
return execSQLs(context.Background(), exec, sqls)
}
return
}
Expand All @@ -184,3 +169,13 @@ func finishTransaction(ctx context.Context, exec sqlexec.SQLExecutor, err error)
}
return errors.Trace(err)
}

func execSQLs(ctx context.Context, exec sqlexec.SQLExecutor, sqls []string) error {
for _, sql := range sqls {
_, err := exec.Execute(ctx, sql)
if err != nil {
return err
}
}
return nil
}
5 changes: 1 addition & 4 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,10 +168,7 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64,
}
}
err = h.SaveMetaToStorage(tbl.PhysicalID, tbl.Count, tbl.ModifyCount)
if err != nil {
return errors.Trace(err)
}
return nil
return err
}

// TableStatsFromJSON loads statistic from JSONTable and return the Table of statistic.
Expand Down
Loading

0 comments on commit 3f4f3d0

Please sign in to comment.