Skip to content

Commit

Permalink
executor: Remove unnecessary information in explain analyze output (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored Jun 2, 2020
1 parent b82e924 commit 71f9ecc
Show file tree
Hide file tree
Showing 8 changed files with 35 additions and 16 deletions.
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

0 comments on commit 71f9ecc

Please sign in to comment.