Skip to content

Commit

Permalink
Merge #57310
Browse files Browse the repository at this point in the history
57310: sql: add KV rows/bytes read to EXPLAIN ANALYZE (PLAN) r=RaduBerinde a=RaduBerinde

Release note: None

Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
  • Loading branch information
craig[bot] and RaduBerinde committed Dec 3, 2020
2 parents ac6463f + 5072655 commit 58e9eb7
Show file tree
Hide file tree
Showing 8 changed files with 53 additions and 10 deletions.
18 changes: 11 additions & 7 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ func (ih *instrumentationHelper) Finish(
}

if ih.traceMetadata != nil && ih.explainPlan != nil {
ih.traceMetadata.addSpans(trace)
ih.traceMetadata.addSpans(trace, cfg.TestingKnobs.DeterministicExplainAnalyze)
ih.traceMetadata.annotateExplain(ih.explainPlan)
}

Expand Down Expand Up @@ -386,7 +386,7 @@ func (m execNodeTraceMetadata) associateNodeWithProcessors(

// addSpans populates the processorTraceMetadata.fields with the statistics
// recorded in a trace.
func (m execNodeTraceMetadata) addSpans(spans []tracingpb.RecordedSpan) {
func (m execNodeTraceMetadata) addSpans(spans []tracingpb.RecordedSpan, makeDeterministic bool) {
// Build a map from <flow-id, processor-id> pair (encoded as a string)
// to the corresponding processorTraceMetadata entry.
processorKeyToMetadata := make(map[string]*processorTraceMetadata)
Expand Down Expand Up @@ -422,6 +422,9 @@ func (m execNodeTraceMetadata) addSpans(spans []tracingpb.RecordedSpan) {
if err := types.UnmarshalAny(span.Stats, &stats); err != nil {
continue
}
if makeDeterministic {
stats.MakeDeterministic()
}
procMetadata.stats = &stats
}
}
Expand All @@ -433,23 +436,24 @@ func (m execNodeTraceMetadata) annotateExplain(plan *explain.Plan) {
walk = func(n *explain.Node) {
wrapped := n.WrappedNode()
if meta, ok := m[wrapped]; ok {
var rowCount uint64
var nodeStats exec.ExecutionStats

incomplete := false
for i := range meta.processors {
stats := meta.processors[i].stats
if stats == nil {
incomplete = true
break
}
rowCount += stats.Output.NumTuples.Value()
nodeStats.RowCount.MaybeAdd(stats.Output.NumTuples)
nodeStats.KVBytesRead.MaybeAdd(stats.KV.BytesRead)
nodeStats.KVRowsRead.MaybeAdd(stats.KV.TuplesRead)
}
// If we didn't get statistics for all processors, we don't show the
// incomplete results. In the future, we may consider an incomplete flag
// if we want to show them with a warning.
if !incomplete {
n.Annotate(exec.ExecutionStatsID, &exec.ExecutionStats{
RowCount: rowCount,
})
n.Annotate(exec.ExecutionStatsID, &nodeStats)
}
}

Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/explain_analyze
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ vectorized: <hidden>
·
• scan
actual row count: 0
KV rows read: 0
KV bytes read: 0 B
missing stats
table: kv@primary
spans: [/2 - ]
Expand All @@ -35,6 +37,8 @@ vectorized: <hidden>
·
• scan
actual row count: 3
KV rows read: 3
KV bytes read: 24 B
missing stats
table: kv@primary
spans: [/2 - ]
Expand Down Expand Up @@ -63,13 +67,17 @@ vectorized: <hidden>
├── • scan
│ columns: (k, v)
│ actual row count: 4
│ KV rows read: 4
│ KV bytes read: 32 B
│ estimated row count: 1000 (missing stats)
│ table: kv@primary
│ spans: FULL SCAN
└── • scan
columns: (a, b)
actual row count: 3
KV rows read: 3
KV bytes read: 24 B
estimated row count: 1000 (missing stats)
table: ab@primary
spans: FULL SCAN
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/optional",
],
)

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/explain/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_library(
"//pkg/util/errorutil",
"//pkg/util/treeprinter",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/dustin/go-humanize",
],
)

Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/opt/exec/explain/emit.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/errors"
"github.com/dustin/go-humanize"
)

// Emit produces the EXPLAIN output against the given OutputBuilder. The
Expand Down Expand Up @@ -328,7 +329,15 @@ func (e *emitter) joinNodeName(algo string, joinType descpb.JoinType) string {
func (e *emitter) emitNodeAttributes(n *Node) error {
if stats, ok := n.annotations[exec.ExecutionStatsID]; ok {
s := stats.(*exec.ExecutionStats)
e.ob.Attr("actual row count", s.RowCount)
if s.RowCount.HasValue() {
e.ob.AddField("actual row count", s.RowCount.String())
}
if s.KVRowsRead.HasValue() {
e.ob.AddField("KV rows read", s.KVRowsRead.String())
}
if s.KVBytesRead.HasValue() {
e.ob.AddField("KV bytes read", humanize.IBytes(s.KVBytesRead.Value()))
}
}

if stats, ok := n.annotations[exec.EstimatedStatsID]; ok {
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/opt/exec/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/optional"
)

// Node represents a node in the execution tree
Expand Down Expand Up @@ -296,7 +297,10 @@ type EstimatedStats struct {
// TODO(radu): can/should we just use execinfrapb.ComponentStats instead?
type ExecutionStats struct {
// RowCount is the number of rows produced by the operator.
RowCount uint64
RowCount optional.Uint

KVBytesRead optional.Uint
KVRowsRead optional.Uint
}

// BuildPlanForExplainFn builds an execution plan against the given
Expand Down
8 changes: 8 additions & 0 deletions pkg/util/optional/uint.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,3 +63,11 @@ func (i *Uint) Set(value uint64) {
func (i *Uint) Add(delta int64) {
*i = MakeUint(uint64(int64(i.Value()) + delta))
}

// MaybeAdd adds the given value, if it is set. Does nothing if other is not
// set.
func (i *Uint) MaybeAdd(other Uint) {
if other.HasValue() {
*i = MakeUint(i.Value() + other.Value())
}
}
10 changes: 9 additions & 1 deletion pkg/util/optional/uint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,15 @@ func TestUint(t *testing.T) {
require.Equal(t, uint64(0), v.Value())
require.Equal(t, v.String(), "<unset>")

v.Add(100)
var other optional.Uint

v.MaybeAdd(other)
require.False(t, v.HasValue())
require.Equal(t, uint64(0), v.Value())
require.Equal(t, v.String(), "<unset>")

other.Set(100)
v.MaybeAdd(other)
require.True(t, v.HasValue())
require.Equal(t, uint64(100), v.Value())
require.Equal(t, v.String(), "100")
Expand Down

0 comments on commit 58e9eb7

Please sign in to comment.