Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: Remove unnecessary information in explain analyze output (#16248) #17350

Merged
merged 4 commits into from
Jun 2, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 4 additions & 2 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/set"
Expand Down Expand Up @@ -243,8 +244,9 @@ func (e *HashAggExec) Close() error {
partialConcurrency = cap(e.partialWorkers)
finalConcurrency = cap(e.finalWorkers)
}
e.runtimeStats.SetConcurrencyInfo("PartialConcurrency", partialConcurrency)
e.runtimeStats.SetConcurrencyInfo("FinalConcurrency", finalConcurrency)
partialConcurrencyInfo := execdetails.NewConcurrencyInfo("PartialConcurrency", partialConcurrency)
finalConcurrencyInfo := execdetails.NewConcurrencyInfo("FinalConcurrency", finalConcurrency)
e.runtimeStats.SetConcurrencyInfo(partialConcurrencyInfo, finalConcurrencyInfo)
}
return e.baseExecutor.Close()
}
Expand Down
3 changes: 2 additions & 1 deletion executor/index_lookup_hash_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
Expand Down Expand Up @@ -295,7 +296,7 @@ func (e *IndexNestedLoopHashJoin) Close() error {
}
if e.runtimeStats != nil {
concurrency := cap(e.joinChkResourceCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
for i := range e.joinChkResourceCh {
close(e.joinChkResourceCh[i])
Expand Down
3 changes: 2 additions & 1 deletion executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/mvmap"
Expand Down Expand Up @@ -681,7 +682,7 @@ func (e *IndexLookUpJoin) Close() error {
e.memTracker = nil
if e.runtimeStats != nil {
concurrency := cap(e.resultCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
return e.baseExecutor.Close()
}
3 changes: 2 additions & 1 deletion executor/index_lookup_merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/stringutil"
Expand Down Expand Up @@ -686,7 +687,7 @@ func (e *IndexLookUpMergeJoin) Close() error {
e.memTracker = nil
if e.runtimeStats != nil {
concurrency := cap(e.resultCh)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
}
return e.baseExecutor.Close()
}
7 changes: 5 additions & 2 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/stringutil"
)
Expand Down Expand Up @@ -138,8 +139,10 @@ func (e *HashJoinExec) Close() error {

if e.runtimeStats != nil {
concurrency := cap(e.joiners)
e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency)
e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String())
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", concurrency))
if e.rowContainer != nil {
e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String())
}
}
err := e.baseExecutor.Close()
return err
Expand Down
5 changes: 3 additions & 2 deletions executor/projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"go.uber.org/zap"
Expand Down Expand Up @@ -308,9 +309,9 @@ func (e *ProjectionExec) Close() error {
}
if e.runtimeStats != nil {
if e.isUnparallelExec() {
e.runtimeStats.SetConcurrencyInfo("Concurrency", 0)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", 0))
} else {
e.runtimeStats.SetConcurrencyInfo("Concurrency", int(e.numWorkers))
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", int(e.numWorkers)))
}
}
return e.baseExecutor.Close()
Expand Down
3 changes: 2 additions & 1 deletion executor/shuffle.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
"github.com/spaolacci/murmur3"
"go.uber.org/zap"
Expand Down Expand Up @@ -144,7 +145,7 @@ func (e *ShuffleExec) Close() error {
e.executed = false

if e.runtimeStats != nil {
e.runtimeStats.SetConcurrencyInfo("ShuffleConcurrency", e.concurrency)
e.runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("ShuffleConcurrency", e.concurrency))
}

err := e.dataSource.Close()
Expand Down
21 changes: 15 additions & 6 deletions util/execdetails/execdetails.go
Original file line number Diff line number Diff line change
Expand Up @@ -360,12 +360,17 @@ type RuntimeStatsColl struct {
readerStats map[string]*ReaderRuntimeStats
}

// concurrencyInfo is used to save the concurrency information of the executor operator
type concurrencyInfo struct {
// ConcurrencyInfo is used to save the concurrency information of the executor operator
type ConcurrencyInfo struct {
concurrencyName string
concurrencyNum int
}

// NewConcurrencyInfo creates new executor's concurrencyInfo.
func NewConcurrencyInfo(name string, num int) *ConcurrencyInfo {
return &ConcurrencyInfo{name, num}
}

// RuntimeStats collects one executor's execution info.
type RuntimeStats struct {
// executor's Next() called times.
Expand All @@ -378,7 +383,7 @@ type RuntimeStats struct {
// protect concurrency
mu sync.Mutex
// executor concurrency information
concurrency []concurrencyInfo
concurrency []*ConcurrencyInfo

// additional information for executors
additionalInfo string
Expand Down Expand Up @@ -466,12 +471,16 @@ func (e *RuntimeStats) SetRowNum(rowNum int64) {
atomic.StoreInt64(&e.rows, rowNum)
}

// SetConcurrencyInfo sets the concurrency information.
// SetConcurrencyInfo sets the concurrency informations.
// We must clear the concurrencyInfo first when we call the SetConcurrencyInfo.
// When the num <= 0, it means the exector operator is not executed parallel.
func (e *RuntimeStats) SetConcurrencyInfo(name string, num int) {
func (e *RuntimeStats) SetConcurrencyInfo(infos ...*ConcurrencyInfo) {
e.mu.Lock()
defer e.mu.Unlock()
e.concurrency = append(e.concurrency, concurrencyInfo{concurrencyName: name, concurrencyNum: num})
e.concurrency = e.concurrency[:0]
for _, info := range infos {
e.concurrency = append(e.concurrency, info)
}
}

// SetAdditionalInfo sets the additional information.
Expand Down