From dd8f79ac3ce548512e4f3f1c00d63c1a4f78c643 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alfonso=20Subiotto=20Marqu=C3=A9s?= Date: Tue, 12 Jun 2018 10:50:44 -0400 Subject: [PATCH] distsql: move tableReader stat collection setup out of Start It makes more sense to have the tableReader instrument itself for stat collection at the same time other setup is performed in newTableReader. Release note: None --- pkg/sql/distsqlrun/tablereader.go | 11 ++++++----- pkg/sql/distsqlrun/tablereader_test.go | 11 ++++++----- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/pkg/sql/distsqlrun/tablereader.go b/pkg/sql/distsqlrun/tablereader.go index a4af4593e5fb..3842d581eb60 100644 --- a/pkg/sql/distsqlrun/tablereader.go +++ b/pkg/sql/distsqlrun/tablereader.go @@ -109,6 +109,12 @@ func newTableReader( tr.spans[i] = s.Span } tr.input = &rowFetcherWrapper{RowFetcher: &tr.fetcher} + + if sp := opentracing.SpanFromContext(flowCtx.EvalCtx.Ctx()); sp != nil && tracing.IsRecording(sp) { + tr.input = NewInputStatCollector(tr.input) + tr.finishTrace = tr.outputStatsToTrace + } + return tr, nil } @@ -207,11 +213,6 @@ func (tr *tableReader) Start(ctx context.Context) context.Context { log.Fatalf(ctx, "tableReader outside of txn") } - if sp := opentracing.SpanFromContext(ctx); sp != nil && tracing.IsRecording(sp) { - tr.input = NewInputStatCollector(tr.input) - tr.finishTrace = tr.outputStatsToTrace - } - // Like every processor, the tableReader will have a context with a log tag // and a span. The underlying fetcher inherits the proc's span, but not the // log tag. diff --git a/pkg/sql/distsqlrun/tablereader_test.go b/pkg/sql/distsqlrun/tablereader_test.go index 2489081d8dda..c18574af9734 100644 --- a/pkg/sql/distsqlrun/tablereader_test.go +++ b/pkg/sql/distsqlrun/tablereader_test.go @@ -329,16 +329,17 @@ func TestLimitScans(t *testing.T) { const limit = 3 post := PostProcessSpec{Limit: limit} - tr, err := newTableReader(&flowCtx, 0 /* processorID */, &spec, &post, nil /* output */) - if err != nil { - t.Fatal(err) - } - // Now we're going to run the tableReader and trace it. tracer := tracing.NewTracer() sp := tracer.StartSpan("root", tracing.Recordable) tracing.StartRecording(sp, tracing.SnowballRecording) ctx := opentracing.ContextWithSpan(context.Background(), sp) + flowCtx.EvalCtx.CtxProvider = tree.FixedCtxProvider{Context: ctx} + + tr, err := newTableReader(&flowCtx, 0 /* processorID */, &spec, &post, nil /* output */) + if err != nil { + t.Fatal(err) + } tr.Start(ctx) rows := 0