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

*: make explain support explain anaylze #7827

Merged
merged 8 commits into from
Oct 12, 2018
Merged

*: make explain support explain anaylze #7827

merged 8 commits into from
Oct 12, 2018

Conversation

lysu
Copy link
Contributor

@lysu lysu commented Oct 2, 2018

What problem does this PR solve?

Currently, our explain command only display the physical plan result without execution.

but sometime we also need execution info to help us find reality plan question or executor performance bottleneck.

this PR add anaylze option to explain command(it just like postgresql's explain anaylze does https://www.postgresql.org/docs/9.1/static/sql-explain.html), and it will cause the statement to be actually executed, not only planned.

it works like this:

mysql> explain analyze select a_id, count(*) from v left join `u` on `u`.id = v.u_id where p_id = 1 and `u`.t > '2018-08-01' and v.t>'2018-08-01' group by a_id;
+------------------------------+---------+------+---------------------------------------------------------------------------------------+---------------------------------------+
| id                           | count   | task | operator info                                                                         | execution_info                        |
+------------------------------+---------+------+---------------------------------------------------------------------------------------+---------------------------------------+
| Projection_7                 | 2.67    | root | test.v.a_id, 5_col_0                                                          | actual_time:0.504215, loops:1, rows:0 |
| └─HashAgg_10                 | 2.67    | root | group by:test.v.a_id, funcs:count(1), firstrow(test.v.a_id)           | time:0.497296, loops:1, rows:0 |
|   └─IndexJoin_18             | 4.17    | root | inner join, inner:TableReader_17, outer key:test.v.u_id, inner key:test.u.id | time:0.394209, loops:1, rows:0 |
|     ├─IndexLookUp_40         | 3.33    | root |                                                                                       | time:0.365342, loops:1, rows:0 |
|     │ ├─IndexScan_37         | 10.00   | cop  | table:v, index:p_id, u_id, range:[1,1], keep order:false, stats:pseudo       |   |
|     │ └─Selection_39         | 3.33    | cop  | gt(test.v.t, 2018-08-01 00:00:00.000000)                                |   |
|     │   └─TableScan_38       | 10.00   | cop  | table:v, keep order:false, stats:pseudo                                            |   |
|     └─TableReader_17         | 3333.33 | root | data:Selection_16                                                                     |   |
|       └─Selection_16         | 3333.33 | cop  | gt(test.u.t, 2018-08-01 00:00:00.000000)                                |   |
|         └─TableScan_15       | 10.00   | cop  | table:u, range: decided by [test.v.u_id], keep order:false, stats:pseudo     |   |
+------------------------------+---------+------+---------------------------------------------------------------------------------------+---------------------------------------+
10 rows in set (0.00 sec)

the difference is with addition execution_info columns, and has be actual executed.

What is changed and how it works?

  • change Executor's Next from method to a func field to make it bindable
  • bind "Normal version Next" or "Explain version Next" once in plan build phase
  • maintain ExecStats(a map), and init ExecStates entry in plan building phase(single thread, so map itself no race condition, but for entry...)
  • add switch on SessionCtx.StmtCtx level to control Normal or Explain execution
  • modify Explain to open switch then actual execute plan and output addition column.

Remain question

  • Cop task need get info from tikv

Check List

Tests

  • Old test
  • Manual test (add detailed scripts or steps below)

Code changes

  • Has exported function/method change
  • Has exported variable/fields change
  • Has interface methods change

Side effects

  • No

Related changes

  • Need to update the documentation
  • Need to be included in the release note

This change is Reviewable

@lysu
Copy link
Contributor Author

lysu commented Oct 2, 2018

/run-all-tests

@shenli
Copy link
Member

shenli commented Oct 2, 2018

/cc @zhexuany
@lysu Is this the same thing with tracing?

@lysu
Copy link
Contributor Author

lysu commented Oct 3, 2018

@shenli I think it's a different perspective of same question.

explain anaylze only focus on the perspective of plan execute stats to help find plan question(so in this "view" we can combine plan detail with execution info), pg like db seems have explan anaylze and tracing at same time.

explain anaylze collect and using stats data(sum of time, rows, count) in executor level, but tracing more low level data(timeline, row per call, call span...so it can find more detail question) and display in custom level. so tracing need do some agg to get the same view, it's not simple and will collect some other nouse info for plan anaylze.

It's in Proposal, and need more disscuss, maybe finally we can let command's stats datasource be tracing-data too. 😆

@zz-jason
Copy link
Member

zz-jason commented Oct 8, 2018

@lysu we don't need to bind different next() functions. we only need to add a if statement in the next() function for every operator to record the executor statistics. and these executor statistics can also be reused in the trace statement.

Copy link
Contributor

@eurekaka eurekaka left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice work, LGTM up till now.

@winoros
Copy link
Member

winoros commented Oct 8, 2018

Side effects
Possible performance regression

Will there be that?

@winoros
Copy link
Member

winoros commented Oct 8, 2018

Do we need to record the loop information?

@eurekaka
Copy link
Contributor

eurekaka commented Oct 8, 2018

Side effects
Possible performance regression

Will there be that?

Deeper call stack I guess.

@lysu
Copy link
Contributor Author

lysu commented Oct 8, 2018

yes~it will get deeper call stack, and current modification way is change Next() from method to function pointer to make wrap Next easier, but calling function pointer seems slower than previous way. if use @zz-jason 's if-way will get better in performance but it needs make modification point to every operator.

and loops seems useful when chunk-size is highly-customized and not easy deduce from rows

@zz-jason
Copy link
Member

zz-jason commented Oct 8, 2018

@lysu both ways need to make modification point to every operator.

@lysu
Copy link
Contributor Author

lysu commented Oct 8, 2018

@zz-jason yes, it need change operator to support cutting point, but "track logic" isn't distribute to every operator just in one place at

	e.nextFunc = func(ctx context.Context, chk *chunk.Chunk) error {
 		start := time.Now()
 		err := nextFunc(ctx, chk)
 		e.execStat.Record(time.Now().Sub(start), chk.NumRows())
 		return err
 	}

and add new operator that follow current idiom will no need take care about "track logic" any more.

metric is classical crosscutting logic just like https://en.wikipedia.org/wiki/Aspect-oriented_programming or golang's http.Handler(prometheus no need call every http handler).

but performance is question, I will change it to if-way later 😆

@lysu lysu removed the status/WIP label Oct 8, 2018
@winoros
Copy link
Member

winoros commented Oct 8, 2018

Will there be one test that test its explain result?

@lysu
Copy link
Contributor Author

lysu commented Oct 9, 2018

@winoros time part is change every time, so I add some simple test in https://github.com/pingcap/tidb/pull/7827/files#diff-ddcc9b9aba1b5bc2d2338389e13a3bd8R40 PTAL

@lysu
Copy link
Contributor Author

lysu commented Oct 9, 2018

/run-all-tests

@lysu
Copy link
Contributor Author

lysu commented Oct 11, 2018

@zz-jason @jackysp PTAL if free

type ExecStats map[string]*ExecStat

// ExecStat collects one executor's execution info.
type ExecStat struct {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. How about: s/ExecStat/RuntimeStats/
  2. Please add comments for each struct field.

}

// NewExecutorStats creates new executor collector.
func NewExecutorStats() ExecStats {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/NewExecutorStats/NewRuntimeStats/

@@ -52,3 +54,55 @@ func (d ExecDetails) String() string {
}
return strings.Join(parts, " ")
}

// ExecStats collects executors's execution info.
type ExecStats map[string]*ExecStat
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why use a map to store the runtime statistics of all the executors? Maybe its simpler to make each executor to hold a ExecStat object?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because I want to get RuntimeStat from a PhysicalPlan, so this map will give ExplainID to RuntimeStat mapping. or do we have better way to get Executor from an PhysicalPlan~? 🐱

And it seem some physical is "1 to N" relationship with executor. (e.g. IndexLookUpExecutor)

Except that map lookup also easy to combine coprocessor's RuntimeStats in furture~?

@@ -112,6 +114,12 @@ func (e *UpdateExec) canNotUpdate(handle types.Datum) bool {

// Next implements the Executor Next interface.
func (e *UpdateExec) Next(ctx context.Context, chk *chunk.Chunk) error {
if e.execStat != nil {
start := time.Now()
defer func() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it can be simplified to:

defer e.execStat.Record(time.Now().Sub(start), chk.NumRows())

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does update statement support the explain statement?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update support it and will execute selectExec part and not modify records...but this line is brainless added code that can be removed too... 😹

@@ -49,6 +50,12 @@ type SimpleExec struct {

// Next implements the Executor Next interface.
func (e *SimpleExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

most of the statements executed by SimpleExec can not be explained.

executor/show.go Outdated
@@ -64,6 +64,12 @@ type ShowExec struct {

// Next implements the Executor Next interface.
func (e *ShowExec) Next(ctx context.Context, chk *chunk.Chunk) error {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the show statement can not be explained.

executor/set.go Outdated
@@ -43,6 +43,12 @@ type SetExecutor struct {

// Next implements the Executor Next interface.
func (e *SetExecutor) Next(ctx context.Context, chk *chunk.Chunk) error {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

@@ -501,6 +502,10 @@ func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGro

// Next implements the Executor Next interface.
func (e *HashAggExec) Next(ctx context.Context, chk *chunk.Chunk) error {
if e.runtimeStat != nil {
start := time.Now()
defer e.runtimeStat.Record(time.Now().Sub(start), chk.NumRows())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Defer function input parameter will be calculated in defer define.

func main() {
        foo()
        foo1()
}

func foo() {
        start := time.Now()
        defer func() {
                fmt.Println(time.Since(start))
        }()
        time.Sleep(1 * time.Second)
}

func foo1() {
        start := time.Now()
        defer fmt.Println(time.Since(start))
        time.Sleep(1 * time.Second)
}

output:

1.001687191s
319ns

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

emmm.....that's right..@zz-jason u mislead me.:rofl:

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

😂I'm sorry

@lysu lysu requested a review from jackysp October 12, 2018 10:52
@@ -15,7 +15,6 @@ package executor

import (
"fmt"

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this modification caused by gofmt?

@@ -178,11 +178,6 @@ func (s *testSuite) TestPrepared(c *C) {
_, _, fields, err = tk.Se.PrepareStmt("update prepare3 set a = ?")
c.Assert(err, IsNil)
c.Assert(len(fields), Equals, 0)

// Coverage.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why remove this test?

@@ -15,7 +15,6 @@ package executor

import (
"encoding/json"

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this modification caused by gofmt?

Copy link
Member

@jackysp jackysp left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@jackysp jackysp merged commit d21f294 into pingcap:master Oct 12, 2018
@zz-jason
Copy link
Member

it seems that there is only one LGTM?

@jackysp
Copy link
Member

jackysp commented Oct 12, 2018

Seems @eurekaka send another LGTM.

case ast.ExplainFormatROW:
retFields := []string{"id", "count", "task", "operator info"}
if explain.Analyze {
retFields = append(retFields, "execution_info")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/execution_info/execution info/

row := []string{e.prettyIdentifier(p.ExplainID(), indent, isLastChild), count, TaskType, operatorInfo}
row := []string{e.prettyIdentifier(p.ExplainID(), indent, isLastChild), count, taskType, operatorInfo}
if e.Analyze {
runtimeStat := e.ctx.GetSessionVars().StmtCtx.RuntimeStats
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/runtimeStat/runtimeStats/

@@ -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),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

e.runtimeStat will always be set, no matter whether it is in the explain analyze statement.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ctx.GetSessionVars().StmtCtx.RuntimeStats == nil so GetRuntimeStat(id) will quick return nil, so I only +1 a set nil, but look more uniform?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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

@@ -75,6 +78,7 @@ type baseExecutor struct {
maxChunkSize int
children []Executor
retFieldTypes []*types.FieldType
runtimeStat *execdetails.RuntimeStat
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/Stat/Stats/
stats is short for statistics in our codebase.

Copy link
Contributor Author

@lysu lysu Oct 12, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but runtimeStats is collection of runtimeStat

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about s/RuntimeStat/ExecutorStats/ and keep RuntimeStats a collection of the ExecutorStats?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it feels stranger 🤣

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

then how about s/RuntimeStats/RuntimeStatsColl/ and s/RuntimeStat/RuntimeStats/?

@@ -659,6 +659,33 @@ func (b *executorBuilder) buildTrace(v *plannercore.Trace) Executor {

// buildExplain builds a explain executor. `e.rows` collects final result to `ExplainExec`.
func (b *executorBuilder) buildExplain(v *plannercore.Explain) Executor {
if v.Analyze {
Copy link
Member

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().

@lysu
Copy link
Contributor Author

lysu commented Oct 12, 2018

@zz-jason now issue another PR to fix this?

@zz-jason
Copy link
Member

@lysu OK

if e == nil {
return ""
}
return fmt.Sprintf("time:%f, loops:%d, rows:%d", time.Duration(e.consume).Seconds()*1e3, e.loop, e.rows)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's better to specify the time unit in the string message. for example: "time:%fms, loops:%d, rows:%d"

return runtimecStat
}

func (e RuntimeStats) String() string {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this method can be removed? BTW, the result is not stable, because the it is composed by a range operation on a map.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's current no use and can be remove

@@ -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")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this collected stats is never used by the ExplainExec

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

e.id + "_tableReader" question~~?

Copy link
Member

Choose a reason for hiding this comment

The 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.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh, this is by design, GetRuntimeStat will pre-init e.id + "_tableReader" slot in RuntimeStatsColl, after this call will fork goroutine create executors, this pre-init prevent race condition for modify RuntimeStatsColl map, so RuntimeStatsColl live without mutex.

Copy link
Member

Choose a reason for hiding this comment

The 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 ExplainExec.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, _tableReader can be removed~

@zz-jason
Copy link
Member

zz-jason commented Oct 12, 2018

another thing is, for index join, its inner child will be built again and again according to the outer join keys. we need to collect all the runtime statistics of the built inner child and aggregate them to only one result to present to the user.

@lysu
Copy link
Contributor Author

lysu commented Oct 12, 2018

yes~ if inner child is unqiue plan id (e.g. IndexLookUp_8), inner create multple times will share same RuntimeStats instance and sum up.

}

// GetRuntimeStat gets execStat for a executor.
func (e RuntimeStats) GetRuntimeStat(planID string) *RuntimeStat {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe we need a mutex to prevent data race on the map, considered the parallel executed inner children of the index join.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we have no way to pre-init inner plainID's entry to make it lock free?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, GetRuntimeStat is low frequence operation...let's lock.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think a lock is much more easier.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
sig/execution SIG execution
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants