Skip to content

Commit

Permalink
executor,session: display the tree-like format of the trace statement (
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored and jackysp committed Aug 6, 2019
1 parent c9502f0 commit 02ab1ed
Show file tree
Hide file tree
Showing 8 changed files with 8 additions and 30 deletions.
1 change: 1 addition & 0 deletions executor/adapter.go
Expand Up @@ -398,6 +398,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, e Executor) (sqlex
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("executor.handleNoDelayExecutor", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

// Check if "tidb_snapshot" is set for the write executors.
Expand Down
1 change: 1 addition & 0 deletions executor/compiler.go
Expand Up @@ -64,6 +64,7 @@ func (c *Compiler) compile(ctx context.Context, stmtNode ast.StmtNode, skipBind
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("executor.Compile", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

if !skipBind {
Expand Down
2 changes: 2 additions & 0 deletions executor/executor.go
Expand Up @@ -195,6 +195,7 @@ func Next(ctx context.Context, e Executor, req *chunk.Chunk) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan(fmt.Sprintf("%T.Next", e), opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
return e.Next(ctx, req)
}
Expand Down Expand Up @@ -907,6 +908,7 @@ func init() {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("executor.EvalSubQuery", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

e := &executorBuilder{is: is, ctx: sctx}
Expand Down
24 changes: 0 additions & 24 deletions executor/trace.go
Expand Up @@ -16,7 +16,6 @@ package executor
import (
"context"
"encoding/json"
"sort"
"time"

"github.com/opentracing/basictracer-go"
Expand Down Expand Up @@ -91,7 +90,6 @@ func (e *TraceExec) Next(ctx context.Context, req *chunk.Chunk) error {
return nil
}
trace := traces[0]
sortTraceByStartTime(trace)
dfsTree(trace, "", false, req)
e.exhausted = true
return nil
Expand Down Expand Up @@ -131,28 +129,6 @@ func drainRecordSet(ctx context.Context, sctx sessionctx.Context, rs sqlexec.Rec
}
}

type sortByStartTime []*appdash.Trace

func (t sortByStartTime) Len() int { return len(t) }
func (t sortByStartTime) Less(i, j int) bool {
return getStartTime(t[j]).After(getStartTime(t[i]))
}
func (t sortByStartTime) Swap(i, j int) { t[i], t[j] = t[j], t[i] }

func getStartTime(trace *appdash.Trace) (t time.Time) {
if e, err := trace.TimespanEvent(); err == nil {
t = e.Start()
}
return
}

func sortTraceByStartTime(trace *appdash.Trace) {
sort.Sort(sortByStartTime(trace.Sub))
for _, t := range trace.Sub {
sortTraceByStartTime(t)
}
}

func dfsTree(t *appdash.Trace, prefix string, isLast bool, chk *chunk.Chunk) {
var newPrefix, suffix string
if len(prefix) == 0 {
Expand Down
6 changes: 0 additions & 6 deletions executor/update.go
Expand Up @@ -17,7 +17,6 @@ import (
"context"
"fmt"

"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand Down Expand Up @@ -129,11 +128,6 @@ func (e *UpdateExec) canNotUpdate(handle types.Datum) bool {

// Next implements the Executor Next interface.
func (e *UpdateExec) Next(ctx context.Context, req *chunk.Chunk) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("update.Next", opentracing.ChildOf(span.Context()))
defer span1.Finish()
}

req.Reset()
if !e.fetched {
err := e.fetchChunkRows(ctx)
Expand Down
2 changes: 2 additions & 0 deletions session/session.go
Expand Up @@ -499,6 +499,7 @@ func (s *session) CommitTxn(ctx context.Context) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("session.CommitTxn", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

var commitDetail *execdetails.CommitDetails
Expand Down Expand Up @@ -1027,6 +1028,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("session.Execute", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
if recordSets, err = s.execute(ctx, sql); err != nil {
s.sessionVars.StmtCtx.AppendError(err)
Expand Down
1 change: 1 addition & 0 deletions session/tidb.go
Expand Up @@ -207,6 +207,7 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement)
span1 := span.Tracer().StartSpan("session.runStmt", opentracing.ChildOf(span.Context()))
span1.LogKV("sql", s.OriginText())
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}
se := sctx.(*session)
defer func() {
Expand Down
1 change: 1 addition & 0 deletions session/txn.go
Expand Up @@ -399,6 +399,7 @@ func (s *session) getTxnFuture(ctx context.Context) *txnFuture {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("session.getTxnFuture", opentracing.ChildOf(span.Context()))
defer span1.Finish()
ctx = opentracing.ContextWithSpan(ctx, span1)
}

oracleStore := s.store.GetOracle()
Expand Down

0 comments on commit 02ab1ed

Please sign in to comment.