From 54d8e8507a9bfa348c7694f08eb07a282334a47f Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 4 Jun 2019 14:44:45 +0800 Subject: [PATCH] =?UTF-8?q?*:=20fix=20bug=20when=20unsigned=20histogram=20?= =?UTF-8?q?meets=20signed=20ranges=20in=20feedback=20(#=E2=80=A6=20(#10695?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- distsql/request_builder.go | 5 +++- executor/executor_test.go | 16 ++++++++++++ executor/table_reader.go | 7 ++++- planner/core/logical_plans.go | 12 +++++---- statistics/feedback.go | 6 ++++- statistics/histogram.go | 48 +++++++++++++++++++++++++++++++---- statistics/update_test.go | 2 +- 7 files changed, 82 insertions(+), 14 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index da1bd1ffc0e57..3d17819a3f945 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -269,7 +269,10 @@ 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(sc, feedbackRanges) + feedbackRanges, ok := fb.Hist().SplitRange(sc, feedbackRanges, true) + if !ok { + fb.Invalidate() + } krs := make([]kv.KeyRange, 0, len(feedbackRanges)) for _, ran := range feedbackRanges { low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() diff --git a/executor/executor_test.go b/executor/executor_test.go index 35e3c41f8181a..134e4603a3dd3 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" @@ -3502,6 +3503,21 @@ func (s *testSuite) TestDoSubquery(c *C) { c.Assert(r, IsNil, Commentf("result of Do not empty")) } +func (s *testSuite) TestUnsignedFeedback(c *C) { + tk := testkit.NewTestKit(c, s.store) + oriProbability := statistics.FeedbackProbability + statistics.FeedbackProbability = 1.0 + defer func() { statistics.FeedbackProbability = oriProbability }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))") + tk.MustExec("insert into t values (1,1),(2,2)") + tk.MustExec("analyze table t") + tk.MustQuery("select count(distinct b) from t").Check(testkit.Rows("2")) + result := tk.MustQuery("explain analyze select count(distinct b) from t") + c.Assert(result.Rows()[2][3], Equals, "table:t, range:[0,+inf], keep order:false") +} + type testOOMSuite struct { store kv.Storage do *domain.Domain diff --git a/executor/table_reader.go b/executor/table_reader.go index 00bc3d87dc3a7..8eebc77527880 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -98,7 +98,12 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { e.resultHandler = &tableResultHandler{} // Split ranges here since the unsigned part and signed part will swap their position when encoding the range to kv ranges. if e.feedback != nil && e.feedback.Hist() != nil { - e.ranges = e.feedback.Hist().SplitRange(e.ctx.GetSessionVars().StmtCtx, e.ranges) + // EncodeInt don't need *statement.Context. + var ok bool + e.ranges, ok = e.feedback.Hist().SplitRange(nil, e.ranges, false) + if !ok { + e.feedback.Invalidate() + } } firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder, e.desc) firstResult, err := e.buildResp(ctx, firstPartRanges) diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 79488341600c1..520b8037b48d6 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -370,19 +370,21 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath) (bool, error) { path.tableFilters = ds.pushedDownConds var pkCol *expression.Column columnLen := len(ds.schema.Columns) - if columnLen > 0 && ds.schema.Columns[columnLen-1].ID == model.ExtraHandleID { - pkCol = ds.schema.Columns[columnLen-1] - } else if ds.tableInfo.PKIsHandle { + isUnsigned := false + if ds.tableInfo.PKIsHandle { if pkColInfo := ds.tableInfo.GetPkColInfo(); pkColInfo != nil { + isUnsigned = mysql.HasUnsignedFlag(pkColInfo.Flag) pkCol = expression.ColInfo2Col(ds.schema.Columns, pkColInfo) } + } else if columnLen > 0 && ds.schema.Columns[columnLen-1].ID == model.ExtraHandleID { + pkCol = ds.schema.Columns[columnLen-1] } if pkCol == nil { - path.ranges = ranger.FullIntRange(false) + path.ranges = ranger.FullIntRange(isUnsigned) return false, nil } - path.ranges = ranger.FullIntRange(mysql.HasUnsignedFlag(pkCol.RetType.Flag)) + path.ranges = ranger.FullIntRange(isUnsigned) if len(ds.pushedDownConds) == 0 { return false, nil } diff --git a/statistics/feedback.go b/statistics/feedback.go index c0af492ec20cb..f0ce12dedd602 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -1164,7 +1164,11 @@ func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCou ran.HighVal[0] = getMaxValue(q.hist.tp) } } - ranges := q.hist.SplitRange(sc, []*ranger.Range{ran}) + ranges, ok := q.Hist().SplitRange(sc, []*ranger.Range{ran}, q.tp == indexType) + if !ok { + logutil.Logger(context.Background()).Debug("type of histogram and ranges mismatch") + return nil + } counts := make([]float64, 0, len(ranges)) sum := 0.0 for i, r := range ranges { diff --git a/statistics/histogram.go b/statistics/histogram.go index 5be059fe3f622..9369f32b3761e 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -503,9 +503,9 @@ 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(sc *stmtctx.StatementContext, ran *ranger.Range) bool { +func validRange(sc *stmtctx.StatementContext, 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 { var err error @@ -527,10 +527,48 @@ func validRange(sc *stmtctx.StatementContext, ran *ranger.Range) bool { return bytes.Compare(low, high) < 0 } +func checkKind(vals []types.Datum, kind byte) bool { + if kind == types.KindString { + kind = types.KindBytes + } + for _, val := range vals { + valKind := val.Kind() + if valKind == types.KindNull || valKind == types.KindMinNotNull || valKind == types.KindMaxValue { + continue + } + if valKind == types.KindString { + valKind = types.KindBytes + } + if valKind != kind { + return false + } + // Only check the first non-null value. + break + } + return true +} + +func (hg *Histogram) typeMatch(ranges []*ranger.Range) bool { + kind := hg.GetLower(0).Kind() + for _, ran := range ranges { + if !checkKind(ran.LowVal, kind) || !checkKind(ran.HighVal, kind) { + return false + } + } + return true +} + // 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(sc *stmtctx.StatementContext, ranges []*ranger.Range) []*ranger.Range { +func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, oldRanges []*ranger.Range, encoded bool) ([]*ranger.Range, bool) { + if !hg.typeMatch(oldRanges) { + return oldRanges, false + } + ranges := make([]*ranger.Range, 0, len(oldRanges)) + for _, ran := range oldRanges { + ranges = append(ranges, ran.Clone()) + } split := make([]*ranger.Range, 0, len(ranges)) for len(ranges) > 0 { // Find the last bound that greater or equal to the LowVal. @@ -574,12 +612,12 @@ func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, ranges []*ranger.R HighExclude: false}) ranges[0].LowVal[0] = upper ranges[0].LowExclude = true - if !validRange(sc, ranges[0]) { + if !validRange(sc, ranges[0], encoded) { ranges = ranges[1:] } } } - return split + return split, true } func (hg *Histogram) bucketCount(idx int) int64 { diff --git a/statistics/update_test.go b/statistics/update_test.go index 816aef606d7f0..9e1d67efffbac 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -604,7 +604,7 @@ func (s *testStatsUpdateSuite) TestSplitRange(c *C) { HighExclude: t.exclude[i+1], }) } - ranges = h.SplitRange(nil, ranges) + ranges, _ = h.SplitRange(nil, ranges, false) var ranStrs []string for _, ran := range ranges { ranStrs = append(ranStrs, ran.String())