diff --git a/pkg/planner/core/partition_pruning_test.go b/pkg/planner/core/partition_pruning_test.go index 4268f890348bf..92b2b251f0a57 100644 --- a/pkg/planner/core/partition_pruning_test.go +++ b/pkg/planner/core/partition_pruning_test.go @@ -578,6 +578,60 @@ func TestPartitionRangeColumnsForExpr(t *testing.T) { } } +func TestPartitionRangeColumnsForExprWithSpecialCollation(t *testing.T) { + tc := prepareTestCtx(t, "create table t (a varchar(255) COLLATE utf8mb4_0900_ai_ci, b varchar(255) COLLATE utf8mb4_unicode_ci)", "a,b") + lessThan := make([][]*expression.Expression, 0, 6) + partDefs := [][]string{ + {"'i'", "'i'"}, + {"MAXVALUE", "MAXVALUE"}, + } + for i := range partDefs { + l := make([]*expression.Expression, 0, 2) + for j := range []int{0, 1} { + v := partDefs[i][j] + var e *expression.Expression + if v == "MAXVALUE" { + e = nil // MAXVALUE + } else { + expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, v, tc.schema, tc.names) + require.NoError(t, err) + e = &expr[0] + } + l = append(l, e) + } + lessThan = append(lessThan, l) + } + pruner := &rangeColumnsPruner{lessThan, tc.columns[:2]} + cases := []struct { + input string + result partitionRangeOR + }{ + {"a = 'q'", partitionRangeOR{{1, 2}}}, + {"a = 'Q'", partitionRangeOR{{1, 2}}}, + {"a = 'a'", partitionRangeOR{{0, 1}}}, + {"a = 'A'", partitionRangeOR{{0, 1}}}, + {"a > 'a'", partitionRangeOR{{0, 2}}}, + {"a > 'q'", partitionRangeOR{{1, 2}}}, + {"a = 'i' and b = 'q'", partitionRangeOR{{1, 2}}}, + {"a = 'i' and b = 'Q'", partitionRangeOR{{1, 2}}}, + {"a = 'i' and b = 'a'", partitionRangeOR{{0, 1}}}, + {"a = 'i' and b = 'A'", partitionRangeOR{{0, 1}}}, + {"a = 'i' and b > 'a'", partitionRangeOR{{0, 2}}}, + {"a = 'i' and b > 'q'", partitionRangeOR{{1, 2}}}, + {"a = 'i' or a = 'h'", partitionRangeOR{{0, 2}}}, + {"a = 'h' and a = 'j'", partitionRangeOR{}}, + } + + for _, ca := range cases { + expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) + require.NoError(t, err) + result := fullRange(len(lessThan)) + e := expression.SplitCNFItems(expr[0]) + result = partitionRangeForCNFExpr(tc.sctx, e, pruner, result) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "unexpected: %v %v != %v", ca.input, ca.result, result) + } +} + func benchmarkRangeColumnsPruner(b *testing.B, parts int) { tc := prepareBenchCtx("create table t (a bigint unsigned, b int, c int)", "a") if tc == nil { diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index 3da234a6486cb..34811b32ffa3b 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -1175,7 +1175,7 @@ func multiColumnRangeColumnsPruner(sctx sessionctx.Context, exprs []expression.E lens = append(lens, columnsPruner.partCols[i].RetType.GetFlen()) } - res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, exprs, columnsPruner.partCols, lens, sctx.GetSessionVars().RangeMaxSize) + res, err := ranger.DetachCondAndBuildRangeForPartition(sctx, exprs, columnsPruner.partCols, lens, sctx.GetSessionVars().RangeMaxSize) if err != nil { return fullRange(len(columnsPruner.lessThan)) } @@ -1190,16 +1190,12 @@ func multiColumnRangeColumnsPruner(sctx sessionctx.Context, exprs []expression.E rangeOr := make([]partitionRange, 0, len(res.Ranges)) - comparer := make([]collate.Collator, 0, len(columnsPruner.partCols)) - for i := range columnsPruner.partCols { - comparer = append(comparer, collate.GetCollator(columnsPruner.partCols[i].RetType.GetCollate())) - } gotError := false // Create a sort.Search where the compare loops over ColumnValues // Loop over the different ranges and extend/include all the partitions found for idx := range res.Ranges { - minComparer := minCmp(sctx, res.Ranges[idx].LowVal, columnsPruner, comparer, res.Ranges[idx].LowExclude, &gotError) - maxComparer := maxCmp(sctx, res.Ranges[idx].HighVal, columnsPruner, comparer, res.Ranges[idx].HighExclude, &gotError) + minComparer := minCmp(sctx, res.Ranges[idx].LowVal, columnsPruner, res.Ranges[idx].Collators, res.Ranges[idx].LowExclude, &gotError) + maxComparer := maxCmp(sctx, res.Ranges[idx].HighVal, columnsPruner, res.Ranges[idx].Collators, res.Ranges[idx].HighExclude, &gotError) if gotError { // the compare function returned error, use all partitions. return fullRange(len(columnsPruner.lessThan)) @@ -1797,10 +1793,9 @@ func makeRangeColumnPruner(columns []*expression.Column, pi *model.PartitionInfo if len(pi.Definitions) != len(from.LessThan) { return nil, errors.Trace(fmt.Errorf("internal error len(pi.Definitions) != len(from.LessThan) %d != %d", len(pi.Definitions), len(from.LessThan))) } - schema := expression.NewSchema(columns...) partCols := make([]*expression.Column, len(offsets)) for i, offset := range offsets { - partCols[i] = schema.Columns[offset] + partCols[i] = columns[offset] } lessThan := make([][]*expression.Expression, 0, len(from.LessThan)) for i := range from.LessThan { diff --git a/pkg/util/ranger/detacher.go b/pkg/util/ranger/detacher.go index 5586aaea06fae..bea2d7fe43b24 100644 --- a/pkg/util/ranger/detacher.go +++ b/pkg/util/ranger/detacher.go @@ -261,7 +261,7 @@ func extractBestCNFItemRanges(sctx sessionctx.Context, conds []expression.Expres // We build ranges for `(a,b) in ((1,1),(1,2))` and get `[1 1, 1 1] [1 2, 1 2]`, which are point ranges and we can // append `c = 1` to the point ranges. However, if we choose to merge consecutive ranges here, we get `[1 1, 1 2]`, // which are not point ranges, and we cannot append `c = 1` anymore. - res, err := detachCondAndBuildRangeWithoutMerging(sctx, tmpConds, cols, lengths, rangeMaxSize, convertToSortKey) + res, err := detachCondAndBuildRange(sctx, tmpConds, cols, lengths, rangeMaxSize, convertToSortKey, false) if err != nil { return nil, nil, err } @@ -418,7 +418,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi if eqOrInCount > 0 { newCols := d.cols[eqOrInCount:] newLengths := d.lengths[eqOrInCount:] - tailRes, err := DetachCondAndBuildRangeForIndex(d.sctx, newConditions, newCols, newLengths, d.rangeMaxSize) + tailRes, err := detachCondAndBuildRange(d.sctx, newConditions, newCols, newLengths, d.rangeMaxSize, d.convertToSortKey, d.mergeConsecutive) if err != nil { return nil, err } @@ -892,16 +892,15 @@ func DetachCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []expre return d.detachCondAndBuildRangeForCols() } -// detachCondAndBuildRangeWithoutMerging detaches the index filters from table filters and uses them to build ranges. -// When building ranges, it doesn't merge consecutive ranges. -func detachCondAndBuildRangeWithoutMerging(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, - lengths []int, rangeMaxSize int64, convertToSortKey bool) (*DetachRangeResult, error) { +// detachCondAndBuildRange detaches the index filters from table filters and uses them to build ranges. +func detachCondAndBuildRange(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, + lengths []int, rangeMaxSize int64, convertToSortKey bool, mergeConsecutive bool) (*DetachRangeResult, error) { d := &rangeDetacher{ sctx: sctx, allConds: conditions, cols: cols, lengths: lengths, - mergeConsecutive: false, + mergeConsecutive: mergeConsecutive, convertToSortKey: convertToSortKey, rangeMaxSize: rangeMaxSize, } @@ -914,7 +913,7 @@ func detachCondAndBuildRangeWithoutMerging(sctx sessionctx.Context, conditions [ // The returned values are encapsulated into a struct DetachRangeResult, see its comments for explanation. func DetachCondAndBuildRangeForPartition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int, rangeMaxSize int64) (*DetachRangeResult, error) { - return detachCondAndBuildRangeWithoutMerging(sctx, conditions, cols, lengths, rangeMaxSize, false) + return detachCondAndBuildRange(sctx, conditions, cols, lengths, rangeMaxSize, false, false) } type rangeDetacher struct {