planner: use code-gen to generate CloneForPlanCache method for Insert/Update/Delete (#55185)
ref pingcap/tidb#54057
This commit is contained in:
@ -811,6 +811,16 @@ type Assignment struct {
|
||||
LazyErr error
|
||||
}
|
||||
|
||||
// Clone clones the Assignment.
|
||||
func (a *Assignment) Clone() *Assignment {
|
||||
return &Assignment{
|
||||
Col: a.Col.Clone().(*Column),
|
||||
ColName: a.ColName,
|
||||
Expr: a.Expr.Clone(),
|
||||
LazyErr: a.LazyErr,
|
||||
}
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of Assignment
|
||||
func (a *Assignment) MemoryUsage() (sum int64) {
|
||||
if a == nil {
|
||||
|
||||
@ -324,17 +324,24 @@ func (p *PhysicalSimpleWrapper) MemoryUsage() (sum int64) {
|
||||
// InsertGeneratedColumns is for completing generated columns in Insert.
|
||||
// We resolve generation expressions in plan, and eval those in executor.
|
||||
type InsertGeneratedColumns struct {
|
||||
Columns []*ast.ColumnName
|
||||
Exprs []expression.Expression
|
||||
OnDuplicates []*expression.Assignment
|
||||
}
|
||||
|
||||
// Copy clones InsertGeneratedColumns.
|
||||
func (i InsertGeneratedColumns) Copy() InsertGeneratedColumns {
|
||||
return InsertGeneratedColumns{
|
||||
Exprs: util.CloneExpressions(i.Exprs),
|
||||
OnDuplicates: util.CloneAssignments(i.OnDuplicates),
|
||||
}
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of InsertGeneratedColumns
|
||||
func (i *InsertGeneratedColumns) MemoryUsage() (sum int64) {
|
||||
if i == nil {
|
||||
return
|
||||
}
|
||||
sum = size.SizeOfSlice*3 + int64(cap(i.Columns)+cap(i.OnDuplicates))*size.SizeOfPointer + int64(cap(i.Exprs))*size.SizeOfInterface
|
||||
sum = size.SizeOfSlice*3 + int64(cap(i.OnDuplicates))*size.SizeOfPointer + int64(cap(i.Exprs))*size.SizeOfInterface
|
||||
|
||||
for _, expr := range i.Exprs {
|
||||
sum += expr.MemoryUsage()
|
||||
@ -349,15 +356,15 @@ func (i *InsertGeneratedColumns) MemoryUsage() (sum int64) {
|
||||
type Insert struct {
|
||||
baseSchemaProducer
|
||||
|
||||
Table table.Table
|
||||
tableSchema *expression.Schema
|
||||
tableColNames types.NameSlice
|
||||
Columns []*ast.ColumnName
|
||||
Table table.Table `plan-cache-clone:"shallow"`
|
||||
tableSchema *expression.Schema `plan-cache-clone:"shallow"`
|
||||
tableColNames types.NameSlice `plan-cache-clone:"shallow"`
|
||||
Columns []*ast.ColumnName `plan-cache-clone:"shallow"`
|
||||
Lists [][]expression.Expression
|
||||
|
||||
OnDuplicate []*expression.Assignment
|
||||
Schema4OnDuplicate *expression.Schema
|
||||
names4OnDuplicate types.NameSlice
|
||||
Schema4OnDuplicate *expression.Schema `plan-cache-clone:"shallow"`
|
||||
names4OnDuplicate types.NameSlice `plan-cache-clone:"shallow"`
|
||||
|
||||
GenCols InsertGeneratedColumns
|
||||
|
||||
@ -372,8 +379,8 @@ type Insert struct {
|
||||
|
||||
RowLen int
|
||||
|
||||
FKChecks []*FKCheck
|
||||
FKCascades []*FKCascade
|
||||
FKChecks []*FKCheck `plan-cache-clone:"must-nil"`
|
||||
FKCascades []*FKCascade `plan-cache-clone:"must-nil"`
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of Insert
|
||||
@ -429,16 +436,19 @@ type Update struct {
|
||||
|
||||
SelectPlan base.PhysicalPlan
|
||||
|
||||
TblColPosInfos TblColPosInfoSlice
|
||||
// TblColPosInfos is for multi-table update statement.
|
||||
// It records the column position of each related table.
|
||||
TblColPosInfos TblColPosInfoSlice `plan-cache-clone:"shallow"`
|
||||
|
||||
// Used when partition sets are given.
|
||||
// e.g. update t partition(p0) set a = 1;
|
||||
PartitionedTable []table.PartitionedTable
|
||||
PartitionedTable []table.PartitionedTable `plan-cache-clone:"must-nil"`
|
||||
|
||||
tblID2Table map[int64]table.Table
|
||||
// tblID2Table stores related tables' info of this Update statement.
|
||||
tblID2Table map[int64]table.Table `plan-cache-clone:"shallow"`
|
||||
|
||||
FKChecks map[int64][]*FKCheck
|
||||
FKCascades map[int64][]*FKCascade
|
||||
FKChecks map[int64][]*FKCheck `plan-cache-clone:"must-nil"`
|
||||
FKCascades map[int64][]*FKCascade `plan-cache-clone:"must-nil"`
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of Update
|
||||
@ -477,10 +487,10 @@ type Delete struct {
|
||||
|
||||
SelectPlan base.PhysicalPlan
|
||||
|
||||
TblColPosInfos TblColPosInfoSlice
|
||||
TblColPosInfos TblColPosInfoSlice `plan-cache-clone:"shallow"`
|
||||
|
||||
FKChecks map[int64][]*FKCheck
|
||||
FKCascades map[int64][]*FKCascade
|
||||
FKChecks map[int64][]*FKCheck `plan-cache-clone:"must-nil"`
|
||||
FKCascades map[int64][]*FKCascade `plan-cache-clone:"must-nil"`
|
||||
}
|
||||
|
||||
// MemoryUsage return the memory usage of Delete
|
||||
|
||||
@ -186,8 +186,10 @@ func (p *Insert) buildOnInsertFKTriggers(ctx base.PlanContext, is infoschema.Inf
|
||||
fkChecks = append(fkChecks, fkCheck)
|
||||
}
|
||||
}
|
||||
p.FKChecks = fkChecks
|
||||
p.FKCascades = fkCascades
|
||||
if len(fkChecks) > 0 || len(fkCascades) > 0 {
|
||||
p.FKChecks = fkChecks
|
||||
p.FKCascades = fkCascades
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -254,8 +256,10 @@ func (updt *Update) buildOnUpdateFKTriggers(ctx base.PlanContext, is infoschema.
|
||||
fkChecks[tid] = append(fkChecks[tid], childFKChecks...)
|
||||
}
|
||||
}
|
||||
updt.FKChecks = fkChecks
|
||||
updt.FKCascades = fkCascades
|
||||
if len(fkChecks) > 0 || len(fkCascades) > 0 {
|
||||
updt.FKChecks = fkChecks
|
||||
updt.FKCascades = fkCascades
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -285,8 +289,10 @@ func (del *Delete) buildOnDeleteFKTriggers(ctx base.PlanContext, is infoschema.I
|
||||
}
|
||||
}
|
||||
}
|
||||
del.FKChecks = fkChecks
|
||||
del.FKCascades = fkCascades
|
||||
if len(fkChecks) > 0 || len(fkCascades) > 0 {
|
||||
del.FKChecks = fkChecks
|
||||
del.FKCascades = fkCascades
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@ -641,6 +641,9 @@ type ColWithCmpFuncManager struct {
|
||||
|
||||
// Copy clones the ColWithCmpFuncManager.
|
||||
func (cwc *ColWithCmpFuncManager) Copy() *ColWithCmpFuncManager {
|
||||
if cwc == nil {
|
||||
return nil
|
||||
}
|
||||
cloned := new(ColWithCmpFuncManager)
|
||||
if cwc.TargetCol != nil {
|
||||
cloned.TargetCol = cwc.TargetCol.Clone().(*expression.Column)
|
||||
|
||||
@ -39,6 +39,7 @@ func TestPlanCacheClone(t *testing.T) {
|
||||
tk1.MustExec(`use test`)
|
||||
tk2.MustExec(`use test`)
|
||||
tk1.MustExec(`create table t (a int, b int, c int, d int, primary key(a), key(b), unique key(d))`)
|
||||
tk1.MustExec(`create table t1 (a int, b int, c int, d int)`)
|
||||
|
||||
for i := -20; i < 20; i++ {
|
||||
tk1.MustExec(fmt.Sprintf("insert into t values (%v,%v,%v,%v)", i, rand.Intn(20), rand.Intn(20), -i))
|
||||
@ -181,13 +182,41 @@ func TestPlanCacheClone(t *testing.T) {
|
||||
`set @a1=1,@b1=1, @a2=2,@b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'select * from t where d in (?,?)'`,
|
||||
`set @a1=1,@b1=1, @a2=2,@b2=2`, `execute st using @a1,@b1`, `execute st using @a2,@b2`)
|
||||
|
||||
// Insert
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'insert into t1 values (?, ?, ?, ?)'`,
|
||||
`set @a=1, @b=2`, `execute st using @a, @a, @a, @a`, `execute st using @b, @b, @b, @b`)
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'insert into t1 select * from t where a<?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'insert into t1 select * from t where a=?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
|
||||
// Delete
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'delete from t1 where a<?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'delete from t1 where a=?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
|
||||
// Update
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'update t1 set a=a+1 where a<?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
testCachedPlanClone(t, tk1, tk2, `prepare st from 'update t1 set a=a+1 where a=?'`,
|
||||
`set @a=1, @b=2`, `execute st using @a`, `execute st using @b`)
|
||||
}
|
||||
|
||||
func testCachedPlanClone(t *testing.T, tk1, tk2 *testkit.TestKit, prep, set, exec1, exec2 string) {
|
||||
isDML := false
|
||||
if strings.Contains(prep, "insert") || strings.Contains(prep, "update") || strings.Contains(prep, "delete") {
|
||||
isDML = true
|
||||
}
|
||||
ctx := context.WithValue(context.Background(), core.PlanCacheKeyEnableInstancePlanCache{}, true)
|
||||
tk1.MustExecWithContext(ctx, prep)
|
||||
tk1.MustExecWithContext(ctx, set)
|
||||
tk1.MustQueryWithContext(ctx, exec1) // generate the first cached plan
|
||||
if isDML {
|
||||
tk1.MustExecWithContext(ctx, exec1)
|
||||
} else {
|
||||
tk1.MustQueryWithContext(ctx, exec1) // generate the first cached plan
|
||||
}
|
||||
|
||||
tk2.MustExecWithContext(ctx, prep)
|
||||
tk2.MustExecWithContext(ctx, set)
|
||||
@ -198,7 +227,11 @@ func testCachedPlanClone(t *testing.T, tk1, tk2 *testkit.TestKit, prep, set, exe
|
||||
".ctx",
|
||||
"*collate"))
|
||||
})
|
||||
tk2.MustQueryWithContext(ctx, exec2)
|
||||
if isDML {
|
||||
tk2.MustExecWithContext(ctx, exec2)
|
||||
} else {
|
||||
tk2.MustQueryWithContext(ctx, exec2)
|
||||
}
|
||||
require.True(t, checked)
|
||||
}
|
||||
|
||||
|
||||
@ -414,16 +414,12 @@ type PlanCacheValue struct {
|
||||
// Since PlanCacheValue.Plan is not read-only, to solve the concurrency problem when sharing the same PlanCacheValue
|
||||
// across multiple sessions, we need to clone the PlanCacheValue for each session.
|
||||
func (v *PlanCacheValue) CloneForInstancePlanCache(ctx context.Context, newCtx base.PlanContext) (*PlanCacheValue, bool) {
|
||||
phyPlan, ok := v.Plan.(base.PhysicalPlan)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
clonedPlan, ok := phyPlan.CloneForPlanCache(newCtx)
|
||||
clonedPlan, ok := v.Plan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
if intest.InTest && ctx.Value(PlanCacheKeyTestClone{}) != nil {
|
||||
ctx.Value(PlanCacheKeyTestClone{}).(func(plan, cloned base.Plan))(phyPlan, clonedPlan)
|
||||
ctx.Value(PlanCacheKeyTestClone{}).(func(plan, cloned base.Plan))(v.Plan, clonedPlan)
|
||||
}
|
||||
cloned := new(PlanCacheValue)
|
||||
*cloned = *v
|
||||
|
||||
@ -194,11 +194,13 @@ func (op *PhysicalTableReader) CloneForPlanCache(newCtx base.PlanContext) (base.
|
||||
return nil, false
|
||||
}
|
||||
cloned.physicalSchemaProducer = *basePlan
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.tablePlan != nil {
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan)
|
||||
cloned.PlanPartInfo = op.PlanPartInfo.Clone()
|
||||
if op.TableScanAndPartitionInfos != nil {
|
||||
@ -216,11 +218,13 @@ func (op *PhysicalIndexReader) CloneForPlanCache(newCtx base.PlanContext) (base.
|
||||
return nil, false
|
||||
}
|
||||
cloned.physicalSchemaProducer = *basePlan
|
||||
indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.indexPlan != nil {
|
||||
indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.indexPlan = indexPlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.indexPlan = indexPlan.(base.PhysicalPlan)
|
||||
cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan)
|
||||
cloned.OutputColumns = util.CloneColumns(op.OutputColumns)
|
||||
cloned.PlanPartInfo = op.PlanPartInfo.Clone()
|
||||
@ -308,19 +312,19 @@ func (op *PhysicalIndexJoin) CloneForPlanCache(newCtx base.PlanContext) (base.Pl
|
||||
return nil, false
|
||||
}
|
||||
cloned.basePhysicalJoin = *basePlan
|
||||
innerPlan, ok := op.innerPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.innerPlan != nil {
|
||||
innerPlan, ok := op.innerPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.innerPlan = innerPlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.innerPlan = innerPlan.(base.PhysicalPlan)
|
||||
cloned.Ranges = op.Ranges.CloneForPlanCache()
|
||||
cloned.KeyOff2IdxOff = make([]int, len(op.KeyOff2IdxOff))
|
||||
copy(cloned.KeyOff2IdxOff, op.KeyOff2IdxOff)
|
||||
cloned.IdxColLens = make([]int, len(op.IdxColLens))
|
||||
copy(cloned.IdxColLens, op.IdxColLens)
|
||||
if op.CompareFilters != nil {
|
||||
cloned.CompareFilters = op.CompareFilters.Copy()
|
||||
}
|
||||
cloned.CompareFilters = op.CompareFilters.Copy()
|
||||
cloned.OuterHashKeys = util.CloneColumns(op.OuterHashKeys)
|
||||
cloned.InnerHashKeys = util.CloneColumns(op.InnerHashKeys)
|
||||
return cloned, true
|
||||
@ -348,16 +352,20 @@ func (op *PhysicalIndexLookUpReader) CloneForPlanCache(newCtx base.PlanContext)
|
||||
return nil, false
|
||||
}
|
||||
cloned.physicalSchemaProducer = *basePlan
|
||||
indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.indexPlan != nil {
|
||||
indexPlan, ok := op.indexPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.indexPlan = indexPlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.indexPlan = indexPlan.(base.PhysicalPlan)
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.tablePlan != nil {
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
cloned.IndexPlans = flattenPushDownPlan(cloned.indexPlan)
|
||||
cloned.TablePlans = flattenPushDownPlan(cloned.tablePlan)
|
||||
if op.ExtraHandleCol != nil {
|
||||
@ -385,11 +393,13 @@ func (op *PhysicalIndexMergeReader) CloneForPlanCache(newCtx base.PlanContext) (
|
||||
return nil, false
|
||||
}
|
||||
cloned.partialPlans = partialPlans
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
if op.tablePlan != nil {
|
||||
tablePlan, ok := op.tablePlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
}
|
||||
cloned.tablePlan = tablePlan.(base.PhysicalPlan)
|
||||
cloned.PartialPlans = make([][]base.PhysicalPlan, len(op.PartialPlans))
|
||||
for i, plan := range cloned.partialPlans {
|
||||
cloned.PartialPlans[i] = flattenPushDownPlan(plan)
|
||||
@ -401,3 +411,73 @@ func (op *PhysicalIndexMergeReader) CloneForPlanCache(newCtx base.PlanContext) (
|
||||
}
|
||||
return cloned, true
|
||||
}
|
||||
|
||||
// CloneForPlanCache implements the base.Plan interface.
|
||||
func (op *Update) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) {
|
||||
cloned := new(Update)
|
||||
*cloned = *op
|
||||
cloned.baseSchemaProducer = *op.baseSchemaProducer.CloneWithNewCtx(newCtx)
|
||||
cloned.OrderedList = util.CloneAssignments(op.OrderedList)
|
||||
if op.SelectPlan != nil {
|
||||
SelectPlan, ok := op.SelectPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.SelectPlan = SelectPlan.(base.PhysicalPlan)
|
||||
}
|
||||
if op.PartitionedTable != nil {
|
||||
return nil, false
|
||||
}
|
||||
if op.FKChecks != nil {
|
||||
return nil, false
|
||||
}
|
||||
if op.FKCascades != nil {
|
||||
return nil, false
|
||||
}
|
||||
return cloned, true
|
||||
}
|
||||
|
||||
// CloneForPlanCache implements the base.Plan interface.
|
||||
func (op *Delete) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) {
|
||||
cloned := new(Delete)
|
||||
*cloned = *op
|
||||
cloned.baseSchemaProducer = *op.baseSchemaProducer.CloneWithNewCtx(newCtx)
|
||||
if op.SelectPlan != nil {
|
||||
SelectPlan, ok := op.SelectPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.SelectPlan = SelectPlan.(base.PhysicalPlan)
|
||||
}
|
||||
if op.FKChecks != nil {
|
||||
return nil, false
|
||||
}
|
||||
if op.FKCascades != nil {
|
||||
return nil, false
|
||||
}
|
||||
return cloned, true
|
||||
}
|
||||
|
||||
// CloneForPlanCache implements the base.Plan interface.
|
||||
func (op *Insert) CloneForPlanCache(newCtx base.PlanContext) (base.Plan, bool) {
|
||||
cloned := new(Insert)
|
||||
*cloned = *op
|
||||
cloned.baseSchemaProducer = *op.baseSchemaProducer.CloneWithNewCtx(newCtx)
|
||||
cloned.Lists = util.CloneExpression2D(op.Lists)
|
||||
cloned.OnDuplicate = util.CloneAssignments(op.OnDuplicate)
|
||||
cloned.GenCols = op.GenCols.Copy()
|
||||
if op.SelectPlan != nil {
|
||||
SelectPlan, ok := op.SelectPlan.CloneForPlanCache(newCtx)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
cloned.SelectPlan = SelectPlan.(base.PhysicalPlan)
|
||||
}
|
||||
if op.FKChecks != nil {
|
||||
return nil, false
|
||||
}
|
||||
if op.FKCascades != nil {
|
||||
return nil, false
|
||||
}
|
||||
return cloned, true
|
||||
}
|
||||
|
||||
@ -38,7 +38,8 @@ func genPlanCloneForPlanCacheCode() ([]byte, error) {
|
||||
PhysicalHashJoin{}, PhysicalMergeJoin{}, PhysicalTableReader{}, PhysicalIndexReader{},
|
||||
PointGetPlan{}, BatchPointGetPlan{}, PhysicalLimit{},
|
||||
PhysicalIndexJoin{}, PhysicalIndexHashJoin{},
|
||||
PhysicalIndexLookUpReader{}, PhysicalIndexMergeReader{}}
|
||||
PhysicalIndexLookUpReader{}, PhysicalIndexMergeReader{},
|
||||
Update{}, Delete{}, Insert{}}
|
||||
c := new(codeGen)
|
||||
c.write(codeGenPrefix)
|
||||
for _, s := range structures {
|
||||
@ -97,10 +98,11 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
|
||||
case "baseimpl.Plan", "core.baseSchemaProducer":
|
||||
c.write("cloned.%v = *op.%v.CloneWithNewCtx(newCtx)", f.Name, f.Name)
|
||||
case "[]expression.Expression", "[]*ranger.Range", "[]*util.ByItems", "[]*expression.Column", "[]model.CIStr",
|
||||
"[]*expression.Constant", "[]*expression.ScalarFunction", "[]property.SortItem", "[]types.Datum", "[]kv.Handle":
|
||||
"[]*expression.Constant", "[]*expression.ScalarFunction", "[]property.SortItem", "[]types.Datum",
|
||||
"[]kv.Handle", "[]*expression.Assignment":
|
||||
structureName := strings.Split(f.Type.String(), ".")[1] + "s"
|
||||
c.write("cloned.%v = util.Clone%v(op.%v)", f.Name, structureName, f.Name)
|
||||
case "[][]*expression.Constant", "[][]types.Datum":
|
||||
case "[][]*expression.Constant", "[][]types.Datum", "[][]expression.Expression":
|
||||
structureName := strings.Split(f.Type.String(), ".")[1]
|
||||
c.write("cloned.%v = util.Clone%v2D(op.%v)", f.Name, structureName, f.Name)
|
||||
case "context.PlanContext":
|
||||
@ -111,10 +113,12 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
|
||||
c.write("}")
|
||||
case "*core.PhysPlanPartInfo", "*core.PushedDownLimit", "*expression.Schema":
|
||||
c.write("cloned.%v = op.%v.Clone()", f.Name, f.Name)
|
||||
case "kv.Handle", "*core.ColWithCmpFuncManager":
|
||||
case "kv.Handle":
|
||||
c.write("if op.%v != nil {", f.Name)
|
||||
c.write("cloned.%v = op.%v.Copy()", f.Name, f.Name)
|
||||
c.write("}")
|
||||
case "*core.ColWithCmpFuncManager", "core.InsertGeneratedColumns":
|
||||
c.write("cloned.%v = op.%v.Copy()", f.Name, f.Name)
|
||||
case "*expression.Column", "*expression.Constant":
|
||||
c.write("if op.%v != nil {", f.Name)
|
||||
c.write("cloned.%v = op.%v.Clone().(%v)", f.Name, f.Name, f.Type.String())
|
||||
@ -125,9 +129,11 @@ func genPlanCloneForPlanCache(x any) ([]byte, error) {
|
||||
c.write("cloned.%v = *inlj.(*PhysicalIndexJoin)", f.Name)
|
||||
c.write("cloned.self = cloned")
|
||||
case "base.PhysicalPlan":
|
||||
c.write("if op.%v != nil {", f.Name)
|
||||
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)
|
||||
c.write("}")
|
||||
case "[]base.PhysicalPlan":
|
||||
c.write("%v, ok := clonePhysicalPlansForPlanCache(newCtx, op.%v)", f.Name, f.Name)
|
||||
c.write("if !ok {return nil, false}")
|
||||
|
||||
@ -3646,7 +3646,6 @@ func (b *PlanBuilder) resolveGeneratedColumns(ctx context.Context, columns []*ta
|
||||
return igc, err
|
||||
}
|
||||
|
||||
igc.Columns = append(igc.Columns, columnName)
|
||||
igc.Exprs = append(igc.Exprs, expr)
|
||||
if onDups == nil {
|
||||
continue
|
||||
|
||||
@ -69,6 +69,18 @@ func CloneExpressions(exprs []expression.Expression) []expression.Expression {
|
||||
return CloneExprs(exprs)
|
||||
}
|
||||
|
||||
// CloneExpression2D uses CloneExprs to clone a 2D slice of expression.Expression.
|
||||
func CloneExpression2D(exprs [][]expression.Expression) [][]expression.Expression {
|
||||
if exprs == nil {
|
||||
return nil
|
||||
}
|
||||
cloned := make([][]expression.Expression, 0, len(exprs))
|
||||
for _, e := range exprs {
|
||||
cloned = append(cloned, CloneExprs(e))
|
||||
}
|
||||
return cloned
|
||||
}
|
||||
|
||||
// CloneScalarFunctions uses (*ScalarFunction).Clone to clone a slice of *ScalarFunction.
|
||||
func CloneScalarFunctions(scalarFuncs []*expression.ScalarFunction) []*expression.ScalarFunction {
|
||||
if scalarFuncs == nil {
|
||||
@ -81,6 +93,18 @@ func CloneScalarFunctions(scalarFuncs []*expression.ScalarFunction) []*expressio
|
||||
return cloned
|
||||
}
|
||||
|
||||
// CloneAssignments uses (*Assignment).Clone to clone a slice of *Assignment.
|
||||
func CloneAssignments(assignments []*expression.Assignment) []*expression.Assignment {
|
||||
if assignments == nil {
|
||||
return nil
|
||||
}
|
||||
cloned := make([]*expression.Assignment, 0, len(assignments))
|
||||
for _, a := range assignments {
|
||||
cloned = append(cloned, a.Clone())
|
||||
}
|
||||
return cloned
|
||||
}
|
||||
|
||||
// CloneCols uses (*Column).Clone to clone a slice of *Column.
|
||||
func CloneCols(cols []*expression.Column) []*expression.Column {
|
||||
if cols == nil {
|
||||
|
||||
Reference in New Issue
Block a user