diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 344c8d71256ba..2708ae59085f1 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -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) + 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()) { + 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 @@ -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 diff --git a/tests/integrationtest/r/planner/core/casetest/partition/partition_pruner.result b/tests/integrationtest/r/planner/core/casetest/partition/partition_pruner.result index a9c29b5f8f035..90d26afc6c781 100644 --- a/tests/integrationtest/r/planner/core/casetest/partition/partition_pruner.result +++ b/tests/integrationtest/r/planner/core/casetest/partition/partition_pruner.result @@ -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`) +(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 diff --git a/tests/integrationtest/t/planner/core/casetest/partition/partition_pruner.test b/tests/integrationtest/t/planner/core/casetest/partition/partition_pruner.test index 0b3a6c10b3b60..0a6929a04bfd8 100644 --- a/tests/integrationtest/t/planner/core/casetest/partition/partition_pruner.test +++ b/tests/integrationtest/t/planner/core/casetest/partition/partition_pruner.test @@ -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'; +