From a453b814587c2eec8ca40007031dba20add30ded Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 15 Jul 2024 15:50:58 +0800 Subject: [PATCH] planner: use code-gen to generate CloneForPlanCache method for some operators (#54601) ref pingcap/tidb#54057 --- build/nogo_config.json | 1 + pkg/expression/schema.go | 3 + pkg/planner/core/BUILD.bazel | 8 +- pkg/planner/core/physical_plans.go | 40 ++--- pkg/planner/core/plan.go | 35 ++++- pkg/planner/core/plan_cache_rebuild_test.go | 6 +- pkg/planner/core/plan_clone_generated.go | 102 +++++++++++++ pkg/planner/core/plan_clone_generator.go | 156 ++++++++++++++++++++ pkg/planner/core/plan_clone_test.go | 49 ++++++ pkg/planner/core/util.go | 11 ++ pkg/planner/util/misc.go | 12 ++ 11 files changed, 397 insertions(+), 26 deletions(-) create mode 100644 pkg/planner/core/plan_clone_generated.go create mode 100644 pkg/planner/core/plan_clone_generator.go create mode 100644 pkg/planner/core/plan_clone_test.go diff --git a/build/nogo_config.json b/build/nogo_config.json index 2b10896859ac3..898b4ba746849 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -522,6 +522,7 @@ "pkg/parser/goyacc/": "ignore goyacc code", "pkg/parser/ast/": "ignore parser/ast code", "pkg/parser/test_driver/": "ignore parser/test_driver code", + "pkg/planner/core/plan_clone_generated.go": "ignore plan_clone_generated code", ".*_test\\.go$": "ignore generated code", ".*_generated\\.go$": "ignore generated code", ".*mock.go$": "ignore generated code", diff --git a/pkg/expression/schema.go b/pkg/expression/schema.go index 12b1cee89f700..3d915c8434742 100644 --- a/pkg/expression/schema.go +++ b/pkg/expression/schema.go @@ -67,6 +67,9 @@ func (s *Schema) String() string { // Clone copies the total schema. func (s *Schema) Clone() *Schema { + if s == nil { + return nil + } cols := make([]*Column, 0, s.Len()) keys := make([]KeyInfo, 0, len(s.Keys)) for _, col := range s.Columns { diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 140078bcc9094..894eddd2983ed 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -64,6 +64,8 @@ go_library( "plan_cache_rebuild.go", "plan_cache_utils.go", "plan_cacheable_checker.go", + "plan_clone_generated.go", + "plan_clone_generator.go", "plan_cost_detail.go", "plan_cost_ver1.go", "plan_cost_ver2.go", @@ -256,6 +258,7 @@ go_test( "plan_cache_rebuild_test.go", "plan_cache_test.go", "plan_cacheable_checker_test.go", + "plan_clone_test.go", "plan_cost_detail_test.go", "plan_cost_ver1_test.go", "plan_cost_ver2_test.go", @@ -271,7 +274,10 @@ go_test( "stringer_test.go", "util_test.go", ], - data = glob(["testdata/**"]), + data = glob([ + "testdata/**", + "*.go", + ]), embed = [":core"], flaky = True, shard_count = 50, diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index e9aeca34becab..f82b386c3ca7d 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -680,26 +680,26 @@ type PhysicalIndexScan struct { // AccessCondition is used to calculate range. AccessCondition []expression.Expression - Table *model.TableInfo - Index *model.IndexInfo + Table *model.TableInfo `plan-cache-clone:"shallow"` // please see comment on genPlanCloneForPlanCacheCode. + Index *model.IndexInfo `plan-cache-clone:"shallow"` IdxCols []*expression.Column IdxColLens []int Ranges []*ranger.Range - Columns []*model.ColumnInfo - DBName model.CIStr + Columns []*model.ColumnInfo `plan-cache-clone:"shallow"` + DBName model.CIStr `plan-cache-clone:"shallow"` - TableAsName *model.CIStr + TableAsName *model.CIStr `plan-cache-clone:"shallow"` // dataSourceSchema is the original schema of DataSource. The schema of index scan in KV and index reader in TiDB // will be different. The schema of index scan will decode all columns of index but the TiDB only need some of them. - dataSourceSchema *expression.Schema + dataSourceSchema *expression.Schema `plan-cache-clone:"shallow"` rangeInfo string // The index scan may be on a partition. physicalTableID int64 - GenExprs map[model.TableItemID]expression.Expression `json:"-"` + GenExprs map[model.TableItemID]expression.Expression `plan-cache-clone:"must-nil"` isPartition bool Desc bool @@ -715,18 +715,18 @@ type PhysicalIndexScan struct { // required by cost model // tblColHists contains all columns before pruning, which are used to calculate row-size - tblColHists *statistics.HistColl + tblColHists *statistics.HistColl `plan-cache-clone:"shallow"` pkIsHandleCol *expression.Column // constColsByCond records the constant part of the index columns caused by the access conds. // e.g. the index is (a, b, c) and there's filter a = 1 and b = 2, then the column a and b are const part. constColsByCond []bool - prop *property.PhysicalProperty + prop *property.PhysicalProperty `plan-cache-clone:"shallow"` // usedStatsInfo records stats status of this physical table. // It's for printing stats related information when display execution plan. - usedStatsInfo *stmtctx.UsedStatsInfoForTable + usedStatsInfo *stmtctx.UsedStatsInfoForTable `plan-cache-clone:"shallow"` } // Clone implements op.PhysicalPlan interface. @@ -857,12 +857,12 @@ type PhysicalTableScan struct { // TODO: remove this field after we support pushing down selection to coprocessor. LateMaterializationFilterCondition []expression.Expression - Table *model.TableInfo - Columns []*model.ColumnInfo - DBName model.CIStr + Table *model.TableInfo `plan-cache-clone:"shallow"` + Columns []*model.ColumnInfo `plan-cache-clone:"shallow"` + DBName model.CIStr `plan-cache-clone:"shallow"` Ranges []*ranger.Range - TableAsName *model.CIStr + TableAsName *model.CIStr `plan-cache-clone:"shallow"` physicalTableID int64 @@ -890,13 +890,13 @@ type PhysicalTableScan struct { PlanPartInfo *PhysPlanPartInfo - SampleInfo *tablesampler.TableSampleInfo + SampleInfo *tablesampler.TableSampleInfo `plan-cache-clone:"must-nil"` // required by cost model // tblCols and tblColHists contains all columns before pruning, which are used to calculate row-size - tblCols []*expression.Column - tblColHists *statistics.HistColl - prop *property.PhysicalProperty + tblCols []*expression.Column `plan-cache-clone:"shallow"` + tblColHists *statistics.HistColl `plan-cache-clone:"shallow"` + prop *property.PhysicalProperty `plan-cache-clone:"shallow"` // constColsByCond records the constant part of the index columns caused by the access conds. // e.g. the index is (a, b, c) and there's filter a = 1 and b = 2, then the column a and b are const part. @@ -905,10 +905,10 @@ type PhysicalTableScan struct { // usedStatsInfo records stats status of this physical table. // It's for printing stats related information when display execution plan. - usedStatsInfo *stmtctx.UsedStatsInfoForTable + usedStatsInfo *stmtctx.UsedStatsInfoForTable `plan-cache-clone:"shallow"` // for runtime filter - runtimeFilterList []*RuntimeFilter + runtimeFilterList []*RuntimeFilter `plan-cache-clone:"must-nil"` // plan with runtime filter is not cached maxWaitTimeMs int } diff --git a/pkg/planner/core/plan.go b/pkg/planner/core/plan.go index b77f6492c3b0f..e37f0cda97453 100644 --- a/pkg/planner/core/plan.go +++ b/pkg/planner/core/plan.go @@ -249,14 +249,14 @@ func getActualProbeCntFromProbeParents(pps []base.PhysicalPlan, statsColl *execd type basePhysicalPlan struct { baseimpl.Plan - childrenReqProps []*property.PhysicalProperty + childrenReqProps []*property.PhysicalProperty `plan-cache-clone:"shallow"` self base.PhysicalPlan children []base.PhysicalPlan // used by the new cost interface planCostInit bool planCost float64 - planCostVer2 costusage.CostVer2 + planCostVer2 costusage.CostVer2 `plan-cache-clone:"shallow"` // probeParents records the IndexJoins and Applys with this operator in their inner children. // Please see comments in op.PhysicalPlan for details. @@ -268,6 +268,37 @@ type basePhysicalPlan struct { TiFlashFineGrainedShuffleStreamCount uint64 } +func (p *basePhysicalPlan) cloneForPlanCacheWithSelf(newCtx base.PlanContext, newSelf base.PhysicalPlan) (*basePhysicalPlan, bool) { + cloned := new(basePhysicalPlan) + *cloned = *p + cloned.SetSCtx(newCtx) + cloned.self = newSelf + cloned.children = make([]base.PhysicalPlan, 0, len(p.children)) + for _, child := range p.children { + clonedChild, ok := child.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + clonedPP, ok := clonedChild.(base.PhysicalPlan) + if !ok { + return nil, false + } + cloned.children = append(cloned.children, clonedPP) + } + for _, probe := range p.probeParents { + clonedProbe, ok := probe.CloneForPlanCache(newCtx) + if !ok { + return nil, false + } + clonedPP, ok := clonedProbe.(base.PhysicalPlan) + if !ok { + return nil, false + } + cloned.probeParents = append(cloned.probeParents, clonedPP) + } + return cloned, true +} + func (p *basePhysicalPlan) cloneWithSelf(newCtx base.PlanContext, newSelf base.PhysicalPlan) (*basePhysicalPlan, error) { base := &basePhysicalPlan{ Plan: p.Plan, diff --git a/pkg/planner/core/plan_cache_rebuild_test.go b/pkg/planner/core/plan_cache_rebuild_test.go index 0f6b79d7385d0..a11229ba2f8e4 100644 --- a/pkg/planner/core/plan_cache_rebuild_test.go +++ b/pkg/planner/core/plan_cache_rebuild_test.go @@ -153,7 +153,7 @@ func TestCheckPlanClone(t *testing.T) { // test tag type S struct { - p1 *int `plan-cache-shallow-clone:"true"` + p1 *int `plan-cache-clone:"shallow"` p2 *int } s1 := new(S) @@ -255,8 +255,8 @@ func planCacheUnclearCloneCheck(v1, v2 reflect.Value, path string, visited map[v return planCacheUnclearCloneCheck(v1.Elem(), v2.Elem(), path, visited) case reflect.Struct: for i, n := 0, v1.NumField(); i < n; i++ { - tag := v1.Type().Field(i).Tag.Get("plan-cache-shallow-clone") - if tag == "true" { + tag := v1.Type().Field(i).Tag.Get("plan-cache-clone") + if tag == "shallow" { continue } fieldName := v1.Type().Field(i).Name diff --git a/pkg/planner/core/plan_clone_generated.go b/pkg/planner/core/plan_clone_generated.go new file mode 100644 index 0000000000000..0bbbef21d8fcd --- /dev/null +++ b/pkg/planner/core/plan_clone_generated.go @@ -0,0 +1,102 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by plan_clone_generator; DO NOT EDIT IT DIRECTLY. + +package core + +import ( + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util" +) + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalTableScan) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalTableScan) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + cloned.AccessCondition = util.CloneExprs(op.AccessCondition) + cloned.filterCondition = util.CloneExprs(op.filterCondition) + cloned.LateMaterializationFilterCondition = util.CloneExprs(op.LateMaterializationFilterCondition) + cloned.Ranges = util.CloneRanges(op.Ranges) + cloned.HandleIdx = make([]int, len(op.HandleIdx)) + copy(cloned.HandleIdx, op.HandleIdx) + cloned.HandleCols = op.HandleCols.Clone(newCtx.GetSessionVars().StmtCtx) + cloned.ByItems = util.CloneByItems(op.ByItems) + cloned.PlanPartInfo = op.PlanPartInfo.Clone() + if op.SampleInfo != nil { + return nil, false + } + cloned.constColsByCond = make([]bool, len(op.constColsByCond)) + copy(cloned.constColsByCond, op.constColsByCond) + if op.runtimeFilterList != nil { + return nil, false + } + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalIndexScan) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalIndexScan) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + cloned.AccessCondition = util.CloneExprs(op.AccessCondition) + cloned.IdxCols = util.CloneCols(op.IdxCols) + cloned.IdxColLens = make([]int, len(op.IdxColLens)) + copy(cloned.IdxColLens, op.IdxColLens) + cloned.Ranges = util.CloneRanges(op.Ranges) + if op.GenExprs != nil { + return nil, false + } + cloned.ByItems = util.CloneByItems(op.ByItems) + cloned.pkIsHandleCol = op.pkIsHandleCol.Clone().(*expression.Column) + cloned.constColsByCond = make([]bool, len(op.constColsByCond)) + copy(cloned.constColsByCond, op.constColsByCond) + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalSelection) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalSelection) + *cloned = *op + basePlan, baseOK := op.basePhysicalPlan.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.basePhysicalPlan = *basePlan + cloned.Conditions = util.CloneExprs(op.Conditions) + return cloned, true +} + +// CloneForPlanCache implements the base.Plan interface. +func (op *PhysicalProjection) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) { + cloned := new(PhysicalProjection) + *cloned = *op + basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan + cloned.Exprs = util.CloneExprs(op.Exprs) + return cloned, true +} diff --git a/pkg/planner/core/plan_clone_generator.go b/pkg/planner/core/plan_clone_generator.go new file mode 100644 index 0000000000000..6196a08ff5aa1 --- /dev/null +++ b/pkg/planner/core/plan_clone_generator.go @@ -0,0 +1,156 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "bytes" + "fmt" + "go/format" + "reflect" +) + +// genPlanCloneForPlanCacheCode generates CloneForPlanCache for all physical plan nodes in plan_clone_generated.go. +// Using code-gen is safer than writing by hand, for example, if someone adds a new field to a struct, +// the code-gen can update the Clone method correctly and automatically. +// To update plan_clone_generated.go, please run TestUpdatePlanCloneCode manually. +// This function relies on Golang field tags to determine whether to shallow clone a field or not. +// If a field is tagged with `plan-cache-clone:"shallow"`, then it will be shallow cloned. +// If a field is tagged with `plan-cache-clone:"must-nil"`, then it will be checked for nil before cloning. +// If a field is not tagged, then it will be deep cloned. +func genPlanCloneForPlanCacheCode() ([]byte, error) { + var structures = []any{PhysicalTableScan{}, PhysicalIndexScan{}, PhysicalSelection{}, PhysicalProjection{}} + c := new(codeGen) + c.write(codeGenPrefix) + for _, s := range structures { + code, err := genPlanCloneForPlanCache(s) + if err != nil { + return nil, err + } + c.write(string(code)) + } + return c.format() +} + +func genPlanCloneForPlanCache(x any) ([]byte, error) { + c := new(codeGen) + vType := reflect.TypeOf(x) + c.write("// CloneForPlanCache implements the base.Plan interface.") + c.write("func (op *%v) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) {", vType.Name()) + c.write("cloned := new(%v)", vType.Name()) + c.write("*cloned = *op") + for i := 0; i < vType.NumField(); i++ { + f := vType.Field(i) + if allowShallowClone(f) { + continue + } + if mustNilField(f) { + c.write(`if op.%v != nil {return nil, false}`, f.Name) + 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": + c.write(`basePlan, baseOK := op.physicalSchemaProducer.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.physicalSchemaProducer = *basePlan`) + case "core.basePhysicalPlan": + c.write(`basePlan, baseOK := op.basePhysicalPlan.cloneForPlanCacheWithSelf(newCtx, cloned) + if !baseOK { + return nil, false + } + cloned.basePhysicalPlan = *basePlan`) + case "[]expression.Expression": + c.write("cloned.%v = util.CloneExprs(op.%v)", f.Name, f.Name) + case "[]*ranger.Range": + c.write("cloned.%v = util.CloneRanges(op.%v)", f.Name, f.Name) + case "[]*util.ByItems": + c.write("cloned.%v = util.CloneByItems(op.%v)", f.Name, f.Name) + case "[]*expression.Column": + c.write("cloned.%v = util.CloneCols(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": + 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) + default: + return nil, fmt.Errorf("can't generate Clone method for type: %v", f.Type.String()) + } + } + c.write("return cloned, true") + c.write("}") + return c.format() +} + +func mustNilField(fType reflect.StructField) bool { + return fType.Tag.Get("plan-cache-clone") == "must-nil" +} + +func allowShallowClone(fType reflect.StructField) bool { + if fType.Tag.Get("plan-cache-clone") == "shallow" { + return true // allow shallow clone for this field + } + switch fType.Type.Kind() { + case reflect.Bool, reflect.Int, reflect.Int8, reflect.Int16, + reflect.Int32, reflect.Int64, reflect.Uint, reflect.Uint8, + reflect.Uint16, reflect.Uint32, reflect.Uint64, + reflect.Float32, reflect.Float64, reflect.String: + return true + default: + return false + } +} + +type codeGen struct { + buffer bytes.Buffer +} + +func (c *codeGen) write(format string, args ...any) { + c.buffer.WriteString(fmt.Sprintf(format, args...)) + c.buffer.WriteString("\n") +} + +func (c *codeGen) format() ([]byte, error) { + return format.Source(c.buffer.Bytes()) +} + +const codeGenPrefix = `// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by plan_clone_generator; DO NOT EDIT IT DIRECTLY. + +package core + +import ( + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util" +) +` diff --git a/pkg/planner/core/plan_clone_test.go b/pkg/planner/core/plan_clone_test.go new file mode 100644 index 0000000000000..c34290ec47d0a --- /dev/null +++ b/pkg/planner/core/plan_clone_test.go @@ -0,0 +1,49 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "bytes" + "os" + "testing" +) + +func TestPlanClone(t *testing.T) { + updatedCode, err := genPlanCloneForPlanCacheCode() + if err != nil { + t.Errorf("Generate CloneForPlanCache code error: %v", err) + return + } + currentCode, err := os.ReadFile("./plan_clone_generated.go") + if err != nil { + t.Errorf("Read current plan_clone_generated.go code error: %v", err) + return + } + if !bytes.Equal(updatedCode, currentCode) { + t.Errorf("plan_clone_generated.go should be updated, please run TestUpdatePlanCloneCode manually to update it.") + } +} + +func TestUpdatePlanCloneCode(t *testing.T) { + t.Skip("only run this test manually to update plan_clone_generated.go") + updatedCode, err := genPlanCloneForPlanCacheCode() + if err != nil { + t.Errorf("Generate CloneForPlanCache code error: %v", err) + return + } + if err := os.WriteFile("./plan_clone_generated.go", updatedCode, 0644); err != nil { + t.Errorf("Write plan_clone_generated.go error: %v", err) + } +} diff --git a/pkg/planner/core/util.go b/pkg/planner/core/util.go index 4db933333d889..1a2af6694bc4a 100644 --- a/pkg/planner/core/util.go +++ b/pkg/planner/core/util.go @@ -98,6 +98,17 @@ type physicalSchemaProducer struct { basePhysicalPlan } +func (s *physicalSchemaProducer) cloneForPlanCacheWithSelf(newCtx base.PlanContext, newSelf base.PhysicalPlan) (*physicalSchemaProducer, bool) { + cloned := new(physicalSchemaProducer) + cloned.schema = s.Schema().Clone() + base, ok := s.basePhysicalPlan.cloneForPlanCacheWithSelf(newCtx, newSelf) + if !ok { + return nil, false + } + cloned.basePhysicalPlan = *base + return cloned, true +} + func (s *physicalSchemaProducer) cloneWithSelf(newCtx base.PlanContext, newSelf base.PhysicalPlan) (*physicalSchemaProducer, error) { base, err := s.basePhysicalPlan.cloneWithSelf(newCtx, newSelf) if err != nil { diff --git a/pkg/planner/util/misc.go b/pkg/planner/util/misc.go index 8083881c1acb1..ac939c8da351f 100644 --- a/pkg/planner/util/misc.go +++ b/pkg/planner/util/misc.go @@ -100,6 +100,18 @@ func CloneRanges(ranges []*ranger.Range) []*ranger.Range { return cloned } +// CloneByItems uses (*ByItems).Clone to clone a slice of *ByItems. +func CloneByItems(byItems []*ByItems) []*ByItems { + if byItems == nil { + return nil + } + cloned := make([]*ByItems, 0, len(byItems)) + for _, item := range byItems { + cloned = append(cloned, item.Clone()) + } + return cloned +} + // QueryTimeRange represents a time range specified by TIME_RANGE hint type QueryTimeRange struct { From time.Time