Skip to content

Commit

Permalink
sql: add nodes for each EXPLAIN ANALYZE operator
Browse files Browse the repository at this point in the history
Show the cluster nodes involved in the execution of each operator.

Note that this info does not show up in the non-analyze EXPLAIN. It is
technically much more challenging to do that because of the indirect
way we do distsql planning. Once we have the new DistSQL exec factory,
we will be able to add it.

Note on the implementation: I started by trying to make execution set
`ComponentID.NodeID` in all cases, but I got stuck in `ProcessorBase`
where we only have a `SQLIDContainer` available. I don't fully
understand the new abstraction and whether the distsql components and
flows should really use SQLIDs instead of NodeIDs.

Unfortunately, there is not much we can do to test this currently
(other than manual testing). I will investigate making the
"deterministic" flag more fine-grained, so that we can hide truly
non-deterministic values (like timings) separately from those that
just vary with the configuration (query distribution).

Example:
```
movr> EXPLAIN ANALYZE SELECT * FROM rides JOIN vehicles ON rides.vehicle_id = vehicles.id;
                    info
--------------------------------------------
  planning time: 158µs
  execution time: 7ms
  distribution: full
  vectorized: true

   hash join
  │ cluster nodes: n1, n2, n3
  │ actual row count: 500
  │ equality: (vehicle_id) = (id)
  │
  ├──  scan
  │     cluster nodes: n1, n2, n3
  │     actual row count: 500
  │     KV rows read: 500
  │     KV bytes read: 86 KiB
  │     missing stats
  │     table: rides@primary
  │     spans: FULL SCAN
  │
  └──  scan
        cluster nodes: n1, n2, n3
        actual row count: 15
        KV rows read: 15
        KV bytes read: 2.3 KiB
        missing stats
        table: vehicles@primary
        spans: FULL SCAN
```

Release note (sql change): EXPLAIN ANALYZE now includes the nodes
which were involved in the execution of each operator in the tree.
  • Loading branch information
RaduBerinde committed Feb 13, 2021
1 parent 005c156 commit 6cdaeda
Show file tree
Hide file tree
Showing 10 changed files with 95 additions and 6 deletions.
7 changes: 5 additions & 2 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -2724,11 +2724,14 @@ func (dsp *DistSQLPlanner) createPhysPlanForPlanNode(

if planCtx.traceMetadata != nil {
processors := make(execComponents, len(plan.ResultRouters))
for i := range plan.ResultRouters {
for i, resultProcIdx := range plan.ResultRouters {
processors[i] = execinfrapb.ProcessorComponentID(
execinfrapb.FlowID{UUID: planCtx.infra.FlowID},
int32(plan.ResultRouters[i]),
int32(resultProcIdx),
)
// TODO(radu): we should set NodeID in all cases, as part of
// ProcessorComponentID.
processors[i].NodeID = plan.Processors[resultProcIdx].Node
}
planCtx.traceMetadata.associateNodeWithComponents(node, processors)
}
Expand Down
24 changes: 20 additions & 4 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
Expand Down Expand Up @@ -219,7 +220,12 @@ func (ih *instrumentationHelper) Finish(
}

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

// TODO(radu): this should be unified with other stmt stats accesses.
Expand Down Expand Up @@ -469,7 +475,7 @@ func (m execNodeTraceMetadata) associateNodeWithComponents(
// annotateExplain aggregates the statistics in the trace and annotates
// explain.Nodes with execution stats.
func (m execNodeTraceMetadata) annotateExplain(
plan *explain.Plan, spans []tracingpb.RecordedSpan, makeDeterministic bool,
plan *explain.Plan, flowInfos []flowInfo, spans []tracingpb.RecordedSpan, makeDeterministic bool,
) {
statsMap := execinfrapb.ExtractStatsFromSpans(spans, makeDeterministic)

Expand All @@ -480,8 +486,15 @@ func (m execNodeTraceMetadata) annotateExplain(
var nodeStats exec.ExecutionStats

incomplete := false
for i := range components {
stats := statsMap[components[i]]
var nodes util.FastIntSet
for _, c := range components {
if c.Type == execinfrapb.ComponentID_PROCESSOR {
nodes.Add(int(c.NodeID))
// Clear the NodeID, which is not set for processors.
// TODO(radu): remove this hack when execution sets NodeID for all components.
c.NodeID = 0
}
stats := statsMap[c]
if stats == nil {
incomplete = true
break
Expand All @@ -494,6 +507,9 @@ func (m execNodeTraceMetadata) annotateExplain(
// incomplete results. In the future, we may consider an incomplete flag
// if we want to show them with a warning.
if !incomplete {
for i, ok := nodes.Next(0); ok; i, ok = nodes.Next(i + 1) {
nodeStats.Nodes = append(nodeStats.Nodes, fmt.Sprintf("n%d", i))
}
n.Annotate(exec.ExecutionStatsID, &nodeStats)
}
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/dist_vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,11 @@ distribution: <hidden>
vectorized: <hidden>
·
• group (scalar)
│ cluster nodes: <hidden>
│ actual row count: 1
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand All @@ -80,12 +82,14 @@ distribution: <hidden>
vectorized: <hidden>
·
• merge join
│ cluster nodes: <hidden>
│ actual row count: 5
│ equality: (k) = (k)
│ left cols are key
│ right cols are key
├── • scan
│ cluster nodes: <hidden>
│ actual row count: 5
│ KV rows read: 5
│ KV bytes read: 40 B
Expand All @@ -94,6 +98,7 @@ vectorized: <hidden>
│ spans: FULL SCAN
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/explain_analyze
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ distribution: <hidden>
vectorized: <hidden>
·
• scan
cluster nodes: <hidden>
actual row count: 0
KV rows read: 0
KV bytes read: 0 B
Expand All @@ -36,6 +37,7 @@ distribution: <hidden>
vectorized: <hidden>
·
• scan
cluster nodes: <hidden>
actual row count: 3
KV rows read: 3
KV bytes read: 24 B
Expand All @@ -59,13 +61,15 @@ vectorized: <hidden>
·
• hash join (inner)
│ columns: (k, v, a, b)
│ cluster nodes: <hidden>
│ actual row count: 2
│ estimated row count: 990 (missing stats)
│ equality: (v) = (a)
│ right cols are key
├── • scan
│ columns: (k, v)
│ cluster nodes: <hidden>
│ actual row count: 4
│ KV rows read: 4
│ KV bytes read: 32 B
Expand All @@ -75,6 +79,7 @@ vectorized: <hidden>
└── • scan
columns: (a, b)
cluster nodes: <hidden>
actual row count: 3
KV rows read: 3
KV bytes read: 24 B
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/explain_analyze_plans
Original file line number Diff line number Diff line change
Expand Up @@ -66,17 +66,20 @@ distribution: <hidden>
vectorized: <hidden>
·
• group
│ cluster nodes: <hidden>
│ actual row count: 5
│ group by: k
│ ordered: +k
└── • merge join
│ cluster nodes: <hidden>
│ actual row count: 5
│ equality: (k) = (k)
│ left cols are key
│ right cols are key
├── • scan
│ cluster nodes: <hidden>
│ actual row count: 5
│ KV rows read: 5
│ KV bytes read: 40 B
Expand All @@ -85,6 +88,7 @@ vectorized: <hidden>
│ spans: FULL SCAN
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand All @@ -106,19 +110,23 @@ distribution: <hidden>
vectorized: <hidden>
·
• sort
│ cluster nodes: <hidden>
│ actual row count: 5
│ order: +w
└── • distinct
│ cluster nodes: <hidden>
│ actual row count: 5
│ distinct on: w
└── • hash join
│ cluster nodes: <hidden>
│ actual row count: 5
│ equality: (k) = (w)
│ left cols are key
├── • scan
│ cluster nodes: <hidden>
│ actual row count: 5
│ KV rows read: 5
│ KV bytes read: 40 B
Expand All @@ -127,6 +135,7 @@ vectorized: <hidden>
│ spans: FULL SCAN
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand All @@ -148,12 +157,15 @@ distribution: <hidden>
vectorized: <hidden>
·
• cross join
│ cluster nodes: <hidden>
│ actual row count: 25
├── • ordinality
│ │ cluster nodes: <hidden>
│ │ actual row count: 5
│ │
│ └── • scan
│ cluster nodes: <hidden>
│ actual row count: 5
│ KV rows read: 5
│ KV bytes read: 40 B
Expand All @@ -162,9 +174,11 @@ vectorized: <hidden>
│ spans: FULL SCAN
└── • ordinality
│ cluster nodes: <hidden>
│ actual row count: 5
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand Down Expand Up @@ -206,9 +220,11 @@ distribution: <hidden>
vectorized: <hidden>
·
• window
│ cluster nodes: <hidden>
│ actual row count: 5
└── • scan
cluster nodes: <hidden>
actual row count: 5
KV rows read: 5
KV bytes read: 40 B
Expand All @@ -231,6 +247,7 @@ distribution: <hidden>
vectorized: <hidden>
·
• scan
cluster nodes: <hidden>
actual row count: 0
KV rows read: 0
KV bytes read: 0 B
Expand Down Expand Up @@ -259,6 +276,7 @@ vectorized: <hidden>
• root
├── • insert
│ │ cluster nodes: <hidden>
│ │ actual row count: 1
│ │ into: child(c, p)
│ │
Expand All @@ -274,9 +292,11 @@ vectorized: <hidden>
│ │ exec mode: one row
│ │
│ └── • group (scalar)
│ │ cluster nodes: <hidden>
│ │ actual row count: 1
│ │
│ └── • scan
│ cluster nodes: <hidden>
│ actual row count: 1
│ KV rows read: 1
│ KV bytes read: 8 B
Expand All @@ -288,9 +308,11 @@ vectorized: <hidden>
└── • constraint-check
└── • error if rows
│ cluster nodes: <hidden>
│ actual row count: 0
└── • lookup join (anti)
│ cluster nodes: <hidden>
│ actual row count: 0
│ KV rows read: 1
│ KV bytes read: 8 B
Expand All @@ -299,10 +321,12 @@ vectorized: <hidden>
│ equality cols are key
└── • filter
│ cluster nodes: <hidden>
│ actual row count: 1
│ filter: column2 IS NOT NULL
└── • scan buffer
cluster nodes: <hidden>
actual row count: 1
label: buffer 1
·
Expand Down
15 changes: 15 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/inverted_index_geospatial
Original file line number Diff line number Diff line change
Expand Up @@ -34,25 +34,30 @@ distribution: <hidden>
vectorized: <hidden>
·
• sort
│ cluster nodes: <hidden>
│ actual row count: 2
│ order: +k
└── • filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ filter: st_intersects('010100002026690000000000000C6A18410000008081844E41', geom)
└── • index join
│ cluster nodes: <hidden>
│ actual row count: 2
│ KV rows read: 2
│ KV bytes read: 16 B
│ table: geo_table@primary
└── • inverted filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ inverted column: geom_inverted_key
│ num spans: 31
└── • scan
cluster nodes: <hidden>
actual row count: 2
KV rows read: 4
KV bytes read: 32 B
Expand Down Expand Up @@ -104,25 +109,30 @@ distribution: <hidden>
vectorized: <hidden>
·
• sort
│ cluster nodes: <hidden>
│ actual row count: 2
│ order: +k
└── • filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ filter: st_intersects('010100002026690000000000000C6A18410000008081844E41', geom)
└── • index join
│ cluster nodes: <hidden>
│ actual row count: 2
│ KV rows read: 2
│ KV bytes read: 16 B
│ table: geo_table@primary
└── • inverted filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ inverted column: geom_inverted_key
│ num spans: 31
└── • scan
cluster nodes: <hidden>
actual row count: 2
KV rows read: 2
KV bytes read: 16 B
Expand Down Expand Up @@ -150,25 +160,30 @@ distribution: <hidden>
vectorized: <hidden>
·
• sort
│ cluster nodes: <hidden>
│ actual row count: 2
│ order: +k
└── • filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ filter: st_intersects('010100002026690000000000000C6A18410000008081844E41', geom)
└── • index join
│ cluster nodes: <hidden>
│ actual row count: 2
│ KV rows read: 2
│ KV bytes read: 16 B
│ table: geo_table@primary
└── • inverted filter
│ cluster nodes: <hidden>
│ actual row count: 2
│ inverted column: geom_inverted_key
│ num spans: 31
└── • scan
cluster nodes: <hidden>
actual row count: 2
KV rows read: 2
KV bytes read: 16 B
Expand Down
Loading

0 comments on commit 6cdaeda

Please sign in to comment.