Skip to content

Commit

Permalink
planner: save explain result of plan in session fo ... (#17687) (#18124)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored Jun 19, 2020
1 parent ffc17bd commit b7181f7
Show file tree
Hide file tree
Showing 6 changed files with 92 additions and 47 deletions.
67 changes: 51 additions & 16 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@ import (
"crypto/tls"
"fmt"
"math"
"strconv"
"sync"

. "github.com/pingcap/check"
"github.com/pingcap/parser/auth"
Expand Down Expand Up @@ -128,35 +130,68 @@ type testPrepareSerialSuite struct {
}

func (s *testPrepareSerialSuite) TestExplainForConnPlanCache(c *C) {
tk := testkit.NewTestKit(c, s.store)
orgEnable := core.PreparedPlanCacheEnabled()
defer func() {
core.SetPreparedPlanCache(orgEnable)
}()
core.SetPreparedPlanCache(true)

var err error
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
tk1 := testkit.NewTestKit(c, s.store)
tk1.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
})
c.Assert(err, IsNil)
tk2 := testkit.NewTestKitWithInit(c, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")
rows := tk.MustQuery("select connection_id()").Rows()
c.Assert(len(rows), Equals, 1)
connID := rows[0][0].(string)
tk.MustExec("prepare stmt from 'select * from t where a = ?'")
tk.MustExec("set @p0='1'")
tk.MustExec("execute stmt using @p0")
tkProcess := tk.Se.ShowProcess()
ps := []*util.ProcessInfo{tkProcess}
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
tk.MustQuery(fmt.Sprintf("explain for connection %s", connID)).Check(testkit.Rows(
tk1.MustExec("use test")
tk1.MustExec("drop table if exists t")
tk1.MustExec("create table t(a int)")
tk1.MustExec("prepare stmt from 'select * from t where a = ?'")
tk1.MustExec("set @p0='1'")

executeQuery := "execute stmt using @p0"
explainQuery := "explain for connection " + strconv.FormatUint(tk1.Se.ShowProcess().ID, 10)
explainResult := testkit.Rows(
"TableReader_7 8000.00 root data:Selection_6",
"└─Selection_6 8000.00 cop[tikv] eq(cast(test.t.a), 1)",
" └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo",
))
)

// Now the ProcessInfo held by mockSessionManager1 will not be updated in real time.
// So it needs to be reset every time before tk2 query.
// TODO: replace mockSessionManager1 with another mockSessionManager.

// single test
tk1.MustExec(executeQuery)
tk2.Se.SetSessionManager(&mockSessionManager1{
PS: []*util.ProcessInfo{tk1.Se.ShowProcess()},
})
tk2.MustQuery(explainQuery).Check(explainResult)

// multiple test, '1000' is both effective and efficient.
repeats := 1000
var wg sync.WaitGroup
wg.Add(2)

go func() {
for i := 0; i < repeats; i++ {
tk1.MustExec(executeQuery)
}
wg.Done()
}()

go func() {
for i := 0; i < repeats; i++ {
tk2.Se.SetSessionManager(&mockSessionManager1{
PS: []*util.ProcessInfo{tk1.Se.ShowProcess()},
})
tk2.MustQuery(explainQuery).Check(explainResult)
}
wg.Done()
}()

wg.Wait()
}

func (s *testPrepareSerialSuite) TestExplainDotForExplainPlan(c *C) {
Expand Down
20 changes: 6 additions & 14 deletions expression/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,11 +102,11 @@ func (c *Constant) Clone() Expression {

// GetType implements Expression interface.
func (c *Constant) GetType() *types.FieldType {
if p := c.ParamMarker; p != nil && !p.ctx.GetSessionVars().StmtCtx.InExplainStmt {
if c.ParamMarker != nil {
// GetType() may be called in multi-threaded context, e.g, in building inner executors of IndexJoin,
// so it should avoid data race. We achieve this by returning different FieldType pointer for each call.
tp := types.NewFieldType(mysql.TypeUnspecified)
dt := p.GetUserVar()
dt := c.ParamMarker.GetUserVar()
types.DefaultParamTypeForValue(dt.GetValue(), tp)
return tp
}
Expand Down Expand Up @@ -170,21 +170,13 @@ func (c *Constant) VecEvalJSON(ctx sessionctx.Context, input *chunk.Chunk, resul
}

func (c *Constant) getLazyDatum(row chunk.Row) (dt types.Datum, isLazy bool, err error) {
if p := c.ParamMarker; p != nil {
if p.ctx.GetSessionVars().StmtCtx.InExplainStmt {
// Since `ParamMarker` is not nil only in prepare/execute context, the query must be `explain for connection` when coming here.
// The PreparedParams may have been reset already, to avoid panic, we just use the pre-evaluated datum for this constant.
return dt, false, nil
}
dt = p.GetUserVar()
isLazy = true
return
if c.ParamMarker != nil {
return c.ParamMarker.GetUserVar(), true, nil
} else if c.DeferredExpr != nil {
dt, err = c.DeferredExpr.Eval(row)
isLazy = true
return
return dt, true, err
}
return
return types.Datum{}, false, nil
}

// Eval implements Expression interface.
Expand Down
23 changes: 19 additions & 4 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -798,6 +798,19 @@ type Explain struct {
explainedPlans map[int]bool
}

// GetExplainRowsForPlan get explain rows for plan.
func GetExplainRowsForPlan(plan Plan) (rows [][]string) {
explain := &Explain{
TargetPlan: plan,
Format: ast.ExplainFormatROW,
Analyze: false,
}
if err := explain.RenderResult(); err != nil {
return rows
}
return explain.Rows
}

// prepareSchema prepares explain's result schema.
func (e *Explain) prepareSchema() error {
var fieldNames []string
Expand Down Expand Up @@ -836,10 +849,12 @@ func (e *Explain) RenderResult() error {
}
switch strings.ToLower(e.Format) {
case ast.ExplainFormatROW:
e.explainedPlans = map[int]bool{}
err := e.explainPlanInRowFormat(e.TargetPlan, "root", "", "", true)
if err != nil {
return err
if e.Rows == nil || e.Analyze {
e.explainedPlans = map[int]bool{}
err := e.explainPlanInRowFormat(e.TargetPlan, "root", "", "", true)
if err != nil {
return err
}
}
case ast.ExplainFormatDOT:
if physicalPlan, ok := e.TargetPlan.(PhysicalPlan); ok {
Expand Down
7 changes: 4 additions & 3 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2855,12 +2855,13 @@ func (b *PlanBuilder) buildTrace(trace *ast.TraceStmt) (Plan, error) {
return p, nil
}

func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, analyze bool, execStmt ast.StmtNode) (Plan, error) {
func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, rows [][]string, analyze bool, execStmt ast.StmtNode) (Plan, error) {
p := &Explain{
TargetPlan: targetPlan,
Format: format,
Analyze: analyze,
ExecStmt: execStmt,
Rows: rows,
}
p.ctx = b.ctx
return p, p.prepareSchema()
Expand All @@ -2886,7 +2887,7 @@ func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (Plan, err
return &Explain{Format: explainFor.Format}, nil
}

return b.buildExplainPlan(targetPlan, explainFor.Format, false, nil)
return b.buildExplainPlan(targetPlan, explainFor.Format, processInfo.PlanExplainRows, false, nil)
}

func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt) (Plan, error) {
Expand All @@ -2898,7 +2899,7 @@ func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt
return nil, err
}

return b.buildExplainPlan(targetPlan, explain.Format, explain.Analyze, explain.Stmt)
return b.buildExplainPlan(targetPlan, explain.Format, nil, explain.Analyze, explain.Stmt)
}

func (b *PlanBuilder) buildSelectInto(ctx context.Context, sel *ast.SelectStmt) (Plan, error) {
Expand Down
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1017,6 +1017,7 @@ func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecu
DB: s.sessionVars.CurrentDB,
Command: command,
Plan: s.currentPlan,
PlanExplainRows: plannercore.GetExplainRowsForPlan(s.currentPlan),
Time: t,
State: s.Status(),
Info: sql,
Expand Down
21 changes: 11 additions & 10 deletions util/processinfo.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,17 @@ import (

// ProcessInfo is a struct used for show processlist statement.
type ProcessInfo struct {
ID uint64
User string
Host string
DB string
Plan interface{}
Time time.Time
Info string
CurTxnStartTS uint64
StmtCtx *stmtctx.StatementContext
StatsInfo func(interface{}) map[string]uint64
ID uint64
User string
Host string
DB string
Plan interface{}
PlanExplainRows [][]string
Time time.Time
Info string
CurTxnStartTS uint64
StmtCtx *stmtctx.StatementContext
StatsInfo func(interface{}) map[string]uint64
// MaxExecutionTime is the timeout for select statement, in milliseconds.
// If the query takes too long, kill it.
MaxExecutionTime uint64
Expand Down

0 comments on commit b7181f7

Please sign in to comment.