planner: use code-gen to generate CloneForPlanCache method for Insert/Update/Delete (#55185)

ref pingcap/tidb#54057
This commit is contained in:
Yuanjia Zhang
2024-08-06 12:21:39 +08:00
committed by GitHub
parent 07c35e8157
commit fa554f8775
10 changed files with 231 additions and 64 deletions

View File

@ -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 {

View File

@ -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

View File

@ -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
}

View File

@ -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)

View File

@ -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)
}

View File

@ -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

View File

@ -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
}

View File

@ -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}")

View File

@ -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

View File

@ -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 {