Skip to content

Commit

Permalink
planner: PhysicalPlan memory trace 9 (#38220)
Browse files Browse the repository at this point in the history
ref #37632
  • Loading branch information
fzzf678 authored Oct 8, 2022
1 parent 4adfaf1 commit e6a1afd
Show file tree
Hide file tree
Showing 2 changed files with 108 additions and 0 deletions.
13 changes: 13 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1890,6 +1890,19 @@ type ShowContents struct {
Limit *ast.Limit // Used for limit Result Set row number.
}

const emptyShowContentsSize = int64(unsafe.Sizeof(ShowContents{}))

// MemoryUsage return the memory usage of ShowContents
func (s *ShowContents) MemoryUsage() (sum int64) {
if s == nil {
return
}

sum = emptyShowContentsSize + int64(len(s.DBName)) + s.Partition.MemoryUsage() + s.IndexName.MemoryUsage() +
int64(cap(s.Roles))*size.SizeOfPointer
return
}

// LogicalShow represents a show plan.
type LogicalShow struct {
logicalSchemaProducer
Expand Down
95 changes: 95 additions & 0 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1963,6 +1963,15 @@ func (p *PhysicalMaxOneRow) Clone() (PhysicalPlan, error) {
return cloned, nil
}

// MemoryUsage return the memory usage of PhysicalMaxOneRow
func (p *PhysicalMaxOneRow) MemoryUsage() (sum int64) {
if p == nil {
return
}

return p.basePhysicalPlan.MemoryUsage()
}

// PhysicalTableDual is the physical operator of dual.
type PhysicalTableDual struct {
physicalSchemaProducer
Expand All @@ -1984,6 +1993,19 @@ func (p *PhysicalTableDual) SetOutputNames(names types.NameSlice) {
p.names = names
}

// MemoryUsage return the memory usage of PhysicalTableDual
func (p *PhysicalTableDual) MemoryUsage() (sum int64) {
if p == nil {
return
}

sum = p.physicalSchemaProducer.MemoryUsage() + size.SizeOfInt + size.SizeOfSlice + int64(cap(p.names))*size.SizeOfPointer
for _, name := range p.names {
sum += name.MemoryUsage()
}
return
}

// PhysicalWindow is the physical operator of window function.
type PhysicalWindow struct {
physicalSchemaProducer
Expand Down Expand Up @@ -2048,6 +2070,27 @@ func (p *PhysicalWindow) Clone() (PhysicalPlan, error) {
return cloned, nil
}

// MemoryUsage return the memory usage of PhysicalWindow
func (p *PhysicalWindow) MemoryUsage() (sum int64) {
if p == nil {
return
}

sum = p.physicalSchemaProducer.MemoryUsage() + size.SizeOfSlice*3 + int64(cap(p.WindowFuncDescs))*size.SizeOfPointer +
size.SizeOfUint8

for _, windowFunc := range p.WindowFuncDescs {
sum += windowFunc.MemoryUsage()
}
for _, item := range p.PartitionBy {
sum += item.MemoryUsage()
}
for _, item := range p.OrderBy {
sum += item.MemoryUsage()
}
return
}

// PhysicalShuffle represents a shuffle plan.
// `Tails` and `DataSources` are the last plan within and the first plan following the "shuffle", respectively,
//
Expand All @@ -2070,6 +2113,30 @@ type PhysicalShuffle struct {
ByItemArrays [][]expression.Expression
}

// MemoryUsage return the memory usage of PhysicalShuffle
func (p *PhysicalShuffle) MemoryUsage() (sum int64) {
if p == nil {
return
}

sum = p.basePhysicalPlan.MemoryUsage() + size.SizeOfInt*2 + size.SizeOfSlice*(3+int64(cap(p.ByItemArrays))) +
int64(cap(p.Tails)+cap(p.DataSources))*size.SizeOfInterface

for _, plan := range p.Tails {
sum += plan.MemoryUsage()
}
for _, plan := range p.DataSources {
sum += plan.MemoryUsage()
}
for _, exprs := range p.ByItemArrays {
sum += int64(cap(exprs)) * size.SizeOfInterface
for _, expr := range exprs {
sum += expr.MemoryUsage()
}
}
return
}

// PartitionSplitterType is the type of `Shuffle` executor splitter, which splits data source into partitions.
type PartitionSplitterType int

Expand All @@ -2091,6 +2158,16 @@ type PhysicalShuffleReceiverStub struct {
DataSource PhysicalPlan
}

// MemoryUsage return the memory usage of PhysicalShuffleReceiverStub
func (p *PhysicalShuffleReceiverStub) MemoryUsage() (sum int64) {
if p == nil {
return
}

sum = p.physicalSchemaProducer.MemoryUsage() + size.SizeOfPointer + size.SizeOfInterface + p.DataSource.MemoryUsage()
return
}

// CollectPlanStatsVersion uses to collect the statistics version of the plan.
func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) map[string]uint64 {
for _, child := range plan.Children() {
Expand Down Expand Up @@ -2123,13 +2200,31 @@ type PhysicalShow struct {
Extractor ShowPredicateExtractor
}

// MemoryUsage return the memory usage of PhysicalShow
func (p *PhysicalShow) MemoryUsage() (sum int64) {
if p == nil {
return
}

sum = p.physicalSchemaProducer.MemoryUsage() + p.ShowContents.MemoryUsage() + size.SizeOfInterface
return
}

// PhysicalShowDDLJobs is for showing DDL job list.
type PhysicalShowDDLJobs struct {
physicalSchemaProducer

JobNumber int64
}

// MemoryUsage return the memory usage of PhysicalShowDDLJobs
func (p *PhysicalShowDDLJobs) MemoryUsage() (sum int64) {
if p == nil {
return
}
return p.physicalSchemaProducer.MemoryUsage() + size.SizeOfInt64
}

// BuildMergeJoinPlan builds a PhysicalMergeJoin from the given fields. Currently, it is only used for test purpose.
func BuildMergeJoinPlan(ctx sessionctx.Context, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin {
baseJoin := basePhysicalJoin{
Expand Down

0 comments on commit e6a1afd

Please sign in to comment.