Skip to content
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

planner, executor: fix batch_point_get with specific table partitions #45748

Merged
merged 2 commits into from
Aug 3, 2023
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
19 changes: 11 additions & 8 deletions executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,7 @@ type BatchPointGetExec struct {
partExpr *tables.PartitionExpr
partPos int
planPhysIDs []int64
singlePart bool
partTblID int64
partTblID []int64
idxVals [][]types.Datum
txn kv.Transaction
lock bool
Expand Down Expand Up @@ -235,9 +234,11 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
}
}

// If this BatchPointGetExec is built only for the specific table partition, skip those filters not matching this partition.
if e.singlePart && e.partTblID != physID {
continue
// If this BatchPointGetExec is built only for the specific table partitions, skip those filters not matching those partitions.
if len(e.partTblID) >= 1 {
if _, found := slices.BinarySearch(e.partTblID, physID); !found {
continue
}
}
idxKey, err1 := EncodeUniqueIndexKey(e.Ctx(), e.tblInfo, e.idxInfo, idxVals, physID)
if err1 != nil && !kv.ErrNotExist.Equal(err1) {
Expand Down Expand Up @@ -379,9 +380,11 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
}
}
}
// If this BatchPointGetExec is built only for the specific table partition, skip those handles not matching this partition.
if e.singlePart && e.partTblID != tID {
continue
// If this BatchPointGetExec is built only for the specific table partitions, skip those handles not matching those partitions.
if len(e.partTblID) >= 1 {
if _, found := slices.BinarySearch(e.partTblID, tID); !found {
continue
}
}
key := tablecodec.EncodeRowKeyWithHandle(tID, handle)
keys = append(keys, key)
Expand Down
1 change: 0 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5217,7 +5217,6 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan
partExpr: plan.PartitionExpr,
partPos: plan.PartitionColPos,
planPhysIDs: plan.PartitionIDs,
singlePart: plan.SinglePart,
partTblID: plan.PartTblID,
columns: plan.Columns,
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,14 +247,23 @@
"select * from thash3 where a in (1,3) and 1 = 1",
"select * from thash3 where a in (1,3) and 1 = 1 order by a",
"select * from thash3 where a in (1,3) and 1 = 1 order by a desc",
"select * from thash3 partition(p0) where a in (1,4)",
"select * from thash3 partition(p1) where a in (2,4)",
"select * from thash3 partition(p0,p1) where a in (2,4)",
"select * from trange3 where a in (1,2) and 1 = 1",
"select * from trange3 where a in (1,3) and 1 = 1",
"select * from trange3 where a in (1,3) and 1 = 1 order by a",
"select * from trange3 where a in (1,3) and 1 = 1 order by a desc",
"select * from trange3 partition(p0) where a in (1,4)",
"select * from trange3 partition(p1) where a in (1,2)",
"select * from trange3 partition(p0,p1) where a in (1,2)",
"select * from tlist3 where a in (1,2) and 1 = 1",
"select * from tlist3 where a in (1,3) and 1 = 1",
"select * from tlist3 where a in (1,2) and 1 = 1 order by a",
"select * from tlist3 where a in (1,2) and 1 = 1 order by a desc"
"select * from tlist3 where a in (1,2) and 1 = 1 order by a desc",
"select * from tlist3 partition(p0) where a in (1,4)",
"select * from tlist3 partition(p1) where a in (1,2)",
"select * from tlist3 partition(p0,p1) where a in (1,2)"
]
},
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2267,6 +2267,42 @@
"1 0"
]
},
{
"SQL": "select * from thash3 partition(p0) where a in (1,4)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:thash3 handle:[1 4], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:thash3 handle:[1 4], keep order:false, desc:false"
],
"Result": [
"4 0"
]
},
{
"SQL": "select * from thash3 partition(p1) where a in (2,4)",
"DynamicPlan": [
"TableReader 2.00 root partition:dual data:TableRangeScan",
"└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[2,2], [4,4], keep order:false, stats:pseudo"
],
"StaticPlan": [
"TableDual 0.00 root rows:0"
],
"Result": null
},
{
"SQL": "select * from thash3 partition(p0,p1) where a in (2,4)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:thash3 handle:[2 4], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:thash3 handle:[2 4], keep order:false, desc:false"
],
"Result": [
"2 0",
"4 0"
]
},
{
"SQL": "select * from trange3 where a in (1,2) and 1 = 1",
"DynamicPlan": [
Expand Down Expand Up @@ -2329,6 +2365,42 @@
"1 0"
]
},
{
"SQL": "select * from trange3 partition(p0) where a in (1,4)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:trange3 handle:[1 4], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:trange3 handle:[1 4], keep order:false, desc:false"
],
"Result": [
"1 0"
]
},
{
"SQL": "select * from trange3 partition(p1) where a in (1,2)",
"DynamicPlan": [
"TableReader 2.00 root partition:dual data:TableRangeScan",
"└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [2,2], keep order:false, stats:pseudo"
],
"StaticPlan": [
"TableDual 0.00 root rows:0"
],
"Result": null
},
{
"SQL": "select * from trange3 partition(p0,p1) where a in (1,2)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:trange3 handle:[1 2], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:trange3 handle:[1 2], keep order:false, desc:false"
],
"Result": [
"1 0",
"2 0"
]
},
{
"SQL": "select * from tlist3 where a in (1,2) and 1 = 1",
"DynamicPlan": [
Expand Down Expand Up @@ -2384,6 +2456,42 @@
"2 0",
"1 0"
]
},
{
"SQL": "select * from tlist3 partition(p0) where a in (1,4)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:tlist3 handle:[1 4], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:tlist3 handle:[1 4], keep order:false, desc:false"
],
"Result": [
"1 0"
]
},
{
"SQL": "select * from tlist3 partition(p1) where a in (1,2)",
"DynamicPlan": [
"TableReader 2.00 root partition:dual data:TableRangeScan",
"└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:false, stats:pseudo"
],
"StaticPlan": [
"TableDual 0.00 root rows:0"
],
"Result": null
},
{
"SQL": "select * from tlist3 partition(p0,p1) where a in (1,2)",
"DynamicPlan": [
"Batch_Point_Get 2.00 root table:tlist3 handle:[1 2], keep order:false, desc:false"
],
"StaticPlan": [
"Batch_Point_Get 2.00 root table:tlist3 handle:[1 2], keep order:false, desc:false"
],
"Result": [
"1 0",
"2 0"
]
}
]
},
Expand Down
20 changes: 18 additions & 2 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -2543,10 +2543,26 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca
TblInfo: ds.TableInfo(),
KeepOrder: !prop.IsSortItemEmpty(),
Columns: ds.Columns,
SinglePart: ds.isPartition,
PartTblID: ds.physicalTableID,
PartitionExpr: getPartitionExpr(ds.SCtx(), ds.TableInfo()),
}
if ds.isPartition {
// static prune
batchPointGetPlan.PartTblID = make([]int64, 1)
batchPointGetPlan.PartTblID[0] = ds.physicalTableID
} else if ds.tableInfo.GetPartitionInfo() != nil {
// dynamic prune
idxs, err := PartitionPruning(ds.SCtx(), ds.table.GetPartitionedTable(), ds.allConds, ds.partitionNames, ds.TblCols, ds.names)
if err != nil || len(idxs) == 0 {
return invalidTask
}
if idxs[0] != FullRange {
batchPointGetPlan.PartTblID = make([]int64, len(idxs))
for i, idx := range idxs {
batchPointGetPlan.PartTblID[i] = ds.tableInfo.GetPartitionInfo().Definitions[idx].ID
}
slices.Sort(batchPointGetPlan.PartTblID)
}
}
if batchPointGetPlan.KeepOrder {
// TODO: support keepOrder for partition table with dynamic pruning
if ds.TableInfo().GetPartitionInfo() != nil && ds.SCtx().GetSessionVars().StmtCtx.UseDynamicPruneMode {
Expand Down
9 changes: 2 additions & 7 deletions planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -329,13 +329,8 @@ type BatchPointGetPlan struct {
Columns []*model.ColumnInfo
cost float64

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

// required by cost model
planCostInit bool
Expand Down