Skip to content

Commit

Permalink
planner: revise optimize trace logic (#30163)
Browse files Browse the repository at this point in the history
  • Loading branch information
Yisaer authored Nov 29, 2021
1 parent 87f4c7d commit 8ace8f6
Show file tree
Hide file tree
Showing 7 changed files with 28 additions and 18 deletions.
1 change: 1 addition & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1689,6 +1689,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.CTEStorageMap = map[int]*CTEStorages{}
sc.IsStaleness = false
sc.LockTableIDs = make(map[int64]struct{})
sc.EnableOptimizeTrace = false
sc.LogicalOptimizeTrace = nil
sc.OptimizerCETrace = nil

Expand Down
15 changes: 11 additions & 4 deletions executor/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
Expand Down Expand Up @@ -111,7 +112,16 @@ func (e *TraceExec) nextOptimizerPlanTrace(ctx context.Context, se sessionctx.Co
if err != nil {
return errors.AddStack(err)
}
e.executeChild(ctx, se.(sqlexec.SQLExecutor))
stmtCtx := se.GetSessionVars().StmtCtx
origin := stmtCtx.EnableOptimizeTrace
stmtCtx.EnableOptimizeTrace = true
defer func() {
stmtCtx.EnableOptimizeTrace = origin
}()
_, _, err = core.OptimizeAstNode(ctx, se, e.stmtNode, se.GetInfoSchema().(infoschema.InfoSchema))
if err != nil {
return err
}
res, err := json.Marshal(se.GetSessionVars().StmtCtx.LogicalOptimizeTrace)
if err != nil {
return errors.AddStack(err)
Expand Down Expand Up @@ -188,11 +198,8 @@ func (e *TraceExec) executeChild(ctx context.Context, se sqlexec.SQLExecutor) {
vars := e.ctx.GetSessionVars()
origin := vars.InRestrictedSQL
vars.InRestrictedSQL = true
originOptimizeTrace := vars.EnableStmtOptimizeTrace
vars.EnableStmtOptimizeTrace = e.optimizerTrace
defer func() {
vars.InRestrictedSQL = origin
vars.EnableStmtOptimizeTrace = originOptimizeTrace
}()
rs, err := se.ExecuteStmt(ctx, e.stmtNode)
if err != nil {
Expand Down
4 changes: 2 additions & 2 deletions planner/core/logical_plan_trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func (s *testPlanSuite) TestLogicalOptimizeWithTraceEnabled(c *C) {
err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is}))
c.Assert(err, IsNil, comment)
sctx := MockContext()
sctx.GetSessionVars().EnableStmtOptimizeTrace = true
sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true
builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{})
domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is)
ctx := context.TODO()
Expand Down Expand Up @@ -111,7 +111,7 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) {
err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is}))
c.Assert(err, IsNil, comment)
sctx := MockContext()
sctx.GetSessionVars().EnableStmtOptimizeTrace = true
sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true
builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{})
domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is)
ctx := context.TODO()
Expand Down
12 changes: 7 additions & 5 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,11 +112,11 @@ func (op *logicalOptimizeOp) appendStepToCurrent(id int, tp, reason, action stri
op.tracer.AppendRuleTracerStepToCurrent(id, tp, reason, action)
}

func (op *logicalOptimizeOp) trackAfterRuleOptimize(after LogicalPlan) {
func (op *logicalOptimizeOp) recordFinalLogicalPlan(final LogicalPlan) {
if op.tracer == nil {
return
}
op.tracer.TrackLogicalPlanAfterRuleOptimize(after.buildLogicalPlanTrace(after))
op.tracer.RecordFinalLogicalPlan(final.buildLogicalPlanTrace(final))
}

// logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc.
Expand Down Expand Up @@ -378,8 +378,10 @@ func enableParallelApply(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPla
func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (LogicalPlan, error) {
opt := defaultLogicalOptimizeOption()
vars := logic.SCtx().GetSessionVars()
if vars.EnableStmtOptimizeTrace {
tracer := &tracing.LogicalOptimizeTracer{Steps: make([]*tracing.LogicalRuleOptimizeTracer, 0)}
if vars.StmtCtx.EnableOptimizeTrace {
tracer := &tracing.LogicalOptimizeTracer{
Steps: make([]*tracing.LogicalRuleOptimizeTracer, 0),
}
opt = opt.withEnableOptimizeTracer(tracer)
defer func() {
vars.StmtCtx.LogicalOptimizeTrace = tracer
Expand All @@ -398,8 +400,8 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic
if err != nil {
return nil, err
}
opt.trackAfterRuleOptimize(logic)
}
opt.recordFinalLogicalPlan(logic)
return logic, err
}

Expand Down
2 changes: 2 additions & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,8 @@ type StatementContext struct {
// InVerboseExplain indicates the statement is "explain format='verbose' ...".
InVerboseExplain bool

// EnableOptimizeTrace indicates whether enable optimizer trace by 'trace plan statement'
EnableOptimizeTrace bool
// LogicalOptimizeTrace indicates the trace for optimize
LogicalOptimizeTrace *tracing.LogicalOptimizeTracer
// EnableOptimizerCETrace indicate if cardinality estimation internal process needs to be traced.
Expand Down
2 changes: 0 additions & 2 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -958,8 +958,6 @@ type SessionVars struct {
data [2]stmtctx.StatementContext
}

// EnableStmtOptimizeTrace indicates whether enable optimizer trace by 'trace plan statement'
EnableStmtOptimizeTrace bool
// Rng stores the rand_seed1 and rand_seed2 for Rand() function
Rng *utilMath.MysqlRng
}
Expand Down
10 changes: 5 additions & 5 deletions util/tracing/opt_trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,8 @@ type LogicalPlanTrace struct {

// LogicalOptimizeTracer indicates the trace for the whole logicalOptimize processing
type LogicalOptimizeTracer struct {
Steps []*LogicalRuleOptimizeTracer `json:"steps"`
FinalLogicalPlan *LogicalPlanTrace `json:"final"`
Steps []*LogicalRuleOptimizeTracer `json:"steps"`
// curRuleTracer indicates the current rule Tracer during optimize by rule
curRuleTracer *LogicalRuleOptimizeTracer
}
Expand All @@ -50,17 +51,16 @@ func (tracer *LogicalOptimizeTracer) AppendRuleTracerStepToCurrent(id int, tp, r
})
}

// TrackLogicalPlanAfterRuleOptimize add plan trace after optimize
func (tracer *LogicalOptimizeTracer) TrackLogicalPlanAfterRuleOptimize(after *LogicalPlanTrace) {
tracer.curRuleTracer.After = after
// RecordFinalLogicalPlan add plan trace after logical optimize
func (tracer *LogicalOptimizeTracer) RecordFinalLogicalPlan(final *LogicalPlanTrace) {
tracer.FinalLogicalPlan = final
}

// LogicalRuleOptimizeTracer indicates the trace for the LogicalPlan tree before and after
// logical rule optimize
type LogicalRuleOptimizeTracer struct {
Index int `json:"index"`
Before *LogicalPlanTrace `json:"before"`
After *LogicalPlanTrace `json:"after"`
RuleName string `json:"name"`
Steps []LogicalRuleOptimizeTraceStep `json:"steps"`
}
Expand Down

0 comments on commit 8ace8f6

Please sign in to comment.