Skip to content

Commit

Permalink
stats: fix histogram boundaries overflow error (#7883) (#7944)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and zz-jason committed Oct 18, 2018
1 parent 52d5ee2 commit 4021862
Show file tree
Hide file tree
Showing 6 changed files with 159 additions and 29 deletions.
104 changes: 84 additions & 20 deletions statistics/feedback.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,20 +243,79 @@ type BucketFeedback struct {
upper *types.Datum // The upper bound of the new bucket.
}

// outOfRange checks if the `val` is between `min` and `max`.
func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, error) {
result, err := val.CompareDatum(sc, min)
if err != nil {
return 0, err
}
if result < 0 {
return result, nil
}
result, err = val.CompareDatum(sc, max)
if err != nil {
return 0, err
}
if result > 0 {
return result, nil
}
return 0, nil
}

// adjustFeedbackBoundaries adjust the feedback boundaries according to the `min` and `max`.
// If the feedback has no intersection with `min` and `max`, we could just skip this feedback.
func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) {
result, err := outOfRange(sc, min, max, f.lower)
if err != nil {
return false, err
}
if result > 0 {
return true, nil
}
if result < 0 {
f.lower = min
}
result, err = outOfRange(sc, min, max, f.upper)
if err != nil {
return false, err
}
if result < 0 {
return true, nil
}
if result > 0 {
f.upper = max
}
return false, nil
}

// buildBucketFeedback build the feedback for each bucket from the histogram feedback.
func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) {
bktID2FB := make(map[int]*BucketFeedback)
if len(feedback.feedback) == 0 {
return bktID2FB, 0
}
total := 0
for _, ran := range feedback.feedback {
idx, _ := h.Bounds.LowerBound(0, ran.lower)
sc := &stmtctx.StatementContext{TimeZone: time.UTC}
kind := feedback.feedback[0].lower.Kind()
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 {
log.Debugf("adjust feedback boundaries failed, err: %v", errors.ErrorStack(err))
continue
}
if skip {
continue
}
idx, _ := h.Bounds.LowerBound(0, fb.lower)
bktIdx := 0
// The last bucket also stores the feedback that falls outside the upper bound.
if idx >= h.Bounds.NumRows()-2 {
bktIdx = h.Len() - 1
} else {
bktIdx = idx / 2
// Make sure that this feedback lies within the bucket.
if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, ran.upper) < 0 {
if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, fb.upper) < 0 {
continue
}
}
Expand All @@ -266,23 +325,23 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket
bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)}
bktID2FB[bktIdx] = bkt
}
bkt.feedback = append(bkt.feedback, ran)
bkt.feedback = append(bkt.feedback, fb)
// Update the bound if necessary.
res, err := bkt.lower.CompareDatum(nil, ran.lower)
res, err := bkt.lower.CompareDatum(nil, fb.lower)
if err != nil {
log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, ran.lower, errors.ErrorStack(err))
log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, fb.lower, errors.ErrorStack(err))
continue
}
if res > 0 {
bkt.lower = ran.lower
bkt.lower = fb.lower
}
res, err = bkt.upper.CompareDatum(nil, ran.upper)
res, err = bkt.upper.CompareDatum(nil, fb.upper)
if err != nil {
log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, ran.upper, errors.ErrorStack(err))
log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, fb.upper, errors.ErrorStack(err))
continue
}
if res < 0 {
bkt.upper = ran.upper
bkt.upper = fb.upper
}
}
return bktID2FB, total
Expand Down Expand Up @@ -528,7 +587,12 @@ func splitBuckets(h *Histogram, feedback *QueryFeedback) ([]bucket, []bool, int6
func UpdateHistogram(h *Histogram, feedback *QueryFeedback) *Histogram {
buckets, isNewBuckets, totalCount := splitBuckets(h, feedback)
buckets = mergeBuckets(buckets, isNewBuckets, float64(totalCount))
return buildNewHistogram(h, buckets)
hist := buildNewHistogram(h, buckets)
// Update the NDV of primary key column.
if feedback.tp == pkType {
hist.NDV = int64(hist.totalRowCount())
}
return hist
}

// UpdateCMSketch updates the CMSketch by feedback.
Expand Down Expand Up @@ -1077,13 +1141,13 @@ func supportColumnType(k byte) bool {
func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) {
switch k {
case types.KindInt64:
max.SetInt64(math.MaxInt64)
max.SetInt64(types.SignedUpperBound[ft.Tp])
case types.KindUint64:
max.SetUint64(math.MaxUint64)
max.SetUint64(types.UnsignedUpperBound[ft.Tp])
case types.KindFloat32:
max.SetFloat32(math.MaxFloat32)
max.SetFloat32(float32(types.GetMaxFloat(ft.Flen, ft.Decimal)))
case types.KindFloat64:
max.SetFloat64(math.MaxFloat64)
max.SetFloat64(types.GetMaxFloat(ft.Flen, ft.Decimal))
case types.KindString, types.KindBytes:
val := types.MaxValueDatum()
bytes, err := codec.EncodeKey(nil, nil, val)
Expand All @@ -1093,7 +1157,7 @@ func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) {
}
max.SetBytes(bytes)
case types.KindMysqlDecimal:
max.SetMysqlDecimal(types.NewMaxOrMinDec(false, mysql.MaxDecimalWidth, 0))
max.SetMysqlDecimal(types.NewMaxOrMinDec(false, ft.Flen, ft.Decimal))
case types.KindMysqlDuration:
max.SetMysqlDuration(types.Duration{Duration: math.MaxInt64})
case types.KindMysqlTime:
Expand All @@ -1109,13 +1173,13 @@ func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) {
func getMinValue(k byte, ft *types.FieldType) (min types.Datum) {
switch k {
case types.KindInt64:
min.SetInt64(math.MinInt64)
min.SetInt64(types.SignedLowerBound[ft.Tp])
case types.KindUint64:
min.SetUint64(0)
case types.KindFloat32:
min.SetFloat32(-math.MaxFloat32)
min.SetFloat32(float32(-types.GetMaxFloat(ft.Flen, ft.Decimal)))
case types.KindFloat64:
min.SetFloat64(-math.MaxFloat64)
min.SetFloat64(-types.GetMaxFloat(ft.Flen, ft.Decimal))
case types.KindString, types.KindBytes:
val := types.MinNotNullDatum()
bytes, err := codec.EncodeKey(nil, nil, val)
Expand All @@ -1125,7 +1189,7 @@ func getMinValue(k byte, ft *types.FieldType) (min types.Datum) {
}
min.SetBytes(bytes)
case types.KindMysqlDecimal:
min.SetMysqlDecimal(types.NewMaxOrMinDec(true, mysql.MaxDecimalWidth, 0))
min.SetMysqlDecimal(types.NewMaxOrMinDec(true, ft.Flen, ft.Decimal))
case types.KindMysqlDuration:
min.SetMysqlDuration(types.Duration{Duration: math.MinInt64})
case types.KindMysqlTime:
Expand Down
2 changes: 1 addition & 1 deletion statistics/feedback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) {
defaultBucketCount = 7
defer func() { defaultBucketCount = originBucketCount }()
c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals,
"column:0 ndv:0 totColSize:0\n"+
"column:0 ndv:10057 totColSize:0\n"+
"num: 10000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+
"num: 8 lower_bound: 2 upper_bound: 7 repeats: 0\n"+
"num: 11 lower_bound: 8 upper_bound: 19 repeats: 0\n"+
Expand Down
4 changes: 0 additions & 4 deletions statistics/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -558,10 +558,6 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch
log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err))
}
}
// Update the NDV of primary key column.
if table.Meta().PKIsHandle && q.tp == pkType {
hist.NDV = int64(hist.totalRowCount())
}
err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms)
return errors.Trace(err)
}
Expand Down
71 changes: 70 additions & 1 deletion statistics/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -594,7 +594,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) {
{
// test primary key feedback
sql: "select * from t where t.a <= 5",
hist: "column:1 ndv:3 totColSize:0\n" +
hist: "column:1 ndv:4 totColSize:0\n" +
"num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" +
"num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n" +
"num: 2 lower_bound: 3 upper_bound: 5 repeats: 0",
Expand Down Expand Up @@ -1118,3 +1118,72 @@ func (s *testStatsUpdateSuite) TestIndexQueryFeedback(c *C) {
c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, t.eqCount)
}
}

func (s *testStatsUpdateSuite) TestFeedbackRanges(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)
h := s.do.StatsHandle()
oriProbability := statistics.FeedbackProbability
oriNumber := statistics.MaxNumberOfRanges
defer func() {
statistics.FeedbackProbability = oriProbability
statistics.MaxNumberOfRanges = oriNumber
}()
statistics.FeedbackProbability = 1

testKit.MustExec("use test")
testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))")
for i := 0; i < 20; i++ {
testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
}
h.HandleDDLEvent(<-h.DDLEventCh())
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
testKit.MustExec("analyze table t with 3 buckets")
for i := 30; i < 40; i++ {
testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
}
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
tests := []struct {
sql string
hist string
colID int64
}{
{
sql: "select * from t where a <= 50 or (a > 130 and a < 140)",
hist: "column:1 ndv:30 totColSize:0\n" +
"num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 50 repeats: 0",
colID: 1,
},
{
sql: "select * from t where a >= 10",
hist: "column:1 ndv:30 totColSize:0\n" +
"num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 127 repeats: 0",
colID: 1,
},
{
sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))",
hist: "column:2 ndv:20 totColSize:20\n" +
"num: 7 lower_bound: -128 upper_bound: 6 repeats: 0\n" +
"num: 7 lower_bound: 7 upper_bound: 13 repeats: 1\n" +
"num: 6 lower_bound: 14 upper_bound: 19 repeats: 1",
colID: 2,
},
}
is := s.do.InfoSchema()
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
for i, t := range tests {
testKit.MustQuery(t.sql)
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
c.Assert(h.DumpStatsFeedbackToKV(), IsNil)
c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil)
c.Assert(err, IsNil)
h.Update(is)
tblInfo := table.Meta()
tbl := h.GetTableStats(tblInfo)
c.Assert(tbl.Columns[t.colID].ToString(0), Equals, tests[i].hist)
}
}
2 changes: 1 addition & 1 deletion types/etc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ func (s *testTypeEtcSuite) TestMaxFloat(c *C) {
}

for _, t := range tbl {
f := getMaxFloat(t.Flen, t.Decimal)
f := GetMaxFloat(t.Flen, t.Decimal)
c.Assert(f, Equals, t.Expect)
}
}
Expand Down
5 changes: 3 additions & 2 deletions types/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,8 @@ func Truncate(f float64, dec int) float64 {
return math.Trunc(tmp) / shift
}

func getMaxFloat(flen int, decimal int) float64 {
// GetMaxFloat gets the max float for given flen and decimal.
func GetMaxFloat(flen int, decimal int) float64 {
intPartLen := flen - decimal
f := math.Pow10(intPartLen)
f -= math.Pow10(-decimal)
Expand All @@ -74,7 +75,7 @@ func TruncateFloat(f float64, flen int, decimal int) (float64, error) {
return 0, ErrOverflow.GenWithStackByArgs("DOUBLE", "")
}

maxF := getMaxFloat(flen, decimal)
maxF := GetMaxFloat(flen, decimal)

if !math.IsInf(f, 0) {
f = Round(f, decimal)
Expand Down

0 comments on commit 4021862

Please sign in to comment.