Skip to content

Commit

Permalink
Merge #40528
Browse files Browse the repository at this point in the history
40528: exec: omit some of the operators from EXPLAIN (VEC) output r=yuzefovich a=yuzefovich

This commits adds NonExplainable marker interface which operators
can choose to implement so that they are omitted from the output
of EXPLAIN (VEC). It is intended that non-core operators will be
marked. Additionally, now a VERBOSE explain option is supported
which overrides the omitting behavior.

Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Sep 7, 2019
2 parents 08455ac + 01e2cd6 commit 82de988
Show file tree
Hide file tree
Showing 18 changed files with 202 additions and 283 deletions.
1 change: 1 addition & 0 deletions pkg/sql/distsqlrun/columnarizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
type columnarizer struct {
ProcessorBase
exec.ZeroInputNode
exec.NonExplainable

input RowSource
da sqlbase.DatumAlloc
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/distsqlrun/materializer.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
// materializer converts an exec.Operator input into a RowSource.
type materializer struct {
ProcessorBase
exec.NonExplainable

input exec.Operator

Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/exec/bool_vec_to_sel.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
// an index to the selection for each true value in the boolean column.
type boolVecToSelOp struct {
OneInputNode
NonExplainable

// outputCol is the boolean output column. It should be shared by other
// operators that write to it.
Expand Down Expand Up @@ -111,6 +112,7 @@ func NewBoolVecToSelOp(input Operator, colIdx int) Operator {
// an operator that can see the inside of its input batch for NewBoolVecToSelOp.
type selBoolOp struct {
OneInputNode
NonExplainable
boolVecToSelOp *boolVecToSelOp
colIdx int
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
// and makes it available to be read multiple times by downstream consumers.
type bufferOp struct {
OneInputNode
NonExplainable

// read is true if someone has read the current batch already.
read bool
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/cancel_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
// occurred. The check happens on every batch.
type CancelChecker struct {
OneInputNode
NonExplainable

// Number of times check() has been called since last context cancellation
// check.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/coalescer.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
// to return full batches of coldata.BatchSize.
type coalescerOp struct {
OneInputNode
NonExplainable

inputTypes []coltypes.T

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/deselector.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
// selection vector, it is a noop.
type deselectorOp struct {
OneInputNode
NonExplainable
inputTypes []coltypes.T

output coldata.Batch
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/fn_op.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
// once per input batch, passing the input batch unmodified along.
type fnOp struct {
OneInputNode
NonExplainable

fn func()
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/mergejoiner.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,7 @@ type mergeJoinInput struct {
// the next batch.
type feedOperator struct {
ZeroInputNode
NonExplainable
batch coldata.Batch
}

Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/exec/operator.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,14 @@ type OpNode interface {
Child(nth int) OpNode
}

// NonExplainable is a marker interface which identifies an Operator that
// should be omitted from the output of EXPLAIN (VEC). Note that VERBOSE
// explain option will override the omitting behavior.
type NonExplainable interface {
// nonExplainableMarker is just a marker method. It should never be called.
nonExplainableMarker()
}

// NewOneInputNode returns an OpNode with a single Operator input.
func NewOneInputNode(input Operator) OneInputNode {
return OneInputNode{input: input}
Expand Down Expand Up @@ -141,6 +149,7 @@ type resettableOperator interface {

type noopOperator struct {
OneInputNode
NonExplainable
}

var _ Operator = &noopOperator{}
Expand All @@ -166,6 +175,7 @@ func (n *noopOperator) reset() {

type zeroOperator struct {
OneInputNode
NonExplainable
}

var _ Operator = &zeroOperator{}
Expand All @@ -188,6 +198,7 @@ func (s *zeroOperator) Next(ctx context.Context) coldata.Batch {

type singleTupleNoInputOperator struct {
ZeroInputNode
NonExplainable
batch coldata.Batch
nexted bool
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/simple_project.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
// columns that aren't needed by later operators.
type simpleProjectOp struct {
OneInputNode
NonExplainable

batch *projectingBatch
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/sort.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ type spooler interface {
// by the general sorter over the whole input.
type allSpooler struct {
OneInputNode
NonExplainable

// inputTypes contains the types of all of the columns from the input.
inputTypes []coltypes.T
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/sort_chunks.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ const (
// buffer when appropriate.
type chunker struct {
OneInputNode
NonExplainable
// inputTypes contains the types of all of the columns from input.
inputTypes []coltypes.T
// inputDone indicates whether input has been fully consumed.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
// StopWatch.
type VectorizedStatsCollector struct {
Operator
NonExplainable
execpb.VectorizedStats

// inputWatch is a single stop watch that is shared with all the input
Expand Down
27 changes: 21 additions & 6 deletions pkg/sql/explain_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@ import (
type explainVecNode struct {
optColumnsSlot

plan planNode
options *tree.ExplainOptions
plan planNode

stmtType tree.StatementType

Expand Down Expand Up @@ -120,27 +121,41 @@ func (n *explainVecNode) startExec(params runParams) error {
sort.Slice(sortedFlows, func(i, j int) bool { return sortedFlows[i].nodeID < sortedFlows[j].nodeID })
tp := treeprinter.New()
root := tp.Child("")
verbose := n.options.Flags.Contains(tree.ExplainFlagVerbose)
for _, flow := range sortedFlows {
node := root.Childf("Node %d", flow.nodeID)
opChains, err := distsqlrun.SupportsVectorized(params.ctx, flowCtx, flow.flow.Processors)
if err != nil {
return err
}
for _, op := range opChains {
formatOpChain(op, node)
formatOpChain(op, node, verbose)
}
}
n.run.lines = tp.FormattedRows()
return nil
}

func formatOpChain(operator exec.OpNode, node treeprinter.Node) {
doFormatOpChain(operator, node.Child(reflect.TypeOf(operator).String()))
func shouldOutput(operator exec.OpNode, verbose bool) bool {
_, nonExplainable := operator.(exec.NonExplainable)
return !nonExplainable || verbose
}
func doFormatOpChain(operator exec.OpNode, node treeprinter.Node) {

func formatOpChain(operator exec.OpNode, node treeprinter.Node, verbose bool) {
if shouldOutput(operator, verbose) {
doFormatOpChain(operator, node.Child(reflect.TypeOf(operator).String()), verbose)
} else {
doFormatOpChain(operator, node, verbose)
}
}
func doFormatOpChain(operator exec.OpNode, node treeprinter.Node, verbose bool) {
for i := 0; i < operator.ChildCount(); i++ {
child := operator.Child(i)
doFormatOpChain(child, node.Child(reflect.TypeOf(child).String()))
if shouldOutput(child, verbose) {
doFormatOpChain(child, node.Child(reflect.TypeOf(child).String()), verbose)
} else {
doFormatOpChain(child, node, verbose)
}
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/dist_vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ statement ok
RESET optimizer

query T
EXPLAIN (VEC) SELECT count(*) FROM kv
EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv
----
├── Node 1
Expand Down Expand Up @@ -134,7 +134,7 @@ EXPLAIN (VEC) SELECT count(*) FROM kv
└── *distsqlrun.colBatchScan

query T
EXPLAIN (VEC) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2
EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2
----
├── Node 1
Expand Down
Loading

0 comments on commit 82de988

Please sign in to comment.