From c4f6de159b069291ef4c90ab8e79386e8680784f Mon Sep 17 00:00:00 2001 From: fzzf678 <108643977+fzzf678@users.noreply.github.com> Date: Fri, 23 Sep 2022 15:29:43 +0800 Subject: [PATCH] planner: PhysicalPlan memory trace 7 (#38030) ref pingcap/tidb#37632 --- executor/benchmark_test.go | 5 ++ executor/builder.go | 4 +- executor/executor_required_rows_test.go | 5 ++ expression/schema.go | 6 ++- planner/core/find_best_task_test.go | 5 ++ planner/core/initialize.go | 14 +++++ planner/core/logical_plans.go | 27 ++++++++++ planner/core/physical_plans.go | 72 +++++++++++++++++++++++++ 8 files changed, 135 insertions(+), 3 deletions(-) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 941c9d8d57d0e..d80f788ae533a 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -102,6 +102,11 @@ func (mp *mockDataPhysicalPlan) SelectBlockOffset() int { return 0 } +// MemoryUsage of mockDataPhysicalPlan is only for testing +func (mp *mockDataPhysicalPlan) MemoryUsage() (sum int64) { + return +} + func buildMockDataPhysicalPlan(ctx sessionctx.Context, srcExec Executor) *mockDataPhysicalPlan { return &mockDataPhysicalPlan{ schema: srcExec.Schema(), diff --git a/executor/builder.go b/executor/builder.go index d0364b1c5851e..08b4b6eaff56d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3965,9 +3965,9 @@ type mockPhysicalIndexReader struct { e Executor } -// MemoryUsage return the memory usage of mockPhysicalIndexReader +// MemoryUsage of mockPhysicalIndexReader is only for testing func (p *mockPhysicalIndexReader) MemoryUsage() (sum int64) { - return // mock operator for testing only + return } func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index b647b64acd576..cbca9914b5bc2 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -861,6 +861,11 @@ func (mp *mockPlan) Schema() *expression.Schema { return mp.exec.Schema() } +// MemoryUsage of mockPlan is only for testing +func (mp *mockPlan) MemoryUsage() (sum int64) { + return +} + func TestVecGroupCheckerDATARACE(t *testing.T) { ctx := mock.NewContext() diff --git a/expression/schema.go b/expression/schema.go index 18577a2293245..2a1b14d3c2aea 100644 --- a/expression/schema.go +++ b/expression/schema.go @@ -17,6 +17,8 @@ package expression import ( "strings" "unsafe" + + "github.com/pingcap/tidb/util/size" ) // KeyInfo stores the columns of one unique key or primary key. @@ -216,17 +218,19 @@ func (s *Schema) MemoryUsage() (sum int64) { return } - sum = emptySchemaSize + sum = emptySchemaSize + int64(cap(s.Columns))*size.SizeOfPointer + int64(cap(s.Keys)+cap(s.UniqueKeys))*size.SizeOfSlice for _, col := range s.Columns { sum += col.MemoryUsage() } for _, cols := range s.Keys { + sum += int64(cap(cols)) * size.SizeOfPointer for _, col := range cols { sum += col.MemoryUsage() } } for _, cols := range s.UniqueKeys { + sum += int64(cap(cols)) * size.SizeOfPointer for _, col := range cols { sum += col.MemoryUsage() } diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index 5507e4ae6dc04..e9067853a8f21 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -128,6 +128,11 @@ func (p *mockPhysicalPlan4Test) attach2Task(tasks ...task) task { return t } +// MemoryUsage of mockPhysicalPlan4Test is only for testing +func (p *mockPhysicalPlan4Test) MemoryUsage() (sum int64) { + return +} + func TestCostOverflow(t *testing.T) { ctx := MockContext() // Plan Tree: mockPlan -> mockDataSource diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 3d0f0f0c12f3e..bb2fc22b60546 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/plancodec" + "github.com/pingcap/tidb/util/size" ) // Init initializes LogicalAggregation. @@ -472,6 +473,19 @@ func (p PhysicalTableSample) Init(ctx sessionctx.Context, offset int) *PhysicalT return &p } +// MemoryUsage return the memory usage of PhysicalTableSample +func (p *PhysicalTableSample) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = p.physicalSchemaProducer.MemoryUsage() + size.SizeOfInterface + size.SizeOfBool + if p.TableSampleInfo != nil { + sum += p.TableSampleInfo.MemoryUsage() + } + return +} + // Init initializes PhysicalIndexReader. func (p PhysicalIndexReader) Init(ctx sessionctx.Context, offset int) *PhysicalIndexReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexReader, &p, offset) diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index b34067a1af519..e7550c0ea0abd 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -16,6 +16,7 @@ package core import ( "math" + "unsafe" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/size" "go.uber.org/zap" ) @@ -1926,6 +1928,31 @@ type CTEClass struct { ColumnMap map[string]*expression.Column } +const emptyCTEClassSize = int64(unsafe.Sizeof(CTEClass{})) + +// MemoryUsage return the memory usage of CTEClass +func (cc *CTEClass) MemoryUsage() (sum int64) { + if cc == nil { + return + } + + sum = emptyCTEClassSize + if cc.seedPartPhysicalPlan != nil { + sum += cc.seedPartPhysicalPlan.MemoryUsage() + } + if cc.recursivePartPhysicalPlan != nil { + sum += cc.recursivePartPhysicalPlan.MemoryUsage() + } + + for _, expr := range cc.pushDownPredicates { + sum += expr.MemoryUsage() + } + for key, val := range cc.ColumnMap { + sum += size.SizeOfString + int64(len(key)) + size.SizeOfPointer + val.MemoryUsage() + } + return +} + // LogicalCTE is for CTE. type LogicalCTE struct { logicalSchemaProducer diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index e949985f0d061..2096b1371be5e 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1513,6 +1513,15 @@ func (p *PhysicalUnionAll) Clone() (PhysicalPlan, error) { return cloned, nil } +// MemoryUsage return the memory usage of PhysicalUnionAll +func (p *PhysicalUnionAll) MemoryUsage() (sum int64) { + if p == nil { + return + } + + return p.physicalSchemaProducer.MemoryUsage() + size.SizeOfBool +} + // AggMppRunMode defines the running mode of aggregation in MPP type AggMppRunMode int @@ -2070,6 +2079,22 @@ type TableSampleInfo struct { Partitions []table.PartitionedTable } +// MemoryUsage return the memory usage of TableSampleInfo +func (t *TableSampleInfo) MemoryUsage() (sum int64) { + if t == nil { + return + } + + sum = size.SizeOfPointer*2 + size.SizeOfSlice + int64(cap(t.Partitions))*size.SizeOfInterface + if t.AstNode != nil { + sum += int64(unsafe.Sizeof(ast.TableSample{})) + } + if t.FullSchema != nil { + sum += t.FullSchema.MemoryUsage() + } + return +} + // NewTableSampleInfo creates a new TableSampleInfo. func NewTableSampleInfo(node *ast.TableSample, fullSchema *expression.Schema, pt []table.PartitionedTable) *TableSampleInfo { if node == nil { @@ -2128,11 +2153,39 @@ func (p *PhysicalCTE) ExplainID() fmt.Stringer { }) } +// MemoryUsage return the memory usage of PhysicalCTE +func (p *PhysicalCTE) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = p.physicalSchemaProducer.MemoryUsage() + p.cteAsName.MemoryUsage() + if p.SeedPlan != nil { + sum += p.SeedPlan.MemoryUsage() + } + if p.RecurPlan != nil { + sum += p.RecurPlan.MemoryUsage() + } + if p.CTE != nil { + sum += p.CTE.MemoryUsage() + } + return +} + // ExplainInfo overrides the ExplainInfo func (p *PhysicalCTETable) ExplainInfo() string { return "Scan on CTE_" + strconv.Itoa(p.IDForStorage) } +// MemoryUsage return the memory usage of PhysicalCTETable +func (p *PhysicalCTETable) MemoryUsage() (sum int64) { + if p == nil { + return + } + + return p.physicalSchemaProducer.MemoryUsage() + size.SizeOfInt +} + // CTEDefinition is CTE definition for explain. type CTEDefinition PhysicalCTE @@ -2157,6 +2210,25 @@ func (p *CTEDefinition) ExplainID() fmt.Stringer { }) } +// MemoryUsage return the memory usage of CTEDefinition +func (p *CTEDefinition) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = p.physicalSchemaProducer.MemoryUsage() + p.cteAsName.MemoryUsage() + if p.SeedPlan != nil { + sum += p.SeedPlan.MemoryUsage() + } + if p.RecurPlan != nil { + sum += p.RecurPlan.MemoryUsage() + } + if p.CTE != nil { + sum += p.CTE.MemoryUsage() + } + return +} + func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOptimizeOp) { candidate := &tracing.CandidatePlanTrace{ PlanTrace: &tracing.PlanTrace{