Skip to content

Commit

Permalink
planner: use code-gen to generate CloneForPlanCache method for some o…
Browse files Browse the repository at this point in the history
…perators (pingcap#54957)

ref pingcap#54057
  • Loading branch information
qw4990 authored and hawkingrei committed Aug 1, 2024
1 parent cc7316d commit 8c0d393
Show file tree
Hide file tree
Showing 3 changed files with 154 additions and 14 deletions.
18 changes: 9 additions & 9 deletions pkg/planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,8 +129,8 @@ type PhysicalTableReader struct {
physicalSchemaProducer

// TablePlans flats the tablePlan to construct executor pb.
TablePlans []base.PhysicalPlan
tablePlan base.PhysicalPlan
TablePlans []base.PhysicalPlan

// StoreType indicates table read from which type of store.
StoreType kv.StoreType
Expand All @@ -144,7 +144,7 @@ type PhysicalTableReader struct {
// Used by partition table.
PlanPartInfo *PhysPlanPartInfo
// Used by MPP, because MPP plan may contain join/union/union all, it is possible that a physical table reader contains more than 1 table scan
TableScanAndPartitionInfos []tableScanAndPartitionInfo
TableScanAndPartitionInfos []tableScanAndPartitionInfo `plan-cache-clone:"must-nil"`
}

// PhysPlanPartInfo indicates partition helper info in physical plan.
Expand Down Expand Up @@ -302,8 +302,8 @@ type PhysicalIndexReader struct {
physicalSchemaProducer

// IndexPlans flats the indexPlan to construct executor pb.
IndexPlans []base.PhysicalPlan
indexPlan base.PhysicalPlan
IndexPlans []base.PhysicalPlan

// OutputColumns represents the columns that index reader should return.
OutputColumns []*expression.Column
Expand Down Expand Up @@ -431,12 +431,12 @@ func (p *PushedDownLimit) MemoryUsage() (sum int64) {
type PhysicalIndexLookUpReader struct {
physicalSchemaProducer

indexPlan base.PhysicalPlan
tablePlan base.PhysicalPlan
// IndexPlans flats the indexPlan to construct executor pb.
IndexPlans []base.PhysicalPlan
// TablePlans flats the tablePlan to construct executor pb.
TablePlans []base.PhysicalPlan
indexPlan base.PhysicalPlan
tablePlan base.PhysicalPlan
Paging bool

ExtraHandleCol *expression.Column
Expand Down Expand Up @@ -582,14 +582,14 @@ type PhysicalIndexMergeReader struct {
// ByItems is used to support sorting the handles returned by partialPlans.
ByItems []*util.ByItems

// PartialPlans flats the partialPlans to construct executor pb.
PartialPlans [][]base.PhysicalPlan
// TablePlans flats the tablePlan to construct executor pb.
TablePlans []base.PhysicalPlan
// partialPlans are the partial plans that have not been flatted. The type of each element is permitted PhysicalIndexScan or PhysicalTableScan.
partialPlans []base.PhysicalPlan
// tablePlan is a PhysicalTableScan to get the table tuples. Current, it must be not nil.
tablePlan base.PhysicalPlan
// PartialPlans flats the partialPlans to construct executor pb.
PartialPlans [][]base.PhysicalPlan
// TablePlans flats the tablePlan to construct executor pb.
TablePlans []base.PhysicalPlan

// Used by partition table.
PlanPartInfo *PhysPlanPartInfo
Expand Down
101 changes: 101 additions & 0 deletions pkg/planner/core/plan_clone_generated.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

49 changes: 44 additions & 5 deletions pkg/planner/core/plan_clone_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"go/format"
"reflect"
"strings"

"github.com/pingcap/tidb/pkg/planner/core/base"
)

// genPlanCloneForPlanCacheCode generates CloneForPlanCache for all physical plan nodes in plan_clone_generated.go.
Expand All @@ -33,7 +35,8 @@ import (
func genPlanCloneForPlanCacheCode() ([]byte, error) {
var structures = []any{PhysicalTableScan{}, PhysicalIndexScan{}, PhysicalSelection{}, PhysicalProjection{},
PhysicalSort{}, PhysicalTopN{}, PhysicalStreamAgg{}, PhysicalHashAgg{},
PhysicalHashJoin{}, PhysicalMergeJoin{}}
PhysicalHashJoin{}, PhysicalMergeJoin{}, PhysicalTableReader{}, PhysicalIndexReader{},
PhysicalIndexLookUpReader{}, PhysicalIndexMergeReader{}}
c := new(codeGen)
c.write(codeGenPrefix)
for _, s := range structures {
Expand Down Expand Up @@ -62,16 +65,32 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
c.write(`if op.%v != nil {return nil, false}`, f.Name)
continue
}

fullFieldName := fmt.Sprintf("%v.%v", vType.String(), vType.Field(i).Name)
switch fullFieldName { // handle some fields specially
case "core.PhysicalTableReader.TablePlans", "core.PhysicalIndexLookUpReader.TablePlans",
"core.PhysicalIndexMergeReader.TablePlans":
c.write("cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan)")
continue
case "core.PhysicalIndexReader.IndexPlans", "core.PhysicalIndexLookUpReader.IndexPlans":
c.write("cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan)")
continue
case "core.PhysicalIndexMergeReader.PartialPlans":
c.write("cloned.PartialPlans = make([][]base.PhysicalPlan, len(op.PartialPlans))")
c.write("for i, plan := range cloned.partialPlans {")
c.write("cloned.PartialPlans[i] = flattenPushDownPlan(plan)")
c.write("}")
continue
}

switch f.Type.String() {
case "[]int", "[]byte", "[]float", "[]bool": // simple slice
c.write("cloned.%v = make(%v, len(op.%v))", f.Name, f.Type, f.Name)
c.write("copy(cloned.%v, op.%v)", f.Name, f.Name)
case "core.physicalSchemaProducer", "core.basePhysicalPlan", "core.basePhysicalAgg", "core.basePhysicalJoin":
fieldName := strings.Split(f.Type.String(), ".")[1]
c.write(`basePlan, baseOK := op.%v.cloneForPlanCacheWithSelf(newCtx, cloned)
if !baseOK {
return nil, false
}
if !baseOK {return nil, false}
cloned.%v = *basePlan`, fieldName, fieldName)
case "[]expression.Expression":
c.write("cloned.%v = util.CloneExprs(op.%v)", f.Name, f.Name)
Expand All @@ -87,10 +106,18 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
c.write("cloned.%v = util.CloneSortItem(op.%v)", f.Name, f.Name)
case "util.HandleCols":
c.write("cloned.%v = op.%v.Clone(newCtx.GetSessionVars().StmtCtx)", f.Name, f.Name)
case "*core.PhysPlanPartInfo":
case "*core.PhysPlanPartInfo", "*core.PushedDownLimit":
c.write("cloned.%v = op.%v.Clone()", f.Name, f.Name)
case "*expression.Column":
c.write("cloned.%v = op.%v.Clone().(*expression.Column)", f.Name, f.Name)
case "base.PhysicalPlan":
c.write("%v, ok := op.%v.CloneForPlanCache(newCtx)", f.Name, f.Name)
c.write("if !ok {return nil, false}")
c.write("cloned.%v = %v.(base.PhysicalPlan)", f.Name, f.Name)
case "[]base.PhysicalPlan":
c.write("%v, ok := clonePhysicalPlansForPlanCache(newCtx, op.%v)", f.Name, f.Name)
c.write("if !ok {return nil, false}")
c.write("cloned.%v = %v", f.Name, f.Name)
default:
return nil, fmt.Errorf("can't generate Clone method for type %v in %v", f.Type.String(), vType.String())
}
Expand All @@ -100,6 +127,18 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
return c.format()
}

func clonePhysicalPlansForPlanCache(newCtx base.PlanContext, plans []base.PhysicalPlan) ([]base.PhysicalPlan, bool) {
clonedPlans := make([]base.PhysicalPlan, len(plans))
for i, plan := range plans {
cloned, ok := plan.CloneForPlanCache(newCtx)
if !ok {
return nil, false
}
clonedPlans[i] = cloned.(base.PhysicalPlan)
}
return clonedPlans, true
}

func mustNilField(fType reflect.StructField) bool {
return fType.Tag.Get("plan-cache-clone") == "must-nil"
}
Expand Down

0 comments on commit 8c0d393

Please sign in to comment.