From 2fd3b0894f57275767f727212f6b8ef96718d1be Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 14 Dec 2021 01:20:35 +0800 Subject: [PATCH] done --- distsql/request_builder.go | 9 ++-- distsql/request_builder_test.go | 47 +++++++++++++----- executor/analyze.go | 4 +- executor/builder.go | 2 + executor/executor_pkg_test.go | 3 +- planner/core/exhaust_physical_plans.go | 13 +++-- planner/core/find_best_task.go | 4 +- planner/util/path_test.go | 16 +++--- statistics/feedback.go | 1 + statistics/handle/handle_test.go | 1 + statistics/handle/update.go | 11 +++-- statistics/handle/update_test.go | 2 + statistics/histogram.go | 10 ++-- statistics/histogram_test.go | 29 +++++------ statistics/selectivity_test.go | 6 ++- statistics/statistics_test.go | 20 +++++--- statistics/table.go | 7 ++- util/collate/collate.go | 15 +++++- util/ranger/points.go | 8 +-- util/ranger/ranger.go | 19 +++++--- util/ranger/types.go | 7 ++- util/ranger/types_test.go | 67 +++++++++++++++++--------- 22 files changed, 201 insertions(+), 100 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 44ff8d237f413..a2c67992868a9 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -382,7 +383,7 @@ func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics // since we need to guarantee each range falls inside the exactly one bucket, `PrefixNext` will make the // high value greater than upper bound, so we store the range here. r := &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}} + HighVal: []types.Datum{types.NewBytesDatum(high)}, Collators: collate.GetBinaryCollatorSlice(1)} feedbackRanges = append(feedbackRanges, r) if !ran.HighExclude { @@ -465,6 +466,7 @@ func SplitRangesAcrossInt64Boundary(ranges []*ranger.Range, keepOrder bool, desc LowVal: ranges[idx].LowVal, LowExclude: ranges[idx].LowExclude, HighVal: []types.Datum{types.NewUintDatum(math.MaxInt64)}, + Collators: ranges[idx].Collators, }) } if !(ranges[idx].HighVal[0].GetUint64() == math.MaxInt64+1 && ranges[idx].HighExclude) { @@ -472,6 +474,7 @@ func SplitRangesAcrossInt64Boundary(ranges []*ranger.Range, keepOrder bool, desc LowVal: []types.Datum{types.NewUintDatum(math.MaxInt64 + 1)}, HighVal: ranges[idx].HighVal, HighExclude: ranges[idx].HighExclude, + Collators: ranges[idx].Collators, }) } if idx < len(ranges) { @@ -585,7 +588,7 @@ func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementCont return nil, err } feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true}) + HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true, Collators: collate.GetBinaryCollatorSlice(1)}) } feedbackRanges, ok := fb.Hist.SplitRange(sc, feedbackRanges, true) if !ok { @@ -624,7 +627,7 @@ func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ra return nil, err } rans = append(rans, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true}) + HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true, Collators: collate.GetBinaryCollatorSlice(1)}) } krs := make([]kv.KeyRange, 0, len(rans)) for _, ran := range rans { diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 1c0c99523344b..81ea73513cfa1 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -26,6 +26,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/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -72,29 +73,34 @@ func TestTableHandlesToKVRanges(t *testing.T) { func TestTableRangesToKVRanges(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(2)}, HighVal: []types.Datum{types.NewIntDatum(4)}, LowExclude: true, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(4)}, HighVal: []types.Datum{types.NewIntDatum(19)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(19)}, HighVal: []types.Datum{types.NewIntDatum(32)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(34)}, HighVal: []types.Datum{types.NewIntDatum(34)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, } @@ -129,29 +135,34 @@ func TestTableRangesToKVRanges(t *testing.T) { func TestIndexRangesToKVRanges(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(2)}, HighVal: []types.Datum{types.NewIntDatum(4)}, LowExclude: true, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(4)}, HighVal: []types.Datum{types.NewIntDatum(19)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(19)}, HighVal: []types.Datum{types.NewIntDatum(32)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(34)}, HighVal: []types.Datum{types.NewIntDatum(34)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, } @@ -188,29 +199,34 @@ func TestIndexRangesToKVRanges(t *testing.T) { func TestRequestBuilder1(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(2)}, HighVal: []types.Datum{types.NewIntDatum(4)}, LowExclude: true, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(4)}, HighVal: []types.Datum{types.NewIntDatum(19)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(19)}, HighVal: []types.Datum{types.NewIntDatum(32)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(34)}, HighVal: []types.Datum{types.NewIntDatum(34)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, } @@ -265,29 +281,34 @@ func TestRequestBuilder1(t *testing.T) { func TestRequestBuilder2(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(2)}, HighVal: []types.Datum{types.NewIntDatum(4)}, LowExclude: true, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(4)}, HighVal: []types.Datum{types.NewIntDatum(19)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(19)}, HighVal: []types.Datum{types.NewIntDatum(32)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, { LowVal: []types.Datum{types.NewIntDatum(34)}, HighVal: []types.Datum{types.NewIntDatum(34)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, } @@ -573,8 +594,9 @@ func TestRequestBuilder8(t *testing.T) { func TestTableRangesToKVRangesWithFbs(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(4)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(4)}, + Collators: collate.GetBinaryCollatorSlice(1), }, } fb := newTestFb() @@ -594,8 +616,9 @@ func TestTableRangesToKVRangesWithFbs(t *testing.T) { func TestIndexRangesToKVRangesWithFbs(t *testing.T) { ranges := []*ranger.Range{ { - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(4)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(4)}, + Collators: collate.GetBinaryCollatorSlice(1), }, } fb := newTestFb() diff --git a/executor/analyze.go b/executor/analyze.go index 9414b2fbade1e..372108764babd 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -2117,7 +2117,7 @@ func analyzeIndexIncremental(idxExec *analyzeIndexIncrementalExec) *statistics.A if err != nil { return &statistics.AnalyzeResults{Err: err, Job: idxExec.job} } - ran := ranger.Range{LowVal: values, HighVal: []types.Datum{types.MaxValueDatum()}} + ran := ranger.Range{LowVal: values, HighVal: []types.Datum{types.MaxValueDatum()}, Collators: collate.GetBinaryCollatorSlice(1)} hist, cms, fms, topN, err := idxExec.buildStats([]*ranger.Range{&ran}, false) if err != nil { return &statistics.AnalyzeResults{Err: err, Job: idxExec.job} @@ -2172,7 +2172,7 @@ func analyzePKIncremental(colExec *analyzePKIncrementalExec) *statistics.Analyze maxVal = types.NewIntDatum(math.MaxInt64) } startPos := *colExec.oldHist.GetUpper(colExec.oldHist.Len() - 1) - ran := ranger.Range{LowVal: []types.Datum{startPos}, LowExclude: true, HighVal: []types.Datum{maxVal}} + ran := ranger.Range{LowVal: []types.Datum{startPos}, LowExclude: true, HighVal: []types.Datum{maxVal}, Collators: collate.GetBinaryCollatorSlice(1)} hists, _, _, _, _, err := colExec.buildStats([]*ranger.Range{&ran}, false) if err != nil { return &statistics.AnalyzeResults{Err: err, Job: colExec.job} diff --git a/executor/builder.go b/executor/builder.go index 109cc49915d58..1525bc931fd0e 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4076,6 +4076,7 @@ func buildRangesForIndexJoin(ctx sessionctx.Context, lookUpContents []*indexJoin ran.HighVal[lastPos] = nextColRan.HighVal[0] ran.LowExclude = nextColRan.LowExclude ran.HighExclude = nextColRan.HighExclude + ran.Collators = nextColRan.Collators tmpDatumRanges = append(tmpDatumRanges, ran.Clone()) } } @@ -4127,6 +4128,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l ran.HighVal[lastPos] = nextColRan.HighVal[0] ran.LowExclude = nextColRan.LowExclude ran.HighExclude = nextColRan.HighExclude + ran.Collators = nextColRan.Collators tmpDatumRanges = append(tmpDatumRanges, ran.Clone()) } } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 1b449f5fed6a6..51453e177b568 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" @@ -212,7 +213,7 @@ func generateIndexRange(vals ...int64) *ranger.Range { lowDatums := generateDatumSlice(vals...) highDatums := make([]types.Datum, len(vals)) copy(highDatums, lowDatums) - return &ranger.Range{LowVal: lowDatums, HighVal: highDatums} + return &ranger.Range{LowVal: lowDatums, HighVal: highDatums, Collators: collate.GetBinaryCollatorSlice(len(lowDatums))} } func generateDatumSlice(vals ...int64) []types.Datum { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index fb6a02d34b484..db9c16e722d78 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1538,6 +1538,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn HighVal: make([]types.Datum, pointLength, pointLength+1), LowExclude: colRan.LowExclude, HighExclude: colRan.HighExclude, + Collators: make([]collate.Collator, pointLength, pointLength+1), } ran.LowVal = append(ran.LowVal, colRan.LowVal[0]) ran.HighVal = append(ran.HighVal, colRan.HighVal[0]) @@ -1546,13 +1547,15 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn } else if haveExtraCol { // Reserve a position for the last col. ranges = append(ranges, &ranger.Range{ - LowVal: make([]types.Datum, pointLength+1), - HighVal: make([]types.Datum, pointLength+1), + LowVal: make([]types.Datum, pointLength+1), + HighVal: make([]types.Datum, pointLength+1), + Collators: make([]collate.Collator, pointLength+1), }) } else { ranges = append(ranges, &ranger.Range{ - LowVal: make([]types.Datum, pointLength), - HighVal: make([]types.Datum, pointLength), + LowVal: make([]types.Datum, pointLength), + HighVal: make([]types.Datum, pointLength), + Collators: make([]collate.Collator, pointLength), }) } sc := ijHelper.join.ctx.GetSessionVars().StmtCtx @@ -1575,6 +1578,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn for _, ran := range ranges { ran.LowVal[i] = oneColumnRan[0].LowVal[0] ran.HighVal[i] = oneColumnRan[0].HighVal[0] + ran.Collators[i] = oneColumnRan[0].Collators[0] } curRangeLen := len(ranges) for ranIdx := 1; ranIdx < len(oneColumnRan); ranIdx++ { @@ -1583,6 +1587,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn newRange := ranges[oldRangeIdx].Clone() newRange.LowVal[i] = oneColumnRan[ranIdx].LowVal[0] newRange.HighVal[i] = oneColumnRan[ranIdx].HighVal[0] + newRange.Collators[i] = oneColumnRan[0].Collators[0] newRanges = append(newRanges, newRange) } if sc.MemTracker != nil && len(newRanges) != 0 { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 59a182f9f8e7c..166d3adc298b3 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1520,7 +1520,7 @@ func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, if i < 0 { return nil, 0, true } - convertedRanges = []*ranger.Range{{LowVal: ranges[i].HighVal, HighVal: []types.Datum{types.MaxValueDatum()}, LowExclude: !ranges[i].HighExclude}} + convertedRanges = []*ranger.Range{{LowVal: ranges[i].HighVal, HighVal: []types.Datum{types.MaxValueDatum()}, LowExclude: !ranges[i].HighExclude, Collators: ranges[i].Collators}} } else { for i = 0; i < len(ranges); i++ { if count+rangeCounts[i] >= expectedCnt { @@ -1531,7 +1531,7 @@ func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, if i == len(ranges) { return nil, 0, true } - convertedRanges = []*ranger.Range{{LowVal: []types.Datum{{}}, HighVal: ranges[i].LowVal, HighExclude: !ranges[i].LowExclude}} + convertedRanges = []*ranger.Range{{LowVal: []types.Datum{{}}, HighVal: ranges[i].LowVal, HighExclude: !ranges[i].LowExclude, Collators: ranges[i].Collators}} } return convertedRanges, count, false } diff --git a/planner/util/path_test.go b/planner/util/path_test.go index c9053dcd4d114..9268ad81f6a23 100644 --- a/planner/util/path_test.go +++ b/planner/util/path_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) @@ -81,16 +82,19 @@ func TestOnlyPointRange(t *testing.T) { nullDatum := types.MinNotNullDatum() nullDatum.SetNull() nullPointRange := ranger.Range{ - LowVal: []types.Datum{*nullDatum.Clone()}, - HighVal: []types.Datum{*nullDatum.Clone()}, + LowVal: []types.Datum{*nullDatum.Clone()}, + HighVal: []types.Datum{*nullDatum.Clone()}, + Collators: collate.GetBinaryCollatorSlice(1), } onePointRange := ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(1)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(1)}, + Collators: collate.GetBinaryCollatorSlice(1), } one2TwoRange := ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), } intHandlePath := &util.AccessPath{IsIntHandlePath: true} diff --git a/statistics/feedback.go b/statistics/feedback.go index 12e986f95bff5..670d7a0509a17 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -236,6 +236,7 @@ func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { LowVal: lowVal, HighVal: highVal, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(len(lowVal)), })) } return ranges, nil diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 70ec989f7bca6..924ff178aee73 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -290,6 +290,7 @@ func (s *testStatsSuite) TestColumnIDs(c *C) { HighVal: []types.Datum{types.NewIntDatum(2)}, LowExclude: false, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), } count, err := statsTbl.GetRowCountByColumnRanges(sctx, tableInfo.Columns[0].ID, []*ranger.Range{ran}) c.Assert(err, IsNil) diff --git a/statistics/handle/update.go b/statistics/handle/update.go index a36f12bbdd7d2..73fb91c4d8979 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -39,6 +39,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/tidb/util/sqlexec" @@ -1176,8 +1177,9 @@ func logForIndex(prefix string, t *statistics.Table, idx *statistics.Index, rang } equalityCount := idx.QueryBytes(bytes) rang := ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + Collators: collate.GetBinaryCollatorSlice(1), } colName := idx.Info.Columns[rangePosition].Name.L // prefer index stats over column stats @@ -1401,8 +1403,9 @@ func (h *Handle) DumpFeedbackForIndex(q *statistics.QueryFeedback, t *statistics } equalityCount := float64(idx.QueryBytes(bytes)) * idx.GetIncreaseFactor(t.Count) rang := &ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + Collators: collate.GetBinaryCollatorSlice(1), } colName := idx.Info.Columns[rangePosition].Name.L var rangeCount float64 diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 3d41f92701593..23a061cd96f6a 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/statistics/handle" "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" @@ -938,6 +939,7 @@ func (s *testStatsSuite) TestSplitRange(c *C) { LowExclude: t.exclude[i], HighVal: []types.Datum{types.NewIntDatum(t.points[i+1])}, HighExclude: t.exclude[i+1], + Collators: collate.GetBinaryCollatorSlice(1), }) } ranges, _ = h.SplitRange(nil, ranges, false) diff --git a/statistics/histogram.go b/statistics/histogram.go index 5e1788da7a1ac..cd053ec070997 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -690,7 +690,9 @@ func (hg *Histogram) SplitRange(sc *stmtctx.StatementContext, oldRanges []*range LowExclude: ranges[0].LowExclude, LowVal: []types.Datum{ranges[0].LowVal[0]}, HighVal: []types.Datum{lower}, - HighExclude: true} + HighExclude: true, + Collators: ranges[0].Collators, + } if validRange(sc, newRange, encoded) { split = append(split, newRange) } @@ -1414,8 +1416,9 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang func (idx *Index) expBackoffEstimation(sctx sessionctx.Context, coll *HistColl, indexRange *ranger.Range) (float64, bool, error) { tmpRan := []*ranger.Range{ { - LowVal: make([]types.Datum, 1), - HighVal: make([]types.Datum, 1), + LowVal: make([]types.Datum, 1), + HighVal: make([]types.Datum, 1), + Collators: make([]collate.Collator, 1), }, } colsIDs := coll.Idx2ColumnIDs[idx.ID] @@ -1428,6 +1431,7 @@ func (idx *Index) expBackoffEstimation(sctx sessionctx.Context, coll *HistColl, for i := 0; i < len(indexRange.LowVal); i++ { tmpRan[0].LowVal[0] = indexRange.LowVal[i] tmpRan[0].HighVal[0] = indexRange.HighVal[i] + tmpRan[0].Collators[0] = indexRange.Collators[0] if i == len(indexRange.LowVal)-1 { tmpRan[0].LowExclude = indexRange.LowExclude tmpRan[0].HighExclude = indexRange.HighExclude diff --git a/statistics/histogram_test.go b/statistics/histogram_test.go index 15e4d696de1ee..ce95ddfa6fcdd 100644 --- a/statistics/histogram_test.go +++ b/statistics/histogram_test.go @@ -23,6 +23,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/mock" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" @@ -46,12 +47,12 @@ func TestNewHistogramBySelectivity(t *testing.T) { } coll.Columns[1] = intCol node := &StatsNode{ID: 1, Tp: PkType, Selectivity: 0.56} - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums(2)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(5), HighVal: types.MakeDatums(6)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(8), HighVal: types.MakeDatums(10)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(13), HighVal: types.MakeDatums(13)}) - node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(25), HighVal: []types.Datum{types.MaxValueDatum()}}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil), Collators: collate.GetBinaryCollatorSlice(1)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums(2), Collators: collate.GetBinaryCollatorSlice(1)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(5), HighVal: types.MakeDatums(6), Collators: collate.GetBinaryCollatorSlice(1)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(8), HighVal: types.MakeDatums(10), Collators: collate.GetBinaryCollatorSlice(1)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(13), HighVal: types.MakeDatums(13), Collators: collate.GetBinaryCollatorSlice(1)}) + node.Ranges = append(node.Ranges, &ranger.Range{LowVal: types.MakeDatums(25), HighVal: []types.Datum{types.MaxValueDatum()}, Collators: collate.GetBinaryCollatorSlice(1)}) intColResult := `column:1 ndv:16 totColSize:0 num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 ndv: 0 num: 11 lower_bound: 6 upper_bound: 8 repeats: 0 ndv: 0 @@ -79,12 +80,12 @@ num: 30 lower_bound: 27 upper_bound: 29 repeats: 0 ndv: 0` stringCol.PreCalculateScalar() coll.Columns[2] = stringCol node2 := &StatsNode{ID: 2, Tp: ColType, Selectivity: 0.6} - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil)}) - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa")}) - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa")}) - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc")}) - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff")}) - node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums(nil), HighVal: types.MakeDatums(nil), Collators: collate.GetBinaryCollatorSlice(1)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: types.MakeDatums("aaa"), Collators: collate.GetBinaryCollatorSlice(1)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("aaaaaaaaaaa"), HighVal: types.MakeDatums("aaaaaaaaaaaaaa"), Collators: collate.GetBinaryCollatorSlice(1)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("bbb"), HighVal: types.MakeDatums("cccc"), Collators: collate.GetBinaryCollatorSlice(1)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ddd"), HighVal: types.MakeDatums("fff"), Collators: collate.GetBinaryCollatorSlice(1)}) + node2.Ranges = append(node2.Ranges, &ranger.Range{LowVal: types.MakeDatums("ggg"), HighVal: []types.Datum{types.MaxValueDatum()}, Collators: collate.GetBinaryCollatorSlice(1)}) stringColResult := `column:2 ndv:9 totColSize:0 num: 60 lower_bound: a upper_bound: aaaabbbb repeats: 0 ndv: 0 num: 52 lower_bound: bbbb upper_bound: fdsfdsfds repeats: 0 ndv: 0 @@ -110,8 +111,8 @@ num: 60 lower_bound: ssssssu upper_bound: yyyyy repeats: 0 ndv: 0` } idx.PreCalculateScalar() node3 := &StatsNode{ID: 0, Tp: IndexType, Selectivity: 0.47} - node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3)}) - node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(10), HighVal: types.MakeDatums(13)}) + node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(2), HighVal: types.MakeDatums(3), Collators: collate.GetBinaryCollatorSlice(1)}) + node3.Ranges = append(node3.Ranges, &ranger.Range{LowVal: types.MakeDatums(10), HighVal: types.MakeDatums(13), Collators: collate.GetBinaryCollatorSlice(1)}) idxResult := `index:0 ndv:7 num: 30 lower_bound: 0 upper_bound: 2 repeats: 10 ndv: 0 diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 3a2e73777f54d..e4b283bcf7232 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/testkit" "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/stretchr/testify/require" ) @@ -123,8 +124,9 @@ func prepareSelectivity(testKit *testkit.TestKit, dom *domain.Domain) (*statisti func getRange(start, end int64) []*ranger.Range { ran := &ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(start)}, - HighVal: []types.Datum{types.NewIntDatum(end)}, + LowVal: []types.Datum{types.NewIntDatum(start)}, + HighVal: []types.Datum{types.NewIntDatum(end)}, + Collators: collate.GetBinaryCollatorSlice(1), } return []*ranger.Range{ran} } diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index c99802bd0314e..9590d972cd004 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -254,8 +254,9 @@ func SubTestColumnRange() func(*testing.T) { }, } ran := []*ranger.Range{{ - LowVal: []types.Datum{{}}, - HighVal: []types.Datum{types.MaxValueDatum()}, + LowVal: []types.Datum{{}}, + HighVal: []types.Datum{types.MaxValueDatum()}, + Collators: collate.GetBinaryCollatorSlice(1), }} count, err := tbl.GetRowCountByColumnRanges(ctx, 0, ran) require.NoError(t, err) @@ -325,8 +326,9 @@ func SubTestIntColumnRanges() func(*testing.T) { }, } ran := []*ranger.Range{{ - LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, - HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, + LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, + HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, + Collators: collate.GetBinaryCollatorSlice(1), }} count, err := tbl.GetRowCountByIntColumnRanges(ctx, 0, ran) require.NoError(t, err) @@ -348,8 +350,9 @@ func SubTestIntColumnRanges() func(*testing.T) { require.Equal(t, 1, int(count)) ran = []*ranger.Range{{ - LowVal: []types.Datum{types.NewUintDatum(0)}, - HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + LowVal: []types.Datum{types.NewUintDatum(0)}, + HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + Collators: collate.GetBinaryCollatorSlice(1), }} count, err = tbl.GetRowCountByIntColumnRanges(ctx, 0, ran) require.NoError(t, err) @@ -419,8 +422,9 @@ func SubTestIndexRanges() func(*testing.T) { }, } ran := []*ranger.Range{{ - LowVal: []types.Datum{types.MinNotNullDatum()}, - HighVal: []types.Datum{types.MaxValueDatum()}, + LowVal: []types.Datum{types.MinNotNullDatum()}, + HighVal: []types.Datum{types.MaxValueDatum()}, + Collators: collate.GetBinaryCollatorSlice(1), }} count, err := tbl.GetRowCountByIndexRanges(ctx, 0, ran) require.NoError(t, err) diff --git a/statistics/table.go b/statistics/table.go index 10e08001c7528..b8bfa8d9aaf5e 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -33,6 +33,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/tidb/util/tracing" @@ -459,7 +460,7 @@ func (t *Table) PseudoAvgCountPerValue() float64 { func GetOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int { for i := range ran.LowVal { a, b := ran.LowVal[i], ran.HighVal[i] - cmp, err := a.CompareDatum(sc, &b) + cmp, err := a.Compare(sc, &b, ran.Collators[0]) if err != nil { return 0 } @@ -607,6 +608,7 @@ func (coll *HistColl) crossValidationSelectivity(sctx sessionctx.Context, idx *I LowExclude: lowExclude, HighVal: []types.Datum{idxPointRange.HighVal[i]}, HighExclude: highExclude, + Collators: []collate.Collator{idxPointRange.Collators[i]}, } rowCount, err := col.GetColumnRowCount(sctx, []*ranger.Range{&rang}, coll.Count, col.IsHandle) @@ -725,6 +727,7 @@ func (coll *HistColl) getIndexRowCount(sctx sessionctx.Context, idxID int64, ind LowExclude: ran.LowExclude, HighVal: []types.Datum{ran.HighVal[rangePosition]}, HighExclude: ran.HighExclude, + Collators: []collate.Collator{ran.Collators[rangePosition]}, } var count float64 var err error @@ -847,7 +850,7 @@ func GetPseudoRowCountByColumnRanges(sc *stmtctx.StatementContext, tableRowCount } else if ran.HighVal[colIdx].Kind() == types.KindMaxValue { rowCount += tableRowCount / pseudoLessRate } else { - compare, err1 := ran.LowVal[colIdx].CompareDatum(sc, &ran.HighVal[colIdx]) + compare, err1 := ran.LowVal[colIdx].Compare(sc, &ran.HighVal[colIdx], ran.Collators[colIdx]) if err1 != nil { return 0, errors.Trace(err1) } diff --git a/util/collate/collate.go b/util/collate/collate.go index 73bdeeaaa20d5..5573db203327c 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -33,7 +33,8 @@ var ( newCollationEnabled int32 // binCollatorInstance is a singleton used for all collations when newCollationEnabled is false. - binCollatorInstance = &binCollator{} + binCollatorInstance = &binCollator{} + binCollatorInstanceSliceWithLen1 = []Collator{binCollatorInstance} // ErrUnsupportedCollation is returned when an unsupported collation is specified. ErrUnsupportedCollation = dbterror.ClassDDL.NewStdErr(mysql.ErrUnknownCollation, mysql.Message("Unsupported collation when new collation is enabled: '%-.64s'", nil)) @@ -161,6 +162,18 @@ func GetBinaryCollator() Collator { return binCollatorInstance } +// GetBinaryCollatorSlice gets the binary collator slice with len n. +func GetBinaryCollatorSlice(n int) []Collator { + if n == 1 { + return binCollatorInstanceSliceWithLen1 + } + collators := make([]Collator, n) + for i := 0; i < n; i++ { + collators[i] = binCollatorInstance + } + return collators +} + // GetCollatorByID get the collator according to id, it will return the binary collator if the corresponding collator doesn't exist. func GetCollatorByID(id int) Collator { if atomic.LoadInt32(&newCollationEnabled) == 1 { diff --git a/util/ranger/points.go b/util/ranger/points.go index 007d9a3469bc5..b2807b2759f7c 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -159,14 +159,14 @@ func getNotNullFullRange() []*point { // So we need to set it to MaxInt64. func FullIntRange(isUnsigned bool) []*Range { if isUnsigned { - return []*Range{{LowVal: []types.Datum{types.NewUintDatum(0)}, HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}}} + return []*Range{{LowVal: []types.Datum{types.NewUintDatum(0)}, HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, Collators: collate.GetBinaryCollatorSlice(1)}} } - return []*Range{{LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} + return []*Range{{LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, Collators: collate.GetBinaryCollatorSlice(1)}} } // FullRange is [null, +∞) for Range. func FullRange() []*Range { - return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}} + return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}, Collators: collate.GetBinaryCollatorSlice(1)}} } // FullNotNullRange is (-∞, +∞) for Range. @@ -176,7 +176,7 @@ func FullNotNullRange() []*Range { // NullRange is [null, null] for Range. func NullRange() []*Range { - return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{{}}}} + return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{{}}, Collators: collate.GetBinaryCollatorSlice(1)}} } // builder is the range builder struct. diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index ac671d3ae32f8..9a089b0622a0c 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -86,6 +86,7 @@ func points2Ranges(sctx sessionctx.Context, rangePoints []*point, tp *types.Fiel LowExclude: startPoint.excl, HighVal: []types.Datum{endPoint.value}, HighExclude: endPoint.excl, + Collators: []collate.Collator{collate.GetCollator(tp.Collate)}, } ranges = append(ranges, ran) } @@ -211,11 +212,15 @@ func appendPoints2IndexRange(sctx sessionctx.Context, origin *Range, rangePoints copy(highVal, origin.HighVal) highVal[len(origin.HighVal)] = endPoint.value + collators := make([]collate.Collator, len(origin.Collators)+1) + copy(collators, origin.Collators) + collators[len(origin.Collators)] = collate.GetCollator(ft.Collate) ir := &Range{ LowVal: lowVal, LowExclude: startPoint.excl, HighVal: highVal, HighExclude: endPoint.excl, + Collators: collators, } newRanges = append(newRanges, ir) } @@ -231,11 +236,13 @@ func appendRanges2PointRanges(pointRanges []*Range, ranges []*Range) []*Range { for _, r := range ranges { lowVal := append(pointRange.LowVal, r.LowVal...) highVal := append(pointRange.HighVal, r.HighVal...) + collators := append(pointRange.Collators, r.Collators...) newRange := &Range{ LowVal: lowVal, LowExclude: r.LowExclude, HighVal: highVal, HighExclude: r.HighExclude, + Collators: collators, } newRanges = append(newRanges, newRange) } @@ -288,6 +295,7 @@ func points2TableRanges(sctx sessionctx.Context, rangePoints []*point, tp *types LowExclude: startPoint.excl, HighVal: []types.Datum{endPoint.value}, HighExclude: endPoint.excl, + Collators: []collate.Collator{collate.GetCollator(tp.Collate)}, } ranges = append(ranges, ran) } @@ -341,7 +349,7 @@ func BuildTableRange(accessConditions []expression.Expression, sctx sessionctx.C // BuildColumnRange builds range from access conditions for general columns. func BuildColumnRange(conds []expression.Expression, sctx sessionctx.Context, tp *types.FieldType, colLen int) ([]*Range, error) { if len(conds) == 0 { - return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}, nil + return FullRange(), nil } return buildColumnRange(conds, sctx, tp, false, colLen) } @@ -631,7 +639,7 @@ func RangesToString(sc *stmtctx.StatementContext, rans []*Range, colNames []stri // sanity check: only last column of the `Range` can be an interval if j < len(ran.LowVal)-1 { - cmp, err := ran.LowVal[j].CompareDatum(sc, &ran.HighVal[j]) + cmp, err := ran.LowVal[j].Compare(sc, &ran.HighVal[j], ran.Collators[i]) if err != nil { return "", errors.New("comparing values error: " + err.Error()) } @@ -639,8 +647,7 @@ func RangesToString(sc *stmtctx.StatementContext, rans []*Range, colNames []stri return "", errors.New("unexpected form of range") } } - - str, err := RangeSingleColToString(sc, ran.LowVal[j], ran.HighVal[j], lowExclude, highExclude, colNames[j]) + str, err := RangeSingleColToString(sc, ran.LowVal[j], ran.HighVal[j], lowExclude, highExclude, colNames[j], ran.Collators[i]) if err != nil { return "false", err } @@ -667,7 +674,7 @@ func RangesToString(sc *stmtctx.StatementContext, rans []*Range, colNames []stri } // RangeSingleColToString prints a single column of a Range into a string which can appear in an SQL as a condition. -func RangeSingleColToString(sc *stmtctx.StatementContext, lowVal, highVal types.Datum, lowExclude, highExclude bool, colName string) (string, error) { +func RangeSingleColToString(sc *stmtctx.StatementContext, lowVal, highVal types.Datum, lowExclude, highExclude bool, colName string, collator collate.Collator) (string, error) { // case 1: low and high are both special values(null, min not null, max value) lowKind := lowVal.Kind() highKind := highVal.Kind() @@ -689,7 +696,7 @@ func RangeSingleColToString(sc *stmtctx.StatementContext, lowVal, highVal types. restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &buf) // case 2: low value and high value are the same, and low value and high value are both inclusive. - cmp, err := lowVal.CompareDatum(sc, &highVal) + cmp, err := lowVal.Compare(sc, &highVal, collator) if err != nil { return "false", errors.Trace(err) } diff --git a/util/ranger/types.go b/util/ranger/types.go index f2bf561f6a3cf..b932650588f86 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" ) // MutableRanges represents a range may change after it is created. @@ -56,6 +57,7 @@ type Range struct { LowExclude bool // Low value is exclusive. HighExclude bool // High value is exclusive. + Collators []collate.Collator } // Width returns the width of this range. @@ -77,6 +79,7 @@ func (ran *Range) Clone() *Range { for i, length := 0, len(ran.HighVal); i < length; i++ { newRange.HighVal = append(newRange.HighVal, ran.HighVal[i]) } + newRange.Collators = append(newRange.Collators, ran.Collators...) return newRange } @@ -95,7 +98,7 @@ func (ran *Range) isPoint(stmtCtx *stmtctx.StatementContext, regardNullAsPoint b if a.Kind() == types.KindMinNotNull || b.Kind() == types.KindMaxValue { return false } - cmp, err := a.CompareDatum(stmtCtx, &b) + cmp, err := a.Compare(stmtCtx, &b, ran.Collators[i]) if err != nil { return false } @@ -203,7 +206,7 @@ func (ran *Range) Encode(sc *stmtctx.StatementContext, lowBuffer, highBuffer []b func (ran *Range) PrefixEqualLen(sc *stmtctx.StatementContext) (int, error) { // Here, len(ran.LowVal) always equal to len(ran.HighVal) for i := 0; i < len(ran.LowVal); i++ { - cmp, err := ran.LowVal[i].CompareDatum(sc, &ran.HighVal[i]) + cmp, err := ran.LowVal[i].Compare(sc, &ran.HighVal[i], ran.Collators[i]) if err != nil { return 0, errors.Trace(err) } diff --git a/util/ranger/types_test.go b/util/ranger/types_test.go index d7e664dd323d8..ab0b3714ca5fe 100644 --- a/util/ranger/types_test.go +++ b/util/ranger/types_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) @@ -31,8 +32,9 @@ func TestRange(t *testing.T) { }{ { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(1)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(1)}, + Collators: collate.GetBinaryCollatorSlice(1), }, str: "[1,1]", }, @@ -41,6 +43,7 @@ func TestRange(t *testing.T) { LowVal: []types.Datum{types.NewIntDatum(1)}, HighVal: []types.Datum{types.NewIntDatum(1)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, str: "[1,1)", }, @@ -50,6 +53,7 @@ func TestRange(t *testing.T) { HighVal: []types.Datum{types.NewIntDatum(2)}, LowExclude: true, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, str: "(1,2)", }, @@ -58,6 +62,7 @@ func TestRange(t *testing.T) { LowVal: []types.Datum{types.NewFloat64Datum(1.1)}, HighVal: []types.Datum{types.NewFloat64Datum(1.9)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, str: "[1.1,1.9)", }, @@ -66,6 +71,7 @@ func TestRange(t *testing.T) { LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: []types.Datum{types.NewIntDatum(1)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, str: "[-inf,1)", }, @@ -80,22 +86,25 @@ func TestRange(t *testing.T) { }{ { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(1)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(1)}, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: true, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewStringDatum("abc")}, - HighVal: []types.Datum{types.NewStringDatum("abc")}, + LowVal: []types.Datum{types.NewStringDatum("abc")}, + HighVal: []types.Datum{types.NewStringDatum("abc")}, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: true, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(1), types.NewIntDatum(1)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(1), types.NewIntDatum(1)}, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: false, }, @@ -104,6 +113,7 @@ func TestRange(t *testing.T) { LowVal: []types.Datum{types.NewIntDatum(1)}, HighVal: []types.Datum{types.NewIntDatum(1)}, LowExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: false, }, @@ -112,13 +122,15 @@ func TestRange(t *testing.T) { LowVal: []types.Datum{types.NewIntDatum(1)}, HighVal: []types.Datum{types.NewIntDatum(1)}, HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: false, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewIntDatum(2)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + Collators: collate.GetBinaryCollatorSlice(1), }, isPoint: false, }, @@ -138,56 +150,63 @@ func TestIsFullRange(t *testing.T) { }{ { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, - HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, + LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, + HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: true, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, - HighVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, + LowVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}, + HighVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: false, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(1)}, - HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + LowVal: []types.Datum{types.NewIntDatum(1)}, + HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: false, }, { ran: ranger.Range{ - LowVal: []types.Datum{*nullDatum.Clone()}, - HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + LowVal: []types.Datum{*nullDatum.Clone()}, + HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: true, }, { ran: ranger.Range{ - LowVal: []types.Datum{*nullDatum.Clone()}, - HighVal: []types.Datum{*nullDatum.Clone()}, + LowVal: []types.Datum{*nullDatum.Clone()}, + HighVal: []types.Datum{*nullDatum.Clone()}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: false, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.MinNotNullDatum()}, - HighVal: []types.Datum{types.MaxValueDatum()}, + LowVal: []types.Datum{types.MinNotNullDatum()}, + HighVal: []types.Datum{types.MaxValueDatum()}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: false, isFullRange: true, }, { ran: ranger.Range{ - LowVal: []types.Datum{types.NewUintDatum(0)}, - HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + LowVal: []types.Datum{types.NewUintDatum(0)}, + HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}, + Collators: collate.GetBinaryCollatorSlice(1), }, unsignedIntHandle: true, isFullRange: true,