Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
104 changes: 21 additions & 83 deletions pkg/planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ import (
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/plancodec"
"github.com/pingcap/tidb/pkg/util/ranger"
"github.com/pingcap/tidb/pkg/util/redact"
"github.com/pingcap/tidb/pkg/util/size"
"github.com/pingcap/tidb/pkg/util/stringutil"
Expand Down Expand Up @@ -341,67 +342,6 @@ func (p *PointGetPlan) LoadTableStats(ctx sessionctx.Context) {
loadTableStats(ctx, p.TblInfo, tableID)
}

// needsPartitionPruning checks if IndexValues can be used by GetPartitionIdxByRow() or if they have already been
// converted to SortKey and would need GetPartitionIdxByRow() to be refactored to work, since it will unconditionally
// convert it again.
// Returns:
// Matching partition
// if done Partition pruning (else not needed, can use GetPartitionIdxByRow() instead)
// error
// TODO: Also supporting BatchPointGet? Problem is that partition ID must be mapped to handle/IndexValue.
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) {
for i := range indexValues {
if tblInfo.Columns[indexInfo.Columns[i].Offset].FieldType.EvalType() != types.ETString ||
indexValues[i].Collation() == tblInfo.Columns[indexInfo.Columns[i].Offset].GetCollate() {
return nil, false, nil
}
}
// convertToPointGet will have the IndexValues already converted to SortKey,
// which will be converted again by GetPartitionIdxByRow, so we need to re-run the pruner
// with the conditions.

// TODO: Is there a simpler way, or existing function for this?!?
tblCols := make([]*expression.Column, 0, len(indexInfo.Columns))
var partNameSlice types.NameSlice
for _, tblCol := range tblInfo.Columns {
found := false
for _, idxCol := range indexCols {
if idxCol.ID == tblCol.ID {
tblCols = append(tblCols, idxCol)
found = true
break
}
}
partNameSlice = append(partNameSlice, &types.FieldName{
ColName: tblCol.Name,
TblName: tblInfo.Name,
DBName: ast.NewCIStr(dbName),
OrigTblName: tblInfo.Name,
OrigColName: tblCol.Name,
})
if !found {
tblCols = append(tblCols, &expression.Column{
ID: tblCol.ID,
OrigName: tblCol.Name.O,
RetType: tblCol.FieldType.Clone(),
})
}
}

partIdx, err := PartitionPruning(sctx.GetPlanCtx(), pt, conds, partitionNames, tblCols, partNameSlice)
if err != nil {
return nil, true, err
}
if len(partIdx) == 1 && partIdx[0] == FullRange {
ret := make([]int, len(tblInfo.Partition.Definitions))
for i := range len(tblInfo.Partition.Definitions) {
ret[i] = i
}
return ret, true, nil
}
return partIdx, true, nil
}

// PrunePartitions will check which partition to use
// returns true if no matching partition
func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) (bool, error) {
Expand Down Expand Up @@ -446,32 +386,29 @@ func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) (bool, error) {
}
row := make([]types.Datum, len(p.TblInfo.Columns))
if p.HandleConstant == nil && len(p.IndexValues) > 0 {
partColsNames := pt.Meta().Partition.Columns
if len(partColsNames) > 0 {
partIdx, done, err := needsPartitionPruning(sctx, p.TblInfo, pt, p.dbName, p.IndexInfo, p.IdxCols, p.IndexValues, p.AccessConditions, p.PartitionNames)
if table.ErrNoPartitionForGivenValue.Equal(err) {
err = nil
partIdx = nil
}
if err != nil {
return false, err
}
if done {
if len(partIdx) == 1 {
p.PartitionIdx = &partIdx[0]
return false, nil
indexValues := p.IndexValues
evalCtx := sctx.GetExprCtx().GetEvalCtx()
// If the plan is created via the fast path, `IdxCols` will be nil here,
// and the fast path does not convert the values to `sortKey`.
for _, col := range p.IdxCols {
// TODO: We could check whether `col` belongs to the partition columns to avoid unnecessary ranger building.
// https://github.com/pingcap/tidb/pull/62002#discussion_r2171420731
if !collate.IsBinCollation(col.GetType(evalCtx).GetCollate()) {
// If a non-binary collation is used, the values in `p.IndexValues` are sort keys and cannot be used for partition pruning.
r, err := ranger.DetachCondAndBuildRangeForPartition(sctx.GetRangerCtx(), p.AccessConditions, p.IdxCols, p.IdxColLens, sctx.GetSessionVars().RangeMaxSize)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why are we looping on p.IdxCols and doing this for the full conditions, and all columns (p.IdxCols)?
Should we also check if the column is part of the partitioning expression and then check if binary collation?

Copy link
Contributor Author

@Defined2014 Defined2014 Jun 27, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a comments here, I think it's not easy to do this right now. And this part will not affect the performance very much.

However, we currently don't have a safe and reliable way to do that. col is of type expression.Column, while the IDs from pt.GetPartitionColumnIDs() are derived from t.Cols() which type is table.Column. It's not always correctly to compare them.

if err != nil {
return false, err
}
if len(partIdx) == 0 {
idx := -1
p.PartitionIdx = &idx
return true, nil
if len(r.Ranges) != 1 || !r.Ranges[0].IsPoint(sctx.GetRangerCtx()) {
Copy link
Contributor

@mjonss mjonss Jun 26, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a test where len(r.Ranges) == 0. Like RANGE partition without MAXVALUE and a condition for value greater than last partition.
I would assume it should return true, nil in such case and use table DUAL.

Copy link
Contributor Author

@Defined2014 Defined2014 Jun 27, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This function isn't related to partition tables — it only processes the AccessConditions and returns the ranger. So even it's not meet any exists partition, the len(r.Ranges) also should equals to 1.

The pruning stage is handled separately here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The root cause of these problems is that the planner uses a different function from this one, and the results of the two are not always consistent. To ensure correctness, we should use the same function here.

return false, errors.Errorf("internal error, build ranger for PointGet failed")
}
return false, errors.Errorf("too many partitions matching for PointGetPlan")
indexValues = r.Ranges[0].LowVal
break
}
}
for i := range p.IndexInfo.Columns {
// TODO: Skip copying non-partitioning columns?
p.IndexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset])
indexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset])
}
} else {
var dVal types.Datum
Expand Down Expand Up @@ -1828,7 +1765,8 @@ func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblNam
if !ok {
return nil, false
}
if binOp.Op == opcode.LogicAnd {
switch binOp.Op {
case opcode.LogicAnd:
nvPairs, isTableDual = getNameValuePairs(ctx, tbl, tblName, nvPairs, binOp.L)
if nvPairs == nil || isTableDual {
return nil, isTableDual
Expand All @@ -1838,7 +1776,7 @@ func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblNam
return nil, isTableDual
}
return nvPairs, isTableDual
} else if binOp.Op == opcode.EQ {
case opcode.EQ:
var (
d types.Datum
colName *ast.ColumnNameExpr
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -928,3 +928,19 @@ Point_Get 1.00 root table:t, partition:p4, index:idx_3(col_4)
select * from t where col_4 in ( 'u^D92@_4' ,null );
col_4
u^D92@_4
drop table if exists t;
CREATE TABLE `t` (
`col_95` char(181) COLLATE gbk_bin NOT NULL DEFAULT 'SaMKHTyg+nlID-X3Y',
PRIMARY KEY (`col_95`) /*T![clustered_index] CLUSTERED */
) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin
PARTITION BY RANGE COLUMNS(`col_95`)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to test other partitioning schemes as well? Like RANGE COLUMNS, LIST [COLUMNS] and KEY?

Copy link
Contributor Author

@Defined2014 Defined2014 Jun 27, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this is related to partition types. And also I think the test cases have already been added in PR #59918.

(PARTITION `p0` VALUES LESS THAN ('6)nvX^uj0UGxqX'),
PARTITION `p1` VALUES LESS THAN ('BHSluf6'),
PARTITION `p2` VALUES LESS THAN (MAXVALUE));
insert into t values ('58y-j)84-&Y*'), ('WNe(rS5uwmvIvFnHw'), ('j9FsMawX5uBro%$p'), ('C(#EQm@J');
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';
id estRows task access object operator info
Point_Get 1.00 root table:t, partition:p2, clustered index:PRIMARY(col_95)
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';
r0
j9FsMawX5uBro%$p
Original file line number Diff line number Diff line change
Expand Up @@ -330,3 +330,19 @@ insert into t values ('u^D92@_4'), (null);
explain format='brief' select * from t where col_4 in ( 'u^D92@_4' ,null );
select * from t where col_4 in ( 'u^D92@_4' ,null );

# Test issue 61965
drop table if exists t;
CREATE TABLE `t` (
`col_95` char(181) COLLATE gbk_bin NOT NULL DEFAULT 'SaMKHTyg+nlID-X3Y',
PRIMARY KEY (`col_95`) /*T![clustered_index] CLUSTERED */
) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin
PARTITION BY RANGE COLUMNS(`col_95`)
(PARTITION `p0` VALUES LESS THAN ('6)nvX^uj0UGxqX'),
PARTITION `p1` VALUES LESS THAN ('BHSluf6'),
PARTITION `p2` VALUES LESS THAN (MAXVALUE));

insert into t values ('58y-j)84-&Y*'), ('WNe(rS5uwmvIvFnHw'), ('j9FsMawX5uBro%$p'), ('C(#EQm@J');

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';
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';