-
Notifications
You must be signed in to change notification settings - Fork 6k
planner: fix PointGetPlan.PrunePartitions
function works with non-binary collate
#62002
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
9900472
9049af5
8cad6da
ba386f6
285a003
8c95709
fa51b55
9feab38
8ef216a
d94c1f7
190f60b
dd2c350
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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" | ||
|
@@ -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) { | ||
|
@@ -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) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why are we looping on There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
|
||
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()) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add a test where There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This function isn't related to partition tables — it only processes the The pruning stage is handled separately here. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? | ||
Defined2014 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
p.IndexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset]) | ||
indexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset]) | ||
} | ||
} else { | ||
var dVal types.Datum | ||
|
@@ -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 | ||
|
@@ -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 | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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`) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
Uh oh!
There was an error while loading. Please reload this page.