Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 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
106 changes: 24 additions & 82 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 All @@ -419,7 +359,6 @@ func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) (bool, error) {
if def.Name.L == p.PartitionNames[0].L {
idx := i
p.PartitionIdx = &idx
break
}
}
return false, nil
Expand All @@ -446,32 +385,34 @@ 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
hasNonBinaryCollate := false
indexValues := p.IndexValues

evalCtx := sctx.GetExprCtx().GetEvalCtx()
for _, col := range p.IdxCols {
switch col.GetType(evalCtx).GetCollate() {
case "utf8mb4_bin", "binary", "ascii_bin", "latin1_bin", "utf8_bin", "utf8mb4_0900_bin":
default:
hasNonBinaryCollate = true
}
if hasNonBinaryCollate {
break
}
}
// If a non-binary collation is used, the values in `p.IndexValues` are sort keys and cannot be used for partition pruning.
if hasNonBinaryCollate {
r, err := ranger.DetachCondAndBuildRange(sctx.GetRangerCtx(), p.AccessConditions, p.IdxCols, p.IdxColLens, sctx.GetSessionVars().RangeMaxSize, false, true)
if err != nil {
return false, err
}
if done {
if len(partIdx) == 1 {
p.PartitionIdx = &partIdx[0]
return false, nil
}
if len(partIdx) == 0 {
idx := -1
p.PartitionIdx = &idx
return true, nil
}
return false, errors.Errorf("too many partitions matching for PointGetPlan")
if len(r.Ranges) != 1 || !r.Ranges[0].IsPoint(sctx.GetRangerCtx()) {
return false, errors.Errorf("internal error, build ranger for PointGet failed")
}
indexValues = r.Ranges[0].LowVal
}
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 +1769,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 +1780,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
10 changes: 5 additions & 5 deletions pkg/util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ func extractBestCNFItemRanges(sctx *rangerctx.RangerContext, conds []expression.
// 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 := detachCondAndBuildRange(sctx, tmpConds, cols, lengths, rangeMaxSize, convertToSortKey, false)
res, err := DetachCondAndBuildRange(sctx, tmpConds, cols, lengths, rangeMaxSize, convertToSortKey, false)
if err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -502,7 +502,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi
if eqOrInCount > 0 {
newCols := d.cols[eqOrInCount:]
newLengths := d.lengths[eqOrInCount:]
tailRes, err := detachCondAndBuildRange(d.sctx, newConditions, newCols, newLengths, d.rangeMaxSize, d.convertToSortKey, d.mergeConsecutive)
tailRes, err := DetachCondAndBuildRange(d.sctx, newConditions, newCols, newLengths, d.rangeMaxSize, d.convertToSortKey, d.mergeConsecutive)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -1030,8 +1030,8 @@ func DetachCondAndBuildRangeForIndex(sctx *rangerctx.RangerContext, conditions [
return d.detachCondAndBuildRangeForCols()
}

// detachCondAndBuildRange detaches the index filters from table filters and uses them to build ranges.
func detachCondAndBuildRange(sctx *rangerctx.RangerContext, conditions []expression.Expression, cols []*expression.Column,
// DetachCondAndBuildRange detaches the index filters from table filters and uses them to build ranges.
func DetachCondAndBuildRange(sctx *rangerctx.RangerContext, conditions []expression.Expression, cols []*expression.Column,
lengths []int, rangeMaxSize int64, convertToSortKey bool, mergeConsecutive bool) (*DetachRangeResult, error) {
d := &rangeDetacher{
sctx: sctx,
Expand All @@ -1051,7 +1051,7 @@ func detachCondAndBuildRange(sctx *rangerctx.RangerContext, conditions []express
// The returned values are encapsulated into a struct DetachRangeResult, see its comments for explanation.
func DetachCondAndBuildRangeForPartition(sctx *rangerctx.RangerContext, conditions []expression.Expression, cols []*expression.Column,
lengths []int, rangeMaxSize int64) (*DetachRangeResult, error) {
return detachCondAndBuildRange(sctx, conditions, cols, lengths, rangeMaxSize, false, false)
return DetachCondAndBuildRange(sctx, conditions, cols, lengths, rangeMaxSize, false, false)
}

type rangeDetacher struct {
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';