Skip to content

Commit

Permalink
*: fix bug when unsigned pk meets feedback ranges (#10307)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored and zz-jason committed Jun 4, 2019
1 parent c0cb290 commit 02526b6
Show file tree
Hide file tree
Showing 7 changed files with 43 additions and 19 deletions.
3 changes: 1 addition & 2 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,6 @@ 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)
krs := make([]kv.KeyRange, 0, len(ranges))
feedbackRanges := make([]*ranger.Range, 0, len(ranges))
for _, ran := range ranges {
Expand Down Expand Up @@ -270,7 +269,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(sc, feedbackRanges)
krs := make([]kv.KeyRange, 0, len(feedbackRanges))
for _, ran := range feedbackRanges {
low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes()
Expand Down
6 changes: 5 additions & 1 deletion executor/table_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/ranger"
tipb "github.com/pingcap/tipb/go-tipb"
"github.com/pingcap/tipb/go-tipb"
"golang.org/x/net/context"
)

Expand Down Expand Up @@ -96,6 +96,10 @@ 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)
}
firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder, e.desc)
firstResult, err := e.buildResp(ctx, firstPartRanges)
if err != nil {
Expand Down
10 changes: 5 additions & 5 deletions planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1130,14 +1130,14 @@ func (s *testPlanSuite) TestColumnPruning(c *C) {
{
sql: "select count(*) from t",
ans: map[int][]string{
1: {"test.t._tidb_rowid"},
1: {"test.t.a"},
},
},
{
sql: "select count(*) from t a join t b where a.a < 1",
ans: map[int][]string{
1: {"test.a.a"},
2: {"test.t._tidb_rowid"},
2: {"test.b.a"},
},
},
{
Expand Down Expand Up @@ -1178,7 +1178,7 @@ func (s *testPlanSuite) TestColumnPruning(c *C) {
{
sql: "select exists (select count(*) from t where b = k.a) from t k",
ans: map[int][]string{
1: {"test.t._tidb_rowid"},
1: {"test.k.a"},
},
},
{
Expand Down Expand Up @@ -1236,8 +1236,8 @@ func (s *testPlanSuite) TestColumnPruning(c *C) {
3: {"test.t21.a"},
5: {"t2.a"},
8: {"test.t01.a"},
10: {"test.t._tidb_rowid"},
12: {"test.t._tidb_rowid"},
10: {"test.t3.a"},
12: {"test.t4.a"},
},
},
}
Expand Down
17 changes: 15 additions & 2 deletions planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/mysql"
)

type columnPruner struct {
Expand Down Expand Up @@ -165,7 +166,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:]...)
Expand All @@ -179,8 +188,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)
}
}

Expand Down
4 changes: 2 additions & 2 deletions statistics/feedback.go
Original file line number Diff line number Diff line change
Expand Up @@ -1141,8 +1141,8 @@ func dumpFeedbackForIndex(h *Handle, q *QueryFeedback, t *Table) error {

func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCount float64) error {
lowIsNull := ran.LowVal[0].IsNull()
sc := &stmtctx.StatementContext{TimeZone: time.UTC}
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)
Expand All @@ -1164,7 +1164,7 @@ func (q *QueryFeedback) dumpRangeFeedback(h *Handle, ran *ranger.Range, rangeCou
ran.HighVal[0] = getMaxValue(q.hist.tp)
}
}
ranges := q.hist.SplitRange([]*ranger.Range{ran})
ranges := q.hist.SplitRange(sc, []*ranger.Range{ran})
counts := make([]float64, 0, len(ranges))
sum := 0.0
for i, r := range ranges {
Expand Down
16 changes: 12 additions & 4 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,12 +503,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(sc *stmtctx.StatementContext, ran *ranger.Range) bool {
var low, high []byte
if ran.LowVal[0].Kind() == types.KindBytes {
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(sc, nil, ran.LowVal[0])
if err != nil {
return false
}
high, err = codec.EncodeKey(sc, nil, ran.HighVal[0])
if err != nil {
return false
}
}
if ran.LowExclude {
low = kv.Key(low).PrefixNext()
Expand All @@ -522,7 +530,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(sc *stmtctx.StatementContext, ranges []*ranger.Range) []*ranger.Range {
split := make([]*ranger.Range, 0, len(ranges))
for len(ranges) > 0 {
// Find the last bound that greater or equal to the LowVal.
Expand Down Expand Up @@ -566,7 +574,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(sc, ranges[0]) {
ranges = ranges[1:]
}
}
Expand Down
6 changes: 3 additions & 3 deletions statistics/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,7 +604,7 @@ func (s *testStatsUpdateSuite) TestSplitRange(c *C) {
HighExclude: t.exclude[i+1],
})
}
ranges = h.SplitRange(ranges)
ranges = h.SplitRange(nil, ranges)
var ranStrs []string
for _, ran := range ranges {
ranStrs = append(ranStrs, ran.String())
Expand Down Expand Up @@ -1344,7 +1344,7 @@ func (s *testStatsUpdateSuite) TestUnsignedFeedbackRanges(c *C) {
statistics.FeedbackProbability = 1

testKit.MustExec("use test")
testKit.MustExec("create table t (a tinyint unsigned, primary key(a))")
testKit.MustExec("create table t (a bigint unsigned, primary key(a))")
for i := 0; i < 20; i++ {
testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i))
}
Expand All @@ -1371,7 +1371,7 @@ func (s *testStatsUpdateSuite) TestUnsignedFeedbackRanges(c *C) {
hist: "column:1 ndv:30 totColSize:0\n" +
"num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 255 repeats: 0",
"num: 14 lower_bound: 16 upper_bound: 18446744073709551615 repeats: 0",
},
}
is := s.do.InfoSchema()
Expand Down

0 comments on commit 02526b6

Please sign in to comment.