Skip to content

Commit

Permalink
executor: support Chunk for ExplainExec (pingcap#5398)
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored and shenli committed Dec 14, 2017
1 parent f1e89cd commit 37eb540
Show file tree
Hide file tree
Showing 4 changed files with 40 additions and 37 deletions.
9 changes: 5 additions & 4 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -424,14 +424,15 @@ func (b *executorBuilder) buildDDL(v *plan.DDL) Executor {
}

func (b *executorBuilder) buildExplain(v *plan.Explain) Executor {
exec := &ExplainExec{
e := &ExplainExec{
baseExecutor: newBaseExecutor(v.Schema(), b.ctx),
}
exec.rows = make([]Row, 0, len(v.Rows))
e.rows = make([][]string, 0, len(v.Rows))
for _, row := range v.Rows {
exec.rows = append(exec.rows, row)
e.rows = append(e.rows, row)
}
return exec
e.supportChk = true
return e
}

func (b *executorBuilder) buildUnionScanExec(v *plan.PhysicalUnionScan) Executor {
Expand Down
33 changes: 29 additions & 4 deletions executor/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,18 @@

package executor

import goctx "golang.org/x/net/context"
import (
"github.com/cznic/mathutil"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
goctx "golang.org/x/net/context"
)

// ExplainExec represents an explain executor.
type ExplainExec struct {
baseExecutor

rows []Row
rows [][]string
cursor int
}

Expand All @@ -28,13 +33,33 @@ func (e *ExplainExec) Next(goCtx goctx.Context) (Row, error) {
if e.cursor >= len(e.rows) {
return nil, nil
}
row := e.rows[e.cursor]
resultRow := make([]types.Datum, 0, len(e.rows[0]))
for i := range e.rows[e.cursor] {
resultRow = append(resultRow, types.NewStringDatum(e.rows[e.cursor][i]))
}
e.cursor++
return row, nil
return resultRow, nil
}

// Close implements the Executor Close interface.
func (e *ExplainExec) Close() error {
e.rows = nil
return nil
}

// NextChunk implements the Executor NextChunk interface.
func (e *ExplainExec) NextChunk(goCtx goctx.Context, chk *chunk.Chunk) error {
chk.Reset()
if e.cursor >= len(e.rows) {
return nil
}

numCurRows := mathutil.Min(e.ctx.GetSessionVars().MaxChunkSize, len(e.rows)-e.cursor)
for i := e.cursor; i < e.cursor+numCurRows; i++ {
for j := range e.rows[i] {
chk.AppendString(j, e.rows[i][j])
}
}
e.cursor += numCurRows
return nil
}
3 changes: 1 addition & 2 deletions plan/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1058,12 +1058,11 @@ func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) Plan {
p := &Explain{StmtPlan: pp}
switch strings.ToLower(explain.Format) {
case ast.ExplainFormatROW:
retFields := []string{"id", "parents", "children", "task", "operator info"}
retFields := []string{"id", "parents", "children", "task", "operator info", "count"}
schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...)
for _, fieldName := range retFields {
schema.Append(buildColumn("", fieldName, mysql.TypeString, mysql.MaxBlobWidth))
}
schema.Append(buildColumn("", "count", mysql.TypeDouble, mysql.MaxRealWidth))
p.SetSchema(schema)
p.explainedPlans = map[int]bool{}
p.prepareRootTaskInfo(p.StmtPlan.(PhysicalPlan))
Expand Down
32 changes: 5 additions & 27 deletions plan/plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,8 @@ package plan

import (
"bytes"
"encoding/json"
"fmt"
"strconv"
"strings"

"github.com/juju/errors"
Expand All @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/auth"
"github.com/pingcap/tidb/util/kvcache"
)
Expand Down Expand Up @@ -284,30 +283,10 @@ type Explain struct {
basePlan

StmtPlan Plan
Rows [][]types.Datum
Rows [][]string
explainedPlans map[int]bool
}

func (e *Explain) prepareExplainInfo(p Plan, parent Plan) error {
for _, child := range p.Children() {
err := e.prepareExplainInfo(child, p)
if err != nil {
return errors.Trace(err)
}
}
explain, err := json.MarshalIndent(p, "", " ")
if err != nil {
return errors.Trace(err)
}
parentStr := ""
if parent != nil {
parentStr = parent.ExplainID()
}
row := types.MakeDatums(p.ExplainID(), string(explain), parentStr)
e.Rows = append(e.Rows, row)
return nil
}

// prepareExplainInfo4DAGTask generates the following information for every plan:
// ["id", "parents", "task", "operator info"].
func (e *Explain) prepareExplainInfo4DAGTask(p PhysicalPlan, taskType string) {
Expand All @@ -323,8 +302,8 @@ func (e *Explain) prepareExplainInfo4DAGTask(p PhysicalPlan, taskType string) {
parentInfo := strings.Join(parentIDs, ",")
childrenInfo := strings.Join(childrenIDs, ",")
operatorInfo := p.ExplainInfo()
count := p.statsProfile().count
row := types.MakeDatums(p.ExplainID(), parentInfo, childrenInfo, taskType, operatorInfo, count)
count := string(strconv.AppendFloat([]byte{}, p.statsProfile().count, 'f', -1, 64))
row := []string{p.ExplainID(), parentInfo, childrenInfo, taskType, operatorInfo, count}
e.Rows = append(e.Rows, row)
}

Expand Down Expand Up @@ -363,8 +342,7 @@ func (e *Explain) prepareDotInfo(p PhysicalPlan) {
e.prepareTaskDot(p, "root", buffer)
buffer.WriteString(fmt.Sprintln("}"))

row := types.MakeDatums(buffer.String())
e.Rows = append(e.Rows, row)
e.Rows = append(e.Rows, []string{buffer.String()})
}

func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Buffer) {
Expand Down

0 comments on commit 37eb540

Please sign in to comment.