Skip to content

Commit 40b72e7

Browse files
authored
partition: Revert support batch_point_get for partition table in dynamic mode (#45891) (#45902)
close #45889
1 parent 203d839 commit 40b72e7

File tree

8 files changed

+284
-163
lines changed

8 files changed

+284
-163
lines changed

executor/batch_point_get.go

Lines changed: 8 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,8 @@ type BatchPointGetExec struct {
5151
partExpr *tables.PartitionExpr
5252
partPos int
5353
planPhysIDs []int64
54-
partTblID []int64
54+
singlePart bool
55+
partTblID int64
5556
idxVals [][]types.Datum
5657
txn kv.Transaction
5758
lock bool
@@ -234,11 +235,9 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
234235
}
235236
}
236237

237-
// If this BatchPointGetExec is built only for the specific table partitions, skip those filters not matching those partitions.
238-
if len(e.partTblID) >= 1 {
239-
if _, found := slices.BinarySearch(e.partTblID, physID); !found {
240-
continue
241-
}
238+
// If this BatchPointGetExec is built only for the specific table partition, skip those filters not matching this partition.
239+
if e.singlePart && e.partTblID != physID {
240+
continue
242241
}
243242
idxKey, err1 := EncodeUniqueIndexKey(e.Ctx(), e.tblInfo, e.idxInfo, idxVals, physID)
244243
if err1 != nil && !kv.ErrNotExist.Equal(err1) {
@@ -380,11 +379,9 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
380379
}
381380
}
382381
}
383-
// If this BatchPointGetExec is built only for the specific table partitions, skip those handles not matching those partitions.
384-
if len(e.partTblID) >= 1 {
385-
if _, found := slices.BinarySearch(e.partTblID, tID); !found {
386-
continue
387-
}
382+
// If this BatchPointGetExec is built only for the specific table partition, skip those handles not matching this partition.
383+
if e.singlePart && e.partTblID != tID {
384+
continue
388385
}
389386
key := tablecodec.EncodeRowKeyWithHandle(tID, handle)
390387
keys = append(keys, key)

executor/builder.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5217,6 +5217,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan
52175217
partExpr: plan.PartitionExpr,
52185218
partPos: plan.PartitionColPos,
52195219
planPhysIDs: plan.PartitionIDs,
5220+
singlePart: plan.SinglePart,
52205221
partTblID: plan.PartTblID,
52215222
columns: plan.Columns,
52225223
}

planner/core/casetest/partition/integration_partition_test.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -292,6 +292,9 @@ func TestBatchPointGetTablePartition(t *testing.T) {
292292
tk.MustExec("create table tlist3(a int, b int, primary key(a)) partition by list(a) (partition p0 values in (0, 1, 2), partition p1 values in (3, 4, 5))")
293293
tk.MustExec("insert into tlist3 values(1,0),(2,0),(3,0),(4,0)")
294294

295+
tk.MustExec("create table issue45889(a int) partition by list(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))")
296+
tk.MustExec("insert into issue45889 values (0),(0),(1),(1),(2),(2),(3),(3)")
297+
295298
var input []string
296299
var output []struct {
297300
SQL string

planner/core/casetest/partition/testdata/integration_partition_suite_in.json

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -263,7 +263,8 @@
263263
"select * from tlist3 where a in (1,2) and 1 = 1 order by a desc",
264264
"select * from tlist3 partition(p0) where a in (1,4)",
265265
"select * from tlist3 partition(p1) where a in (1,2)",
266-
"select * from tlist3 partition(p0,p1) where a in (1,2)"
266+
"select * from tlist3 partition(p0,p1) where a in (1,2)",
267+
"select _tidb_rowid, a from issue45889 where _tidb_rowid in (7, 8)"
267268
]
268269
},
269270
{

planner/core/casetest/partition/testdata/integration_partition_suite_out.json

Lines changed: 194 additions & 89 deletions
Large diffs are not rendered by default.

planner/core/casetest/partition/testdata/partition_pruner_out.json

Lines changed: 18 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -740,8 +740,9 @@
740740
],
741741
"Plan": [
742742
"HashJoin 2.20 root inner join, equal:[eq(test_partition.t4.id, test_partition.t5.id)]",
743-
"├─Selection(Build) 2.00 root not(isnull(test_partition.t4.id))",
744-
"│ └─Batch_Point_Get 2.00 root table:t4 handle:[1 3], keep order:false, desc:false",
743+
"├─TableReader(Build) 2.00 root partition:p0 data:Selection",
744+
"│ └─Selection 2.00 cop[tikv] not(isnull(test_partition.t4.id))",
745+
"│ └─TableRangeScan 2.00 cop[tikv] table:t4 range:[1,1], [3,3], keep order:false",
745746
"└─IndexLookUp(Probe) 3.64 root partition:p0,p1 ",
746747
" ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t5, index:a(a, b) range:[1 1,1 1], [1 6,1 6], [6 1,6 1], [6 6,6 6], keep order:false",
747748
" └─Selection(Probe) 3.64 cop[tikv] not(isnull(test_partition.t5.id))",
@@ -755,10 +756,12 @@
755756
],
756757
"Plan": [
757758
"HashJoin 2.00 root inner join, equal:[eq(test_partition.t4.id, test_partition.t4.id)]",
758-
"├─Selection(Build) 2.00 root not(isnull(test_partition.t4.id))",
759-
"│ └─Batch_Point_Get 2.00 root table:t4 handle:[1 4], keep order:false, desc:false",
760-
"└─Selection(Probe) 4.00 root not(isnull(test_partition.t4.id))",
761-
" └─Batch_Point_Get 4.00 root table:t4 handle:[1 3 9 100], keep order:false, desc:false"
759+
"├─TableReader(Build) 2.00 root partition:p0 data:Selection",
760+
"│ └─Selection 2.00 cop[tikv] not(isnull(test_partition.t4.id))",
761+
"│ └─TableRangeScan 2.00 cop[tikv] table:t1 range:[1,1], [4,4], keep order:false",
762+
"└─TableReader(Probe) 4.00 root partition:p0,p1 data:Selection",
763+
" └─Selection 4.00 cop[tikv] not(isnull(test_partition.t4.id))",
764+
" └─TableRangeScan 4.00 cop[tikv] table:t2 range:[1,1], [3,3], [9,9], [100,100], keep order:false"
762765
]
763766
},
764767
{
@@ -1373,8 +1376,9 @@
13731376
],
13741377
"Plan": [
13751378
"HashJoin 2.17 root inner join, equal:[eq(default_partition.t4.id, default_partition.t5.id)]",
1376-
"├─Selection(Build) 2.00 root not(isnull(default_partition.t4.id))",
1377-
"│ └─Batch_Point_Get 2.00 root table:t4 handle:[1 3], keep order:false, desc:false",
1379+
"├─TableReader(Build) 2.00 root partition:p0 data:Selection",
1380+
"│ └─Selection 2.00 cop[tikv] not(isnull(default_partition.t4.id))",
1381+
"│ └─TableRangeScan 2.00 cop[tikv] table:t4 range:[1,1], [3,3], keep order:false",
13781382
"└─IndexLookUp(Probe) 3.69 root partition:p0,p1 ",
13791383
" ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t5, index:a(a, b) range:[1 1,1 1], [1 6,1 6], [6 1,6 1], [6 6,6 6], keep order:false",
13801384
" └─Selection(Probe) 3.69 cop[tikv] not(isnull(default_partition.t5.id))",
@@ -1388,10 +1392,12 @@
13881392
],
13891393
"Plan": [
13901394
"HashJoin 2.00 root inner join, equal:[eq(default_partition.t4.id, default_partition.t4.id)]",
1391-
"├─Selection(Build) 2.00 root not(isnull(default_partition.t4.id))",
1392-
"│ └─Batch_Point_Get 2.00 root table:t4 handle:[1 4], keep order:false, desc:false",
1393-
"└─Selection(Probe) 4.00 root not(isnull(default_partition.t4.id))",
1394-
" └─Batch_Point_Get 4.00 root table:t4 handle:[1 3 9 100], keep order:false, desc:false"
1395+
"├─TableReader(Build) 2.00 root partition:p0 data:Selection",
1396+
"│ └─Selection 2.00 cop[tikv] not(isnull(default_partition.t4.id))",
1397+
"│ └─TableRangeScan 2.00 cop[tikv] table:t1 range:[1,1], [4,4], keep order:false",
1398+
"└─TableReader(Probe) 4.00 root partition:p0,p1 data:Selection",
1399+
" └─Selection 4.00 cop[tikv] not(isnull(default_partition.t4.id))",
1400+
" └─TableRangeScan 4.00 cop[tikv] table:t2 range:[1,1], [3,3], [9,9], [100,100], keep order:false"
13951401
]
13961402
},
13971403
{

planner/core/find_best_task.go

Lines changed: 16 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -1136,12 +1136,19 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
11361136
}
11371137
}
11381138
}
1139-
var partColName *model.CIStr
1139+
var hashPartColName *model.CIStr
11401140
if tblInfo := ds.table.Meta(); canConvertPointGet && tblInfo.GetPartitionInfo() != nil {
1141+
// partition table with dynamic prune not support batchPointGet
1142+
if canConvertPointGet && len(path.Ranges) > 1 && ds.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() {
1143+
canConvertPointGet = false
1144+
}
11411145
if canConvertPointGet && len(path.Ranges) > 1 {
1142-
// not support some complex situation, like `by HASH( col DIV 80 )` etc.
1143-
partColName = getPartitionColumnName(getPartitionExpr(ds.SCtx(), tblInfo), tblInfo)
1144-
if partColName == nil {
1146+
// We can only build batch point get for hash partitions on a simple column now. This is
1147+
// decided by the current implementation of `BatchPointGetExec::initialize()`, specifically,
1148+
// the `getPhysID()` function. Once we optimize that part, we can come back and enable
1149+
// BatchPointGet plan for more cases.
1150+
hashPartColName = getHashOrKeyPartitionColumnName(ds.SCtx(), tblInfo)
1151+
if hashPartColName == nil {
11451152
canConvertPointGet = false
11461153
}
11471154
}
@@ -1170,7 +1177,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
11701177
if len(path.Ranges) == 1 {
11711178
pointGetTask = ds.convertToPointGet(prop, candidate)
11721179
} else {
1173-
pointGetTask = ds.convertToBatchPointGet(prop, candidate, partColName)
1180+
pointGetTask = ds.convertToBatchPointGet(prop, candidate, hashPartColName)
11741181
}
11751182

11761183
// Batch/PointGet plans may be over-optimized, like `a>=1(?) and a<=1(?)` --> `a=1` --> PointGet(a=1).
@@ -2526,7 +2533,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida
25262533
return rTsk
25272534
}
25282535

2529-
func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, candidate *candidatePath, partColName *model.CIStr) (task task) {
2536+
func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, candidate *candidatePath, hashPartColName *model.CIStr) (task task) {
25302537
if !prop.IsSortItemEmpty() && !candidate.isMatchProp {
25312538
return invalidTask
25322539
}
@@ -2543,31 +2550,11 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca
25432550
TblInfo: ds.TableInfo(),
25442551
KeepOrder: !prop.IsSortItemEmpty(),
25452552
Columns: ds.Columns,
2553+
SinglePart: ds.isPartition,
2554+
PartTblID: ds.physicalTableID,
25462555
PartitionExpr: getPartitionExpr(ds.SCtx(), ds.TableInfo()),
25472556
}
2548-
if ds.isPartition {
2549-
// static prune
2550-
batchPointGetPlan.PartTblID = make([]int64, 1)
2551-
batchPointGetPlan.PartTblID[0] = ds.physicalTableID
2552-
} else if ds.tableInfo.GetPartitionInfo() != nil {
2553-
// dynamic prune
2554-
idxs, err := PartitionPruning(ds.SCtx(), ds.table.GetPartitionedTable(), ds.allConds, ds.partitionNames, ds.TblCols, ds.names)
2555-
if err != nil || len(idxs) == 0 {
2556-
return invalidTask
2557-
}
2558-
if idxs[0] != FullRange {
2559-
batchPointGetPlan.PartTblID = make([]int64, len(idxs))
2560-
for i, idx := range idxs {
2561-
batchPointGetPlan.PartTblID[i] = ds.tableInfo.GetPartitionInfo().Definitions[idx].ID
2562-
}
2563-
slices.Sort(batchPointGetPlan.PartTblID)
2564-
}
2565-
}
25662557
if batchPointGetPlan.KeepOrder {
2567-
// TODO: support keepOrder for partition table with dynamic pruning
2568-
if ds.TableInfo().GetPartitionInfo() != nil && ds.SCtx().GetSessionVars().StmtCtx.UseDynamicPruneMode {
2569-
return invalidTask
2570-
}
25712558
batchPointGetPlan.Desc = prop.SortItems[0].Desc
25722559
}
25732560
rTsk := &rootTask{}
@@ -2589,7 +2576,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca
25892576
batchPointGetPlan.IndexInfo = candidate.path.Index
25902577
batchPointGetPlan.IdxCols = candidate.path.IdxCols
25912578
batchPointGetPlan.IdxColLens = candidate.path.IdxColLens
2592-
batchPointGetPlan.PartitionColPos = getColumnPosInIndex(candidate.path.Index, partColName)
2579+
batchPointGetPlan.PartitionColPos = getColumnPosInIndex(candidate.path.Index, hashPartColName)
25932580
for _, ran := range candidate.path.Ranges {
25942581
batchPointGetPlan.IndexValues = append(batchPointGetPlan.IndexValues, ran.LowVal)
25952582
}

planner/core/point_get_plan.go

Lines changed: 42 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -329,8 +329,13 @@ type BatchPointGetPlan struct {
329329
Columns []*model.ColumnInfo
330330
cost float64
331331

332-
// PartTblID is the table IDs for the specific table partitions.
333-
PartTblID []int64
332+
// SinglePart indicates whether this BatchPointGetPlan is just for a single partition, instead of the whole partition table.
333+
// If the BatchPointGetPlan is built in fast path, this value is false; if the plan is generated in physical optimization for a partition,
334+
// this value would be true. This value would decide the behavior of BatchPointGetExec, i.e, whether to compute the table ID of the partition
335+
// on the fly.
336+
SinglePart bool
337+
// PartTblID is the table ID for the specific table partition.
338+
PartTblID int64
334339

335340
// required by cost model
336341
planCostInit bool
@@ -1887,53 +1892,39 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionExpr *tables.Partition
18871892
return 0, nil
18881893
}
18891894

1890-
partitionColName := getPartitionColumnName(partitionExpr, tbl)
1891-
if partitionColName == nil {
1892-
return 0, errors.Errorf("unsupported partition type in BatchGet")
1893-
}
1894-
1895-
return getColumnPosInIndex(idx, partitionColName), nil
1896-
}
1897-
1898-
func getPartitionColumnName(partitionExpr *tables.PartitionExpr, tbl *model.TableInfo) *model.CIStr {
1899-
if partitionExpr == nil {
1900-
return nil
1901-
}
1902-
1903-
pi := tbl.GetPartitionInfo()
19041895
var partitionColName model.CIStr
19051896
switch pi.Type {
19061897
case model.PartitionTypeHash:
19071898
col, ok := partitionExpr.OrigExpr.(*ast.ColumnNameExpr)
19081899
if !ok {
1909-
return nil
1900+
return 0, errors.Errorf("unsupported partition type in BatchGet")
19101901
}
19111902
partitionColName = col.Name.Name
19121903
case model.PartitionTypeKey:
19131904
if len(partitionExpr.KeyPartCols) != 1 {
1914-
return nil
1905+
return 0, errors.Errorf("unsupported partition type in BatchGet")
19151906
}
19161907
colInfo := findColNameByColID(tbl.Columns, partitionExpr.KeyPartCols[0])
19171908
partitionColName = colInfo.Name
19181909
case model.PartitionTypeRange:
19191910
// left range columns partition for future development
19201911
col, ok := partitionExpr.Expr.(*expression.Column)
19211912
if !(ok && len(pi.Columns) == 0) {
1922-
return nil
1913+
return 0, errors.Errorf("unsupported partition type in BatchGet")
19231914
}
19241915
colInfo := findColNameByColID(tbl.Columns, col)
19251916
partitionColName = colInfo.Name
19261917
case model.PartitionTypeList:
19271918
// left list columns partition for future development
19281919
locateExpr, ok := partitionExpr.ForListPruning.LocateExpr.(*expression.Column)
19291920
if !(ok && partitionExpr.ForListPruning.ColPrunes == nil) {
1930-
return nil
1921+
return 0, errors.Errorf("unsupported partition type in BatchGet")
19311922
}
19321923
colInfo := findColNameByColID(tbl.Columns, locateExpr)
19331924
partitionColName = colInfo.Name
19341925
}
19351926

1936-
return &partitionColName
1927+
return getColumnPosInIndex(idx, &partitionColName), nil
19371928
}
19381929

19391930
// getColumnPosInIndex gets the column's position in the index.
@@ -1966,6 +1957,36 @@ func getPartitionExpr(ctx sessionctx.Context, tbl *model.TableInfo) *tables.Part
19661957
return partTable.PartitionExpr()
19671958
}
19681959

1960+
func getHashOrKeyPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *model.CIStr {
1961+
pi := tbl.GetPartitionInfo()
1962+
if pi == nil {
1963+
return nil
1964+
}
1965+
if pi.Type != model.PartitionTypeHash && pi.Type != model.PartitionTypeKey {
1966+
return nil
1967+
}
1968+
is := ctx.GetInfoSchema().(infoschema.InfoSchema)
1969+
table, ok := is.TableByID(tbl.ID)
1970+
if !ok {
1971+
return nil
1972+
}
1973+
// PartitionExpr don't need columns and names for hash partition.
1974+
partitionExpr := table.(partitionTable).PartitionExpr()
1975+
if pi.Type == model.PartitionTypeKey {
1976+
// used to judge whether the key partition contains only one field
1977+
if len(pi.Columns) != 1 {
1978+
return nil
1979+
}
1980+
return &pi.Columns[0]
1981+
}
1982+
expr := partitionExpr.OrigExpr
1983+
col, ok := expr.(*ast.ColumnNameExpr)
1984+
if !ok {
1985+
return nil
1986+
}
1987+
return &col.Name.Name
1988+
}
1989+
19691990
func findColNameByColID(cols []*model.ColumnInfo, col *expression.Column) *model.ColumnInfo {
19701991
for _, c := range cols {
19711992
if c.ID == col.ID {

0 commit comments

Comments
 (0)