-
Notifications
You must be signed in to change notification settings - Fork 5.9k
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
*: make explain
support explain anaylze
#7827
Changes from all commits
57105b8
28db228
30e0278
2873ff1
cd6e960
0de7f6e
a1b68b4
e3411ca
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ import ( | |
"sort" | ||
"sync" | ||
"sync/atomic" | ||
"time" | ||
"unsafe" | ||
|
||
"github.com/pingcap/tidb/distsql" | ||
|
@@ -243,6 +244,10 @@ func (e *IndexReaderExecutor) Close() error { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *IndexReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
err := e.result.Next(ctx, chk) | ||
if err != nil { | ||
e.feedback.Invalidate() | ||
|
@@ -453,6 +458,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k | |
func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-chan *lookupTableTask) { | ||
lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency | ||
e.tblWorkerWg.Add(lookupConcurrencyLimit) | ||
e.baseExecutor.ctx.GetSessionVars().StmtCtx.RuntimeStats.GetRuntimeStat(e.id + "_tableReader") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this collected stats is never used by the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Because you only retrieve the runtime statistics of an operator by the operator id. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. oh, this is by design, There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we don't need to collect the runtime statistics of the table worker of the IndexLookupExecutor, because it is never used and presented to the user in the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok, |
||
for i := 0; i < lookupConcurrencyLimit; i++ { | ||
worker := &tableWorker{ | ||
workCh: workCh, | ||
|
@@ -512,6 +518,10 @@ func (e *IndexLookUpExecutor) Close() error { | |
|
||
// Next implements Exec Next interface. | ||
func (e *IndexLookUpExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.Reset() | ||
for { | ||
resultTask, err := e.getResultTask() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ import ( | |
"runtime" | ||
"sync" | ||
"sync/atomic" | ||
"time" | ||
|
||
"github.com/cznic/mathutil" | ||
"github.com/pingcap/tidb/ast" | ||
|
@@ -38,13 +39,15 @@ import ( | |
"github.com/pingcap/tidb/types" | ||
"github.com/pingcap/tidb/util/admin" | ||
"github.com/pingcap/tidb/util/chunk" | ||
"github.com/pingcap/tidb/util/execdetails" | ||
"github.com/pingcap/tidb/util/memory" | ||
"github.com/pkg/errors" | ||
log "github.com/sirupsen/logrus" | ||
"golang.org/x/net/context" | ||
) | ||
|
||
var ( | ||
_ Executor = &baseExecutor{} | ||
_ Executor = &CheckTableExec{} | ||
_ Executor = &HashAggExec{} | ||
_ Executor = &LimitExec{} | ||
|
@@ -75,6 +78,7 @@ type baseExecutor struct { | |
maxChunkSize int | ||
children []Executor | ||
retFieldTypes []*types.FieldType | ||
runtimeStat *execdetails.RuntimeStat | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. s/Stat/Stats/ There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. but runtimeStats is collection of runtimeStat There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. how about There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it feels stranger 🤣 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. then how about |
||
} | ||
|
||
// Open initializes children recursively and "childrenResults" according to children's schemas. | ||
|
@@ -130,6 +134,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id strin | |
schema: schema, | ||
initCap: ctx.GetSessionVars().MaxChunkSize, | ||
maxChunkSize: ctx.GetSessionVars().MaxChunkSize, | ||
runtimeStat: ctx.GetSessionVars().StmtCtx.RuntimeStats.GetRuntimeStat(id), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it's doesn't +1 memory set, because it's in struct initializer |
||
} | ||
if schema != nil { | ||
cols := schema.Columns | ||
|
@@ -172,6 +177,10 @@ type CancelDDLJobsExec struct { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.GrowAndReset(e.maxChunkSize) | ||
if e.cursor >= len(e.jobIDs) { | ||
return nil | ||
|
@@ -614,6 +623,10 @@ type LimitExec struct { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *LimitExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.Reset() | ||
if e.cursor >= e.end { | ||
return nil | ||
|
@@ -733,6 +746,10 @@ func (e *TableDualExec) Open(ctx context.Context) error { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *TableDualExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.Reset() | ||
if e.numReturned >= e.numDualRows { | ||
return nil | ||
|
@@ -784,6 +801,10 @@ func (e *SelectionExec) Close() error { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *SelectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.GrowAndReset(e.maxChunkSize) | ||
|
||
if !e.batched { | ||
|
@@ -859,6 +880,10 @@ type TableScanExec struct { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *TableScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.GrowAndReset(e.maxChunkSize) | ||
if e.isVirtualTable { | ||
return errors.Trace(e.nextChunk4InfoSchema(ctx, chk)) | ||
|
@@ -959,6 +984,10 @@ func (e *MaxOneRowExec) Open(ctx context.Context) error { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *MaxOneRowExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.Reset() | ||
if e.evaluated { | ||
return nil | ||
|
@@ -1101,6 +1130,10 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { | |
|
||
// Next implements the Executor Next interface. | ||
func (e *UnionExec) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
if e.runtimeStat != nil { | ||
start := time.Now() | ||
defer func() { e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows()) }() | ||
} | ||
chk.GrowAndReset(e.maxChunkSize) | ||
if !e.initialized { | ||
e.initialize(ctx) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should move this to the function body of
ExplainExec.Next()
.