Skip to content

Commit ce538d6

Browse files
authored
planner: fix PointGetPlan.PrunePartitions function works with non-binary collate (#62002)
close #59827, close #61965
1 parent a386a88 commit ce538d6

File tree

3 files changed

+53
-83
lines changed

3 files changed

+53
-83
lines changed

pkg/planner/core/point_get_plan.go

Lines changed: 21 additions & 83 deletions
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,7 @@ import (
6161
"github.com/pingcap/tidb/pkg/util/intest"
6262
"github.com/pingcap/tidb/pkg/util/logutil"
6363
"github.com/pingcap/tidb/pkg/util/plancodec"
64+
"github.com/pingcap/tidb/pkg/util/ranger"
6465
"github.com/pingcap/tidb/pkg/util/redact"
6566
"github.com/pingcap/tidb/pkg/util/size"
6667
"github.com/pingcap/tidb/pkg/util/stringutil"
@@ -341,67 +342,6 @@ func (p *PointGetPlan) LoadTableStats(ctx sessionctx.Context) {
341342
loadTableStats(ctx, p.TblInfo, tableID)
342343
}
343344

344-
// needsPartitionPruning checks if IndexValues can be used by GetPartitionIdxByRow() or if they have already been
345-
// converted to SortKey and would need GetPartitionIdxByRow() to be refactored to work, since it will unconditionally
346-
// convert it again.
347-
// Returns:
348-
// Matching partition
349-
// if done Partition pruning (else not needed, can use GetPartitionIdxByRow() instead)
350-
// error
351-
// TODO: Also supporting BatchPointGet? Problem is that partition ID must be mapped to handle/IndexValue.
352-
func needsPartitionPruning(sctx sessionctx.Context, tblInfo *model.TableInfo, pt table.PartitionedTable, dbName string, indexInfo *model.IndexInfo, indexCols []*expression.Column, indexValues []types.Datum, conds []expression.Expression, partitionNames []ast.CIStr) ([]int, bool, error) {
353-
for i := range indexValues {
354-
if tblInfo.Columns[indexInfo.Columns[i].Offset].FieldType.EvalType() != types.ETString ||
355-
indexValues[i].Collation() == tblInfo.Columns[indexInfo.Columns[i].Offset].GetCollate() {
356-
return nil, false, nil
357-
}
358-
}
359-
// convertToPointGet will have the IndexValues already converted to SortKey,
360-
// which will be converted again by GetPartitionIdxByRow, so we need to re-run the pruner
361-
// with the conditions.
362-
363-
// TODO: Is there a simpler way, or existing function for this?!?
364-
tblCols := make([]*expression.Column, 0, len(indexInfo.Columns))
365-
var partNameSlice types.NameSlice
366-
for _, tblCol := range tblInfo.Columns {
367-
found := false
368-
for _, idxCol := range indexCols {
369-
if idxCol.ID == tblCol.ID {
370-
tblCols = append(tblCols, idxCol)
371-
found = true
372-
break
373-
}
374-
}
375-
partNameSlice = append(partNameSlice, &types.FieldName{
376-
ColName: tblCol.Name,
377-
TblName: tblInfo.Name,
378-
DBName: ast.NewCIStr(dbName),
379-
OrigTblName: tblInfo.Name,
380-
OrigColName: tblCol.Name,
381-
})
382-
if !found {
383-
tblCols = append(tblCols, &expression.Column{
384-
ID: tblCol.ID,
385-
OrigName: tblCol.Name.O,
386-
RetType: tblCol.FieldType.Clone(),
387-
})
388-
}
389-
}
390-
391-
partIdx, err := PartitionPruning(sctx.GetPlanCtx(), pt, conds, partitionNames, tblCols, partNameSlice)
392-
if err != nil {
393-
return nil, true, err
394-
}
395-
if len(partIdx) == 1 && partIdx[0] == FullRange {
396-
ret := make([]int, len(tblInfo.Partition.Definitions))
397-
for i := range len(tblInfo.Partition.Definitions) {
398-
ret[i] = i
399-
}
400-
return ret, true, nil
401-
}
402-
return partIdx, true, nil
403-
}
404-
405345
// PrunePartitions will check which partition to use
406346
// returns true if no matching partition
407347
func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) (bool, error) {
@@ -446,32 +386,29 @@ func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) (bool, error) {
446386
}
447387
row := make([]types.Datum, len(p.TblInfo.Columns))
448388
if p.HandleConstant == nil && len(p.IndexValues) > 0 {
449-
partColsNames := pt.Meta().Partition.Columns
450-
if len(partColsNames) > 0 {
451-
partIdx, done, err := needsPartitionPruning(sctx, p.TblInfo, pt, p.dbName, p.IndexInfo, p.IdxCols, p.IndexValues, p.AccessConditions, p.PartitionNames)
452-
if table.ErrNoPartitionForGivenValue.Equal(err) {
453-
err = nil
454-
partIdx = nil
455-
}
456-
if err != nil {
457-
return false, err
458-
}
459-
if done {
460-
if len(partIdx) == 1 {
461-
p.PartitionIdx = &partIdx[0]
462-
return false, nil
389+
indexValues := p.IndexValues
390+
evalCtx := sctx.GetExprCtx().GetEvalCtx()
391+
// If the plan is created via the fast path, `IdxCols` will be nil here,
392+
// and the fast path does not convert the values to `sortKey`.
393+
for _, col := range p.IdxCols {
394+
// TODO: We could check whether `col` belongs to the partition columns to avoid unnecessary ranger building.
395+
// https://github.com/pingcap/tidb/pull/62002#discussion_r2171420731
396+
if !collate.IsBinCollation(col.GetType(evalCtx).GetCollate()) {
397+
// If a non-binary collation is used, the values in `p.IndexValues` are sort keys and cannot be used for partition pruning.
398+
r, err := ranger.DetachCondAndBuildRangeForPartition(sctx.GetRangerCtx(), p.AccessConditions, p.IdxCols, p.IdxColLens, sctx.GetSessionVars().RangeMaxSize)
399+
if err != nil {
400+
return false, err
463401
}
464-
if len(partIdx) == 0 {
465-
idx := -1
466-
p.PartitionIdx = &idx
467-
return true, nil
402+
if len(r.Ranges) != 1 || !r.Ranges[0].IsPoint(sctx.GetRangerCtx()) {
403+
return false, errors.Errorf("internal error, build ranger for PointGet failed")
468404
}
469-
return false, errors.Errorf("too many partitions matching for PointGetPlan")
405+
indexValues = r.Ranges[0].LowVal
406+
break
470407
}
471408
}
472409
for i := range p.IndexInfo.Columns {
473410
// TODO: Skip copying non-partitioning columns?
474-
p.IndexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset])
411+
indexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset])
475412
}
476413
} else {
477414
var dVal types.Datum
@@ -1828,7 +1765,8 @@ func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblNam
18281765
if !ok {
18291766
return nil, false
18301767
}
1831-
if binOp.Op == opcode.LogicAnd {
1768+
switch binOp.Op {
1769+
case opcode.LogicAnd:
18321770
nvPairs, isTableDual = getNameValuePairs(ctx, tbl, tblName, nvPairs, binOp.L)
18331771
if nvPairs == nil || isTableDual {
18341772
return nil, isTableDual
@@ -1838,7 +1776,7 @@ func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblNam
18381776
return nil, isTableDual
18391777
}
18401778
return nvPairs, isTableDual
1841-
} else if binOp.Op == opcode.EQ {
1779+
case opcode.EQ:
18421780
var (
18431781
d types.Datum
18441782
colName *ast.ColumnNameExpr

tests/integrationtest/r/planner/core/casetest/partition/partition_pruner.result

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -928,3 +928,19 @@ Point_Get 1.00 root table:t, partition:p4, index:idx_3(col_4)
928928
select * from t where col_4 in ( 'u^D92@_4' ,null );
929929
col_4
930930
u^D92@_4
931+
drop table if exists t;
932+
CREATE TABLE `t` (
933+
`col_95` char(181) COLLATE gbk_bin NOT NULL DEFAULT 'SaMKHTyg+nlID-X3Y',
934+
PRIMARY KEY (`col_95`) /*T![clustered_index] CLUSTERED */
935+
) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin
936+
PARTITION BY RANGE COLUMNS(`col_95`)
937+
(PARTITION `p0` VALUES LESS THAN ('6)nvX^uj0UGxqX'),
938+
PARTITION `p1` VALUES LESS THAN ('BHSluf6'),
939+
PARTITION `p2` VALUES LESS THAN (MAXVALUE));
940+
insert into t values ('58y-j)84-&Y*'), ('WNe(rS5uwmvIvFnHw'), ('j9FsMawX5uBro%$p'), ('C(#EQm@J');
941+
explain format='brief' select t.col_95 as r0 from t where t.col_95 between 'Dyw=*7nigCMh' and 'Im0*7sZ' or t.col_95 in ( '58y-j)84-&Y*' ,'WNe(rS5uwmvIvFnHw' ,'j9FsMawX5uBro%$p' ,'C(#EQm@J' ) group by t.col_95 having t.col_95 between '%^2' and '38ABfC-' or t.col_95 between 'eKCAE$d2x_hxscj' and 'zcw35^ATEEp1md=L';
942+
id estRows task access object operator info
943+
Point_Get 1.00 root table:t, partition:p2, clustered index:PRIMARY(col_95)
944+
select t.col_95 as r0 from t where t.col_95 between 'Dyw=*7nigCMh' and 'Im0*7sZ' or t.col_95 in ( '58y-j)84-&Y*' ,'WNe(rS5uwmvIvFnHw' ,'j9FsMawX5uBro%$p' ,'C(#EQm@J' ) group by t.col_95 having t.col_95 between '%^2' and '38ABfC-' or t.col_95 between 'eKCAE$d2x_hxscj' and 'zcw35^ATEEp1md=L';
945+
r0
946+
j9FsMawX5uBro%$p

tests/integrationtest/t/planner/core/casetest/partition/partition_pruner.test

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -330,3 +330,19 @@ insert into t values ('u^D92@_4'), (null);
330330
explain format='brief' select * from t where col_4 in ( 'u^D92@_4' ,null );
331331
select * from t where col_4 in ( 'u^D92@_4' ,null );
332332

333+
# Test issue 61965
334+
drop table if exists t;
335+
CREATE TABLE `t` (
336+
`col_95` char(181) COLLATE gbk_bin NOT NULL DEFAULT 'SaMKHTyg+nlID-X3Y',
337+
PRIMARY KEY (`col_95`) /*T![clustered_index] CLUSTERED */
338+
) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin
339+
PARTITION BY RANGE COLUMNS(`col_95`)
340+
(PARTITION `p0` VALUES LESS THAN ('6)nvX^uj0UGxqX'),
341+
PARTITION `p1` VALUES LESS THAN ('BHSluf6'),
342+
PARTITION `p2` VALUES LESS THAN (MAXVALUE));
343+
344+
insert into t values ('58y-j)84-&Y*'), ('WNe(rS5uwmvIvFnHw'), ('j9FsMawX5uBro%$p'), ('C(#EQm@J');
345+
346+
explain format='brief' select t.col_95 as r0 from t where t.col_95 between 'Dyw=*7nigCMh' and 'Im0*7sZ' or t.col_95 in ( '58y-j)84-&Y*' ,'WNe(rS5uwmvIvFnHw' ,'j9FsMawX5uBro%$p' ,'C(#EQm@J' ) group by t.col_95 having t.col_95 between '%^2' and '38ABfC-' or t.col_95 between 'eKCAE$d2x_hxscj' and 'zcw35^ATEEp1md=L';
347+
select t.col_95 as r0 from t where t.col_95 between 'Dyw=*7nigCMh' and 'Im0*7sZ' or t.col_95 in ( '58y-j)84-&Y*' ,'WNe(rS5uwmvIvFnHw' ,'j9FsMawX5uBro%$p' ,'C(#EQm@J' ) group by t.col_95 having t.col_95 between '%^2' and '38ABfC-' or t.col_95 between 'eKCAE$d2x_hxscj' and 'zcw35^ATEEp1md=L';
348+

0 commit comments

Comments
 (0)