Skip to content

Commit

Permalink
*: fix the estimation error on normal column when collation enabled (#…
Browse files Browse the repository at this point in the history
…18104) (#18311)

Co-authored-by: Yiding Cui <winoros@gmail.com>
Co-authored-by: Arenatlx <ailinsilence4@gmail.com>
  • Loading branch information
3 people authored Sep 8, 2020
1 parent a04ba17 commit ddc6c0d
Show file tree
Hide file tree
Showing 8 changed files with 117 additions and 9 deletions.
32 changes: 24 additions & 8 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -776,7 +777,23 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, mo
func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*ranger.Range, modifyCount int64, pkIsHandle bool) (float64, error) {
var rowCount float64
for _, rg := range ranges {
cmp, err := rg.LowVal[0].CompareDatum(sc, &rg.HighVal[0])
highVal := *rg.HighVal[0].Clone()
lowVal := *rg.LowVal[0].Clone()
if highVal.Kind() == types.KindString {
highVal.SetBytesAsString(collate.GetCollator(
highVal.Collation()).Key(highVal.GetString()),
highVal.Collation(),
uint32(highVal.Length()),
)
}
if lowVal.Kind() == types.KindString {
lowVal.SetBytesAsString(collate.GetCollator(
lowVal.Collation()).Key(lowVal.GetString()),
lowVal.Collation(),
uint32(lowVal.Length()),
)
}
cmp, err := lowVal.CompareDatum(sc, &highVal)
if err != nil {
return 0, errors.Trace(err)
}
Expand All @@ -789,15 +806,15 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range
continue
}
var cnt float64
cnt, err = c.equalRowCount(sc, rg.LowVal[0], modifyCount)
cnt, err = c.equalRowCount(sc, lowVal, modifyCount)
if err != nil {
return 0, errors.Trace(err)
}
rowCount += cnt
}
continue
}
rangeVals := enumRangeValues(rg.LowVal[0], rg.HighVal[0], rg.LowExclude, rg.HighExclude)
rangeVals := enumRangeValues(lowVal, highVal, rg.LowExclude, rg.HighExclude)
// The small range case.
if rangeVals != nil {
for _, val := range rangeVals {
Expand All @@ -810,26 +827,25 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range
continue
}
// The interval case.
lowVal, highVal := rg.LowVal[0], rg.HighVal[0]
cnt := c.BetweenRowCount(lowVal, highVal)
if (c.outOfRange(lowVal) && !lowVal.IsNull()) || c.outOfRange(highVal) {
cnt += outOfRangeEQSelectivity(outOfRangeBetweenRate, modifyCount, int64(c.TotalRowCount())) * c.TotalRowCount()
}
// `betweenRowCount` returns count for [l, h) range, we adjust cnt for boudaries here.
// Note that, `cnt` does not include null values, we need specially handle cases
// where null is the lower bound.
if rg.LowExclude && !rg.LowVal[0].IsNull() {
lowCnt, err := c.equalRowCount(sc, rg.LowVal[0], modifyCount)
if rg.LowExclude && !lowVal.IsNull() {
lowCnt, err := c.equalRowCount(sc, lowVal, modifyCount)
if err != nil {
return 0, errors.Trace(err)
}
cnt -= lowCnt
}
if !rg.LowExclude && rg.LowVal[0].IsNull() {
if !rg.LowExclude && lowVal.IsNull() {
cnt += float64(c.NullCount)
}
if !rg.HighExclude {
highCnt, err := c.equalRowCount(sc, rg.HighVal[0], modifyCount)
highCnt, err := c.equalRowCount(sc, highVal, modifyCount)
if err != nil {
return 0, errors.Trace(err)
}
Expand Down
15 changes: 15 additions & 0 deletions statistics/sample.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/fastrand"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -201,6 +202,8 @@ type SampleBuilder struct {
MaxFMSketchSize int64
CMSketchDepth int32
CMSketchWidth int32
Collators []collate.Collator
ColsFieldType []*types.FieldType
}

// CollectColumnStats collects sample from the result set using Reservoir Sampling algorithm,
Expand Down Expand Up @@ -245,6 +248,18 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder,
datums = datums[1:]
}
for i, val := range datums {
if s.Collators[i] != nil && !val.IsNull() {
decodedVal, err := tablecodec.DecodeColumnValue(val.GetBytes(), s.ColsFieldType[i], s.Sc.TimeZone)
if err != nil {
return nil, nil, err
}
decodedVal.SetBytesAsString(s.Collators[i].Key(decodedVal.GetString()), decodedVal.Collation(), uint32(decodedVal.Length()))
encodedKey, err := tablecodec.EncodeValue(s.Sc, nil, decodedVal)
if err != nil {
return nil, nil, err
}
val.SetBytes(encodedKey)
}
err = collectors[i].collect(s.Sc, val)
if err != nil {
return nil, nil, errors.Trace(err)
Expand Down
7 changes: 7 additions & 0 deletions statistics/sample_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/sqlexec"
)
Expand Down Expand Up @@ -65,6 +66,8 @@ func (s *testSampleSuite) TestCollectColumnStats(c *C) {
MaxFMSketchSize: 1000,
CMSketchWidth: 2048,
CMSketchDepth: 8,
Collators: make([]collate.Collator, 1),
ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)},
}
c.Assert(s.rs.Close(), IsNil)
collectors, pkBuilder, err := builder.CollectColumnStats()
Expand All @@ -86,6 +89,8 @@ func (s *testSampleSuite) TestMergeSampleCollector(c *C) {
MaxFMSketchSize: 1000,
CMSketchWidth: 2048,
CMSketchDepth: 8,
Collators: make([]collate.Collator, 2),
ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong), types.NewFieldType(mysql.TypeLonglong)},
}
c.Assert(s.rs.Close(), IsNil)
sc := &stmtctx.StatementContext{TimeZone: time.Local}
Expand All @@ -112,6 +117,8 @@ func (s *testSampleSuite) TestCollectorProtoConversion(c *C) {
MaxFMSketchSize: 1000,
CMSketchWidth: 2048,
CMSketchDepth: 8,
Collators: make([]collate.Collator, 2),
ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong), types.NewFieldType(mysql.TypeLonglong)},
}
c.Assert(s.rs.Close(), IsNil)
collectors, pkBuilder, err := builder.CollectColumnStats()
Expand Down
30 changes: 29 additions & 1 deletion statistics/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
Expand All @@ -49,7 +50,7 @@ import (

const eps = 1e-9

var _ = Suite(&testStatsSuite{})
var _ = SerialSuites(&testStatsSuite{})

type testStatsSuite struct {
store kv.Storage
Expand Down Expand Up @@ -639,3 +640,30 @@ func (s *testStatsSuite) TestSelectivityGreedyAlgo(c *C) {
c.Assert(len(usedSets), Equals, 1)
c.Assert(usedSets[0].ID, Equals, int64(1))
}

func (s *testStatsSuite) TestCollationColumnEstimate(c *C) {
defer cleanEnv(c, s.store, s.do)
tk := testkit.NewTestKit(c, s.store)
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a varchar(20) collate utf8mb4_general_ci)")
tk.MustExec("insert into t values('aaa'), ('bbb'), ('AAA'), ('BBB')")
h := s.do.StatsHandle()
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
tk.MustExec("analyze table t")
tk.MustExec("explain select * from t where a = 'aaa'")
c.Assert(h.LoadNeededHistograms(), IsNil)
var (
input []string
output [][]string
)
s.testData.GetTestCases(c, &input, &output)
for i := 0; i < len(input); i++ {
s.testData.OnRecord(func() {
output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(input[i]).Rows())
})
tk.MustQuery(input[i]).Check(testkit.Rows(output[i]...))
}
}
3 changes: 3 additions & 0 deletions statistics/statistics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -277,6 +278,8 @@ func (s *testStatisticsSuite) TestBuild(c *C) {
ColLen: 1,
MaxSampleSize: 1000,
MaxFMSketchSize: 1000,
Collators: make([]collate.Collator, 1),
ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)},
}
c.Assert(s.pk.Close(), IsNil)
collectors, _, err := builder.CollectColumnStats()
Expand Down
8 changes: 8 additions & 0 deletions statistics/testdata/stats_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -44,5 +44,13 @@
"cases": [
"explain select * from t where kid = 1"
]
},
{
"name": "TestCollationColumnEstimate",
"cases": [
"show stats_buckets",
"explain select * from t where a='aÄa'",
"explain select * from t where a > 'aÄa'"
]
}
]
19 changes: 19 additions & 0 deletions statistics/testdata/stats_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -87,5 +87,24 @@
"└─IndexRangeScan_5 7.00 cop[tikv] table:t, index:kid(kid, pid) range:[1,1], keep order:false"
]
]
},
{
"Name": "TestCollationColumnEstimate",
"Cases": [
[
"test t a 0 0 2 2 \u0000A\u0000A\u0000A \u0000A\u0000A\u0000A",
"test t a 0 1 4 2 \u0000B\u0000B\u0000B \u0000B\u0000B\u0000B"
],
[
"TableReader_7 2.00 root data:Selection_6",
"└─Selection_6 2.00 cop[tikv] eq(test.t.a, \"aÄa\")",
" └─TableFullScan_5 4.00 cop[tikv] table:t keep order:false"
],
[
"TableReader_7 2.00 root data:Selection_6",
"└─Selection_6 2.00 cop[tikv] gt(test.t.a, \"aÄa\")",
" └─TableFullScan_5 4.00 cop[tikv] table:t keep order:false"
]
]
}
]
12 changes: 12 additions & 0 deletions store/mockstore/mocktikv/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,8 +188,18 @@ func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe
numCols := len(columns)
if columns[0].GetPkHandle() {
pkID = columns[0].ColumnId
columns = columns[1:]
numCols--
}
collators := make([]collate.Collator, numCols)
fts := make([]*types.FieldType, numCols)
for i, col := range columns {
ft := fieldTypeFromPBColumn(col)
fts[i] = ft
if ft.EvalType() == types.ETString {
collators[i] = collate.GetCollator(ft.Collate)
}
}
colReq := analyzeReq.ColReq
builder := statistics.SampleBuilder{
Sc: sc,
Expand All @@ -198,6 +208,8 @@ func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe
MaxBucketSize: colReq.BucketSize,
MaxFMSketchSize: colReq.SketchSize,
MaxSampleSize: colReq.SampleSize,
Collators: collators,
ColsFieldType: fts,
}
if pkID != -1 {
builder.PkBuilder = statistics.NewSortedBuilder(sc, builder.MaxBucketSize, pkID, types.NewFieldType(mysql.TypeBlob))
Expand Down

0 comments on commit ddc6c0d

Please sign in to comment.