diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 9516604c812e0..c5df402de3126 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -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 @@ -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. @@ -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 @@ -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 @@ -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 diff --git a/pkg/planner/core/plan_clone_generated.go b/pkg/planner/core/plan_clone_generated.go index abe78ba7f0bf4..8630835ca39c2 100644 --- a/pkg/planner/core/plan_clone_generated.go +++ b/pkg/planner/core/plan_clone_generated.go @@ -180,3 +180,104 @@ func (op *PhysicalMergeJoin) CloneForPlanCache(newCtx base.PlanContext) (base.Pl cloned.basePhysicalJoin = *basePlan return cloned, true } + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalTableReader) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalTableReader) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + cloned.tablePlan = tablePlan.(base.PhysicalPlan) + cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan) + cloned.PlanPartInfo = op.PlanPartInfo.Clone() + if op.TableScanAndPartitionInfos != nil { + return nil, false + } + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalIndexReader) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalIndexReader) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + cloned.indexPlan = indexPlan.(base.PhysicalPlan) + cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan) + cloned.OutputColumns = util.CloneCols(op.OutputColumns) + cloned.PlanPartInfo = op.PlanPartInfo.Clone() + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalIndexLookUpReader) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalIndexLookUpReader) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + cloned.indexPlan = indexPlan.(base.PhysicalPlan) + tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + cloned.tablePlan = tablePlan.(base.PhysicalPlan) + cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan) + cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan) + cloned.ExtraHandleCol = op.ExtraHandleCol.Clone().(*expression.Column) + cloned.PushedLimit = op.PushedLimit.Clone() + cloned.CommonHandleCols = util.CloneCols(op.CommonHandleCols) + cloned.PlanPartInfo = op.PlanPartInfo.Clone() + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalIndexMergeReader) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalIndexMergeReader) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + cloned.PushedLimit = op.PushedLimit.Clone() + cloned.ByItems = util.CloneByItems(op.ByItems) + partialPlans, ok := clonePhysicalPlansForPlanCache(newCtx, op.partialPlans) + if !ok { + return nil, false + } + cloned.partialPlans = partialPlans + tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + cloned.tablePlan = tablePlan.(base.PhysicalPlan) + cloned.PartialPlans = make([][]base.PhysicalPlan, len(op.PartialPlans)) + for i, plan := range cloned.partialPlans { + cloned.PartialPlans[i] = flattenPushDownPlan(plan) + } + cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan) + cloned.PlanPartInfo = op.PlanPartInfo.Clone() + cloned.HandleCols = op.HandleCols.Clone(newCtx.GetSessionVars().StmtCtx) + return cloned, true +} diff --git a/pkg/planner/core/plan_clone_generator.go b/pkg/planner/core/plan_clone_generator.go index c9059e6e3140a..6ade146509232 100644 --- a/pkg/planner/core/plan_clone_generator.go +++ b/pkg/planner/core/plan_clone_generator.go @@ -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. @@ -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 { @@ -62,6 +65,24 @@ 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) @@ -69,9 +90,7 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) { 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) @@ -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()) } @@ -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" }