Skip to content

Commit

Permalink
add more info in slow log
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx committed Jul 21, 2019
1 parent 7eaea30 commit 26d6269
Show file tree
Hide file tree
Showing 4 changed files with 162 additions and 3 deletions.
11 changes: 9 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -391,16 +391,23 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
if len(sessVars.StmtCtx.IndexIDs) > 0 {
indexIDs = strings.Replace(fmt.Sprintf("%v", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1)
}
var planString string
if sessVars.Plan != nil {
if plan, ok := sessVars.Plan.(plannercore.Plan); ok {
planString = plannercore.ToStringWithCount(plan)
}
}
execDetail := sessVars.StmtCtx.GetExecDetails()
copTaskInfo := sessVars.StmtCtx.CopTasksDetails()
statsInfos := a.getStatsInfo()
memMax := sessVars.StmtCtx.MemTracker.MaxConsumed()
joinVars := fmt.Sprintf("idxBatch %d hjCon %d", sessVars.IndexJoinBatchSize,sessVars.HashJoinConcurrency)
if costTime < threshold {
_, digest := sessVars.StmtCtx.SQLDigest()
logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql))
logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql, planString, joinVars))
} else {
_, digest := sessVars.StmtCtx.SQLDigest()
logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql))
logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql, planString, joinVars))
metrics.TotalQueryProcHistogram.Observe(costTime.Seconds())
metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds())
metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds())
Expand Down
141 changes: 141 additions & 0 deletions planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,3 +226,144 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) {
strs = append(strs, str)
return strs, idxs
}

// ToStringWithCount explains a Plan, returns description string.
func ToStringWithCount(p Plan) string {
strs, _ := toStringWithCount(p, []string{}, []int{})
return strings.Join(strs, "->")
}

func toStringWithCount(in Plan, strs []string, idxs []int) ([]string, []int) {
switch x := in.(type) {
case LogicalPlan:
if len(x.Children()) > 1 {
idxs = append(idxs, len(strs))
}

for _, c := range x.Children() {
strs, idxs = toString(c, strs, idxs)
}
case PhysicalPlan:
if len(x.Children()) > 1 {
idxs = append(idxs, len(strs))
}

for _, c := range x.Children() {
strs, idxs = toString(c, strs, idxs)
}
}

var str string
switch x := in.(type) {
case *CheckTable:
str = "CheckTable"
case *PhysicalIndexScan:
str = fmt.Sprintf("Index(%s.%s) %.2f", x.Table.Name.L, x.Index.Name.L, x.StatsCount())
case *PhysicalTableScan:
str = fmt.Sprintf("Table(%s) %.2f", x.Table.Name.L, x.StatsCount())
case *PhysicalHashJoin:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
strs = strs[:idx]
idxs = idxs[:last]
if x.InnerChildIdx == 0 {
str = "RightHashJoin{" + strings.Join(children, "->") + "}"
} else {
str = "LeftHashJoin{" + strings.Join(children, "->") + "}"
}
for _, eq := range x.EqualConditions {
l := eq.GetArgs()[0].String()
r := eq.GetArgs()[1].String()
str += fmt.Sprintf("(%s,%s)", l, r)
}
str += fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalMergeJoin:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
strs = strs[:idx]
idxs = idxs[:last]
id := "MergeJoin"
switch x.JoinType {
case SemiJoin:
id = "MergeSemiJoin"
case AntiSemiJoin:
id = "MergeAntiSemiJoin"
case LeftOuterSemiJoin:
id = "MergeLeftOuterSemiJoin"
case AntiLeftOuterSemiJoin:
id = "MergeAntiLeftOuterSemiJoin"
case LeftOuterJoin:
id = "MergeLeftOuterJoin"
case RightOuterJoin:
id = "MergeRightOuterJoin"
case InnerJoin:
id = "MergeInnerJoin"
}
str = id + "{" + strings.Join(children, "->") + "}"
for i := range x.LeftKeys {
l := x.LeftKeys[i].String()
r := x.RightKeys[i].String()
str += fmt.Sprintf("(%s,%s)", l, r)
}
str += fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalApply:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
strs = strs[:idx]
idxs = idxs[:last]
str = "Apply{" + strings.Join(children, "->") + "}" + fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalMaxOneRow:
str = "MaxOneRow"
case *PhysicalLimit:
str = "Limit" + fmt.Sprintf("%v\n", x.StatsCount())
case *PhysicalSort:
str = "Sort" + fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalUnionAll:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
strs = strs[:idx]
str = "UnionAll{" + strings.Join(children, "->") + "}" + fmt.Sprintf("%.2f", x.StatsCount())
idxs = idxs[:last]
case *PhysicalSelection:
str = fmt.Sprintf("Sel(%s) %.2f", x.Conditions, x.StatsCount())
case *PhysicalProjection:
str = "Projection"
case *PhysicalTopN:
str = fmt.Sprintf("TopN(%v,%d,%d) %.2f", x.ByItems, x.Offset, x.Count, x.StatsCount())
case *PhysicalTableDual:
str = "Dual"
case *PhysicalHashAgg:
str = "HashAgg" + fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalStreamAgg:
str = "StreamAgg" + fmt.Sprintf("%.2f", x.StatsCount())
case *PhysicalTableReader:
str = fmt.Sprintf("TableReader(%s) %.2f", ToStringWithCount(x.tablePlan), x.StatsCount())
case *PhysicalIndexReader:
str = fmt.Sprintf("IndexReader(%s) %.2f", ToStringWithCount(x.indexPlan), x.StatsCount())
case *PhysicalIndexLookUpReader:
str = fmt.Sprintf("IndexLookUp(%s, %s) %.2f", ToStringWithCount(x.indexPlan), ToStringWithCount(x.tablePlan), x.StatsCount())
case *PhysicalUnionScan:
str = fmt.Sprintf("UnionScan(%s) %.2f", x.Conditions, x.StatsCount())
case *PhysicalIndexJoin:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
strs = strs[:idx]
idxs = idxs[:last]
str = "IndexJoin{" + strings.Join(children, "->") + "}"
for i := range x.OuterJoinKeys {
l := x.OuterJoinKeys[i]
r := x.InnerJoinKeys[i]
str += fmt.Sprintf("(%s,%s)", l, r)
}
str += fmt.Sprintf(" %.2f", x.StatsCount())
default:
str = fmt.Sprintf("%T", in)
}
strs = append(strs, str)
return strs, idxs
}
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -877,6 +877,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec
zap.String("sql", sql))
return nil, errors.Trace(err)
}
s.sessionVars.Plan = stmt.Plan
metrics.SessionExecuteCompileDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds())

// Step3: Execute the physical plan.
Expand Down
12 changes: 11 additions & 1 deletion sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -319,6 +319,8 @@ type SessionVars struct {

// SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse.
SlowQueryFile string

Plan interface{}
}

// ConnectionInfo present connection used by audit.
Expand Down Expand Up @@ -800,6 +802,8 @@ const (
SlowLogCopWaitMax = "Cop_wait_max"
// SlowLogMemMax is the max number bytes of memory used in this statement.
SlowLogMemMax = "Mem_max"
SlowLogPlan = "Plan"
SlowLogJoinVars = "Join_Var"
)

// SlowLogFormat uses for formatting slow log.
Expand All @@ -821,7 +825,7 @@ const (
// # Memory_max: 4096
// select * from t_slim;
func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest string,
statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, memMax int64, sql string) string {
statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, memMax int64, sql string, planString string, joinVars string) string {
var buf bytes.Buffer
execDetailStr := execDetail.String()
buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n")
Expand Down Expand Up @@ -879,6 +883,12 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe
if memMax > 0 {
buf.WriteString(SlowLogPrefixStr + SlowLogMemMax + SlowLogSpaceMarkStr + strconv.FormatInt(memMax, 10) + "\n")
}
if len(planString) > 0 {
buf.WriteString(SlowLogPrefixStr + SlowLogPlan+ SlowLogSpaceMarkStr + planString + "\n")
}
if len(joinVars) > 0 {
buf.WriteString(SlowLogPrefixStr + SlowLogJoinVars+ SlowLogSpaceMarkStr + joinVars + "\n")
}
if len(sql) == 0 {
sql = ";"
}
Expand Down

0 comments on commit 26d6269

Please sign in to comment.