From 786869b6ab99ce8295119c35a528f01f9b23ad01 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 18 Mar 2021 19:23:37 -0700 Subject: [PATCH 1/5] colexec: plumb statsCollectors directly in most cases This commit is mostly a mechanical change which replaces `getStats` functions with using the stats collectors directly for the root materializer and the hash routers. The outbox still expects a callback because of the need to append flow-level stats on the last outbox. Additionally, the drain helper of the materializer has been updated so that the stats are retrieved before the metadata sources are drained. This will be an invariant enforced by the checkers in the follow-up commits. Release note: None --- pkg/sql/colexec/colbuilder/execplan.go | 2 +- pkg/sql/colexec/colbuilder/execplan_test.go | 2 +- pkg/sql/colexec/materializer.go | 46 +++++----- pkg/sql/colexec/materializer_test.go | 8 +- pkg/sql/colexec/types_integration_test.go | 2 +- pkg/sql/colflow/routers.go | 25 +++--- pkg/sql/colflow/routers_test.go | 10 +-- pkg/sql/colflow/vectorized_flow.go | 84 +++++++++---------- .../colflow/vectorized_flow_shutdown_test.go | 4 +- .../vectorized_meta_propagation_test.go | 2 +- .../vectorized_panic_propagation_test.go | 4 +- pkg/sql/distsql/columnar_utils_test.go | 2 +- .../vectorized_panic_propagation_test.go | 2 +- pkg/sql/sem/tree/eval_test/eval_test.go | 2 +- 14 files changed, 94 insertions(+), 101 deletions(-) diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 9795e561e1f3..b5298067a8a5 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -98,7 +98,7 @@ func wrapRowSources( input, inputTypes[i], nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ metadataSources, nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/colexec/colbuilder/execplan_test.go b/pkg/sql/colexec/colbuilder/execplan_test.go index d4d0f2394c34..de0974ed4311 100644 --- a/pkg/sql/colexec/colbuilder/execplan_test.go +++ b/pkg/sql/colexec/colbuilder/execplan_test.go @@ -126,7 +126,7 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { r.Op, []*types.T{types.Int}, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSources */ nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index 3eed27657280..b3697ddf743a 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -76,8 +76,8 @@ type drainHelper struct { // are noops. ctx context.Context - getStats func() []*execinfrapb.ComponentStats - sources colexecop.MetadataSources + statsCollectors []VectorizedStatsCollector + sources colexecop.MetadataSources bufferedMeta []execinfrapb.ProducerMetadata } @@ -92,10 +92,10 @@ var drainHelperPool = sync.Pool{ } func newDrainHelper( - getStats func() []*execinfrapb.ComponentStats, sources colexecop.MetadataSources, + statsCollectors []VectorizedStatsCollector, sources colexecop.MetadataSources, ) *drainHelper { d := drainHelperPool.Get().(*drainHelper) - d.getStats = getStats + d.statsCollectors = statsCollectors d.sources = sources return d } @@ -118,6 +118,20 @@ func (d *drainHelper) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) // The drainHelper wasn't Start()'ed, so this operation is a noop. return nil, nil } + if len(d.statsCollectors) > 0 { + // If statsCollectors is non-nil, then the drainHelper is responsible + // for attaching the execution statistics to the span. Note that we + // neither retrieve the trace from the span (via sp.GetRecording()) nor + // propagate the trace as a metadata here - that is left to the + // materializer (more precisely, to the embedded ProcessorBase) which is + // necessary in order to not collect same trace data twice. + if sp := tracing.SpanFromContext(d.ctx); sp != nil { + for _, s := range d.statsCollectors { + sp.RecordStructured(s.GetStats()) + } + } + d.statsCollectors = nil + } if d.bufferedMeta == nil { d.bufferedMeta = d.sources.DrainMeta(d.ctx) if d.bufferedMeta == nil { @@ -134,25 +148,7 @@ func (d *drainHelper) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) } // ConsumerDone implements the RowSource interface. -func (d *drainHelper) ConsumerDone() { - if d.ctx == nil { - // The drainHelper wasn't Start()'ed, so this operation is a noop. - return - } - if d.getStats != nil { - // If getStats is non-nil, then the drainHelper is responsible for - // attaching the execution statistics to the span, yet we don't get the - // recording from the span - that is left to the materializer (more - // precisely to the embedded ProcessorBase) which is necessary in order - // to not collect same trace data twice. - if sp := tracing.SpanFromContext(d.ctx); sp != nil { - for _, s := range d.getStats() { - sp.RecordStructured(s) - } - } - d.getStats = nil - } -} +func (d *drainHelper) ConsumerDone() {} // ConsumerClosed implements the RowSource interface. func (d *drainHelper) ConsumerClosed() {} @@ -198,7 +194,7 @@ func NewMaterializer( input colexecop.Operator, typs []*types.T, output execinfra.RowReceiver, - getStats func() []*execinfrapb.ComponentStats, + statsCollectors []VectorizedStatsCollector, metadataSources []colexecop.MetadataSource, toClose []colexecop.Closer, cancelFlow func() context.CancelFunc, @@ -208,7 +204,7 @@ func NewMaterializer( ProcessorBase: m.ProcessorBase, input: input, typs: typs, - drainHelper: newDrainHelper(getStats, metadataSources), + drainHelper: newDrainHelper(statsCollectors, metadataSources), converter: colconv.NewAllVecToDatumConverter(len(typs)), row: make(rowenc.EncDatumRow, len(typs)), closers: toClose, diff --git a/pkg/sql/colexec/materializer_test.go b/pkg/sql/colexec/materializer_test.go index 93393953dc72..e7881af976db 100644 --- a/pkg/sql/colexec/materializer_test.go +++ b/pkg/sql/colexec/materializer_test.go @@ -68,7 +68,7 @@ func TestColumnarizeMaterialize(t *testing.T) { c, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSources */ nil, /* toClose */ nil, /* cancelFlow */ @@ -153,7 +153,7 @@ func BenchmarkMaterializer(b *testing.B) { input, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSources */ nil, /* toClose */ nil, /* cancelFlow */ @@ -208,7 +208,7 @@ func TestMaterializerNextErrorAfterConsumerDone(t *testing.T) { &colexecop.CallbackOperator{}, nil, /* typ */ nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ []colexecop.MetadataSource{metadataSource}, nil, /* toClose */ nil, /* cancelFlow */ @@ -258,7 +258,7 @@ func BenchmarkColumnarizeMaterialize(b *testing.B) { c, types, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSources */ nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index a9717e76517c..d71c8fdcdac2 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -93,7 +93,7 @@ func TestSQLTypesIntegration(t *testing.T) { arrowOp, typs, output, - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSources */ nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index 57faba6afc59..6ee4a17f65e9 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" @@ -408,11 +409,11 @@ type HashRouter struct { // One output for each stream. outputs []routerOutput - // getStats, when non-nil, will be called by the hash router to retrieve the - // execution statistics which are then propagated as - // execinfrapb.ProducerMetadata object. This will be done right before - // draining metadataSources. - getStats func() []*execinfrapb.ComponentStats + // statsCollectors, when non-nil, will be retrieved from by the hash router + // and the execution statistics will then be propagated as + // execinfrapb.ProducerMetadata object right before draining + // metadataSources. + statsCollectors []colexec.VectorizedStatsCollector // metadataSources is a slice of colexecop.MetadataSources that need to be // drained when the HashRouter terminates. metadataSources colexecop.MetadataSources @@ -465,7 +466,7 @@ func NewHashRouter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, - getStats func() []*execinfrapb.ComponentStats, + statsCollectors []colexec.VectorizedStatsCollector, toDrain []colexecop.MetadataSource, toClose []colexecop.Closer, ) (*HashRouter, []colexecop.DrainableOperator) { @@ -498,7 +499,7 @@ func NewHashRouter( outputs[i] = op outputsAsOps[i] = op } - return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, getStats, toDrain, toClose), outputsAsOps + return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, statsCollectors, toDrain, toClose), outputsAsOps } func newHashRouterWithOutputs( @@ -506,7 +507,7 @@ func newHashRouterWithOutputs( hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, - getStats func() []*execinfrapb.ComponentStats, + statsCollectors []colexec.VectorizedStatsCollector, toDrain []colexecop.MetadataSource, toClose []colexecop.Closer, ) *HashRouter { @@ -514,7 +515,7 @@ func newHashRouterWithOutputs( OneInputNode: colexecop.NewOneInputNode(input), hashCols: hashCols, outputs: outputs, - getStats: getStats, + statsCollectors: statsCollectors, metadataSources: toDrain, closers: toClose, unblockedEventsChan: unblockEventsChan, @@ -622,10 +623,8 @@ func (r *HashRouter) Run(ctx context.Context) { r.cancelOutputs(ctx, err) } if span != nil { - if r.getStats != nil { - for _, s := range r.getStats() { - span.RecordStructured(s) - } + for _, s := range r.statsCollectors { + span.RecordStructured(s.GetStats()) } if trace := span.GetRecording(); len(trace) > 0 { meta := execinfrapb.GetProducerMeta() diff --git a/pkg/sql/colflow/routers_test.go b/pkg/sql/colflow/routers_test.go index b1c74a4b67a9..27089713264f 100644 --- a/pkg/sql/colflow/routers_test.go +++ b/pkg/sql/colflow/routers_test.go @@ -737,7 +737,7 @@ func TestHashRouterComputesDestination(t *testing.T) { } } - r := newHashRouterWithOutputs(in, []uint32{0}, nil /* ch */, outputs, nil /* getStats */, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs(in, []uint32{0}, nil /* ch */, outputs, nil /* statsCollectors */, nil /* toDrain */, nil /* toClose */) for r.processNextBatch(ctx) { } @@ -780,7 +780,7 @@ func TestHashRouterCancellation(t *testing.T) { in := colexecop.NewRepeatableBatchSource(tu.testAllocator, batch, typs) unbufferedCh := make(chan struct{}) - r := newHashRouterWithOutputs(in, []uint32{0}, unbufferedCh, routerOutputs, nil /* getStats */, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs(in, []uint32{0}, unbufferedCh, routerOutputs, nil /* statsCollectors */, nil /* toDrain */, nil /* toClose */) t.Run("BeforeRun", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) @@ -890,7 +890,7 @@ func TestHashRouterOneOutput(t *testing.T) { queueCfg, colexecop.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* toDrain */ nil, /* toClose */ ) @@ -1084,7 +1084,7 @@ func TestHashRouterRandom(t *testing.T) { hashCols, unblockEventsChan, outputs, - nil, /* getStats */ + nil, /* statsCollectors */ []colexecop.MetadataSource{ colexectestutils.CallbackMetadataSource{ DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { @@ -1317,7 +1317,7 @@ func BenchmarkHashRouter(b *testing.B) { queueCfg, &colexecop.TestingSemaphore{}, diskAccounts, - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* toDrain */ nil, /* toClose */ ) diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index edd65a407287..484cb4d16308 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" @@ -355,17 +356,41 @@ func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( return newNetworkVectorizedStatsCollector(op, component, inputWatch, inbox, latency) } -// finishVectorizedStatsCollectors finishes the given stats collectors and -// returns all of their stats. -func finishVectorizedStatsCollectors( +// makeGetStatsFnForOutbox creates a function that will retrieve all execution +// statistics that the outbox is responsible for, nil is returned if stats are +// not being collected. +func (s *vectorizedFlowCreator) makeGetStatsFnForOutbox( + flowCtx *execinfra.FlowCtx, statsCollectors []colexec.VectorizedStatsCollector, -) []*execinfrapb.ComponentStats { - // TODO(yuzefovich): consider pooling ComponentStats objects. - result := make([]*execinfrapb.ComponentStats, 0, len(statsCollectors)) - for _, vsc := range statsCollectors { - result = append(result, vsc.GetStats()) + originNodeID roachpb.NodeID, +) func() []*execinfrapb.ComponentStats { + if !s.recordingStats { + return nil + } + return func() []*execinfrapb.ComponentStats { + lastOutboxOnRemoteNode := atomic.AddInt32(&s.numOutboxesDrained, 1) == atomic.LoadInt32(&s.numOutboxes) && !s.isGatewayNode + numResults := len(statsCollectors) + if lastOutboxOnRemoteNode { + numResults++ + } + result := make([]*execinfrapb.ComponentStats, 0, numResults) + for _, s := range statsCollectors { + result = append(result, s.GetStats()) + } + if lastOutboxOnRemoteNode { + // At the last outbox, we can accurately retrieve stats for the + // whole flow from parent monitors. These stats are added to a + // flow-level span. + result = append(result, &execinfrapb.ComponentStats{ + Component: execinfrapb.FlowComponentID(base.SQLInstanceID(originNodeID), flowCtx.ID), + FlowStats: execinfrapb.FlowStats{ + MaxMemUsage: optional.MakeUint(uint64(flowCtx.EvalCtx.Mon.MaximumBytes())), + MaxDiskUsage: optional.MakeUint(uint64(flowCtx.DiskMonitor.MaximumBytes())), + }, + }) + } + return result } - return result } type runFn func(context.Context, context.CancelFunc) @@ -694,7 +719,7 @@ func (s *vectorizedFlowCreator) setupRouter( input colexecop.Operator, outputTyps []*types.T, output *execinfrapb.OutputRouterSpec, - getStats func() []*execinfrapb.ComponentStats, + statsCollectors []colexec.VectorizedStatsCollector, metadataSources []colexecop.MetadataSource, toClose []colexecop.Closer, factory coldata.ColumnFactory, @@ -721,7 +746,7 @@ func (s *vectorizedFlowCreator) setupRouter( router, outputs := NewHashRouter( allocators, input, outputTyps, output.HashColumns, execinfra.GetWorkMemLimit(flowCtx.Cfg), s.diskQueueCfg, s.fdSemaphore, - diskAccounts, getStats, metadataSources, toClose, + diskAccounts, statsCollectors, metadataSources, toClose, ) runRouter := func(ctx context.Context, _ context.CancelFunc) { router.Run(logtags.AddTag(ctx, "hashRouterID", strings.Join(streamIDs, ","))) @@ -970,9 +995,7 @@ func (s *vectorizedFlowCreator) setupOutput( op, opOutputTypes, output, - func() []*execinfrapb.ComponentStats { - return finishVectorizedStatsCollectors(statsCollectors) - }, + statsCollectors, metadataSources, toClose, factory, @@ -993,29 +1016,10 @@ func (s *vectorizedFlowCreator) setupOutput( } case execinfrapb.StreamEndpointSpec_REMOTE: // Set up an Outbox. - var getStats func() []*execinfrapb.ComponentStats - if s.recordingStats { - // If recording stats, we add a metadata source that will generate all - // stats data as metadata for the stats collectors created so far. - getStats = func() []*execinfrapb.ComponentStats { - result := finishVectorizedStatsCollectors(statsCollectors) - if atomic.AddInt32(&s.numOutboxesDrained, 1) == atomic.LoadInt32(&s.numOutboxes) && !s.isGatewayNode { - // At the last outbox, we can accurately retrieve stats for - // the whole flow from parent monitors. These stats are - // added to a flow-level span. - result = append(result, &execinfrapb.ComponentStats{ - Component: execinfrapb.FlowComponentID(base.SQLInstanceID(outputStream.OriginNodeID), flowCtx.ID), - FlowStats: execinfrapb.FlowStats{ - MaxMemUsage: optional.MakeUint(uint64(flowCtx.EvalCtx.Mon.MaximumBytes())), - MaxDiskUsage: optional.MakeUint(uint64(flowCtx.DiskMonitor.MaximumBytes())), - }, - }) - } - return result - } - } outbox, err := s.setupRemoteOutputStream( - ctx, flowCtx, op, opOutputTypes, outputStream, getStats, metadataSources, toClose, factory, + ctx, flowCtx, op, opOutputTypes, outputStream, + s.makeGetStatsFnForOutbox(flowCtx, statsCollectors, outputStream.OriginNodeID), + metadataSources, toClose, factory, ) if err != nil { return err @@ -1025,19 +1029,13 @@ func (s *vectorizedFlowCreator) setupOutput( s.leaves = append(s.leaves, outbox) case execinfrapb.StreamEndpointSpec_SYNC_RESPONSE: // Make the materializer, which will write to the given receiver. - var getStats func() []*execinfrapb.ComponentStats - if s.recordingStats { - getStats = func() []*execinfrapb.ComponentStats { - return finishVectorizedStatsCollectors(statsCollectors) - } - } proc, err := colexec.NewMaterializer( flowCtx, pspec.ProcessorID, op, opOutputTypes, s.syncFlowConsumer, - getStats, + statsCollectors, metadataSources, toClose, s.getCancelFlowFn, diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index 99e2b38bafd9..39cd9ad01a45 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -203,7 +203,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { queueCfg, &colexecop.TestingSemaphore{}, diskAccounts, - nil, /* getStats */ + nil, /* statsCollectors */ toDrain, nil, /* toClose */ ) @@ -361,7 +361,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { materializerInput, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ []colexecop.MetadataSource{materializerMetadataSource}, []colexecop.Closer{callbackCloser{closeCb: func() error { materializerCalledClose = true diff --git a/pkg/sql/colflow/vectorized_meta_propagation_test.go b/pkg/sql/colflow/vectorized_meta_propagation_test.go index fd40e6915fb2..8a94eb73c790 100644 --- a/pkg/sql/colflow/vectorized_meta_propagation_test.go +++ b/pkg/sql/colflow/vectorized_meta_propagation_test.go @@ -79,7 +79,7 @@ func TestVectorizedMetaPropagation(t *testing.T) { noop, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ []colexecop.MetadataSource{col}, nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index b6a894bf5a9c..b9f32437fd04 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -61,7 +61,7 @@ func TestVectorizedInternalPanic(t *testing.T) { vee, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSourceQueue */ nil, /* toClose */ nil, /* cancelFlow */ @@ -108,7 +108,7 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { nvee, typs, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSourceQueue */ nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index a60b8800ad0a..5a201e94805d 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -175,7 +175,7 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { result.Op, args.pspec.ResultTypes, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ result.MetadataSources, result.ToClose, nil, /* cancelFlow */ diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index 9f95e9a12ab9..cbdf42bb6a20 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -60,7 +60,7 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { }, nil, /* typs */ &distsqlutils.RowBuffer{}, - nil, /* getStats */ + nil, /* statsCollectors */ nil, /* metadataSourceQueue */ nil, /* toClose */ nil, /* cancelFlow */ diff --git a/pkg/sql/sem/tree/eval_test/eval_test.go b/pkg/sql/sem/tree/eval_test/eval_test.go index 1a4490864567..aae4750b3b54 100644 --- a/pkg/sql/sem/tree/eval_test/eval_test.go +++ b/pkg/sql/sem/tree/eval_test/eval_test.go @@ -203,7 +203,7 @@ func TestEval(t *testing.T) { result.Op, []*types.T{typedExpr.ResolvedType()}, nil, /* output */ - nil, /* getStats */ + nil, /* statsCollectors */ result.MetadataSources, nil, /* toClose */ nil, /* cancelFlow */ From c3b16171e2a886a9a92a1d824226d06e89fffeed Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 18 Mar 2021 20:59:59 -0700 Subject: [PATCH 2/5] colexecargs: tightly couple root op with metadata sources and closers This commit introduces a utility struct `OpWithMetaInfo` that has a root operator as well as the metadata sources and closers present in the tree handling of which (draining and closing, respectively) hasn't been claimed yet. The struct will be the same (and will replace) `opDAGWithMetaSources` and `SynchronizerInput` by the follow-up commit. The motivation behind this change is that we want to attribute stats collectors, metadata sources, and - why not - closers to the tree of operators in which they are present. Release note: None --- pkg/sql/colexec/colbuilder/BUILD.bazel | 1 - pkg/sql/colexec/colbuilder/execplan.go | 174 +++++++-------- pkg/sql/colexec/colbuilder/execplan_test.go | 7 +- pkg/sql/colexec/colexecargs/op_creation.go | 57 ++--- .../colexec/colexecbase/ordinality_test.go | 4 +- .../colexecbase/simple_project_test.go | 2 +- pkg/sql/colexec/colexectestutils/BUILD.bazel | 1 + pkg/sql/colexec/colexectestutils/args.go | 26 +++ .../colexec/colexectestutils/proj_utils.go | 4 +- .../colexecwindow/window_functions_test.go | 6 +- pkg/sql/colexec/crossjoiner_test.go | 12 +- pkg/sql/colexec/external_distinct_test.go | 6 +- .../colexec/external_hash_aggregator_test.go | 4 +- pkg/sql/colexec/external_hash_joiner_test.go | 6 +- pkg/sql/colexec/external_sort.go | 2 +- pkg/sql/colexec/external_sort_test.go | 6 +- pkg/sql/colexec/hashjoiner_test.go | 10 +- pkg/sql/colexec/is_null_ops_test.go | 6 +- pkg/sql/colexec/materializer.go | 13 +- pkg/sql/colexec/materializer_test.go | 20 +- pkg/sql/colexec/mergejoiner_test.go | 6 +- pkg/sql/colexec/ordered_synchronizer.eg.go | 8 +- pkg/sql/colexec/ordered_synchronizer_test.go | 4 +- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 8 +- .../parallel_unordered_synchronizer.go | 18 +- .../parallel_unordered_synchronizer_test.go | 8 +- .../colexec/serial_unordered_synchronizer.go | 6 +- .../serial_unordered_synchronizer_test.go | 7 +- pkg/sql/colexec/types_integration_test.go | 5 +- pkg/sql/colflow/colbatch_scan_test.go | 4 +- pkg/sql/colflow/routers.go | 17 +- pkg/sql/colflow/routers_test.go | 45 ++-- pkg/sql/colflow/vectorized_flow.go | 199 +++++++----------- .../colflow/vectorized_flow_shutdown_test.go | 29 +-- pkg/sql/colflow/vectorized_flow_space_test.go | 19 +- .../vectorized_meta_propagation_test.go | 8 +- .../vectorized_panic_propagation_test.go | 9 +- pkg/sql/distsql/BUILD.bazel | 1 + pkg/sql/distsql/columnar_utils_test.go | 7 +- .../vectorized_panic_propagation_test.go | 7 +- pkg/sql/sem/tree/eval_test/eval_test.go | 12 +- 41 files changed, 392 insertions(+), 402 deletions(-) create mode 100644 pkg/sql/colexec/colexectestutils/args.go diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index 85fb93347d51..b501271b6496 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -57,7 +57,6 @@ go_test( "//pkg/sql/catalog/catalogkv", "//pkg/sql/colexec", "//pkg/sql/colexec/colexecargs", - "//pkg/sql/colexecop", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/randgen", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index b5298067a8a5..841f4b2fc859 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -47,7 +47,7 @@ import ( "github.com/cockroachdb/errors" ) -func checkNumIn(inputs []colexecop.Operator, numIn int) error { +func checkNumIn(inputs []colexecargs.OpWithMetaInfo, numIn int) error { if len(inputs) != numIn { return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs)) } @@ -61,46 +61,50 @@ func checkNumIn(inputs []colexecop.Operator, numIn int) error { func wrapRowSources( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []colexecop.Operator, + inputs []colexecargs.OpWithMetaInfo, inputTypes [][]*types.T, - args *colexecargs.NewColOperatorArgs, + streamingMemAccount *mon.BoundAccount, + processorID int32, newToWrap func([]execinfra.RowSource) (execinfra.RowSource, error), materializerSafeToRelease bool, factory coldata.ColumnFactory, ) (*colexec.Columnarizer, []execinfra.Releasable, error) { var toWrapInputs []execinfra.RowSource var releasables []execinfra.Releasable - for i, input := range inputs { + for i := range inputs { + inputInfo := &inputs[i] // Optimization: if the input is a Columnarizer, its input is // necessarily a execinfra.RowSource, so remove the unnecessary // conversion. - if c, ok := input.(*colexec.Columnarizer); ok { + if c, ok := inputInfo.Root.(*colexec.Columnarizer); ok { // Since this Columnarizer has been previously added to Closers and // MetadataSources, this call ensures that all future calls are noops. // Modifying the slices at this stage is difficult. c.MarkAsRemovedFromFlow() toWrapInputs = append(toWrapInputs, c.Input()) } else { - var metadataSources colexecop.MetadataSources - if len(args.MetadataSources) > i { - // In some testing paths, MetadataSources might be left unset, - // so we check whether the slice has ith element. In the - // production setting though the length of Inputs is always the - // same as the length of MetadataSources. - metadataSources = args.MetadataSources[i] - // We pass on the responsibility of draining metadata sources to - // the materializer. - args.MetadataSources[i] = nil - } + inputInfoCopy := *inputInfo + // We pass on the responsibility of draining metadata sources and + // closing the closers to the materializer. + // TODO(yuzefovich): possibly set the length to 0 in order to be + // able to pool the underlying slice. + inputInfo.MetadataSources = nil + inputInfo.ToClose = nil + // Note that this materializer is *not* added to the set of + // releasables because in some cases it could be released before + // being closed. Namely, this would occur if we have a subquery + // with LocalPlanNode core and a materializer is added in order to + // wrap that core - what will happen is that all releasables are put + // back into their pools upon the subquery's flow cleanup, yet the + // subquery planNode tree isn't closed yet since its closure is down + // when the main planNode tree is being closed. toWrapInput, err := colexec.NewMaterializer( flowCtx, - args.Spec.ProcessorID, - input, + processorID, + inputInfoCopy, inputTypes[i], nil, /* output */ nil, /* statsCollectors */ - metadataSources, - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { @@ -125,11 +129,11 @@ func wrapRowSources( var c *colexec.Columnarizer if proc.MustBeStreaming() { c, err = colexec.NewStreamingColumnarizer( - ctx, colmem.NewAllocator(ctx, args.StreamingMemAccount, factory), flowCtx, args.Spec.ProcessorID, toWrap, + ctx, colmem.NewAllocator(ctx, streamingMemAccount, factory), flowCtx, processorID, toWrap, ) } else { c, err = colexec.NewBufferingColumnarizer( - ctx, colmem.NewAllocator(ctx, args.StreamingMemAccount, factory), flowCtx, args.Spec.ProcessorID, toWrap, + ctx, colmem.NewAllocator(ctx, streamingMemAccount, factory), flowCtx, processorID, toWrap, ) } return c, releasables, err @@ -562,7 +566,7 @@ func (r opResult) createAndWrapRowSource( ctx context.Context, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, - inputs []colexecop.Operator, + inputs []colexecargs.OpWithMetaInfo, inputTypes [][]*types.T, spec *execinfrapb.ProcessorSpec, factory coldata.ColumnFactory, @@ -587,7 +591,8 @@ func (r opResult) createAndWrapRowSource( flowCtx, inputs, inputTypes, - args, + args.StreamingMemAccount, + spec.ProcessorID, func(inputs []execinfra.RowSource) (execinfra.RowSource, error) { // We provide a slice with a single nil as 'outputs' parameter // because all processors expect a single output. Passing nil is ok @@ -618,11 +623,11 @@ func (r opResult) createAndWrapRowSource( if err != nil { return err } - r.Op = c + r.Root = c if args.TestingKnobs.PlanInvariantsCheckers { - r.Op = colexec.NewInvariantsChecker(r.Op) + r.Root = colexec.NewInvariantsChecker(r.Root) } - r.MetadataSources = append(r.MetadataSources, r.Op.(colexecop.MetadataSource)) + r.MetadataSources = append(r.MetadataSources, r.Root.(colexecop.MetadataSource)) r.ToClose = append(r.ToClose, c) r.Releasables = append(r.Releasables, releasables...) return nil @@ -729,7 +734,7 @@ func NewColOperator( if err := checkNumIn(inputs, 1); err != nil { return r, err } - result.Op = colexecop.NewNoop(inputs[0]) + result.Root = colexecop.NewNoop(inputs[0].Root) result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) @@ -743,7 +748,7 @@ func NewColOperator( core.Values.NumRows, len(core.Values.Columns), ) } - result.Op = colexecutils.NewFixedNumTuplesNoInputOp(streamingAllocator, int(core.Values.NumRows), nil /* opToInitialize */) + result.Root = colexecutils.NewFixedNumTuplesNoInputOp(streamingAllocator, int(core.Values.NumRows), nil /* opToInitialize */) result.ColumnTypes = make([]*types.T, len(core.Values.Columns)) for i, col := range core.Values.Columns { result.ColumnTypes[i] = col.Type @@ -766,12 +771,12 @@ func NewColOperator( if log.V(1) { log.Infof(ctx, "made op %T\n", scanOp) } - result.Op = scanOp + result.Root = scanOp if args.TestingKnobs.PlanInvariantsCheckers { - result.Op = colexec.NewInvariantsChecker(result.Op) + result.Root = colexec.NewInvariantsChecker(result.Root) } result.KVReader = scanOp - result.MetadataSources = append(result.MetadataSources, result.Op.(colexecop.MetadataSource)) + result.MetadataSources = append(result.MetadataSources, result.Root.(colexecop.MetadataSource)) result.Releasables = append(result.Releasables, scanOp) // We want to check for cancellation once per input batch, and @@ -780,7 +785,7 @@ func NewColOperator( // are extremely fast. However, some of the long-running operators // (for example, sorter) are still responsible for doing the // cancellation check on their own while performing long operations. - result.Op = colexecutils.NewCancelChecker(result.Op) + result.Root = colexecutils.NewCancelChecker(result.Root) result.ColumnTypes = scanOp.ResultTypes result.ToClose = append(result.ToClose, scanOp) @@ -791,7 +796,7 @@ func NewColOperator( result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) - result.Op = inputs[0] + result.Root = inputs[0].Root if err := result.planAndMaybeWrapFilter( ctx, flowCtx, evalCtx, args, spec.ProcessorID, core.Filterer.Filter, factory, ); err != nil { @@ -814,14 +819,14 @@ func NewColOperator( // TableReader, so we end up creating an orphaned colBatchScan. // We should avoid that. Ideally the optimizer would not plan a // scan in this unusual case. - result.Op, err = colexecutils.NewFixedNumTuplesNoInputOp(streamingAllocator, 1 /* numTuples */, inputs[0]), nil + result.Root, err = colexecutils.NewFixedNumTuplesNoInputOp(streamingAllocator, 1 /* numTuples */, inputs[0].Root), nil // We make ColumnTypes non-nil so that sanity check doesn't // panic. result.ColumnTypes = []*types.T{} break } if aggSpec.IsRowCount() { - result.Op, err = colexec.NewCountOp(streamingAllocator, inputs[0]), nil + result.Root, err = colexec.NewCountOp(streamingAllocator, inputs[0].Root), nil result.ColumnTypes = []*types.T{types.Int} break } @@ -834,7 +839,7 @@ func NewColOperator( inputTypes := make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(inputTypes, spec.Input[0].ColumnTypes) newAggArgs := &colexecagg.NewAggregatorArgs{ - Input: inputs[0], + Input: inputs[0].Root, InputTypes: inputTypes, Spec: aggSpec, EvalCtx: evalCtx, @@ -869,7 +874,7 @@ func NewColOperator( evalCtx.SingleDatumAggMemAccount = hashAggregatorUnlimitedMemAccount // The second argument is nil because we disable the // tracking of the input tuples. - result.Op, err = colexec.NewHashAggregator(newAggArgs, nil /* newSpillingQueueArgs */) + result.Root, err = colexec.NewHashAggregator(newAggArgs, nil /* newSpillingQueueArgs */) } else { // We will divide the available memory equally between the // two usages - the hash aggregation itself and the input @@ -914,8 +919,8 @@ func NewColOperator( // case, the wrapped aggregate functions might hit a memory // error even when used by the external hash aggregator). evalCtx.SingleDatumAggMemAccount = ehaMemAccount - result.Op = colexec.NewOneInputDiskSpiller( - inputs[0], inMemoryHashAggregator.(colexecop.BufferingInMemoryOperator), + result.Root = colexec.NewOneInputDiskSpiller( + inputs[0].Root, inMemoryHashAggregator.(colexecop.BufferingInMemoryOperator), hashAggregatorMemMonitorName, func(input colexecop.Operator) colexecop.Operator { newAggArgs := *newAggArgs @@ -941,9 +946,9 @@ func NewColOperator( evalCtx.SingleDatumAggMemAccount = streamingMemAccount newAggArgs.Allocator = streamingAllocator newAggArgs.MemAccount = streamingMemAccount - result.Op, err = colexec.NewOrderedAggregator(newAggArgs) + result.Root, err = colexec.NewOrderedAggregator(newAggArgs) } - result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) + result.ToClose = append(result.ToClose, result.Root.(colexecop.Closer)) case core.Distinct != nil: if err := checkNumIn(inputs, 1); err != nil { @@ -952,7 +957,7 @@ func NewColOperator( result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) if len(core.Distinct.OrderedColumns) == len(core.Distinct.DistinctColumns) { - result.Op, err = colexecbase.NewOrderedDistinct(inputs[0], core.Distinct.OrderedColumns, result.ColumnTypes) + result.Root, err = colexecbase.NewOrderedDistinct(inputs[0].Root, core.Distinct.OrderedColumns, result.ColumnTypes) } else { // We have separate unit tests that instantiate in-memory // distinct operators, so we don't need to look at @@ -967,12 +972,12 @@ func NewColOperator( // of distinct tuples in the input is about 0.01 or less. allocator := colmem.NewAllocator(ctx, distinctMemAccount, factory) inMemoryUnorderedDistinct := colexec.NewUnorderedDistinct( - allocator, inputs[0], core.Distinct.DistinctColumns, result.ColumnTypes, + allocator, inputs[0].Root, core.Distinct.DistinctColumns, result.ColumnTypes, ) edOpName := "external-distinct" diskAccount := result.createDiskAccount(ctx, flowCtx, edOpName, spec.ProcessorID) - result.Op = colexec.NewOneInputDiskSpiller( - inputs[0], inMemoryUnorderedDistinct.(colexecop.BufferingInMemoryOperator), + result.Root = colexec.NewOneInputDiskSpiller( + inputs[0].Root, inMemoryUnorderedDistinct.(colexecop.BufferingInMemoryOperator), distinctMemMonitorName, func(input colexecop.Operator) colexecop.Operator { unlimitedAllocator := colmem.NewAllocator( @@ -991,7 +996,7 @@ func NewColOperator( }, args.TestingKnobs.SpillingCallbackFn, ) - result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) + result.ToClose = append(result.ToClose, result.Root.(colexecop.Closer)) } case core.Ordinality != nil: @@ -999,7 +1004,7 @@ func NewColOperator( return r, err } outputIdx := len(spec.Input[0].ColumnTypes) - result.Op = colexecbase.NewOrdinalityOp(streamingAllocator, inputs[0], outputIdx) + result.Root = colexecbase.NewOrdinalityOp(streamingAllocator, inputs[0].Root, outputIdx) result.ColumnTypes = appendOneType(spec.Input[0].ColumnTypes, types.Int) case core.HashJoiner != nil: @@ -1019,17 +1024,17 @@ func NewColOperator( crossJoinerMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID) crossJoinerDiskAcc := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator(ctx, crossJoinerMemAccount, factory) - result.Op = colexecjoin.NewCrossJoiner( + result.Root = colexecjoin.NewCrossJoiner( unlimitedAllocator, memoryLimit, args.DiskQueueCfg, args.FDSemaphore, core.HashJoiner.Type, - inputs[0], inputs[1], + inputs[0].Root, inputs[1].Root, leftTypes, rightTypes, crossJoinerDiskAcc, ) - result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) + result.ToClose = append(result.ToClose, result.Root.(colexecop.Closer)) } else { var hashJoinerMemMonitorName string var hashJoinerMemAccount *mon.BoundAccount @@ -1057,19 +1062,19 @@ func NewColOperator( inMemoryHashJoiner := colexecjoin.NewHashJoiner( colmem.NewAllocator(ctx, hashJoinerMemAccount, factory), - hashJoinerUnlimitedAllocator, hjSpec, inputs[0], inputs[1], + hashJoinerUnlimitedAllocator, hjSpec, inputs[0].Root, inputs[1].Root, colexecjoin.HashJoinerInitialNumBuckets, memoryLimit, ) if useStreamingMemAccountForBuffering || args.TestingKnobs.DiskSpillingDisabled { // We will not be creating a disk-backed hash joiner because // we're running a test that explicitly asked for only // in-memory hash joiner. - result.Op = inMemoryHashJoiner + result.Root = inMemoryHashJoiner } else { opName := "external-hash-joiner" diskAccount := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) - result.Op = colexec.NewTwoInputDiskSpiller( - inputs[0], inputs[1], inMemoryHashJoiner.(colexecop.BufferingInMemoryOperator), + result.Root = colexec.NewTwoInputDiskSpiller( + inputs[0].Root, inputs[1].Root, inMemoryHashJoiner.(colexecop.BufferingInMemoryOperator), hashJoinerMemMonitorName, func(inputOne, inputTwo colexecop.Operator) colexecop.Operator { unlimitedAllocator := colmem.NewAllocator( @@ -1136,7 +1141,7 @@ func NewColOperator( mj, err := colexecjoin.NewMergeJoinOp( unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, args.FDSemaphore, - joinType, inputs[0], inputs[1], leftTypes, rightTypes, + joinType, inputs[0].Root, inputs[1].Root, leftTypes, rightTypes, core.MergeJoiner.LeftOrdering.Columns, core.MergeJoiner.RightOrdering.Columns, diskAccount, ) @@ -1144,7 +1149,7 @@ func NewColOperator( return r, err } - result.Op = mj + result.Root = mj result.ToClose = append(result.ToClose, mj.(colexecop.Closer)) result.ColumnTypes = core.MergeJoiner.Type.MakeOutputTypes(leftTypes, rightTypes) @@ -1160,12 +1165,12 @@ func NewColOperator( if err := checkNumIn(inputs, 1); err != nil { return r, err } - input := inputs[0] + input := inputs[0].Root result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) ordering := core.Sorter.OutputOrdering matchLen := core.Sorter.OrderingMatchLen - result.Op, err = result.createDiskBackedSort( + result.Root, err = result.createDiskBackedSort( ctx, flowCtx, args, input, result.ColumnTypes, ordering, matchLen, 0, /* maxNumberPartitions */ spec.ProcessorID, post, "" /* opNamePrefix */, factory, ) @@ -1175,7 +1180,7 @@ func NewColOperator( return r, err } opNamePrefix := "window-" - input := inputs[0] + input := inputs[0].Root result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) for _, wf := range core.Windower.WindowFns { @@ -1234,9 +1239,9 @@ func NewColOperator( outputIdx := int(wf.OutputColIdx + tempColOffset) switch windowFn { case execinfrapb.WindowerSpec_ROW_NUMBER: - result.Op = colexecwindow.NewRowNumberOperator(streamingAllocator, input, outputIdx, partitionColIdx) + result.Root = colexecwindow.NewRowNumberOperator(streamingAllocator, input, outputIdx, partitionColIdx) case execinfrapb.WindowerSpec_RANK, execinfrapb.WindowerSpec_DENSE_RANK: - result.Op, err = colexecwindow.NewRankOperator( + result.Root, err = colexecwindow.NewRankOperator( streamingAllocator, input, windowFn, wf.Ordering.Columns, outputIdx, partitionColIdx, peersColIdx, ) @@ -1250,7 +1255,7 @@ func NewColOperator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID), factory, ) diskAcc := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) - result.Op, err = colexecwindow.NewRelativeRankOperator( + result.Root, err = colexecwindow.NewRelativeRankOperator( unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, args.FDSemaphore, input, typs, windowFn, wf.Ordering.Columns, outputIdx, partitionColIdx, peersColIdx, diskAcc, @@ -1258,7 +1263,7 @@ func NewColOperator( // NewRelativeRankOperator sometimes returns a constOp when // there are no ordering columns, so we check that the // returned operator is a Closer. - if c, ok := result.Op.(colexecop.Closer); ok { + if c, ok := result.Root.(colexecop.Closer); ok { result.ToClose = append(result.ToClose, c) } default: @@ -1274,7 +1279,7 @@ func NewColOperator( projection = append(projection, i) } projection = append(projection, wf.OutputColIdx+tempColOffset) - result.Op = colexecbase.NewSimpleProjectOp(result.Op, int(wf.OutputColIdx+tempColOffset), projection) + result.Root = colexecbase.NewSimpleProjectOp(result.Root, int(wf.OutputColIdx+tempColOffset), projection) } _, returnType, err := execinfrapb.GetWindowFunctionInfo(wf.Func, []*types.T{}...) @@ -1282,7 +1287,7 @@ func NewColOperator( return r, err } result.ColumnTypes = appendOneType(result.ColumnTypes, returnType) - input = result.Op + input = result.Root } default: @@ -1295,14 +1300,14 @@ func NewColOperator( } if log.V(1) { - log.Infof(ctx, "made op %T\n", result.Op) + log.Infof(ctx, "made op %T\n", result.Root) } // Note: at this point, it is legal for ColumnTypes to be empty (it is // legal for empty rows to be passed between processors). ppr := postProcessResult{ - Op: result.Op, + Op: result.Root, ColumnTypes: result.ColumnTypes, } err = ppr.planPostProcessSpec(ctx, flowCtx, evalCtx, args, post, factory) @@ -1349,10 +1354,10 @@ func NewColOperator( for i := range args.Spec.ResultTypes { expected, actual := args.Spec.ResultTypes[i], r.ColumnTypes[i] if !actual.Identical(expected) { - input := r.Op + input := r.Root castedIdx := len(r.ColumnTypes) resultTypes := appendOneType(r.ColumnTypes, expected) - r.Op, err = colexecbase.GetCastOperator( + r.Root, err = colexecbase.GetCastOperator( streamingAllocator, input, i, castedIdx, actual, expected, ) if err != nil { @@ -1367,7 +1372,7 @@ func NewColOperator( post.RenderExprs[j].Expr = fmt.Sprintf("@%d", j+1) } post.RenderExprs[castedIdx].Expr = fmt.Sprintf("@%d::%s", i+1, expected.SQLStandardName()) - result.Op = input + result.Root = input if err = result.wrapPostProcessSpec(ctx, flowCtx, args, post, resultTypes, factory, err); err != nil { return r, err } @@ -1388,19 +1393,20 @@ func NewColOperator( } } if projection != nil { - r.Op, r.ColumnTypes = addProjection(r.Op, r.ColumnTypes, projection) + r.Root, r.ColumnTypes = addProjection(r.Root, r.ColumnTypes, projection) } if args.TestingKnobs.PlanInvariantsCheckers { // Plan an invariants checker if it isn't already the root of the tree. - if _, isInvariantsChecker := r.Op.(*colexec.InvariantsChecker); !isInvariantsChecker { - r.Op = colexec.NewInvariantsChecker(r.Op) + if _, isInvariantsChecker := r.Root.(*colexec.InvariantsChecker); !isInvariantsChecker { + r.Root = colexec.NewInvariantsChecker(r.Root) } } - // Handle the metadata sources from the input trees. Note that it is - // possible that we have created a materializer which took over draining - // the sources given to us by the caller. - for i := range args.MetadataSources { - r.MetadataSources = append(r.MetadataSources, args.MetadataSources[i]...) + // Handle the metadata sources and the closers from the input trees. Note + // that it is possible that we have created materializers which took over + // the responsibility over those objects. + for i := range inputs { + r.MetadataSources = append(r.MetadataSources, inputs[i].MetadataSources...) + r.ToClose = append(r.ToClose, inputs[i].ToClose...) } if util.CrdbTestBuild { r.AssertInvariants() @@ -1420,7 +1426,7 @@ func (r opResult) planAndMaybeWrapFilter( factory coldata.ColumnFactory, ) error { op, err := planFilterExpr( - ctx, flowCtx, evalCtx, r.Op, r.ColumnTypes, filter, args.StreamingMemAccount, factory, args.ExprHelper, + ctx, flowCtx, evalCtx, r.Root, r.ColumnTypes, filter, args.StreamingMemAccount, factory, args.ExprHelper, ) if err != nil { // ON expression planning failed. Fall back to planning the filter @@ -1443,11 +1449,11 @@ func (r opResult) planAndMaybeWrapFilter( ResultTypes: args.Spec.ResultTypes, } return r.createAndWrapRowSource( - ctx, flowCtx, args, []colexecop.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, + ctx, flowCtx, args, []colexecargs.OpWithMetaInfo{r.OpWithMetaInfo}, [][]*types.T{r.ColumnTypes}, filtererSpec, factory, err, ) } - r.Op = op + r.Root = op return nil } @@ -1473,7 +1479,7 @@ func (r opResult) wrapPostProcessSpec( ResultTypes: resultTypes, } return r.createAndWrapRowSource( - ctx, flowCtx, args, []colexecop.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, + ctx, flowCtx, args, []colexecargs.OpWithMetaInfo{r.OpWithMetaInfo}, [][]*types.T{r.ColumnTypes}, noopSpec, factory, causeToWrap, ) } @@ -1616,7 +1622,7 @@ type postProcessResult struct { } func (r opResult) updateWithPostProcessResult(ppr postProcessResult) { - r.Op = ppr.Op + r.Root = ppr.Op r.ColumnTypes = make([]*types.T, len(ppr.ColumnTypes)) copy(r.ColumnTypes, ppr.ColumnTypes) } diff --git a/pkg/sql/colexec/colbuilder/execplan_test.go b/pkg/sql/colexec/colbuilder/execplan_test.go index de0974ed4311..edcb7fc33895 100644 --- a/pkg/sql/colexec/colbuilder/execplan_test.go +++ b/pkg/sql/colexec/colbuilder/execplan_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/randgen" @@ -114,7 +113,7 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { Post: execinfrapb.PostProcessSpec{RenderExprs: []execinfrapb.Expression{{Expr: "@1 - 1"}}}, ResultTypes: []*types.T{types.Int}, }, - Inputs: []colexecop.Operator{r.Op}, + Inputs: []colexecargs.OpWithMetaInfo{{Root: r.Root}}, StreamingMemAccount: &streamingMemAcc, } r, err = NewColOperator(ctx, flowCtx, args) @@ -123,12 +122,10 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { m, err := colexec.NewMaterializer( flowCtx, 0, /* processorID */ - r.Op, + r.OpWithMetaInfo, []*types.T{types.Int}, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSources */ - nil, /* toClose */ nil, /* cancelFlow */ ) require.NoError(t, err) diff --git a/pkg/sql/colexec/colexecargs/op_creation.go b/pkg/sql/colexec/colexecargs/op_creation.go index e25fd362cbfa..2e6d02fb2896 100644 --- a/pkg/sql/colexec/colexecargs/op_creation.go +++ b/pkg/sql/colexec/colexecargs/op_creation.go @@ -32,25 +32,33 @@ import ( var TestNewColOperator func(ctx context.Context, flowCtx *execinfra.FlowCtx, args *NewColOperatorArgs, ) (r *NewColOperatorResult, err error) +// OpWithMetaInfo stores a colexecop.Operator together with miscellaneous meta +// information about the tree rooted in that operator. +// TODO(yuzefovich): figure out the story about pooling these objects. +type OpWithMetaInfo struct { + Root colexecop.Operator + // MetadataSources are all sources of the metadata that are present in the + // tree rooted in Root for which the responsibility of draining hasn't been + // claimed yet. + MetadataSources colexecop.MetadataSources + // ToClose are all colexecop.Closers that are present in the tree rooted in + // Root for which the responsibility of closing hasn't been claimed yet. + ToClose colexecop.Closers +} + // NewColOperatorArgs is a helper struct that encompasses all of the input // arguments to NewColOperator call. type NewColOperatorArgs struct { Spec *execinfrapb.ProcessorSpec - Inputs []colexecop.Operator + Inputs []OpWithMetaInfo StreamingMemAccount *mon.BoundAccount ProcessorConstructor execinfra.ProcessorConstructor LocalProcessors []execinfra.LocalProcessor - // MetadataSources are all sources of the metadata that are present in the - // trees rooted in Inputs. The slice has the same length as Inputs. If - // NewColOperator call creates a colexec.Materializer, then it will take - // over the responsibility of draining the sources; otherwise, they will be - // returned in NewColOperatorResult. - MetadataSources []colexecop.MetadataSources - DiskQueueCfg colcontainer.DiskQueueCfg - FDSemaphore semaphore.Semaphore - ExprHelper *ExprHelper - Factory coldata.ColumnFactory - TestingKnobs struct { + DiskQueueCfg colcontainer.DiskQueueCfg + FDSemaphore semaphore.Semaphore + ExprHelper *ExprHelper + Factory coldata.ColumnFactory + TestingKnobs struct { // SpillingCallbackFn will be called when the spilling from an in-memory // to disk-backed operator occurs. It should only be set in tests. SpillingCallbackFn func() @@ -90,18 +98,9 @@ type NewColOperatorArgs struct { // NewColOperatorResult is a helper struct that encompasses all of the return // values of NewColOperator call. type NewColOperatorResult struct { - Op colexecop.Operator + OpWithMetaInfo KVReader colexecop.KVReader ColumnTypes []*types.T - // MetadataSources are all sources of the metadata that are present in the - // tree rooted in Op that the caller must drain. These can be - // colfetcher.ColBatchScan or colexec.Columnarizer operators that were - // created during NewColOperator call, but it also might include all of the - // sources from NewColOperatorArgs.MetadataSources if no metadata draining - // component was created. - MetadataSources colexecop.MetadataSources - // ToClose is a slice of components that need to be Closed. - ToClose []colexecop.Closer OpMonitors []*mon.BytesMonitor OpAccounts []*mon.BoundAccount Releasables []execinfra.Releasable @@ -141,12 +140,14 @@ func (r *NewColOperatorResult) Release() { releasable.Release() } *r = NewColOperatorResult{ - ColumnTypes: r.ColumnTypes[:0], - MetadataSources: r.MetadataSources[:0], - ToClose: r.ToClose[:0], - OpMonitors: r.OpMonitors[:0], - OpAccounts: r.OpAccounts[:0], - Releasables: r.Releasables[:0], + OpWithMetaInfo: OpWithMetaInfo{ + MetadataSources: r.OpWithMetaInfo.MetadataSources[:0], + ToClose: r.OpWithMetaInfo.ToClose[:0], + }, + ColumnTypes: r.ColumnTypes[:0], + OpMonitors: r.OpMonitors[:0], + OpAccounts: r.OpAccounts[:0], + Releasables: r.Releasables[:0], } newColOperatorResultPool.Put(r) } diff --git a/pkg/sql/colexec/colexecbase/ordinality_test.go b/pkg/sql/colexec/colexecbase/ordinality_test.go index 89d5b2342fab..9072f65cc108 100644 --- a/pkg/sql/colexec/colexecbase/ordinality_test.go +++ b/pkg/sql/colexec/colexecbase/ordinality_test.go @@ -115,12 +115,12 @@ func createTestOrdinalityOperator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecop.Operator{input}, + Inputs: []colexecargs.OpWithMetaInfo{{Root: input}}, StreamingMemAccount: testMemAcc, } result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) if err != nil { return nil, err } - return result.Op, nil + return result.Root, nil } diff --git a/pkg/sql/colexec/colexecbase/simple_project_test.go b/pkg/sql/colexec/colexecbase/simple_project_test.go index 9c26abc6036b..35853da67594 100644 --- a/pkg/sql/colexec/colexecbase/simple_project_test.go +++ b/pkg/sql/colexec/colexecbase/simple_project_test.go @@ -118,7 +118,7 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { var input colexecop.Operator parallelUnorderedSynchronizerInputs := make([]colexec.SynchronizerInput, len(inputs)) for i := range parallelUnorderedSynchronizerInputs { - parallelUnorderedSynchronizerInputs[i].Op = inputs[i] + parallelUnorderedSynchronizerInputs[i].Root = inputs[i] } input = colexec.NewParallelUnorderedSynchronizer(parallelUnorderedSynchronizerInputs, &wg) input = colexecbase.NewSimpleProjectOp(input, len(inputTypes), []uint32{0}) diff --git a/pkg/sql/colexec/colexectestutils/BUILD.bazel b/pkg/sql/colexec/colexectestutils/BUILD.bazel index 4b1101d7f9f6..f39e2bbc84d3 100644 --- a/pkg/sql/colexec/colexectestutils/BUILD.bazel +++ b/pkg/sql/colexec/colexectestutils/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "colexectestutils", srcs = [ + "args.go", "proj_utils.go", "utils.go", ], diff --git a/pkg/sql/colexec/colexectestutils/args.go b/pkg/sql/colexec/colexectestutils/args.go new file mode 100644 index 000000000000..cb38622bf7a6 --- /dev/null +++ b/pkg/sql/colexec/colexectestutils/args.go @@ -0,0 +1,26 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexectestutils + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" +) + +// MakeInputs is a utility function that populates a slice of +// colexecargs.OpWithMetaInfo objects based on sources. +func MakeInputs(sources []colexecop.Operator) []colexecargs.OpWithMetaInfo { + inputs := make([]colexecargs.OpWithMetaInfo, len(sources)) + for i := range sources { + inputs[i].Root = sources[i] + } + return inputs +} diff --git a/pkg/sql/colexec/colexectestutils/proj_utils.go b/pkg/sql/colexec/colexectestutils/proj_utils.go index bc708e42a76e..f8cb92b98d2d 100644 --- a/pkg/sql/colexec/colexectestutils/proj_utils.go +++ b/pkg/sql/colexec/colexectestutils/proj_utils.go @@ -98,7 +98,7 @@ func CreateTestProjectingOperator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecop.Operator{input}, + Inputs: []colexecargs.OpWithMetaInfo{{Root: input}}, StreamingMemAccount: testMemAcc, } if canFallbackToRowexec { @@ -108,5 +108,5 @@ func CreateTestProjectingOperator( if err != nil { return nil, err } - return result.Op, nil + return result.Root, nil } diff --git a/pkg/sql/colexec/colexecwindow/window_functions_test.go b/pkg/sql/colexec/colexecwindow/window_functions_test.go index 7bd537268484..58fffa80510f 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_test.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_test.go @@ -274,7 +274,7 @@ func TestWindowFunctions(t *testing.T) { } { log.Infof(ctx, "spillForced=%t/%s", spillForced, tc.windowerSpec.WindowFns[0].Func.String()) var semsToCheck []semaphore.Semaphore - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.UnorderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.UnorderedVerifier, func(sources []colexecop.Operator) (colexecop.Operator, error) { tc.init() ct := make([]*types.T, len(tc.tuples[0])) for i := range ct { @@ -297,7 +297,7 @@ func TestWindowFunctions(t *testing.T) { sem := colexecop.NewTestingSemaphore(relativeRankNumRequiredFDs) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: inputs, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, FDSemaphore: sem, @@ -307,7 +307,7 @@ func TestWindowFunctions(t *testing.T) { result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) accounts = append(accounts, result.OpAccounts...) monitors = append(monitors, result.OpMonitors...) - return result.Op, err + return result.Root, err }) for i, sem := range semsToCheck { require.Equal(t, 0, sem.GetCount(), "sem still reports open FDs at index %d", i) diff --git a/pkg/sql/colexec/crossjoiner_test.go b/pkg/sql/colexec/crossjoiner_test.go index cb99418c0ba6..43da592b9f4b 100644 --- a/pkg/sql/colexec/crossjoiner_test.go +++ b/pkg/sql/colexec/crossjoiner_test.go @@ -373,7 +373,7 @@ func TestCrossJoiner(t *testing.T) { spec := createSpecForHashJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: sources, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, FDSemaphore: colexecop.NewTestingSemaphore(externalHJMinPartitions), @@ -384,7 +384,7 @@ func TestCrossJoiner(t *testing.T) { } accounts = append(accounts, result.OpAccounts...) monitors = append(monitors, result.OpMonitors...) - return result.Op, nil + return result.Root, nil }) } } @@ -444,7 +444,7 @@ func BenchmarkCrossJoiner(b *testing.B) { args := &colexecargs.NewColOperatorArgs{ Spec: spec, // Inputs will be set below. - Inputs: []colexecop.Operator{nil, nil}, + Inputs: []colexecargs.OpWithMetaInfo{{}, {}}, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, FDSemaphore: colexecop.NewTestingSemaphore(VecMaxOpenFDsLimit), @@ -459,14 +459,14 @@ func BenchmarkCrossJoiner(b *testing.B) { b.SetBytes(int64(8 * nOutputRows * (len(tc.leftOutCols) + len(tc.rightOutCols)))) b.ResetTimer() for i := 0; i < b.N; i++ { - args.Inputs[0] = colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) - args.Inputs[1] = colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) + args.Inputs[0].Root = colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) + args.Inputs[1].Root = colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) require.NoError(b, err) accounts = append(accounts, result.OpAccounts...) monitors = append(monitors, result.OpMonitors...) require.NoError(b, err) - cj := result.Op + cj := result.Root cj.Init() for b := cj.Next(ctx); b.Length() > 0; b = cj.Next(ctx) { } diff --git a/pkg/sql/colexec/external_distinct_test.go b/pkg/sql/colexec/external_distinct_test.go index 1e265fc310c9..26387c08df84 100644 --- a/pkg/sql/colexec/external_distinct_test.go +++ b/pkg/sql/colexec/external_distinct_test.go @@ -352,7 +352,7 @@ func BenchmarkExternalDistinct(b *testing.B) { func createExternalDistinct( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []colexecop.Operator, + sources []colexecop.Operator, typs []*types.T, distinctCols []uint32, outputOrdering execinfrapb.Ordering, @@ -375,7 +375,7 @@ func createExternalDistinct( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: input, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: diskQueueCfg, FDSemaphore: testingSemaphore, @@ -383,5 +383,5 @@ func createExternalDistinct( args.TestingKnobs.SpillingCallbackFn = spillingCallbackFn args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) - return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err + return result.Root, result.OpAccounts, result.OpMonitors, result.ToClose, err } diff --git a/pkg/sql/colexec/external_hash_aggregator_test.go b/pkg/sql/colexec/external_hash_aggregator_test.go index 25c975c8e25b..113cf06088c2 100644 --- a/pkg/sql/colexec/external_hash_aggregator_test.go +++ b/pkg/sql/colexec/external_hash_aggregator_test.go @@ -243,12 +243,12 @@ func createExternalHashAggregator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecop.Operator{newAggArgs.Input}, + Inputs: []colexecargs.OpWithMetaInfo{{Root: newAggArgs.Input}}, StreamingMemAccount: testMemAcc, DiskQueueCfg: diskQueueCfg, FDSemaphore: testingSemaphore, } args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) - return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err + return result.Root, result.OpAccounts, result.OpMonitors, result.ToClose, err } diff --git a/pkg/sql/colexec/external_hash_joiner_test.go b/pkg/sql/colexec/external_hash_joiner_test.go index 29805bcbb9b3..bb9a7ad9aa16 100644 --- a/pkg/sql/colexec/external_hash_joiner_test.go +++ b/pkg/sql/colexec/external_hash_joiner_test.go @@ -295,7 +295,7 @@ func createDiskBackedHashJoiner( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.ProcessorSpec, - inputs []colexecop.Operator, + sources []colexecop.Operator, spillingCallbackFn func(), diskQueueCfg colcontainer.DiskQueueCfg, numForcedRepartitions int, @@ -304,7 +304,7 @@ func createDiskBackedHashJoiner( ) (colexecop.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecop.Closer, error) { args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: inputs, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: diskQueueCfg, FDSemaphore: testingSemaphore, @@ -316,5 +316,5 @@ func createDiskBackedHashJoiner( args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions args.TestingKnobs.DelegateFDAcquisitions = delegateFDAcquisitions result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) - return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err + return result.Root, result.OpAccounts, result.OpMonitors, result.ToClose, err } diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index 78ea2acc4455..590fbac0d7f5 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -582,7 +582,7 @@ func (s *externalSorter) createMergerForPartitions( s.createPartitionerToOperators(n) syncInputs := make([]SynchronizerInput, n) for i := range syncInputs { - syncInputs[i].Op = s.partitionerToOperators[i] + syncInputs[i].Root = s.partitionerToOperators[i] } if log.V(2) { var b strings.Builder diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index 3ddad4369053..7415807b0a5f 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -455,7 +455,7 @@ func BenchmarkExternalSort(b *testing.B) { func createDiskBackedSorter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []colexecop.Operator, + sources []colexecop.Operator, typs []*types.T, ordCols []execinfrapb.Ordering_Column, matchLen int, @@ -482,7 +482,7 @@ func createDiskBackedSorter( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: input, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: diskQueueCfg, FDSemaphore: testingSemaphore, @@ -491,5 +491,5 @@ func createDiskBackedSorter( args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions args.TestingKnobs.DelegateFDAcquisitions = delegateFDAcquisitions result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) - return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err + return result.Root, result.OpAccounts, result.OpMonitors, result.ToClose, err } diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index e61e99c947b3..fc999d14c25d 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -1015,7 +1015,7 @@ func TestHashJoiner(t *testing.T) { spec := createSpecForHashJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: sources, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, } args.TestingKnobs.UseStreamingMemAccountForBuffering = true @@ -1024,7 +1024,7 @@ func TestHashJoiner(t *testing.T) { if err != nil { return nil, err } - return result.Op, nil + return result.Root, nil }) } } @@ -1164,15 +1164,15 @@ func TestHashJoinerProjection(t *testing.T) { rightSource := colexectestutils.NewOpTestInput(testAllocator, 1, rightTuples, rightTypes) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecop.Operator{leftSource, rightSource}, + Inputs: []colexecargs.OpWithMetaInfo{{Root: leftSource}, {Root: rightSource}}, StreamingMemAccount: testMemAcc, } args.TestingKnobs.UseStreamingMemAccountForBuffering = true args.TestingKnobs.DiskSpillingDisabled = true hjOp, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) require.NoError(t, err) - hjOp.Op.Init() - for b := hjOp.Op.Next(ctx); b.Length() > 0; b = hjOp.Op.Next(ctx) { + hjOp.Root.Init() + for b := hjOp.Root.Next(ctx); b.Length() > 0; b = hjOp.Root.Next(ctx) { // The output types should be {Int64, Int64, Bool, Decimal, Float64, Bytes} // and we check this explicitly. b.ColVec(0).Int64() diff --git a/pkg/sql/colexec/is_null_ops_test.go b/pkg/sql/colexec/is_null_ops_test.go index 0a0816bef2a2..4a080978b020 100644 --- a/pkg/sql/colexec/is_null_ops_test.go +++ b/pkg/sql/colexec/is_null_ops_test.go @@ -229,7 +229,7 @@ func TestIsNullSelOp(t *testing.T) { for _, c := range testCases { log.Infof(ctx, "%s", c.desc) - opConstructor := func(input []colexecop.Operator) (colexecop.Operator, error) { + opConstructor := func(sources []colexecop.Operator) (colexecop.Operator, error) { typs := []*types.T{types.Int} spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: typs}}, @@ -242,14 +242,14 @@ func TestIsNullSelOp(t *testing.T) { } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: input, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, } result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) if err != nil { return nil, err } - return result.Op, nil + return result.Root, nil } colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{c.inputTuples}, c.outputTuples, colexectestutils.OrderedVerifier, opConstructor) } diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index b3697ddf743a..eb098780527b 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -180,8 +181,6 @@ var materializerEmptyPostProcessSpec = &execinfrapb.PostProcessSpec{} // - typs is the output types scheme. // - getStats (when tracing is enabled) returns all of the execution statistics // of operators which the materializer is responsible for. -// - metadataSources are all of the metadata sources that are planned on the -// same node as the Materializer and that need to be drained. // - cancelFlow should return the context cancellation function that cancels // the context of the flow (i.e. it is Flow.ctxCancel). It should only be // non-nil in case of a root Materializer (i.e. not when we're wrapping a row @@ -191,23 +190,21 @@ var materializerEmptyPostProcessSpec = &execinfrapb.PostProcessSpec{} func NewMaterializer( flowCtx *execinfra.FlowCtx, processorID int32, - input colexecop.Operator, + input colexecargs.OpWithMetaInfo, typs []*types.T, output execinfra.RowReceiver, statsCollectors []VectorizedStatsCollector, - metadataSources []colexecop.MetadataSource, - toClose []colexecop.Closer, cancelFlow func() context.CancelFunc, ) (*Materializer, error) { m := materializerPool.Get().(*Materializer) *m = Materializer{ ProcessorBase: m.ProcessorBase, - input: input, + input: input.Root, typs: typs, - drainHelper: newDrainHelper(statsCollectors, metadataSources), + drainHelper: newDrainHelper(statsCollectors, input.MetadataSources), converter: colconv.NewAllVecToDatumConverter(len(typs)), row: make(rowenc.EncDatumRow, len(typs)), - closers: toClose, + closers: input.ToClose, } if err := m.ProcessorBase.InitWithEvalCtx( diff --git a/pkg/sql/colexec/materializer_test.go b/pkg/sql/colexec/materializer_test.go index e7881af976db..e5dbcfb395fd 100644 --- a/pkg/sql/colexec/materializer_test.go +++ b/pkg/sql/colexec/materializer_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -65,12 +66,10 @@ func TestColumnarizeMaterialize(t *testing.T) { m, err := NewMaterializer( flowCtx, 1, /* processorID */ - c, + colexecargs.OpWithMetaInfo{Root: c}, typs, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSources */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { @@ -150,12 +149,10 @@ func BenchmarkMaterializer(b *testing.B) { m, err := NewMaterializer( flowCtx, 0, /* processorID */ - input, + colexecargs.OpWithMetaInfo{Root: input}, typs, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSources */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { @@ -205,12 +202,13 @@ func TestMaterializerNextErrorAfterConsumerDone(t *testing.T) { m, err := NewMaterializer( flowCtx, 0, /* processorID */ - &colexecop.CallbackOperator{}, + colexecargs.OpWithMetaInfo{ + Root: &colexecop.CallbackOperator{}, + MetadataSources: colexecop.MetadataSources{metadataSource}, + }, nil, /* typ */ nil, /* output */ nil, /* statsCollectors */ - []colexecop.MetadataSource{metadataSource}, - nil, /* toClose */ nil, /* cancelFlow */ ) require.NoError(t, err) @@ -255,12 +253,10 @@ func BenchmarkColumnarizeMaterialize(b *testing.B) { m, err := NewMaterializer( flowCtx, 1, /* processorID */ - c, + colexecargs.OpWithMetaInfo{Root: c}, types, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSources */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 1d06c8487265..1b662f211ed0 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -1660,11 +1660,11 @@ func TestMergeJoiner(t *testing.T) { runner(t, testAllocator, []colexectestutils.Tuples{tc.leftTuples, tc.rightTuples}, [][]*types.T{tc.leftTypes, tc.rightTypes}, tc.expected, verifier, - func(input []colexecop.Operator) (colexecop.Operator, error) { + func(sources []colexecop.Operator) (colexecop.Operator, error) { spec := createSpecForMergeJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: input, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, FDSemaphore: colexecop.NewTestingSemaphore(mjFDLimit), @@ -1676,7 +1676,7 @@ func TestMergeJoiner(t *testing.T) { } accounts = append(accounts, result.OpAccounts...) monitors = append(monitors, result.OpMonitors...) - return result.Op, nil + return result.Root, nil }) } } diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index d02170ed99c8..5552f7616260 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -89,7 +89,7 @@ func (o *OrderedSynchronizer) ChildCount(verbose bool) int { // Child implements the execinfrapb.OpNode interface. func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { - return o.inputs[nth].Op + return o.inputs[nth].Root } // NewOrderedSynchronizer creates a new OrderedSynchronizer. @@ -117,7 +117,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { o.inputBatches = make([]coldata.Batch, len(o.inputs)) o.heap = make([]int, 0, len(o.inputs)) for i := range o.inputs { - o.inputBatches[i] = o.inputs[i].Op.Next(ctx) + o.inputBatches[i] = o.inputs[i].Root.Next(ctx) o.updateComparators(i) if o.inputBatches[i].Length() > 0 { o.heap = append(o.heap, i) @@ -239,7 +239,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputIndices[minBatch]+1 < o.inputBatches[minBatch].Length() { o.inputIndices[minBatch]++ } else { - o.inputBatches[minBatch] = o.inputs[minBatch].Op.Next(ctx) + o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(ctx) o.inputIndices[minBatch] = 0 o.updateComparators(minBatch) } @@ -351,7 +351,7 @@ func (o *OrderedSynchronizer) Init() { o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) for i := range o.inputs { - o.inputs[i].Op.Init() + o.inputs[i].Root.Init() } o.comparators = make([]vecComparator, len(o.ordering)) for i := range o.ordering { diff --git a/pkg/sql/colexec/ordered_synchronizer_test.go b/pkg/sql/colexec/ordered_synchronizer_test.go index 1bc0517c8ab8..aab68505a9c9 100644 --- a/pkg/sql/colexec/ordered_synchronizer_test.go +++ b/pkg/sql/colexec/ordered_synchronizer_test.go @@ -185,7 +185,7 @@ func TestOrderedSyncRandomInput(t *testing.T) { } inputs := make([]SynchronizerInput, numInputs) for i := range inputs { - inputs[i].Op = colexectestutils.NewOpTestInput(testAllocator, batchSize, sources[i], typs) + inputs[i].Root = colexectestutils.NewOpTestInput(testAllocator, batchSize, sources[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} op, err := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) @@ -215,7 +215,7 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { inputs := make([]SynchronizerInput, len(batches)) for i := range batches { - inputs[i].Op = colexecop.NewRepeatableBatchSource(testAllocator, batches[i], typs) + inputs[i].Root = colexecop.NewRepeatableBatchSource(testAllocator, batches[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index e3ed8c22d5a7..4e889f9dd152 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -109,7 +109,7 @@ func (o *OrderedSynchronizer) ChildCount(verbose bool) int { // Child implements the execinfrapb.OpNode interface. func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { - return o.inputs[nth].Op + return o.inputs[nth].Root } // NewOrderedSynchronizer creates a new OrderedSynchronizer. @@ -137,7 +137,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { o.inputBatches = make([]coldata.Batch, len(o.inputs)) o.heap = make([]int, 0, len(o.inputs)) for i := range o.inputs { - o.inputBatches[i] = o.inputs[i].Op.Next(ctx) + o.inputBatches[i] = o.inputs[i].Root.Next(ctx) o.updateComparators(i) if o.inputBatches[i].Length() > 0 { o.heap = append(o.heap, i) @@ -189,7 +189,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputIndices[minBatch]+1 < o.inputBatches[minBatch].Length() { o.inputIndices[minBatch]++ } else { - o.inputBatches[minBatch] = o.inputs[minBatch].Op.Next(ctx) + o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(ctx) o.inputIndices[minBatch] = 0 o.updateComparators(minBatch) } @@ -244,7 +244,7 @@ func (o *OrderedSynchronizer) Init() { o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) for i := range o.inputs { - o.inputs[i].Op.Init() + o.inputs[i].Root.Init() } o.comparators = make([]vecComparator, len(o.ordering)) for i := range o.ordering { diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index e17b1febadc5..a03d69f273df 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -16,6 +16,7 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -104,7 +105,7 @@ func (s *ParallelUnorderedSynchronizer) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (s *ParallelUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { - return s.inputs[nth].Op + return s.inputs[nth].Root } // SynchronizerInput is a wrapper over a colexecop.Operator that a @@ -112,8 +113,7 @@ func (s *ParallelUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.O // colexecop.MetadataSources may also be specified, in which case // DrainMeta will be called from the same goroutine. type SynchronizerInput struct { - // Op is the input Operator. - Op colexecop.Operator + colexecargs.OpWithMetaInfo // StatsCollectors are all vectorized stats collectors in the input tree. // The field is currently being used *only* to track all of the stats // collectors in the input tree, and the synchronizers should *not* access @@ -121,18 +121,12 @@ type SynchronizerInput struct { // TODO(yuzefovich): actually move the logic of getting stats into the // synchronizers. StatsCollectors []VectorizedStatsCollector - // MetadataSources are metadata sources in the input tree that should be - // drained in the same goroutine as Op. - MetadataSources colexecop.MetadataSources - // ToClose are Closers in the input tree that should be closed in the same - // goroutine as Op. - ToClose colexecop.Closers } func operatorsToSynchronizerInputs(ops []colexecop.Operator) []SynchronizerInput { result := make([]SynchronizerInput, len(ops)) for i := range result { - result[i].Op = ops[i] + result[i].Root = ops[i] } return result } @@ -175,7 +169,7 @@ func NewParallelUnorderedSynchronizer( // Init is part of the Operator interface. func (s *ParallelUnorderedSynchronizer) Init() { for _, input := range s.inputs { - input.Op.Init() + input.Root.Init() } } @@ -199,7 +193,7 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { for i, input := range s.inputs { s.nextBatch[i] = func(input SynchronizerInput, inputIdx int) func() { return func() { - s.batches[inputIdx] = input.Op.Next(ctx) + s.batches[inputIdx] = input.Root.Next(ctx) } }(input, i) s.externalWaitGroup.Add(1) diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index 67a17ea3baf8..0d1f9d5ec1d4 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -58,7 +58,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { typs, ) source.ResetBatchesToReturn(numBatches) - inputs[i].Op = source + inputs[i].Root = source inputIdx := i inputs[i].MetadataSources = []colexecop.MetadataSource{ colexectestutils.CallbackMetadataSource{DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { @@ -154,7 +154,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { ctx := context.Background() inputs := make([]SynchronizerInput, 6) - inputs[0].Op = &colexecop.CallbackOperator{NextCb: func(context.Context) coldata.Batch { + inputs[0].Root = &colexecop.CallbackOperator{NextCb: func(context.Context) coldata.Batch { colexecerror.InternalError(errors.New(expectedErr)) // This code is unreachable, but the compiler cannot infer that. return nil @@ -163,7 +163,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { acc := testMemMonitor.MakeBoundAccount() defer acc.Close(ctx) func(allocator *colmem.Allocator) { - inputs[i].Op = &colexecop.CallbackOperator{ + inputs[i].Root = &colexecop.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { // All inputs that do not encounter an error will continue to return // batches. @@ -204,7 +204,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { for i := range inputs { batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - inputs[i].Op = colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) + inputs[i].Root = colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) } var wg sync.WaitGroup ctx, cancelFn := context.WithCancel(context.Background()) diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index ba60b6efd831..333ceae892b4 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer.go @@ -44,7 +44,7 @@ func (s *SerialUnorderedSynchronizer) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { - return s.inputs[nth].Op + return s.inputs[nth].Root } // NewSerialUnorderedSynchronizer creates a new SerialUnorderedSynchronizer. @@ -58,7 +58,7 @@ func NewSerialUnorderedSynchronizer(inputs []SynchronizerInput) *SerialUnordered // Init is part of the Operator interface. func (s *SerialUnorderedSynchronizer) Init() { for _, input := range s.inputs { - input.Op.Init() + input.Root.Init() } } @@ -68,7 +68,7 @@ func (s *SerialUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch { if s.curSerialInputIdx == len(s.inputs) { return coldata.ZeroBatch } - b := s.inputs[s.curSerialInputIdx].Op.Next(ctx) + b := s.inputs[s.curSerialInputIdx].Root.Next(ctx) if b.Length() == 0 { s.curSerialInputIdx++ } else { diff --git a/pkg/sql/colexec/serial_unordered_synchronizer_test.go b/pkg/sql/colexec/serial_unordered_synchronizer_test.go index 9ca8097df489..ced6dc2571a9 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -43,15 +44,15 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.ResetBatchesToReturn(numBatches) inputIdx := i - inputs[i] = SynchronizerInput{ - Op: source, + inputs[i] = SynchronizerInput{OpWithMetaInfo: colexecargs.OpWithMetaInfo{ + Root: source, MetadataSources: []colexecop.MetadataSource{ colexectestutils.CallbackMetadataSource{ DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { return []execinfrapb.ProducerMetadata{{Err: errors.Errorf("input %d test-induced metadata", inputIdx)}} }, }, - }, + }}, } } s := NewSerialUnorderedSynchronizer(inputs) diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index d71c8fdcdac2..a6dc49629ef1 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -90,12 +91,10 @@ func TestSQLTypesIntegration(t *testing.T) { materializer, err := NewMaterializer( flowCtx, 1, /* processorID */ - arrowOp, + colexecargs.OpWithMetaInfo{Root: arrowOp}, typs, output, nil, /* statsCollectors */ - nil, /* metadataSources */ - nil, /* toClose */ nil, /* cancelFlow */ ) require.NoError(t, err) diff --git a/pkg/sql/colflow/colbatch_scan_test.go b/pkg/sql/colflow/colbatch_scan_test.go index adaa7c233a7d..ed445f8b757f 100644 --- a/pkg/sql/colflow/colbatch_scan_test.go +++ b/pkg/sql/colflow/colbatch_scan_test.go @@ -93,7 +93,7 @@ func TestColBatchScanMeta(t *testing.T) { if err != nil { t.Fatal(err) } - tr := res.Op + tr := res.Root tr.Init() meta := tr.(*colexecutils.CancelChecker).Input.(*colfetcher.ColBatchScan).DrainMeta(ctx) var txnFinalStateSeen bool @@ -165,7 +165,7 @@ func BenchmarkColBatchScan(b *testing.B) { if err != nil { b.Fatal(err) } - tr := res.Op + tr := res.Root tr.Init() b.StartTimer() for { diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index 6ee4a17f65e9..1901d7eec107 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" @@ -459,7 +460,7 @@ type HashRouter struct { // needs to have a separate disk account. func NewHashRouter( unlimitedAllocators []*colmem.Allocator, - input colexecop.Operator, + input colexecargs.OpWithMetaInfo, types []*types.T, hashCols []uint32, memoryLimit int64, @@ -467,8 +468,6 @@ func NewHashRouter( fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, statsCollectors []colexec.VectorizedStatsCollector, - toDrain []colexecop.MetadataSource, - toClose []colexecop.Closer, ) (*HashRouter, []colexecop.DrainableOperator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) @@ -499,25 +498,23 @@ func NewHashRouter( outputs[i] = op outputsAsOps[i] = op } - return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, statsCollectors, toDrain, toClose), outputsAsOps + return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, statsCollectors), outputsAsOps } func newHashRouterWithOutputs( - input colexecop.Operator, + input colexecargs.OpWithMetaInfo, hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, statsCollectors []colexec.VectorizedStatsCollector, - toDrain []colexecop.MetadataSource, - toClose []colexecop.Closer, ) *HashRouter { r := &HashRouter{ - OneInputNode: colexecop.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input.Root), hashCols: hashCols, outputs: outputs, statsCollectors: statsCollectors, - metadataSources: toDrain, - closers: toClose, + metadataSources: input.MetadataSources, + closers: input.ToClose, unblockedEventsChan: unblockEventsChan, // waitForMetadata is a buffered channel to avoid blocking if nobody will // read the metadata. diff --git a/pkg/sql/colflow/routers_test.go b/pkg/sql/colflow/routers_test.go index 27089713264f..e7e6d5e5ba2b 100644 --- a/pkg/sql/colflow/routers_test.go +++ b/pkg/sql/colflow/routers_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -737,7 +738,12 @@ func TestHashRouterComputesDestination(t *testing.T) { } } - r := newHashRouterWithOutputs(in, []uint32{0}, nil /* ch */, outputs, nil /* statsCollectors */, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs( + colexecargs.OpWithMetaInfo{Root: in}, + []uint32{0}, /* hashCols */ + nil /* ch */, outputs, + nil, /* statsCollectors */ + ) for r.processNextBatch(ctx) { } @@ -780,7 +786,13 @@ func TestHashRouterCancellation(t *testing.T) { in := colexecop.NewRepeatableBatchSource(tu.testAllocator, batch, typs) unbufferedCh := make(chan struct{}) - r := newHashRouterWithOutputs(in, []uint32{0}, unbufferedCh, routerOutputs, nil /* statsCollectors */, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs( + colexecargs.OpWithMetaInfo{Root: in}, + []uint32{0}, /* hashCols */ + unbufferedCh, + routerOutputs, + nil, /* statsCollectors */ + ) t.Run("BeforeRun", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) @@ -883,7 +895,9 @@ func TestHashRouterOneOutput(t *testing.T) { defer diskAcc.Close(ctx) r, routerOutputs := NewHashRouter( []*colmem.Allocator{tu.testAllocator}, - colexectestutils.NewOpFixedSelTestInput(tu.testAllocator, sel, len(sel), data, typs), + colexecargs.OpWithMetaInfo{ + Root: colexectestutils.NewOpFixedSelTestInput(tu.testAllocator, sel, len(sel), data, typs), + }, typs, []uint32{0}, /* hashCols */ mtc.bytes, @@ -891,8 +905,6 @@ func TestHashRouterOneOutput(t *testing.T) { colexecop.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, nil, /* statsCollectors */ - nil, /* toDrain */ - nil, /* toClose */ ) if len(routerOutputs) != 1 { @@ -1080,19 +1092,20 @@ func TestHashRouterRandom(t *testing.T) { const hashRouterMetadataMsg = "hash router test metadata" r := newHashRouterWithOutputs( - inputs[0], + colexecargs.OpWithMetaInfo{ + Root: inputs[0], + MetadataSources: []colexecop.MetadataSource{ + colexectestutils.CallbackMetadataSource{ + DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { + return []execinfrapb.ProducerMetadata{{Err: errors.New(hashRouterMetadataMsg)}} + }, + }, + }, + }, hashCols, unblockEventsChan, outputs, nil, /* statsCollectors */ - []colexecop.MetadataSource{ - colexectestutils.CallbackMetadataSource{ - DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { - return []execinfrapb.ProducerMetadata{{Err: errors.New(hashRouterMetadataMsg)}} - }, - }, - }, - nil, /* toClose */ ) var ( @@ -1310,7 +1323,7 @@ func BenchmarkHashRouter(b *testing.B) { } r, outputs := NewHashRouter( allocators, - input, + colexecargs.OpWithMetaInfo{Root: input}, typs, []uint32{0}, /* hashCols */ 64<<20, /* memoryLimit */ @@ -1318,8 +1331,6 @@ func BenchmarkHashRouter(b *testing.B) { &colexecop.TestingSemaphore{}, diskAccounts, nil, /* statsCollectors */ - nil, /* toDrain */ - nil, /* toClose */ ) b.SetBytes(8 * int64(coldata.BatchSize()) * int64(numInputBatches)) // We expect distribution to not change. This is a sanity check that diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 484cb4d16308..d3acade4aea0 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -317,7 +317,7 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( op colexecop.Operator, kvReader colexecop.KVReader, - inputs []colexecop.Operator, + inputs []colexecargs.OpWithMetaInfo, component execinfrapb.ComponentID, monitors []*mon.BytesMonitor, ) (colexec.VectorizedStatsCollector, error) { @@ -332,7 +332,7 @@ func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( } inputStatsCollectors := make([]childStatsCollector, len(inputs)) for i, input := range inputs { - sc, ok := input.(childStatsCollector) + sc, ok := input.Root.(childStatsCollector) if !ok { return nil, errors.New("unexpectedly an input is not collecting stats") } @@ -419,10 +419,8 @@ type flowCreatorHelper interface { // as the metadataSources and closers in this DAG that need to be drained and // closed. type opDAGWithMetaSources struct { - rootOperator colexecop.Operator + colexecargs.OpWithMetaInfo statsCollectors []colexec.VectorizedStatsCollector - metadataSources []colexecop.MetadataSource - toClose []colexecop.Closer } // remoteComponentCreator is an interface that abstracts the constructors for @@ -514,8 +512,6 @@ type vectorizedFlowCreator struct { // expected number of components are closed. numClosers int32 numClosed int32 - - inputsScratch []colexecop.Operator } var _ execinfra.Releasable = &vectorizedFlowCreator{} @@ -564,7 +560,6 @@ func newVectorizedFlowCreator( releasables: creator.releasables, diskQueueCfg: diskQueueCfg, fdSemaphore: fdSemaphore, - inputsScratch: creator.inputsScratch, } return creator } @@ -599,7 +594,6 @@ func (s *vectorizedFlowCreator) Release() { monitors: s.monitors[:0], accounts: s.accounts[:0], releasables: s.releasables[:0], - inputsScratch: s.inputsScratch[:0], } vectorizedFlowCreatorPool.Put(s) } @@ -656,17 +650,15 @@ func (s *vectorizedFlowCreator) newStreamingMemAccount( func (s *vectorizedFlowCreator) setupRemoteOutputStream( ctx context.Context, flowCtx *execinfra.FlowCtx, - op colexecop.Operator, + op colexecargs.OpWithMetaInfo, outputTyps []*types.T, stream *execinfrapb.StreamEndpointSpec, - getStats func() []*execinfrapb.ComponentStats, - metadataSources []colexecop.MetadataSource, - toClose []colexecop.Closer, factory coldata.ColumnFactory, + getStats func() []*execinfrapb.ComponentStats, ) (execinfra.OpNode, error) { outbox, err := s.remoteComponentCreator.newOutbox( colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx), factory), - op, outputTyps, getStats, metadataSources, toClose, + op.Root, outputTyps, getStats, op.MetadataSources, op.ToClose, ) if err != nil { return nil, err @@ -716,12 +708,10 @@ func (s *vectorizedFlowCreator) setupRemoteOutputStream( func (s *vectorizedFlowCreator) setupRouter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colexecop.Operator, + input colexecargs.OpWithMetaInfo, outputTyps []*types.T, output *execinfrapb.OutputRouterSpec, statsCollectors []colexec.VectorizedStatsCollector, - metadataSources []colexecop.MetadataSource, - toClose []colexecop.Closer, factory coldata.ColumnFactory, ) error { if output.Type != execinfrapb.OutputRouterSpec_BY_HASH { @@ -744,9 +734,8 @@ func (s *vectorizedFlowCreator) setupRouter( } diskMon, diskAccounts := s.createDiskAccounts(ctx, flowCtx, mmName, len(output.Streams)) router, outputs := NewHashRouter( - allocators, input, outputTyps, output.HashColumns, - execinfra.GetWorkMemLimit(flowCtx.Cfg), s.diskQueueCfg, s.fdSemaphore, - diskAccounts, statsCollectors, metadataSources, toClose, + allocators, input, outputTyps, output.HashColumns, execinfra.GetWorkMemLimit(flowCtx.Cfg), + s.diskQueueCfg, s.fdSemaphore, diskAccounts, statsCollectors, ) runRouter := func(ctx context.Context, _ context.CancelFunc) { router.Run(logtags.AddTag(ctx, "hashRouterID", strings.Join(streamIDs, ","))) @@ -766,16 +755,22 @@ func (s *vectorizedFlowCreator) setupRouter( // Note that here we pass in nil 'toClose' slice because hash // router is responsible for closing all of the idempotent closers. if _, err := s.setupRemoteOutputStream( - ctx, flowCtx, op, outputTyps, stream, nil, /* getStats */ - []colexecop.MetadataSource{op}, nil /* toClose */, factory, + ctx, flowCtx, colexecargs.OpWithMetaInfo{ + Root: op, + MetadataSources: colexecop.MetadataSources{op}, + }, outputTyps, stream, factory, nil, /* getStats */ ); err != nil { return err } case execinfrapb.StreamEndpointSpec_LOCAL: foundLocalOutput = true - localOp := colexecop.Operator(op) + opWithMetaInfo := colexecargs.OpWithMetaInfo{ + Root: op, + MetadataSources: colexecop.MetadataSources{op}, + // toClose will be closed by the HashRouter. + ToClose: nil, + } var statsCollectors []colexec.VectorizedStatsCollector - ms := colexecop.MetadataSource(op) if s.recordingStats { mons := []*mon.BytesMonitor{hashRouterMemMonitor, diskMon} // Wrap local outputs with vectorized stats collectors when recording @@ -788,15 +783,12 @@ func (s *vectorizedFlowCreator) setupRouter( if err != nil { return err } - localOp = vsc + opWithMetaInfo.Root = vsc statsCollectors = []colexec.VectorizedStatsCollector{vsc} } s.streamIDToInputOp[stream.StreamID] = opDAGWithMetaSources{ - rootOperator: localOp, + OpWithMetaInfo: opWithMetaInfo, statsCollectors: statsCollectors, - metadataSources: []colexecop.MetadataSource{ms}, - // toClose will be closed by the HashRouter. - toClose: nil, } } } @@ -820,20 +812,14 @@ func (s *vectorizedFlowCreator) setupInput( input execinfrapb.InputSyncSpec, opt flowinfra.FuseOpt, factory coldata.ColumnFactory, -) ( - colexecop.Operator, - []colexec.VectorizedStatsCollector, - []colexecop.MetadataSource, - []colexecop.Closer, - error, -) { +) (colexecargs.OpWithMetaInfo, []colexec.VectorizedStatsCollector, error) { inputStreamOps := make([]colexec.SynchronizerInput, 0, len(input.Streams)) // Before we can safely use types we received over the wire in the // operators, we need to make sure they are hydrated. In row execution // engine it is done during the processor initialization, but operators // don't do that. if err := s.typeResolver.HydrateTypeSlice(ctx, input.ColumnTypes); err != nil { - return nil, nil, nil, nil, err + return colexecargs.OpWithMetaInfo{}, nil, err } for _, inputStream := range input.Streams { @@ -841,16 +827,14 @@ func (s *vectorizedFlowCreator) setupInput( case execinfrapb.StreamEndpointSpec_LOCAL: in := s.streamIDToInputOp[inputStream.StreamID] inputStreamOps = append(inputStreamOps, colexec.SynchronizerInput{ - Op: in.rootOperator, + OpWithMetaInfo: in.OpWithMetaInfo, StatsCollectors: in.statsCollectors, - MetadataSources: in.metadataSources, - ToClose: in.toClose, }) case execinfrapb.StreamEndpointSpec_REMOTE: // If the input is remote, the input operator does not exist in // streamIDToInputOp. Create an inbox. if err := s.checkInboundStreamID(inputStream.StreamID); err != nil { - return nil, nil, nil, nil, err + return colexecargs.OpWithMetaInfo{}, nil, err } // Retrieve the latency from the origin node (the one that has the @@ -871,16 +855,20 @@ func (s *vectorizedFlowCreator) setupInput( ) if err != nil { - return nil, nil, nil, nil, err + return colexecargs.OpWithMetaInfo{}, nil, err } s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup) op := colexecop.Operator(inbox) - var statsCollectors []colexec.VectorizedStatsCollector ms := colexecop.MetadataSource(inbox) if util.CrdbTestBuild { op = colexec.NewInvariantsChecker(op) ms = op.(colexecop.MetadataSource) } + opWithMetaInfo := colexecargs.OpWithMetaInfo{ + Root: op, + MetadataSources: colexecop.MetadataSources{ms}, + } + var statsCollectors []colexec.VectorizedStatsCollector if s.recordingStats { // Note: we can't use flowCtx.StreamComponentID because the stream does // not originate from this node (we are the target node). @@ -888,22 +876,19 @@ func (s *vectorizedFlowCreator) setupInput( base.SQLInstanceID(inputStream.OriginNodeID), flowCtx.ID, inputStream.StreamID, ) vsc := s.wrapWithNetworkVectorizedStatsCollector(op, inbox, compID, latency) - op = vsc + opWithMetaInfo.Root = vsc statsCollectors = []colexec.VectorizedStatsCollector{vsc} } inputStreamOps = append(inputStreamOps, colexec.SynchronizerInput{ - Op: op, + OpWithMetaInfo: opWithMetaInfo, StatsCollectors: statsCollectors, - MetadataSources: []colexecop.MetadataSource{ms}, }) default: - return nil, nil, nil, nil, errors.Errorf("unsupported input stream type %s", inputStream.Type) + return colexecargs.OpWithMetaInfo{}, nil, errors.Errorf("unsupported input stream type %s", inputStream.Type) } } - op := inputStreamOps[0].Op + opWithMetaInfo := inputStreamOps[0].OpWithMetaInfo statsCollectors := inputStreamOps[0].StatsCollectors - metaSources := inputStreamOps[0].MetadataSources - toClose := inputStreamOps[0].ToClose if len(inputStreamOps) > 1 { statsInputs := inputStreamOps if input.Type == execinfrapb.InputSyncSpec_ORDERED { @@ -913,25 +898,32 @@ func (s *vectorizedFlowCreator) setupInput( input.ColumnTypes, execinfrapb.ConvertToColumnOrdering(input.Ordering), ) if err != nil { - return nil, nil, nil, nil, err + return colexecargs.OpWithMetaInfo{}, nil, err + } + opWithMetaInfo = colexecargs.OpWithMetaInfo{ + Root: os, + MetadataSources: colexecop.MetadataSources{os}, + ToClose: colexecop.Closers{os}, } - op = os - metaSources = []colexecop.MetadataSource{os} - toClose = []colexecop.Closer{os} } else { if opt == flowinfra.FuseAggressively { sync := colexec.NewSerialUnorderedSynchronizer(inputStreamOps) - op = sync - metaSources = []colexecop.MetadataSource{sync} - toClose = []colexecop.Closer{sync} + opWithMetaInfo = colexecargs.OpWithMetaInfo{ + Root: sync, + MetadataSources: colexecop.MetadataSources{sync}, + ToClose: colexecop.Closers{sync}, + } } else { sync := colexec.NewParallelUnorderedSynchronizer(inputStreamOps, s.waitGroup) - op = sync - metaSources = []colexecop.MetadataSource{sync} - // toClose is set to nil because the ParallelUnorderedSynchronizer takes - // care of closing these components itself since they need to be closed - // from the same goroutine as Next. - toClose = nil + opWithMetaInfo = colexecargs.OpWithMetaInfo{ + Root: sync, + MetadataSources: colexecop.MetadataSources{sync}, + // ToClose is set to nil because the + // ParallelUnorderedSynchronizer takes care of closing these + // components itself since they need to be closed from the + // same goroutine as Next. + ToClose: nil, + } s.operatorConcurrency = true } // Don't use the unordered synchronizer's inputs for stats collection @@ -940,23 +932,23 @@ func (s *vectorizedFlowCreator) setupInput( statsInputs = nil } if util.CrdbTestBuild { - op = colexec.NewInvariantsChecker(op) - metaSources[0] = op.(colexecop.MetadataSource) + opWithMetaInfo.Root = colexec.NewInvariantsChecker(opWithMetaInfo.Root) + opWithMetaInfo.MetadataSources[0] = opWithMetaInfo.Root.(colexecop.MetadataSource) } if s.recordingStats { - statsInputsAsOps := make([]colexecop.Operator, len(statsInputs)) + statsInputsAsOps := make([]colexecargs.OpWithMetaInfo, len(statsInputs)) for i := range statsInputs { - statsInputsAsOps[i] = statsInputs[i].Op + statsInputsAsOps[i].Root = statsInputs[i].Root } // TODO(asubiotto): Once we have IDs for synchronizers, plumb them into // this stats collector to display stats. vsc, err := s.wrapWithVectorizedStatsCollectorBase( - op, nil /* kvReader */, statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ + opWithMetaInfo.Root, nil /* kvReader */, statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ ) if err != nil { - return nil, nil, nil, nil, err + return colexecargs.OpWithMetaInfo{}, nil, err } - op = vsc + opWithMetaInfo.Root = vsc // Accumulate the stats collectors from all of the input trees and // add the wrapped synchronizer. // TODO(yuzefovich): move the stats collection from inputs into the @@ -968,7 +960,7 @@ func (s *vectorizedFlowCreator) setupInput( statsCollectors = append(statsCollectors, vsc) } } - return op, statsCollectors, metaSources, toClose, nil + return opWithMetaInfo, statsCollectors, nil } // setupOutput sets up any necessary infrastructure according to the output spec @@ -980,11 +972,9 @@ func (s *vectorizedFlowCreator) setupOutput( ctx context.Context, flowCtx *execinfra.FlowCtx, pspec *execinfrapb.ProcessorSpec, - op colexecop.Operator, + opWithMetaInfo colexecargs.OpWithMetaInfo, opOutputTypes []*types.T, statsCollectors []colexec.VectorizedStatsCollector, - metadataSources []colexecop.MetadataSource, - toClose []colexecop.Closer, factory coldata.ColumnFactory, ) error { output := &pspec.Output[0] @@ -992,12 +982,10 @@ func (s *vectorizedFlowCreator) setupOutput( return s.setupRouter( ctx, flowCtx, - op, + opWithMetaInfo, opOutputTypes, output, statsCollectors, - metadataSources, - toClose, factory, ) } @@ -1009,17 +997,14 @@ func (s *vectorizedFlowCreator) setupOutput( switch outputStream.Type { case execinfrapb.StreamEndpointSpec_LOCAL: s.streamIDToInputOp[outputStream.StreamID] = opDAGWithMetaSources{ - rootOperator: op, + OpWithMetaInfo: opWithMetaInfo, statsCollectors: statsCollectors, - metadataSources: metadataSources, - toClose: toClose, } case execinfrapb.StreamEndpointSpec_REMOTE: // Set up an Outbox. outbox, err := s.setupRemoteOutputStream( - ctx, flowCtx, op, opOutputTypes, outputStream, + ctx, flowCtx, opWithMetaInfo, opOutputTypes, outputStream, factory, s.makeGetStatsFnForOutbox(flowCtx, statsCollectors, outputStream.OriginNodeID), - metadataSources, toClose, factory, ) if err != nil { return err @@ -1032,12 +1017,10 @@ func (s *vectorizedFlowCreator) setupOutput( proc, err := colexec.NewMaterializer( flowCtx, pspec.ProcessorID, - op, + opWithMetaInfo, opOutputTypes, s.syncFlowConsumer, statsCollectors, - metadataSources, - toClose, s.getCancelFlowFn, ) if err != nil { @@ -1102,25 +1085,10 @@ func (s *vectorizedFlowCreator) setupFlow( return } + var inputs []colexecargs.OpWithMetaInfo var inputsStatsCollectors [][]colexec.VectorizedStatsCollector - // metadataSources contains all the MetadataSources that need to be - // drained. If in a given loop iteration no component that can drain - // metadata from these sources is found, the metadataSources should - // be added as part of one of the last unconnected inputDAGs in - // streamIDToInputOp. This is to avoid cycles. - // - // The length of the slice is equal to the number of inputs (1 for - // most of processors, but could also be 0 or 2), and we keep the - // sources present in different input trees separately in order to - // prevent premature draining (in case of 2 inputs). - var metadataSources []colexecop.MetadataSources - // toClose is similar to metadataSources with the difference that - // these components do not produce metadata and should be Closed - // even during non-graceful termination. - var toClose []colexecop.Closer - inputs := s.inputsScratch[:0] for i := range pspec.Input { - input, vsc, ms, closers, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) + input, vsc, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) if localErr != nil { err = localErr return @@ -1128,8 +1096,6 @@ func (s *vectorizedFlowCreator) setupFlow( if vsc != nil { inputsStatsCollectors = append(inputsStatsCollectors, vsc) } - metadataSources = append(metadataSources, ms) - toClose = append(toClose, closers...) inputs = append(inputs, input) } @@ -1145,7 +1111,6 @@ func (s *vectorizedFlowCreator) setupFlow( StreamingMemAccount: s.newStreamingMemAccount(flowCtx), ProcessorConstructor: rowexec.NewProcessor, LocalProcessors: localProcessors, - MetadataSources: metadataSources, DiskQueueCfg: s.diskQueueCfg, FDSemaphore: s.fdSemaphore, ExprHelper: s.exprHelper, @@ -1167,27 +1132,25 @@ func (s *vectorizedFlowCreator) setupFlow( return } if flowCtx.EvalCtx.SessionData.TestingVectorizeInjectPanics { - result.Op = newPanicInjector(result.Op) + result.Root = newPanicInjector(result.Root) } if flowCtx.Cfg != nil && flowCtx.Cfg.TestingKnobs.CheckVectorizedFlowIsClosedCorrectly { - for _, closer := range result.ToClose { - func(c colexecop.Closer) { + toCloseCopy := append(colexecop.Closers{}, result.ToClose...) + for i := range toCloseCopy { + func(idx int) { closed := false - toClose = append(toClose, &callbackCloser{closeCb: func(ctx context.Context) error { + result.ToClose[idx] = &callbackCloser{closeCb: func(ctx context.Context) error { if !closed { closed = true atomic.AddInt32(&s.numClosed, 1) } - return c.Close(ctx) - }}) - }(closer) + return toCloseCopy[idx].Close(ctx) + }} + }(i) } s.numClosers += int32(len(result.ToClose)) - } else { - toClose = append(toClose, result.ToClose...) } - op := result.Op var statsCollectors []colexec.VectorizedStatsCollector if s.recordingStats { // Note: if the original op is a Columnarizer, this will result in two @@ -1195,13 +1158,13 @@ func (s *vectorizedFlowCreator) setupFlow( // is prepared to union the stats. // TODO(radu): find a way to clean this up. vsc, err := s.wrapWithVectorizedStatsCollectorBase( - op, result.KVReader, inputs, flowCtx.ProcessorComponentID(pspec.ProcessorID), + result.Root, result.KVReader, inputs, flowCtx.ProcessorComponentID(pspec.ProcessorID), result.OpMonitors, ) if err != nil { return } - op = vsc + result.Root = vsc for i := range inputs { statsCollectors = append(statsCollectors, inputsStatsCollectors[i]...) } @@ -1209,7 +1172,7 @@ func (s *vectorizedFlowCreator) setupFlow( } if err = s.setupOutput( - ctx, flowCtx, pspec, op, result.ColumnTypes, statsCollectors, result.MetadataSources, toClose, factory, + ctx, flowCtx, pspec, result.OpWithMetaInfo, result.ColumnTypes, statsCollectors, factory, ); err != nil { return } diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index 39cd9ad01a45..ecf2933380c9 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow" @@ -196,7 +197,10 @@ func TestVectorizedFlowShutdown(t *testing.T) { } hashRouter, hashRouterOutputs := colflow.NewHashRouter( allocators, - hashRouterInput, + colexecargs.OpWithMetaInfo{ + Root: hashRouterInput, + MetadataSources: toDrain, + }, typs, []uint32{0}, /* hashCols */ 64<<20, /* memoryLimit */ @@ -204,8 +208,6 @@ func TestVectorizedFlowShutdown(t *testing.T) { &colexecop.TestingSemaphore{}, diskAccounts, nil, /* statsCollectors */ - toDrain, - nil, /* toClose */ ) for i := 0; i < numInboxes; i++ { inboxMemAccount := testMemMonitor.MakeBoundAccount() @@ -221,9 +223,10 @@ func TestVectorizedFlowShutdown(t *testing.T) { synchronizerInputs = append( synchronizerInputs, colexec.SynchronizerInput{ - Op: colexecop.Operator(inbox), - MetadataSources: []colexecop.MetadataSource{inbox}, - }, + OpWithMetaInfo: colexecargs.OpWithMetaInfo{ + Root: colexecop.Operator(inbox), + MetadataSources: []colexecop.MetadataSource{inbox}, + }}, ) } synchronizer := colexec.NewParallelUnorderedSynchronizer(synchronizerInputs, &wg) @@ -358,15 +361,17 @@ func TestVectorizedFlowShutdown(t *testing.T) { materializer, err := colexec.NewMaterializer( flowCtx, 1, /* processorID */ - materializerInput, + colexecargs.OpWithMetaInfo{ + Root: materializerInput, + MetadataSources: colexecop.MetadataSources{materializerMetadataSource}, + ToClose: colexecop.Closers{callbackCloser{closeCb: func() error { + materializerCalledClose = true + return nil + }}}, + }, typs, nil, /* output */ nil, /* statsCollectors */ - []colexecop.MetadataSource{materializerMetadataSource}, - []colexecop.Closer{callbackCloser{closeCb: func() error { - materializerCalledClose = true - return nil - }}}, /* toClose */ func() context.CancelFunc { return cancelLocal }, ) require.NoError(t, err) diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index c4e264b14d71..68f0d16ea0d6 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -72,9 +73,9 @@ func TestVectorizeInternalMemorySpaceError(t *testing.T) { for _, tc := range testCases { for _, success := range []bool{true, false} { t.Run(fmt.Sprintf("%s-success-expected-%t", tc.desc, success), func(t *testing.T) { - inputs := []colexecop.Operator{colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */, nil /* opToInitialize */)} + sources := []colexecop.Operator{colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */, nil /* opToInitialize */)} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */, nil /* opToInitialize */)) + sources = append(sources, colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */, nil /* opToInitialize */)) } memMon := mon.NewMonitor("MemoryMonitor", mon.MemoryResource, nil, nil, 0, math.MaxInt64, st) if success { @@ -87,7 +88,7 @@ func TestVectorizeInternalMemorySpaceError(t *testing.T) { defer acc.Close(ctx) args := &colexecargs.NewColOperatorArgs{ Spec: tc.spec, - Inputs: inputs, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: &acc, } var setupErr error @@ -194,9 +195,9 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { for _, success := range []bool{true, false} { expectNoMemoryError := success || tc.spillingSupported t.Run(fmt.Sprintf("%s-success-expected-%t", tc.desc, expectNoMemoryError), func(t *testing.T) { - inputs := []colexecop.Operator{colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)} + sources := []colexecop.Operator{colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)) + sources = append(sources, colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)) } memMon := mon.NewMonitor("MemoryMonitor", mon.MemoryResource, nil, nil, 0, math.MaxInt64, st) flowCtx.Cfg.TestingKnobs = execinfra.TestingKnobs{} @@ -218,7 +219,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { defer acc.Close(ctx) args := &colexecargs.NewColOperatorArgs{ Spec: tc.spec, - Inputs: inputs, + Inputs: colexectestutils.MakeInputs(sources), StreamingMemAccount: &acc, FDSemaphore: colexecop.NewTestingSemaphore(256), } @@ -239,9 +240,9 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { require.NoError(t, err) }); err == nil { err = colexecerror.CatchVectorizedRuntimeError(func() { - result.Op.Init() - result.Op.Next(ctx) - result.Op.Next(ctx) + result.Root.Init() + result.Root.Next(ctx) + result.Root.Next(ctx) }) } if result != nil { diff --git a/pkg/sql/colflow/vectorized_meta_propagation_test.go b/pkg/sql/colflow/vectorized_meta_propagation_test.go index 8a94eb73c790..988e33d1e0d9 100644 --- a/pkg/sql/colflow/vectorized_meta_propagation_test.go +++ b/pkg/sql/colflow/vectorized_meta_propagation_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -76,12 +77,13 @@ func TestVectorizedMetaPropagation(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 2, /* processorID */ - noop, + colexecargs.OpWithMetaInfo{ + Root: noop, + MetadataSources: colexecop.MetadataSources{col}, + }, typs, nil, /* output */ nil, /* statsCollectors */ - []colexecop.MetadataSource{col}, - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index b9f32437fd04..628e4081a93f 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -58,12 +59,10 @@ func TestVectorizedInternalPanic(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 1, /* processorID */ - vee, + colexecargs.OpWithMetaInfo{Root: vee}, typs, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSourceQueue */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { @@ -105,12 +104,10 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 1, /* processorID */ - nvee, + colexecargs.OpWithMetaInfo{Root: nvee}, typs, nil, /* output */ nil, /* statsCollectors */ - nil, /* metadataSourceQueue */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/distsql/BUILD.bazel b/pkg/sql/distsql/BUILD.bazel index a6e8aba27eae..a8d9982e6b2d 100644 --- a/pkg/sql/distsql/BUILD.bazel +++ b/pkg/sql/distsql/BUILD.bazel @@ -63,6 +63,7 @@ go_test( "//pkg/sql/colexec", "//pkg/sql/colexec/colbuilder", "//pkg/sql/colexec/colexecargs", + "//pkg/sql/colexec/colexectestutils", "//pkg/sql/colexec/colexecwindow", "//pkg/sql/colexecop", "//pkg/sql/colflow", diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index 5a201e94805d..7f111060794c 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -138,7 +139,7 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { constructorArgs := &colexecargs.NewColOperatorArgs{ Spec: args.pspec, - Inputs: columnarizers, + Inputs: colexectestutils.MakeInputs(columnarizers), StreamingMemAccount: &acc, DiskQueueCfg: colcontainer.DiskQueueCfg{ FS: tempFS, @@ -172,12 +173,10 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { outColOp, err := colexec.NewMaterializer( flowCtx, int32(len(args.inputs))+2, - result.Op, + result.OpWithMetaInfo, args.pspec.ResultTypes, nil, /* output */ nil, /* statsCollectors */ - result.MetadataSources, - result.ToClose, nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index cbdf42bb6a20..2a7f5568bdc5 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -53,16 +54,14 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 0, /* processorID */ - &colexecop.CallbackOperator{ + colexecargs.OpWithMetaInfo{Root: &colexecop.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { panic("") }, - }, + }}, nil, /* typs */ &distsqlutils.RowBuffer{}, nil, /* statsCollectors */ - nil, /* metadataSourceQueue */ - nil, /* toClose */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/sem/tree/eval_test/eval_test.go b/pkg/sql/sem/tree/eval_test/eval_test.go index aae4750b3b54..8efd7785e570 100644 --- a/pkg/sql/sem/tree/eval_test/eval_test.go +++ b/pkg/sql/sem/tree/eval_test/eval_test.go @@ -174,8 +174,8 @@ func TestEval(t *testing.T) { }, ResultTypes: []*types.T{typedExpr.ResolvedType()}, }, - Inputs: []colexecop.Operator{ - &colexecop.CallbackOperator{ + Inputs: []colexecargs.OpWithMetaInfo{{ + Root: &colexecop.CallbackOperator{ NextCb: func(_ context.Context) coldata.Batch { if batchesReturned > 0 { return coldata.ZeroBatch @@ -185,8 +185,8 @@ func TestEval(t *testing.T) { batch.SetLength(1) batchesReturned++ return batch - }, - }, + }}, + }, }, StreamingMemAccount: &acc, // Unsupported post processing specs are wrapped and run through the @@ -200,12 +200,10 @@ func TestEval(t *testing.T) { mat, err := colexec.NewMaterializer( flowCtx, 0, /* processorID */ - result.Op, + result.OpWithMetaInfo, []*types.T{typedExpr.ResolvedType()}, nil, /* output */ nil, /* statsCollectors */ - result.MetadataSources, - nil, /* toClose */ nil, /* cancelFlow */ ) require.NoError(t, err) From c868e920c797a96bfef30bd9b45c9dd10c64cf3e Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 19 Mar 2021 20:59:22 -0700 Subject: [PATCH 3/5] colexec: clean up stats collection from wrapped processors Previously, if we had wrapped row-execution processors in the vectorized flow, during the stats collection two `ComponentStats` objects would get emitted for each wrapped processor - one by the processor itself and another by the vectorized stats collector that is wrapping the columnarizer. Both stats objects would have the same ComponentID, so they would get Unioned. This commit cleans up the situation by making the columnarizer "hijack" the `ExecStatsForTrace` function from the wrapped processor. This required a bit of plumbing, namely, we need to: - track the "root" columnarizer separately in `NewColOperator` call so that later the vectorized stats collectors could augment the ComponentStats object created by the wrapped processor with additional information - provide an interface for the columnarizer to hijack the stats function. Release note: None --- pkg/sql/colexec/BUILD.bazel | 1 - pkg/sql/colexec/colbuilder/execplan.go | 1 + pkg/sql/colexec/colexecargs/op_creation.go | 14 +++--- pkg/sql/colexec/columnarizer.go | 33 ++++++++++++++ pkg/sql/colexec/materializer.go | 6 +-- .../parallel_unordered_synchronizer.go | 2 +- pkg/sql/colexec/stats.go | 22 ---------- pkg/sql/colexecop/operator.go | 10 +++++ pkg/sql/colflow/routers.go | 7 ++- pkg/sql/colflow/stats.go | 31 ++++++++----- pkg/sql/colflow/stats_test.go | 10 ++--- pkg/sql/colflow/vectorized_flow.go | 44 +++++++++---------- pkg/sql/execinfra/processorsbase.go | 26 +++++++++-- pkg/sql/execinfrapb/component_stats.go | 11 +++-- 14 files changed, 137 insertions(+), 81 deletions(-) delete mode 100644 pkg/sql/colexec/stats.go diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 2e2b6f099b4a..3830ae7bbf56 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -31,7 +31,6 @@ go_library( "sort_chunks.go", "sort_utils.go", "sorttopk.go", - "stats.go", "tuple_proj_op.go", "unordered_distinct.go", "utils.go", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 841f4b2fc859..62610774f4e1 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -624,6 +624,7 @@ func (r opResult) createAndWrapRowSource( return err } r.Root = c + r.Columnarizer = c if args.TestingKnobs.PlanInvariantsCheckers { r.Root = colexec.NewInvariantsChecker(r.Root) } diff --git a/pkg/sql/colexec/colexecargs/op_creation.go b/pkg/sql/colexec/colexecargs/op_creation.go index 2e6d02fb2896..893a7eb0ad7d 100644 --- a/pkg/sql/colexec/colexecargs/op_creation.go +++ b/pkg/sql/colexec/colexecargs/op_creation.go @@ -99,11 +99,15 @@ type NewColOperatorArgs struct { // values of NewColOperator call. type NewColOperatorResult struct { OpWithMetaInfo - KVReader colexecop.KVReader - ColumnTypes []*types.T - OpMonitors []*mon.BytesMonitor - OpAccounts []*mon.BoundAccount - Releasables []execinfra.Releasable + KVReader colexecop.KVReader + // Columnarizer is the root colexec.Columnarizer, if needed, that is hidden + // behind the stats collector interface. We need to track it separately from + // all other stats collectors since it requires special handling. + Columnarizer colexecop.VectorizedStatsCollector + ColumnTypes []*types.T + OpMonitors []*mon.BytesMonitor + OpAccounts []*mon.BoundAccount + Releasables []execinfra.Releasable } var _ execinfra.Releasable = &NewColOperatorResult{} diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index dda0c1bbd8e1..d7546f3cc8a4 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -45,6 +45,8 @@ const ( // reading the input in chunks of size coldata.BatchSize() and converting each // chunk into a coldata.Batch column by column. type Columnarizer struct { + // TODO(yuzefovich): consider whether embedding ProcessorBase into the + // columnarizers makes sense. execinfra.ProcessorBase colexecop.NonExplainable @@ -68,6 +70,7 @@ type Columnarizer struct { } var _ colexecop.Operator = &Columnarizer{} +var _ colexecop.VectorizedStatsCollector = &Columnarizer{} // NewBufferingColumnarizer returns a new Columnarizer that will be buffering up // rows before emitting them as output batches. @@ -155,7 +158,37 @@ func (c *Columnarizer) Init() { c.ctx = c.StartInternalNoSpan(c.ctx) c.input.Start(c.ctx) c.initStatus = colexecop.OperatorInitialized + if execStatsHijacker, ok := c.input.(execinfra.ExecStatsForTraceHijacker); ok { + // The columnarizer is now responsible for propagating the execution + // stats of the wrapped processor. + // + // Note that this columnarizer cannot be removed from the flow + // because it will have a vectorized stats collector planned on top, + // so the optimization of wrapRowSources() in execplan.go will never + // trigger. We check this assumption with an assertion below in the + // test setting. + // + // Still, just to be safe, we delay the hijacking until Init so that + // in case the assumption is wrong, we still get the stats from the + // wrapped processor. + c.ExecStatsForTrace = execStatsHijacker.HijackExecStatsForTrace() + } + } +} + +// GetStats is part of the colexecop.VectorizedStatsCollector interface. +func (c *Columnarizer) GetStats() *execinfrapb.ComponentStats { + if c.removedFromFlow && util.CrdbTestBuild { + colexecerror.InternalError(errors.AssertionFailedf( + "unexpectedly the columnarizer was removed from the flow when stats are being collected", + )) + } + if !c.removedFromFlow && c.ExecStatsForTrace != nil { + s := c.ExecStatsForTrace() + s.Component = c.FlowCtx.ProcessorComponentID(c.ProcessorID) + return s } + return nil } // Next is part of the Operator interface. diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index eb098780527b..e471d4433343 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -77,7 +77,7 @@ type drainHelper struct { // are noops. ctx context.Context - statsCollectors []VectorizedStatsCollector + statsCollectors []colexecop.VectorizedStatsCollector sources colexecop.MetadataSources bufferedMeta []execinfrapb.ProducerMetadata @@ -93,7 +93,7 @@ var drainHelperPool = sync.Pool{ } func newDrainHelper( - statsCollectors []VectorizedStatsCollector, sources colexecop.MetadataSources, + statsCollectors []colexecop.VectorizedStatsCollector, sources colexecop.MetadataSources, ) *drainHelper { d := drainHelperPool.Get().(*drainHelper) d.statsCollectors = statsCollectors @@ -193,7 +193,7 @@ func NewMaterializer( input colexecargs.OpWithMetaInfo, typs []*types.T, output execinfra.RowReceiver, - statsCollectors []VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, cancelFlow func() context.CancelFunc, ) (*Materializer, error) { m := materializerPool.Get().(*Materializer) diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index a03d69f273df..85134197f3ad 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -120,7 +120,7 @@ type SynchronizerInput struct { // it themselves. // TODO(yuzefovich): actually move the logic of getting stats into the // synchronizers. - StatsCollectors []VectorizedStatsCollector + StatsCollectors []colexecop.VectorizedStatsCollector } func operatorsToSynchronizerInputs(ops []colexecop.Operator) []SynchronizerInput { diff --git a/pkg/sql/colexec/stats.go b/pkg/sql/colexec/stats.go deleted file mode 100644 index 5c1a4ef16f75..000000000000 --- a/pkg/sql/colexec/stats.go +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexec - -import ( - "github.com/cockroachdb/cockroach/pkg/sql/colexecop" - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" -) - -// VectorizedStatsCollector is the common interface implemented by collectors. -type VectorizedStatsCollector interface { - colexecop.Operator - GetStats() *execinfrapb.ComponentStats -} diff --git a/pkg/sql/colexecop/operator.go b/pkg/sql/colexecop/operator.go index 5b965b0546e6..5d9c22f4f41b 100644 --- a/pkg/sql/colexecop/operator.go +++ b/pkg/sql/colexecop/operator.go @@ -343,3 +343,13 @@ func (s MetadataSources) DrainMeta(ctx context.Context) []execinfrapb.ProducerMe } return result } + +// VectorizedStatsCollector is the common interface implemented by several +// variations of the execution statistics collectors. At the moment of writing +// we have two variants: the "default" option (for all Operators) and the +// "network" option (strictly for colrpc.Inboxes). +type VectorizedStatsCollector interface { + Operator + // GetStats returns the execution statistics of a single Operator. + GetStats() *execinfrapb.ComponentStats +} diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index 1901d7eec107..bf47aeea3926 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" @@ -414,7 +413,7 @@ type HashRouter struct { // and the execution statistics will then be propagated as // execinfrapb.ProducerMetadata object right before draining // metadataSources. - statsCollectors []colexec.VectorizedStatsCollector + statsCollectors []colexecop.VectorizedStatsCollector // metadataSources is a slice of colexecop.MetadataSources that need to be // drained when the HashRouter terminates. metadataSources colexecop.MetadataSources @@ -467,7 +466,7 @@ func NewHashRouter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, - statsCollectors []colexec.VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, ) (*HashRouter, []colexecop.DrainableOperator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) @@ -506,7 +505,7 @@ func newHashRouterWithOutputs( hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, - statsCollectors []colexec.VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, ) *HashRouter { r := &HashRouter{ OneInputNode: colexecop.NewOneInputNode(input.Root), diff --git a/pkg/sql/colflow/stats.go b/pkg/sql/colflow/stats.go index a2f8ce0891bc..8410a0006c6f 100644 --- a/pkg/sql/colflow/stats.go +++ b/pkg/sql/colflow/stats.go @@ -15,7 +15,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" @@ -27,7 +26,7 @@ import ( ) // childStatsCollector gives access to the stopwatches of a -// colexec.VectorizedStatsCollector's childStatsCollectors. +// colexecop.VectorizedStatsCollector's childStatsCollectors. type childStatsCollector interface { getElapsedTime() time.Duration } @@ -113,25 +112,27 @@ func (bic *batchInfoCollector) getElapsedTime() time.Duration { return bic.stopwatch.Elapsed() } -// newVectorizedStatsCollector creates a colexec.VectorizedStatsCollector which -// wraps 'op' that corresponds to a component with either ProcessorID or +// newVectorizedStatsCollector creates a colexecop.VectorizedStatsCollector +// which wraps 'op' that corresponds to a component with either ProcessorID or // StreamID 'id' (with 'idTagKey' distinguishing between the two). 'kvReader' is // a component (either an operator or a wrapped processor) that performs KV // reads that is present in the chain of operators rooted at 'op'. func newVectorizedStatsCollector( op colexecop.Operator, kvReader colexecop.KVReader, + columnarizer colexecop.VectorizedStatsCollector, id execinfrapb.ComponentID, inputWatch *timeutil.StopWatch, memMonitors []*mon.BytesMonitor, diskMonitors []*mon.BytesMonitor, inputStatsCollectors []childStatsCollector, -) colexec.VectorizedStatsCollector { +) colexecop.VectorizedStatsCollector { // TODO(cathymw): Refactor to have specialized stats collectors for // memory/disk stats and IO operators. return &vectorizedStatsCollectorImpl{ batchInfoCollector: makeBatchInfoCollector(op, id, inputWatch, inputStatsCollectors), kvReader: kvReader, + columnarizer: columnarizer, memMonitors: memMonitors, diskMonitors: diskMonitors, } @@ -143,15 +144,25 @@ type vectorizedStatsCollectorImpl struct { batchInfoCollector kvReader colexecop.KVReader + columnarizer colexecop.VectorizedStatsCollector memMonitors []*mon.BytesMonitor diskMonitors []*mon.BytesMonitor } -// GetStats is part of the colexec.VectorizedStatsCollector interface. +// GetStats is part of the colexecop.VectorizedStatsCollector interface. func (vsc *vectorizedStatsCollectorImpl) GetStats() *execinfrapb.ComponentStats { numBatches, numTuples, time := vsc.batchInfoCollector.finish() - s := &execinfrapb.ComponentStats{Component: vsc.componentID} + var s *execinfrapb.ComponentStats + if vsc.columnarizer != nil { + s = vsc.columnarizer.GetStats() + } + if s == nil { + // Either there was no root columnarizer or it has been + // removed from the flow (in which case the columnarizer will return + // nil). Create a new stats object. + s = &execinfrapb.ComponentStats{Component: vsc.componentID} + } for _, memMon := range vsc.memMonitors { s.Exec.MaxAllocatedMem.Add(memMon.MaximumBytes()) @@ -183,7 +194,7 @@ func (vsc *vectorizedStatsCollectorImpl) GetStats() *execinfrapb.ComponentStats } // newNetworkVectorizedStatsCollector creates a new -// colexec.VectorizedStatsCollector for streams. In addition to the base stats, +// colexecop.VectorizedStatsCollector for streams. In addition to the base stats, // newNetworkVectorizedStatsCollector collects the network latency for a stream. func newNetworkVectorizedStatsCollector( op colexecop.Operator, @@ -191,7 +202,7 @@ func newNetworkVectorizedStatsCollector( inputWatch *timeutil.StopWatch, inbox *colrpc.Inbox, latency time.Duration, -) colexec.VectorizedStatsCollector { +) colexecop.VectorizedStatsCollector { return &networkVectorizedStatsCollectorImpl{ batchInfoCollector: makeBatchInfoCollector(op, id, inputWatch, nil /* childStatsCollectors */), inbox: inbox, @@ -208,7 +219,7 @@ type networkVectorizedStatsCollectorImpl struct { latency time.Duration } -// GetStats is part of the colexec.VectorizedStatsCollector interface. +// GetStats is part of the colexecop.VectorizedStatsCollector interface. func (nvsc *networkVectorizedStatsCollectorImpl) GetStats() *execinfrapb.ComponentStats { numBatches, numTuples, time := nvsc.batchInfoCollector.finish() diff --git a/pkg/sql/colflow/stats_test.go b/pkg/sql/colflow/stats_test.go index 29513fece2cf..91aa7b657238 100644 --- a/pkg/sql/colflow/stats_test.go +++ b/pkg/sql/colflow/stats_test.go @@ -40,7 +40,7 @@ func TestNumBatches(t *testing.T) { nBatches := 10 noop := colexecop.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())) vsc := newVectorizedStatsCollector( - noop, nil /* kvReader */, execinfrapb.ComponentID{}, + noop, nil /* kvReader */, nil /* columnarizer */, execinfrapb.ComponentID{}, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ ) @@ -66,7 +66,7 @@ func TestNumTuples(t *testing.T) { for _, batchSize := range []int{1, 16, 1024} { noop := colexecop.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, batchSize)) vsc := newVectorizedStatsCollector( - noop, nil /* kvReader */, execinfrapb.ComponentID{}, + noop, nil /* kvReader */, nil /* columnarizer */, execinfrapb.ComponentID{}, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ ) @@ -102,7 +102,7 @@ func TestVectorizedStatsCollector(t *testing.T) { timeSource: timeSource, } leftInput := newVectorizedStatsCollector( - leftSource, nil /* kvReader */, execinfrapb.ComponentID{ID: 0}, + leftSource, nil /* kvReader */, nil /* columnarizer */, execinfrapb.ComponentID{ID: 0}, timeutil.NewTestStopWatch(timeSource.Now), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ ) @@ -111,7 +111,7 @@ func TestVectorizedStatsCollector(t *testing.T) { timeSource: timeSource, } rightInput := newVectorizedStatsCollector( - rightSource, nil /* kvReader */, execinfrapb.ComponentID{ID: 1}, + rightSource, nil /* kvReader */, nil /* columnarizer */, execinfrapb.ComponentID{ID: 1}, timeutil.NewTestStopWatch(timeSource.Now), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ ) @@ -132,7 +132,7 @@ func TestVectorizedStatsCollector(t *testing.T) { } mjStatsCollector := newVectorizedStatsCollector( - timeAdvancingMergeJoiner, nil /* kvReader */, execinfrapb.ComponentID{ID: 2}, + timeAdvancingMergeJoiner, nil /* kvReader */, nil /* columnarizer */, execinfrapb.ComponentID{ID: 2}, mjInputWatch, nil /* memMonitors */, nil, /* diskMonitors */ []childStatsCollector{leftInput.(childStatsCollector), rightInput.(childStatsCollector)}, ) diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index d3acade4aea0..d850894cb708 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -311,16 +311,17 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { } // wrapWithVectorizedStatsCollectorBase creates a new -// colexec.VectorizedStatsCollectorBase that wraps op and connects the newly +// colexecop.VectorizedStatsCollector that wraps op and connects the newly // created wrapper with those corresponding to operators in inputs (the latter // must have already been wrapped). func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( op colexecop.Operator, kvReader colexecop.KVReader, + columnarizer colexecop.VectorizedStatsCollector, inputs []colexecargs.OpWithMetaInfo, component execinfrapb.ComponentID, monitors []*mon.BytesMonitor, -) (colexec.VectorizedStatsCollector, error) { +) (colexecop.VectorizedStatsCollector, error) { inputWatch := timeutil.NewStopWatch() var memMonitors, diskMonitors []*mon.BytesMonitor for _, m := range monitors { @@ -339,7 +340,7 @@ func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( inputStatsCollectors[i] = sc } return newVectorizedStatsCollector( - op, kvReader, component, inputWatch, + op, kvReader, columnarizer, component, inputWatch, memMonitors, diskMonitors, inputStatsCollectors, ), nil } @@ -351,7 +352,7 @@ func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( inbox *colrpc.Inbox, component execinfrapb.ComponentID, latency time.Duration, -) colexec.VectorizedStatsCollector { +) colexecop.VectorizedStatsCollector { inputWatch := timeutil.NewStopWatch() return newNetworkVectorizedStatsCollector(op, component, inputWatch, inbox, latency) } @@ -361,7 +362,7 @@ func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( // not being collected. func (s *vectorizedFlowCreator) makeGetStatsFnForOutbox( flowCtx *execinfra.FlowCtx, - statsCollectors []colexec.VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, originNodeID roachpb.NodeID, ) func() []*execinfrapb.ComponentStats { if !s.recordingStats { @@ -420,7 +421,7 @@ type flowCreatorHelper interface { // closed. type opDAGWithMetaSources struct { colexecargs.OpWithMetaInfo - statsCollectors []colexec.VectorizedStatsCollector + statsCollectors []colexecop.VectorizedStatsCollector } // remoteComponentCreator is an interface that abstracts the constructors for @@ -711,7 +712,7 @@ func (s *vectorizedFlowCreator) setupRouter( input colexecargs.OpWithMetaInfo, outputTyps []*types.T, output *execinfrapb.OutputRouterSpec, - statsCollectors []colexec.VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, factory coldata.ColumnFactory, ) error { if output.Type != execinfrapb.OutputRouterSpec_BY_HASH { @@ -770,21 +771,21 @@ func (s *vectorizedFlowCreator) setupRouter( // toClose will be closed by the HashRouter. ToClose: nil, } - var statsCollectors []colexec.VectorizedStatsCollector + var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { mons := []*mon.BytesMonitor{hashRouterMemMonitor, diskMon} // Wrap local outputs with vectorized stats collectors when recording // stats. This is mostly for compatibility but will provide some useful // information (e.g. output stall time). vsc, err := s.wrapWithVectorizedStatsCollectorBase( - op, nil /* kvReader */, nil, /* inputs */ + op, nil /* kvReader */, nil /* columnarizer */, nil, /* inputs */ flowCtx.StreamComponentID(stream.StreamID), mons, ) if err != nil { return err } opWithMetaInfo.Root = vsc - statsCollectors = []colexec.VectorizedStatsCollector{vsc} + statsCollectors = []colexecop.VectorizedStatsCollector{vsc} } s.streamIDToInputOp[stream.StreamID] = opDAGWithMetaSources{ OpWithMetaInfo: opWithMetaInfo, @@ -812,7 +813,7 @@ func (s *vectorizedFlowCreator) setupInput( input execinfrapb.InputSyncSpec, opt flowinfra.FuseOpt, factory coldata.ColumnFactory, -) (colexecargs.OpWithMetaInfo, []colexec.VectorizedStatsCollector, error) { +) (colexecargs.OpWithMetaInfo, []colexecop.VectorizedStatsCollector, error) { inputStreamOps := make([]colexec.SynchronizerInput, 0, len(input.Streams)) // Before we can safely use types we received over the wire in the // operators, we need to make sure they are hydrated. In row execution @@ -868,7 +869,7 @@ func (s *vectorizedFlowCreator) setupInput( Root: op, MetadataSources: colexecop.MetadataSources{ms}, } - var statsCollectors []colexec.VectorizedStatsCollector + var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { // Note: we can't use flowCtx.StreamComponentID because the stream does // not originate from this node (we are the target node). @@ -877,7 +878,7 @@ func (s *vectorizedFlowCreator) setupInput( ) vsc := s.wrapWithNetworkVectorizedStatsCollector(op, inbox, compID, latency) opWithMetaInfo.Root = vsc - statsCollectors = []colexec.VectorizedStatsCollector{vsc} + statsCollectors = []colexecop.VectorizedStatsCollector{vsc} } inputStreamOps = append(inputStreamOps, colexec.SynchronizerInput{ OpWithMetaInfo: opWithMetaInfo, @@ -943,7 +944,8 @@ func (s *vectorizedFlowCreator) setupInput( // TODO(asubiotto): Once we have IDs for synchronizers, plumb them into // this stats collector to display stats. vsc, err := s.wrapWithVectorizedStatsCollectorBase( - opWithMetaInfo.Root, nil /* kvReader */, statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ + opWithMetaInfo.Root, nil /* kvReader */, nil, /* columnarizer */ + statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ ) if err != nil { return colexecargs.OpWithMetaInfo{}, nil, err @@ -974,7 +976,7 @@ func (s *vectorizedFlowCreator) setupOutput( pspec *execinfrapb.ProcessorSpec, opWithMetaInfo colexecargs.OpWithMetaInfo, opOutputTypes []*types.T, - statsCollectors []colexec.VectorizedStatsCollector, + statsCollectors []colexecop.VectorizedStatsCollector, factory coldata.ColumnFactory, ) error { output := &pspec.Output[0] @@ -1086,7 +1088,7 @@ func (s *vectorizedFlowCreator) setupFlow( } var inputs []colexecargs.OpWithMetaInfo - var inputsStatsCollectors [][]colexec.VectorizedStatsCollector + var inputsStatsCollectors [][]colexecop.VectorizedStatsCollector for i := range pspec.Input { input, vsc, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) if localErr != nil { @@ -1151,15 +1153,11 @@ func (s *vectorizedFlowCreator) setupFlow( s.numClosers += int32(len(result.ToClose)) } - var statsCollectors []colexec.VectorizedStatsCollector + var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { - // Note: if the original op is a Columnarizer, this will result in two - // sets of stats for the same processor. The code that processes stats - // is prepared to union the stats. - // TODO(radu): find a way to clean this up. vsc, err := s.wrapWithVectorizedStatsCollectorBase( - result.Root, result.KVReader, inputs, flowCtx.ProcessorComponentID(pspec.ProcessorID), - result.OpMonitors, + result.Root, result.KVReader, result.Columnarizer, inputs, + flowCtx.ProcessorComponentID(pspec.ProcessorID), result.OpMonitors, ) if err != nil { return diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index b51378f04109..1d261d71d739 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -443,7 +443,7 @@ type ProcessorConstructor func( type ProcessorBase struct { self RowSource - processorID int32 + ProcessorID int32 Out ProcOutputHelper FlowCtx *FlowCtx @@ -688,6 +688,24 @@ func (pb *ProcessorBase) popTrailingMeta() *execinfrapb.ProducerMetadata { return nil } +// ExecStatsForTraceHijacker is an interface that allows us to hijack +// ExecStatsForTrace function from the ProcessorBase. +type ExecStatsForTraceHijacker interface { + // HijackExecStatsForTrace returns ExecStatsForTrace function, if set, and + // sets it to nil. The caller becomes responsible for collecting and + // propagating the execution statistics. + HijackExecStatsForTrace() func() *execinfrapb.ComponentStats +} + +var _ ExecStatsForTraceHijacker = &ProcessorBase{} + +// HijackExecStatsForTrace is a part of the ExecStatsForTraceHijacker interface. +func (pb *ProcessorBase) HijackExecStatsForTrace() func() *execinfrapb.ComponentStats { + execStatsForTrace := pb.ExecStatsForTrace + pb.ExecStatsForTrace = nil + return execStatsForTrace +} + // moveToTrailingMeta switches the processor to the "trailing meta" state: only // trailing metadata is returned from now on. For simplicity, processors are // encouraged to always use MoveToDraining() instead of this method, even when @@ -713,7 +731,7 @@ func (pb *ProcessorBase) moveToTrailingMeta() { if pb.span != nil { if pb.ExecStatsForTrace != nil { if stats := pb.ExecStatsForTrace(); stats != nil { - stats.Component = pb.FlowCtx.ProcessorComponentID(pb.processorID) + stats.Component = pb.FlowCtx.ProcessorComponentID(pb.ProcessorID) pb.span.RecordStructured(stats) } } @@ -834,7 +852,7 @@ func (pb *ProcessorBase) InitWithEvalCtx( pb.self = self pb.FlowCtx = flowCtx pb.EvalCtx = evalCtx - pb.processorID = processorID + pb.ProcessorID = processorID pb.MemMonitor = memMonitor pb.trailingMetaCallback = opts.TrailingMetaCallback if opts.InputsToDrain != nil { @@ -904,7 +922,7 @@ func (pb *ProcessorBase) startImpl( pb.Ctx, pb.span = ProcessorSpan(ctx, spanName) if pb.span != nil && pb.span.IsVerbose() { pb.span.SetTag(execinfrapb.FlowIDTagKey, pb.FlowCtx.ID.String()) - pb.span.SetTag(execinfrapb.ProcessorIDTagKey, pb.processorID) + pb.span.SetTag(execinfrapb.ProcessorIDTagKey, pb.ProcessorID) } } else { pb.Ctx = ctx diff --git a/pkg/sql/execinfrapb/component_stats.go b/pkg/sql/execinfrapb/component_stats.go index 6ba31ca796b3..676fe48a7c8f 100644 --- a/pkg/sql/execinfrapb/component_stats.go +++ b/pkg/sql/execinfrapb/component_stats.go @@ -352,9 +352,14 @@ func ExtractStatsFromSpans( statsMap[stats.Component] = &stats } else { // In the vectorized flow we can have multiple statistics - // entries for one component. Merge the stats together. - // TODO(radu): figure out a way to emit the statistics correctly - // in the first place. + // entries for one componentID because a single processor is + // represented by multiple components (e.g. when hash/merge + // joins have an ON expression that is not supported natively - + // we will plan the row-execution filterer processor then). + // + // Merge the stats together. + // TODO(yuzefovich): remove this once such edge cases are no + // longer present. statsMap[stats.Component] = existing.Union(&stats) } }) From 450bb8482e04c515cffd60afc7903f4cbd794ccf Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 19 Mar 2021 11:28:38 -0700 Subject: [PATCH 4/5] colexec: finish stats collection refactor This commit plumbs the stats collectors into `OpWithMetaInfo` object which allows us to get rid of separate `opDAGWithMetaSources` and `SynchronizerInput` structs as well as more clealy define the ownership of the stats collectors. Now all components can drain their input trees are also responsible for collecting the stats, if applicable, before draining. This invariant will be enforced by the follow-up commit. Notable changes are to the synchronizers - now all of them (ordered, serial unordered, parallel unordered) will be collecting stats. This required creating a tracing span on the first call to `Next` so that we had a way to propagate stats. That a-bit-hacky logic will be removed once the `Operator` interface is updated to take in the context object in `Init`. Release note: None --- pkg/sql/colexec/colbuilder/execplan.go | 15 +-- pkg/sql/colexec/colbuilder/execplan_test.go | 1 - pkg/sql/colexec/colexecargs/op_creation.go | 4 + .../colexecbase/simple_project_test.go | 3 +- pkg/sql/colexec/external_sort.go | 18 +--- pkg/sql/colexec/materializer.go | 3 +- pkg/sql/colexec/materializer_test.go | 4 - pkg/sql/colexec/ordered_synchronizer.eg.go | 45 +++++++-- pkg/sql/colexec/ordered_synchronizer_test.go | 14 ++- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 45 +++++++-- .../parallel_unordered_synchronizer.go | 50 ++++------ .../parallel_unordered_synchronizer_test.go | 7 +- .../colexec/serial_unordered_synchronizer.go | 44 +++++++-- .../serial_unordered_synchronizer_test.go | 16 ++- pkg/sql/colexec/types_integration_test.go | 1 - pkg/sql/colflow/routers.go | 32 ++---- pkg/sql/colflow/routers_test.go | 8 +- pkg/sql/colflow/vectorized_flow.go | 99 +++++-------------- .../colflow/vectorized_flow_shutdown_test.go | 13 +-- .../vectorized_meta_propagation_test.go | 1 - .../vectorized_panic_propagation_test.go | 2 - pkg/sql/distsql/columnar_utils_test.go | 1 - .../vectorized_panic_propagation_test.go | 1 - pkg/sql/execinfra/base.go | 11 +++ pkg/sql/sem/tree/eval_test/eval_test.go | 1 - 25 files changed, 226 insertions(+), 213 deletions(-) diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 62610774f4e1..ccd157f8d20d 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -84,10 +84,11 @@ func wrapRowSources( toWrapInputs = append(toWrapInputs, c.Input()) } else { inputInfoCopy := *inputInfo - // We pass on the responsibility of draining metadata sources and - // closing the closers to the materializer. + // We pass on the ownership over the meta components to the + // materializer. // TODO(yuzefovich): possibly set the length to 0 in order to be - // able to pool the underlying slice. + // able to pool the underlying slices. + inputInfo.StatsCollectors = nil inputInfo.MetadataSources = nil inputInfo.ToClose = nil // Note that this materializer is *not* added to the set of @@ -104,7 +105,6 @@ func wrapRowSources( inputInfoCopy, inputTypes[i], nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { @@ -1402,10 +1402,11 @@ func NewColOperator( r.Root = colexec.NewInvariantsChecker(r.Root) } } - // Handle the metadata sources and the closers from the input trees. Note - // that it is possible that we have created materializers which took over - // the responsibility over those objects. + // Handle the metadata components from the input trees. Note that it is + // possible that we have created materializers which took over the + // responsibility over those objects. for i := range inputs { + r.StatsCollectors = append(r.StatsCollectors, inputs[i].StatsCollectors...) r.MetadataSources = append(r.MetadataSources, inputs[i].MetadataSources...) r.ToClose = append(r.ToClose, inputs[i].ToClose...) } diff --git a/pkg/sql/colexec/colbuilder/execplan_test.go b/pkg/sql/colexec/colbuilder/execplan_test.go index edcb7fc33895..d4bcf5da6fbe 100644 --- a/pkg/sql/colexec/colbuilder/execplan_test.go +++ b/pkg/sql/colexec/colbuilder/execplan_test.go @@ -125,7 +125,6 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { r.OpWithMetaInfo, []*types.T{types.Int}, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) require.NoError(t, err) diff --git a/pkg/sql/colexec/colexecargs/op_creation.go b/pkg/sql/colexec/colexecargs/op_creation.go index 893a7eb0ad7d..3a9341e9e609 100644 --- a/pkg/sql/colexec/colexecargs/op_creation.go +++ b/pkg/sql/colexec/colexecargs/op_creation.go @@ -37,6 +37,10 @@ var TestNewColOperator func(ctx context.Context, flowCtx *execinfra.FlowCtx, arg // TODO(yuzefovich): figure out the story about pooling these objects. type OpWithMetaInfo struct { Root colexecop.Operator + // StatsCollectors are all stats collectors that are present in the tree + // rooted in Root for which the responsibility of retrieving stats hasn't + // been claimed yet. + StatsCollectors []colexecop.VectorizedStatsCollector // MetadataSources are all sources of the metadata that are present in the // tree rooted in Root for which the responsibility of draining hasn't been // claimed yet. diff --git a/pkg/sql/colexec/colexecbase/simple_project_test.go b/pkg/sql/colexec/colexecbase/simple_project_test.go index 35853da67594..45226711bc3c 100644 --- a/pkg/sql/colexec/colexecbase/simple_project_test.go +++ b/pkg/sql/colexec/colexecbase/simple_project_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -116,7 +117,7 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, inputTuples, [][]*types.T{inputTypes, inputTypes}, expected, colexectestutils.UnorderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { var input colexecop.Operator - parallelUnorderedSynchronizerInputs := make([]colexec.SynchronizerInput, len(inputs)) + parallelUnorderedSynchronizerInputs := make([]colexecargs.OpWithMetaInfo, len(inputs)) for i := range parallelUnorderedSynchronizerInputs { parallelUnorderedSynchronizerInputs[i].Root = inputs[i] } diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index 590fbac0d7f5..85db910442ee 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -373,10 +374,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { } n++ } - merger, err := s.createMergerForPartitions(ctx, n) - if err != nil { - colexecerror.InternalError(err) - } + merger := s.createMergerForPartitions(ctx, n) merger.Init() s.numPartitions -= n s.partitionsInfo.totalSize[s.numPartitions] = 0 @@ -411,11 +409,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { s.createPartitionerToOperators(s.numPartitions) s.emitter = s.partitionerToOperators[0] } else { - var err error - s.emitter, err = s.createMergerForPartitions(ctx, s.numPartitions) - if err != nil { - colexecerror.InternalError(err) - } + s.emitter = s.createMergerForPartitions(ctx, s.numPartitions) } s.emitter.Init() s.state = externalSorterEmitting @@ -576,11 +570,9 @@ func (s *externalSorter) createPartitionerToOperators(n int) { // createMergerForPartitions creates an ordered synchronizer that will merge // the last n current partitions. -func (s *externalSorter) createMergerForPartitions( - ctx context.Context, n int, -) (colexecop.Operator, error) { +func (s *externalSorter) createMergerForPartitions(ctx context.Context, n int) colexecop.Operator { s.createPartitionerToOperators(n) - syncInputs := make([]SynchronizerInput, n) + syncInputs := make([]colexecargs.OpWithMetaInfo, n) for i := range syncInputs { syncInputs[i].Root = s.partitionerToOperators[i] } diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index e471d4433343..366df9ef53e9 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -193,7 +193,6 @@ func NewMaterializer( input colexecargs.OpWithMetaInfo, typs []*types.T, output execinfra.RowReceiver, - statsCollectors []colexecop.VectorizedStatsCollector, cancelFlow func() context.CancelFunc, ) (*Materializer, error) { m := materializerPool.Get().(*Materializer) @@ -201,7 +200,7 @@ func NewMaterializer( ProcessorBase: m.ProcessorBase, input: input.Root, typs: typs, - drainHelper: newDrainHelper(statsCollectors, input.MetadataSources), + drainHelper: newDrainHelper(input.StatsCollectors, input.MetadataSources), converter: colconv.NewAllVecToDatumConverter(len(typs)), row: make(rowenc.EncDatumRow, len(typs)), closers: input.ToClose, diff --git a/pkg/sql/colexec/materializer_test.go b/pkg/sql/colexec/materializer_test.go index e5dbcfb395fd..83757581bad8 100644 --- a/pkg/sql/colexec/materializer_test.go +++ b/pkg/sql/colexec/materializer_test.go @@ -69,7 +69,6 @@ func TestColumnarizeMaterialize(t *testing.T) { colexecargs.OpWithMetaInfo{Root: c}, typs, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { @@ -152,7 +151,6 @@ func BenchmarkMaterializer(b *testing.B) { colexecargs.OpWithMetaInfo{Root: input}, typs, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { @@ -208,7 +206,6 @@ func TestMaterializerNextErrorAfterConsumerDone(t *testing.T) { }, nil, /* typ */ nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) require.NoError(t, err) @@ -256,7 +253,6 @@ func BenchmarkColumnarizeMaterialize(b *testing.B) { colexecargs.OpWithMetaInfo{Root: c}, types, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 5552f7616260..5c3c82846451 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" @@ -23,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -30,9 +32,12 @@ import ( // stream of rows, ordered according to a set of columns. The rows in each input // stream are assumed to be ordered according to the same set of columns. type OrderedSynchronizer struct { + ctx context.Context + span *tracing.Span + allocator *colmem.Allocator memoryLimit int64 - inputs []SynchronizerInput + inputs []colexecargs.OpWithMetaInfo ordering colinfo.ColumnOrdering typs []*types.T canonicalTypeFamilies []types.Family @@ -97,10 +102,10 @@ func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { func NewOrderedSynchronizer( allocator *colmem.Allocator, memoryLimit int64, - inputs []SynchronizerInput, + inputs []colexecargs.OpWithMetaInfo, typs []*types.T, ordering colinfo.ColumnOrdering, -) (*OrderedSynchronizer, error) { +) *OrderedSynchronizer { return &OrderedSynchronizer{ allocator: allocator, memoryLimit: memoryLimit, @@ -108,16 +113,27 @@ func NewOrderedSynchronizer( ordering: ordering, typs: typs, canonicalTypeFamilies: typeconv.ToCanonicalTypeFamilies(typs), - }, nil + } +} + +// maybeStartTracingSpan stores the context and possibly starts a tracing span +// on its first call and is a noop on all consequent calls. +// TODO(yuzefovich): remove this once ctx is passed in Init. +func (o *OrderedSynchronizer) maybeStartTracingSpan(ctx context.Context) { + if o.ctx == nil { + // It is the very first call to maybeStartTracingSpan. + o.ctx, o.span = execinfra.ProcessorSpan(ctx, "ordered sync") + } } // Next is part of the Operator interface. func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputBatches == nil { + o.maybeStartTracingSpan(ctx) o.inputBatches = make([]coldata.Batch, len(o.inputs)) o.heap = make([]int, 0, len(o.inputs)) for i := range o.inputs { - o.inputBatches[i] = o.inputs[i].Root.Next(ctx) + o.inputBatches[i] = o.inputs[i].Root.Next(o.ctx) o.updateComparators(i) if o.inputBatches[i].Length() > 0 { o.heap = append(o.heap, i) @@ -239,7 +255,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputIndices[minBatch]+1 < o.inputBatches[minBatch].Length() { o.inputIndices[minBatch]++ } else { - o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(ctx) + o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(o.ctx) o.inputIndices[minBatch] = 0 o.updateComparators(minBatch) } @@ -362,8 +378,20 @@ func (o *OrderedSynchronizer) Init() { func (o *OrderedSynchronizer) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { var bufferedMeta []execinfrapb.ProducerMetadata + // It is possible that Next was never called, yet the tracing is enabled. + o.maybeStartTracingSpan(ctx) + if o.span != nil { + for i := range o.inputs { + for _, stats := range o.inputs[i].StatsCollectors { + o.span.RecordStructured(stats.GetStats()) + } + } + if meta := execinfra.GetTraceDataAsMetadata(o.span); meta != nil { + bufferedMeta = append(bufferedMeta, *meta) + } + } for _, input := range o.inputs { - bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(ctx)...) + bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(o.ctx)...) } return bufferedMeta } @@ -372,6 +400,9 @@ func (o *OrderedSynchronizer) Close(ctx context.Context) error { for _, input := range o.inputs { input.ToClose.CloseAndLogOnErr(ctx, "ordered synchronizer") } + if o.span != nil { + o.span.Finish() + } return nil } diff --git a/pkg/sql/colexec/ordered_synchronizer_test.go b/pkg/sql/colexec/ordered_synchronizer_test.go index aab68505a9c9..f578c85e80b8 100644 --- a/pkg/sql/colexec/ordered_synchronizer_test.go +++ b/pkg/sql/colexec/ordered_synchronizer_test.go @@ -18,13 +18,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" ) // Adapted from the same-named test in the rowflow package. @@ -147,7 +147,7 @@ func TestOrderedSync(t *testing.T) { typs[i] = types.Int } colexectestutils.RunTests(t, testAllocator, tc.sources, tc.expected, colexectestutils.OrderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { - return NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, operatorsToSynchronizerInputs(inputs), typs, tc.ordering) + return NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, colexectestutils.MakeInputs(inputs), typs, tc.ordering), nil }) } } @@ -183,13 +183,12 @@ func TestOrderedSyncRandomInput(t *testing.T) { } sources[sourceIdx] = append(sources[sourceIdx], t) } - inputs := make([]SynchronizerInput, numInputs) + inputs := make([]colexecargs.OpWithMetaInfo, numInputs) for i := range inputs { inputs[i].Root = colexectestutils.NewOpTestInput(testAllocator, batchSize, sources[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} - op, err := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) - require.NoError(t, err) + op := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) op.Init() out := colexectestutils.NewOpTestOutput(op, expected) if err := out.Verify(); err != nil { @@ -213,14 +212,13 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { batch.ColVec(0).Int64()[i/numInputs] = i } - inputs := make([]SynchronizerInput, len(batches)) + inputs := make([]colexecargs.OpWithMetaInfo, len(batches)) for i := range batches { inputs[i].Root = colexecop.NewRepeatableBatchSource(testAllocator, batches[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} - op, err := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) - require.NoError(b, err) + op := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) op.Init() b.SetBytes(8 * int64(coldata.BatchSize()) * numInputs) diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 4e889f9dd152..2d05da9cf9da 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -34,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -55,9 +57,12 @@ const _TYPE_WIDTH = 0 // stream of rows, ordered according to a set of columns. The rows in each input // stream are assumed to be ordered according to the same set of columns. type OrderedSynchronizer struct { + ctx context.Context + span *tracing.Span + allocator *colmem.Allocator memoryLimit int64 - inputs []SynchronizerInput + inputs []colexecargs.OpWithMetaInfo ordering colinfo.ColumnOrdering typs []*types.T canonicalTypeFamilies []types.Family @@ -117,10 +122,10 @@ func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { func NewOrderedSynchronizer( allocator *colmem.Allocator, memoryLimit int64, - inputs []SynchronizerInput, + inputs []colexecargs.OpWithMetaInfo, typs []*types.T, ordering colinfo.ColumnOrdering, -) (*OrderedSynchronizer, error) { +) *OrderedSynchronizer { return &OrderedSynchronizer{ allocator: allocator, memoryLimit: memoryLimit, @@ -128,16 +133,27 @@ func NewOrderedSynchronizer( ordering: ordering, typs: typs, canonicalTypeFamilies: typeconv.ToCanonicalTypeFamilies(typs), - }, nil + } +} + +// maybeStartTracingSpan stores the context and possibly starts a tracing span +// on its first call and is a noop on all consequent calls. +// TODO(yuzefovich): remove this once ctx is passed in Init. +func (o *OrderedSynchronizer) maybeStartTracingSpan(ctx context.Context) { + if o.ctx == nil { + // It is the very first call to maybeStartTracingSpan. + o.ctx, o.span = execinfra.ProcessorSpan(ctx, "ordered sync") + } } // Next is part of the Operator interface. func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputBatches == nil { + o.maybeStartTracingSpan(ctx) o.inputBatches = make([]coldata.Batch, len(o.inputs)) o.heap = make([]int, 0, len(o.inputs)) for i := range o.inputs { - o.inputBatches[i] = o.inputs[i].Root.Next(ctx) + o.inputBatches[i] = o.inputs[i].Root.Next(o.ctx) o.updateComparators(i) if o.inputBatches[i].Length() > 0 { o.heap = append(o.heap, i) @@ -189,7 +205,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if o.inputIndices[minBatch]+1 < o.inputBatches[minBatch].Length() { o.inputIndices[minBatch]++ } else { - o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(ctx) + o.inputBatches[minBatch] = o.inputs[minBatch].Root.Next(o.ctx) o.inputIndices[minBatch] = 0 o.updateComparators(minBatch) } @@ -255,8 +271,20 @@ func (o *OrderedSynchronizer) Init() { func (o *OrderedSynchronizer) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { var bufferedMeta []execinfrapb.ProducerMetadata + // It is possible that Next was never called, yet the tracing is enabled. + o.maybeStartTracingSpan(ctx) + if o.span != nil { + for i := range o.inputs { + for _, stats := range o.inputs[i].StatsCollectors { + o.span.RecordStructured(stats.GetStats()) + } + } + if meta := execinfra.GetTraceDataAsMetadata(o.span); meta != nil { + bufferedMeta = append(bufferedMeta, *meta) + } + } for _, input := range o.inputs { - bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(ctx)...) + bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(o.ctx)...) } return bufferedMeta } @@ -265,6 +293,9 @@ func (o *OrderedSynchronizer) Close(ctx context.Context) error { for _, input := range o.inputs { input.ToClose.CloseAndLogOnErr(ctx, "ordered synchronizer") } + if o.span != nil { + o.span.Finish() + } return nil } diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index 85134197f3ad..25a2a0498918 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -12,6 +12,7 @@ package colexec import ( "context" + "fmt" "sync" "sync/atomic" @@ -21,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -59,7 +61,7 @@ const ( // ParallelUnorderedSynchronizer is an Operator that combines multiple Operator streams // into one. type ParallelUnorderedSynchronizer struct { - inputs []SynchronizerInput + inputs []colexecargs.OpWithMetaInfo // readNextBatch is a slice of channels, where each channel corresponds to the // input at the same index in inputs. It is used as a barrier for input // goroutines to wait on until the Next goroutine signals that it is safe to @@ -108,29 +110,6 @@ func (s *ParallelUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.O return s.inputs[nth].Root } -// SynchronizerInput is a wrapper over a colexecop.Operator that a -// synchronizer goroutine will be calling Next on. An accompanying -// colexecop.MetadataSources may also be specified, in which case -// DrainMeta will be called from the same goroutine. -type SynchronizerInput struct { - colexecargs.OpWithMetaInfo - // StatsCollectors are all vectorized stats collectors in the input tree. - // The field is currently being used *only* to track all of the stats - // collectors in the input tree, and the synchronizers should *not* access - // it themselves. - // TODO(yuzefovich): actually move the logic of getting stats into the - // synchronizers. - StatsCollectors []colexecop.VectorizedStatsCollector -} - -func operatorsToSynchronizerInputs(ops []colexecop.Operator) []SynchronizerInput { - result := make([]SynchronizerInput, len(ops)) - for i := range result { - result[i].Root = ops[i] - } - return result -} - // NewParallelUnorderedSynchronizer creates a new ParallelUnorderedSynchronizer. // On the first call to Next, len(inputs) goroutines are spawned to read each // input asynchronously (to not be limited by a slow input). These will @@ -138,7 +117,7 @@ func operatorsToSynchronizerInputs(ops []colexecop.Operator) []SynchronizerInput // guaranteed that these spawned goroutines will have completed on any error or // zero-length batch received from Next. func NewParallelUnorderedSynchronizer( - inputs []SynchronizerInput, wg *sync.WaitGroup, + inputs []colexecargs.OpWithMetaInfo, wg *sync.WaitGroup, ) *ParallelUnorderedSynchronizer { readNextBatch := make([]chan struct{}, len(inputs)) for i := range readNextBatch { @@ -191,7 +170,7 @@ func (s *ParallelUnorderedSynchronizer) setState(state parallelUnorderedSynchron // affected by slow inputs. func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { for i, input := range s.inputs { - s.nextBatch[i] = func(input SynchronizerInput, inputIdx int) func() { + s.nextBatch[i] = func(input colexecargs.OpWithMetaInfo, inputIdx int) func() { return func() { s.batches[inputIdx] = input.Root.Next(ctx) } @@ -201,8 +180,13 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { // TODO(asubiotto): Most inputs are Inboxes, and these have handler // goroutines just sitting around waiting for cancellation. I wonder if we // could reuse those goroutines to push batches to batchCh directly. - go func(input SynchronizerInput, inputIdx int) { + go func(ctx context.Context, input colexecargs.OpWithMetaInfo, inputIdx int) { + var span *tracing.Span + ctx, span = execinfra.ProcessorSpan(ctx, fmt.Sprintf("parallel unordered sync input %d", inputIdx)) defer func() { + if span != nil { + span.Finish() + } if int(atomic.AddUint32(&s.numFinishedInputs, 1)) == len(s.inputs) { close(s.batchCh) } @@ -244,8 +228,16 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { msg = &unorderedSynchronizerMsg{ inputIdx: inputIdx, } + if span != nil { + for _, s := range input.StatsCollectors { + span.RecordStructured(s.GetStats()) + } + if meta := execinfra.GetTraceDataAsMetadata(span); meta != nil { + msg.meta = append(msg.meta, *meta) + } + } if input.MetadataSources != nil { - msg.meta = input.MetadataSources.DrainMeta(ctx) + msg.meta = append(msg.meta, input.MetadataSources.DrainMeta(ctx)...) } if msg.meta == nil { // Initialize msg.meta to be non-nil, which is a signal that @@ -283,7 +275,7 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { return } } - }(input, i) + }(ctx, input, i) } } diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index 0d1f9d5ec1d4..3d45872015de 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -50,7 +51,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { numBatches = rng.Intn(maxBatches) + 1 ) - inputs := make([]SynchronizerInput, numInputs) + inputs := make([]colexecargs.OpWithMetaInfo, numInputs) for i := range inputs { source := colexecop.NewRepeatableBatchSource( testAllocator, @@ -153,7 +154,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { const expectedErr = "first input error" ctx := context.Background() - inputs := make([]SynchronizerInput, 6) + inputs := make([]colexecargs.OpWithMetaInfo, 6) inputs[0].Root = &colexecop.CallbackOperator{NextCb: func(context.Context) coldata.Batch { colexecerror.InternalError(errors.New(expectedErr)) // This code is unreachable, but the compiler cannot infer that. @@ -200,7 +201,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { const numInputs = 6 typs := []*types.T{types.Int} - inputs := make([]SynchronizerInput, numInputs) + inputs := make([]colexecargs.OpWithMetaInfo, numInputs) for i := range inputs { batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index 333ceae892b4..1fb7160307bd 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer.go @@ -14,9 +14,11 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/tracing" ) // SerialUnorderedSynchronizer is an Operator that combines multiple Operator @@ -26,7 +28,10 @@ import ( // undesirable - for example when the whole query is planned on the gateway and // we want to run it in the RootTxn. type SerialUnorderedSynchronizer struct { - inputs []SynchronizerInput + ctx context.Context + span *tracing.Span + + inputs []colexecargs.OpWithMetaInfo // curSerialInputIdx indicates the index of the current input being consumed. curSerialInputIdx int } @@ -48,10 +53,11 @@ func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpN } // NewSerialUnorderedSynchronizer creates a new SerialUnorderedSynchronizer. -func NewSerialUnorderedSynchronizer(inputs []SynchronizerInput) *SerialUnorderedSynchronizer { +func NewSerialUnorderedSynchronizer( + inputs []colexecargs.OpWithMetaInfo, +) *SerialUnorderedSynchronizer { return &SerialUnorderedSynchronizer{ - inputs: inputs, - curSerialInputIdx: 0, + inputs: inputs, } } @@ -62,13 +68,24 @@ func (s *SerialUnorderedSynchronizer) Init() { } } +// maybeStartTracingSpan stores the context and possibly starts a tracing span +// on its first call and is a noop on all consequent calls. +// TODO(yuzefovich): remove this once ctx is passed in Init. +func (s *SerialUnorderedSynchronizer) maybeStartTracingSpan(ctx context.Context) { + if s.ctx == nil { + // It is the very first call to maybeStartTracingSpan. + s.ctx, s.span = execinfra.ProcessorSpan(ctx, "serial unordered sync") + } +} + // Next is part of the Operator interface. func (s *SerialUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch { + s.maybeStartTracingSpan(ctx) for { if s.curSerialInputIdx == len(s.inputs) { return coldata.ZeroBatch } - b := s.inputs[s.curSerialInputIdx].Root.Next(ctx) + b := s.inputs[s.curSerialInputIdx].Root.Next(s.ctx) if b.Length() == 0 { s.curSerialInputIdx++ } else { @@ -82,8 +99,20 @@ func (s *SerialUnorderedSynchronizer) DrainMeta( ctx context.Context, ) []execinfrapb.ProducerMetadata { var bufferedMeta []execinfrapb.ProducerMetadata + // It is possible that Next was never called, yet the tracing is enabled. + s.maybeStartTracingSpan(ctx) + if s.span != nil { + for i := range s.inputs { + for _, stats := range s.inputs[i].StatsCollectors { + s.span.RecordStructured(stats.GetStats()) + } + } + if meta := execinfra.GetTraceDataAsMetadata(s.span); meta != nil { + bufferedMeta = append(bufferedMeta, *meta) + } + } for _, input := range s.inputs { - bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(ctx)...) + bufferedMeta = append(bufferedMeta, input.MetadataSources.DrainMeta(s.ctx)...) } return bufferedMeta } @@ -93,5 +122,8 @@ func (s *SerialUnorderedSynchronizer) Close(ctx context.Context) error { for _, input := range s.inputs { input.ToClose.CloseAndLogOnErr(ctx, "serial unordered synchronizer") } + if s.span != nil { + s.span.Finish() + } return nil } diff --git a/pkg/sql/colexec/serial_unordered_synchronizer_test.go b/pkg/sql/colexec/serial_unordered_synchronizer_test.go index ced6dc2571a9..f74bd9629759 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -38,21 +38,19 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { const numBatches = 4 typs := []*types.T{types.Int} - inputs := make([]SynchronizerInput, numInputs) + inputs := make([]colexecargs.OpWithMetaInfo, numInputs) for i := range inputs { batch := coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.ResetBatchesToReturn(numBatches) inputIdx := i - inputs[i] = SynchronizerInput{OpWithMetaInfo: colexecargs.OpWithMetaInfo{ - Root: source, - MetadataSources: []colexecop.MetadataSource{ - colexectestutils.CallbackMetadataSource{ - DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { - return []execinfrapb.ProducerMetadata{{Err: errors.Errorf("input %d test-induced metadata", inputIdx)}} - }, + inputs[i].Root = source + inputs[i].MetadataSources = []colexecop.MetadataSource{ + colexectestutils.CallbackMetadataSource{ + DrainMetaCb: func(_ context.Context) []execinfrapb.ProducerMetadata { + return []execinfrapb.ProducerMetadata{{Err: errors.Errorf("input %d test-induced metadata", inputIdx)}} }, - }}, + }, } } s := NewSerialUnorderedSynchronizer(inputs) diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index a6dc49629ef1..bc6a01088d7e 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -94,7 +94,6 @@ func TestSQLTypesIntegration(t *testing.T) { colexecargs.OpWithMetaInfo{Root: arrowOp}, typs, output, - nil, /* statsCollectors */ nil, /* cancelFlow */ ) require.NoError(t, err) diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index bf47aeea3926..c83e4fc0f6c4 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -404,22 +404,14 @@ const ( // returned by the constructor. type HashRouter struct { colexecop.OneInputNode + // inputMetaInfo contains all of the meta components that the hash router + // is responsible for. Root field is exactly the same as OneInputNode.Input. + inputMetaInfo colexecargs.OpWithMetaInfo // hashCols is a slice of indices of the columns used for hashing. hashCols []uint32 // One output for each stream. outputs []routerOutput - // statsCollectors, when non-nil, will be retrieved from by the hash router - // and the execution statistics will then be propagated as - // execinfrapb.ProducerMetadata object right before draining - // metadataSources. - statsCollectors []colexecop.VectorizedStatsCollector - // metadataSources is a slice of colexecop.MetadataSources that need to be - // drained when the HashRouter terminates. - metadataSources colexecop.MetadataSources - // closers is a slice of Closers that need to be closed when the hash router - // terminates. - closers colexecop.Closers // unblockedEventsChan is a channel shared between the HashRouter and its // outputs. outputs send events on this channel when they are unblocked by a @@ -466,7 +458,6 @@ func NewHashRouter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, - statsCollectors []colexecop.VectorizedStatsCollector, ) (*HashRouter, []colexecop.DrainableOperator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) @@ -497,7 +488,7 @@ func NewHashRouter( outputs[i] = op outputsAsOps[i] = op } - return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, statsCollectors), outputsAsOps + return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs), outputsAsOps } func newHashRouterWithOutputs( @@ -505,15 +496,12 @@ func newHashRouterWithOutputs( hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, - statsCollectors []colexecop.VectorizedStatsCollector, ) *HashRouter { r := &HashRouter{ OneInputNode: colexecop.NewOneInputNode(input.Root), + inputMetaInfo: input, hashCols: hashCols, outputs: outputs, - statsCollectors: statsCollectors, - metadataSources: input.MetadataSources, - closers: input.ToClose, unblockedEventsChan: unblockEventsChan, // waitForMetadata is a buffered channel to avoid blocking if nobody will // read the metadata. @@ -619,22 +607,20 @@ func (r *HashRouter) Run(ctx context.Context) { r.cancelOutputs(ctx, err) } if span != nil { - for _, s := range r.statsCollectors { + for _, s := range r.inputMetaInfo.StatsCollectors { span.RecordStructured(s.GetStats()) } - if trace := span.GetRecording(); len(trace) > 0 { - meta := execinfrapb.GetProducerMeta() - meta.TraceData = trace + if meta := execinfra.GetTraceDataAsMetadata(span); meta != nil { r.bufferedMeta = append(r.bufferedMeta, *meta) } } - r.bufferedMeta = append(r.bufferedMeta, r.metadataSources.DrainMeta(ctx)...) + r.bufferedMeta = append(r.bufferedMeta, r.inputMetaInfo.MetadataSources.DrainMeta(ctx)...) // Non-blocking send of metadata so that one of the outputs can return it // in DrainMeta. r.waitForMetadata <- r.bufferedMeta close(r.waitForMetadata) - r.closers.CloseAndLogOnErr(ctx, "hash router") + r.inputMetaInfo.ToClose.CloseAndLogOnErr(ctx, "hash router") } // processNextBatch reads the next batch from its input, hashes it and adds diff --git a/pkg/sql/colflow/routers_test.go b/pkg/sql/colflow/routers_test.go index e7e6d5e5ba2b..0e5797ed8213 100644 --- a/pkg/sql/colflow/routers_test.go +++ b/pkg/sql/colflow/routers_test.go @@ -741,8 +741,8 @@ func TestHashRouterComputesDestination(t *testing.T) { r := newHashRouterWithOutputs( colexecargs.OpWithMetaInfo{Root: in}, []uint32{0}, /* hashCols */ - nil /* ch */, outputs, - nil, /* statsCollectors */ + nil, /* unblockEventsChan */ + outputs, ) for r.processNextBatch(ctx) { } @@ -791,7 +791,6 @@ func TestHashRouterCancellation(t *testing.T) { []uint32{0}, /* hashCols */ unbufferedCh, routerOutputs, - nil, /* statsCollectors */ ) t.Run("BeforeRun", func(t *testing.T) { @@ -904,7 +903,6 @@ func TestHashRouterOneOutput(t *testing.T) { queueCfg, colexecop.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, - nil, /* statsCollectors */ ) if len(routerOutputs) != 1 { @@ -1105,7 +1103,6 @@ func TestHashRouterRandom(t *testing.T) { hashCols, unblockEventsChan, outputs, - nil, /* statsCollectors */ ) var ( @@ -1330,7 +1327,6 @@ func BenchmarkHashRouter(b *testing.B) { queueCfg, &colexecop.TestingSemaphore{}, diskAccounts, - nil, /* statsCollectors */ ) b.SetBytes(8 * int64(coldata.BatchSize()) * int64(numInputBatches)) // We expect distribution to not change. This is a sanity check that diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index d850894cb708..24135ab43b58 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -416,14 +416,6 @@ type flowCreatorHelper interface { getCancelFlowFn() context.CancelFunc } -// opDAGWithMetaSources is a helper struct that stores an operator DAG as well -// as the metadataSources and closers in this DAG that need to be drained and -// closed. -type opDAGWithMetaSources struct { - colexecargs.OpWithMetaInfo - statsCollectors []colexecop.VectorizedStatsCollector -} - // remoteComponentCreator is an interface that abstracts the constructors for // several components in a remote flow. Mostly for testing purposes. type remoteComponentCreator interface { @@ -465,7 +457,7 @@ type vectorizedFlowCreator struct { flowCreatorHelper remoteComponentCreator - streamIDToInputOp map[execinfrapb.StreamID]opDAGWithMetaSources + streamIDToInputOp map[execinfrapb.StreamID]colexecargs.OpWithMetaInfo streamIDToSpecIdx map[execinfrapb.StreamID]int recordingStats bool isGatewayNode bool @@ -520,7 +512,7 @@ var _ execinfra.Releasable = &vectorizedFlowCreator{} var vectorizedFlowCreatorPool = sync.Pool{ New: func() interface{} { return &vectorizedFlowCreator{ - streamIDToInputOp: make(map[execinfrapb.StreamID]opDAGWithMetaSources), + streamIDToInputOp: make(map[execinfrapb.StreamID]colexecargs.OpWithMetaInfo), streamIDToSpecIdx: make(map[execinfrapb.StreamID]int), exprHelper: colexecargs.NewExprHelper(), } @@ -712,7 +704,6 @@ func (s *vectorizedFlowCreator) setupRouter( input colexecargs.OpWithMetaInfo, outputTyps []*types.T, output *execinfrapb.OutputRouterSpec, - statsCollectors []colexecop.VectorizedStatsCollector, factory coldata.ColumnFactory, ) error { if output.Type != execinfrapb.OutputRouterSpec_BY_HASH { @@ -735,8 +726,9 @@ func (s *vectorizedFlowCreator) setupRouter( } diskMon, diskAccounts := s.createDiskAccounts(ctx, flowCtx, mmName, len(output.Streams)) router, outputs := NewHashRouter( - allocators, input, outputTyps, output.HashColumns, execinfra.GetWorkMemLimit(flowCtx.Cfg), - s.diskQueueCfg, s.fdSemaphore, diskAccounts, statsCollectors, + allocators, input, outputTyps, output.HashColumns, + execinfra.GetWorkMemLimit(flowCtx.Cfg), + s.diskQueueCfg, s.fdSemaphore, diskAccounts, ) runRouter := func(ctx context.Context, _ context.CancelFunc) { router.Run(logtags.AddTag(ctx, "hashRouterID", strings.Join(streamIDs, ","))) @@ -768,10 +760,9 @@ func (s *vectorizedFlowCreator) setupRouter( opWithMetaInfo := colexecargs.OpWithMetaInfo{ Root: op, MetadataSources: colexecop.MetadataSources{op}, - // toClose will be closed by the HashRouter. + // ToClose will be closed by the hash router. ToClose: nil, } - var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { mons := []*mon.BytesMonitor{hashRouterMemMonitor, diskMon} // Wrap local outputs with vectorized stats collectors when recording @@ -785,12 +776,9 @@ func (s *vectorizedFlowCreator) setupRouter( return err } opWithMetaInfo.Root = vsc - statsCollectors = []colexecop.VectorizedStatsCollector{vsc} - } - s.streamIDToInputOp[stream.StreamID] = opDAGWithMetaSources{ - OpWithMetaInfo: opWithMetaInfo, - statsCollectors: statsCollectors, + opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} } + s.streamIDToInputOp[stream.StreamID] = opWithMetaInfo } } if !foundLocalOutput { @@ -813,29 +801,26 @@ func (s *vectorizedFlowCreator) setupInput( input execinfrapb.InputSyncSpec, opt flowinfra.FuseOpt, factory coldata.ColumnFactory, -) (colexecargs.OpWithMetaInfo, []colexecop.VectorizedStatsCollector, error) { - inputStreamOps := make([]colexec.SynchronizerInput, 0, len(input.Streams)) +) (colexecargs.OpWithMetaInfo, error) { + inputStreamOps := make([]colexecargs.OpWithMetaInfo, 0, len(input.Streams)) // Before we can safely use types we received over the wire in the // operators, we need to make sure they are hydrated. In row execution // engine it is done during the processor initialization, but operators // don't do that. if err := s.typeResolver.HydrateTypeSlice(ctx, input.ColumnTypes); err != nil { - return colexecargs.OpWithMetaInfo{}, nil, err + return colexecargs.OpWithMetaInfo{}, err } for _, inputStream := range input.Streams { switch inputStream.Type { case execinfrapb.StreamEndpointSpec_LOCAL: in := s.streamIDToInputOp[inputStream.StreamID] - inputStreamOps = append(inputStreamOps, colexec.SynchronizerInput{ - OpWithMetaInfo: in.OpWithMetaInfo, - StatsCollectors: in.statsCollectors, - }) + inputStreamOps = append(inputStreamOps, in) case execinfrapb.StreamEndpointSpec_REMOTE: // If the input is remote, the input operator does not exist in // streamIDToInputOp. Create an inbox. if err := s.checkInboundStreamID(inputStream.StreamID); err != nil { - return colexecargs.OpWithMetaInfo{}, nil, err + return colexecargs.OpWithMetaInfo{}, err } // Retrieve the latency from the origin node (the one that has the @@ -856,7 +841,7 @@ func (s *vectorizedFlowCreator) setupInput( ) if err != nil { - return colexecargs.OpWithMetaInfo{}, nil, err + return colexecargs.OpWithMetaInfo{}, err } s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup) op := colexecop.Operator(inbox) @@ -869,7 +854,6 @@ func (s *vectorizedFlowCreator) setupInput( Root: op, MetadataSources: colexecop.MetadataSources{ms}, } - var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { // Note: we can't use flowCtx.StreamComponentID because the stream does // not originate from this node (we are the target node). @@ -878,29 +862,22 @@ func (s *vectorizedFlowCreator) setupInput( ) vsc := s.wrapWithNetworkVectorizedStatsCollector(op, inbox, compID, latency) opWithMetaInfo.Root = vsc - statsCollectors = []colexecop.VectorizedStatsCollector{vsc} + opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} } - inputStreamOps = append(inputStreamOps, colexec.SynchronizerInput{ - OpWithMetaInfo: opWithMetaInfo, - StatsCollectors: statsCollectors, - }) + inputStreamOps = append(inputStreamOps, opWithMetaInfo) default: - return colexecargs.OpWithMetaInfo{}, nil, errors.Errorf("unsupported input stream type %s", inputStream.Type) + return colexecargs.OpWithMetaInfo{}, errors.Errorf("unsupported input stream type %s", inputStream.Type) } } - opWithMetaInfo := inputStreamOps[0].OpWithMetaInfo - statsCollectors := inputStreamOps[0].StatsCollectors + opWithMetaInfo := inputStreamOps[0] if len(inputStreamOps) > 1 { statsInputs := inputStreamOps if input.Type == execinfrapb.InputSyncSpec_ORDERED { - os, err := colexec.NewOrderedSynchronizer( + os := colexec.NewOrderedSynchronizer( colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx), factory), execinfra.GetWorkMemLimit(flowCtx.Cfg), inputStreamOps, input.ColumnTypes, execinfrapb.ConvertToColumnOrdering(input.Ordering), ) - if err != nil { - return colexecargs.OpWithMetaInfo{}, nil, err - } opWithMetaInfo = colexecargs.OpWithMetaInfo{ Root: os, MetadataSources: colexecop.MetadataSources{os}, @@ -948,21 +925,13 @@ func (s *vectorizedFlowCreator) setupInput( statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ ) if err != nil { - return colexecargs.OpWithMetaInfo{}, nil, err + return colexecargs.OpWithMetaInfo{}, err } opWithMetaInfo.Root = vsc - // Accumulate the stats collectors from all of the input trees and - // add the wrapped synchronizer. - // TODO(yuzefovich): move the stats collection from inputs into the - // synchronizer itself. - statsCollectors = nil - for _, input := range inputStreamOps { - statsCollectors = append(statsCollectors, input.StatsCollectors...) - } - statsCollectors = append(statsCollectors, vsc) + opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} } } - return opWithMetaInfo, statsCollectors, nil + return opWithMetaInfo, nil } // setupOutput sets up any necessary infrastructure according to the output spec @@ -976,7 +945,6 @@ func (s *vectorizedFlowCreator) setupOutput( pspec *execinfrapb.ProcessorSpec, opWithMetaInfo colexecargs.OpWithMetaInfo, opOutputTypes []*types.T, - statsCollectors []colexecop.VectorizedStatsCollector, factory coldata.ColumnFactory, ) error { output := &pspec.Output[0] @@ -987,7 +955,6 @@ func (s *vectorizedFlowCreator) setupOutput( opWithMetaInfo, opOutputTypes, output, - statsCollectors, factory, ) } @@ -998,15 +965,12 @@ func (s *vectorizedFlowCreator) setupOutput( outputStream := &output.Streams[0] switch outputStream.Type { case execinfrapb.StreamEndpointSpec_LOCAL: - s.streamIDToInputOp[outputStream.StreamID] = opDAGWithMetaSources{ - OpWithMetaInfo: opWithMetaInfo, - statsCollectors: statsCollectors, - } + s.streamIDToInputOp[outputStream.StreamID] = opWithMetaInfo case execinfrapb.StreamEndpointSpec_REMOTE: // Set up an Outbox. outbox, err := s.setupRemoteOutputStream( ctx, flowCtx, opWithMetaInfo, opOutputTypes, outputStream, factory, - s.makeGetStatsFnForOutbox(flowCtx, statsCollectors, outputStream.OriginNodeID), + s.makeGetStatsFnForOutbox(flowCtx, opWithMetaInfo.StatsCollectors, outputStream.OriginNodeID), ) if err != nil { return err @@ -1022,7 +986,6 @@ func (s *vectorizedFlowCreator) setupOutput( opWithMetaInfo, opOutputTypes, s.syncFlowConsumer, - statsCollectors, s.getCancelFlowFn, ) if err != nil { @@ -1088,16 +1051,12 @@ func (s *vectorizedFlowCreator) setupFlow( } var inputs []colexecargs.OpWithMetaInfo - var inputsStatsCollectors [][]colexecop.VectorizedStatsCollector for i := range pspec.Input { - input, vsc, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) + input, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) if localErr != nil { err = localErr return } - if vsc != nil { - inputsStatsCollectors = append(inputsStatsCollectors, vsc) - } inputs = append(inputs, input) } @@ -1153,7 +1112,6 @@ func (s *vectorizedFlowCreator) setupFlow( s.numClosers += int32(len(result.ToClose)) } - var statsCollectors []colexecop.VectorizedStatsCollector if s.recordingStats { vsc, err := s.wrapWithVectorizedStatsCollectorBase( result.Root, result.KVReader, result.Columnarizer, inputs, @@ -1163,14 +1121,11 @@ func (s *vectorizedFlowCreator) setupFlow( return } result.Root = vsc - for i := range inputs { - statsCollectors = append(statsCollectors, inputsStatsCollectors[i]...) - } - statsCollectors = append(statsCollectors, vsc) + result.StatsCollectors = append(result.StatsCollectors, vsc) } if err = s.setupOutput( - ctx, flowCtx, pspec, result.OpWithMetaInfo, result.ColumnTypes, statsCollectors, factory, + ctx, flowCtx, pspec, result.OpWithMetaInfo, result.ColumnTypes, factory, ); err != nil { return } diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index ecf2933380c9..002996f22e32 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -164,7 +164,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { numInboxes = numHashRouterOutputs + 3 inboxes = make([]*colrpc.Inbox, 0, numInboxes+1) handleStreamErrCh = make([]chan error, numInboxes+1) - synchronizerInputs = make([]colexec.SynchronizerInput, 0, numInboxes) + synchronizerInputs = make([]colexecargs.OpWithMetaInfo, 0, numInboxes) streamID = 0 addAnotherRemote = rng.Float64() < 0.5 ) @@ -207,7 +207,6 @@ func TestVectorizedFlowShutdown(t *testing.T) { queueCfg, &colexecop.TestingSemaphore{}, diskAccounts, - nil, /* statsCollectors */ ) for i := 0; i < numInboxes; i++ { inboxMemAccount := testMemMonitor.MakeBoundAccount() @@ -222,11 +221,10 @@ func TestVectorizedFlowShutdown(t *testing.T) { inboxes = append(inboxes, inbox) synchronizerInputs = append( synchronizerInputs, - colexec.SynchronizerInput{ - OpWithMetaInfo: colexecargs.OpWithMetaInfo{ - Root: colexecop.Operator(inbox), - MetadataSources: []colexecop.MetadataSource{inbox}, - }}, + colexecargs.OpWithMetaInfo{ + Root: colexecop.Operator(inbox), + MetadataSources: []colexecop.MetadataSource{inbox}, + }, ) } synchronizer := colexec.NewParallelUnorderedSynchronizer(synchronizerInputs, &wg) @@ -371,7 +369,6 @@ func TestVectorizedFlowShutdown(t *testing.T) { }, typs, nil, /* output */ - nil, /* statsCollectors */ func() context.CancelFunc { return cancelLocal }, ) require.NoError(t, err) diff --git a/pkg/sql/colflow/vectorized_meta_propagation_test.go b/pkg/sql/colflow/vectorized_meta_propagation_test.go index 988e33d1e0d9..7682be150ded 100644 --- a/pkg/sql/colflow/vectorized_meta_propagation_test.go +++ b/pkg/sql/colflow/vectorized_meta_propagation_test.go @@ -83,7 +83,6 @@ func TestVectorizedMetaPropagation(t *testing.T) { }, typs, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index 628e4081a93f..8687c85eb19d 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -62,7 +62,6 @@ func TestVectorizedInternalPanic(t *testing.T) { colexecargs.OpWithMetaInfo{Root: vee}, typs, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { @@ -107,7 +106,6 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { colexecargs.OpWithMetaInfo{Root: nvee}, typs, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index 7f111060794c..08b5572bca97 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -176,7 +176,6 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { result.OpWithMetaInfo, args.pspec.ResultTypes, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index 2a7f5568bdc5..17335911e8a7 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -61,7 +61,6 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { }}, nil, /* typs */ &distsqlutils.RowBuffer{}, - nil, /* statsCollectors */ nil, /* cancelFlow */ ) if err != nil { diff --git a/pkg/sql/execinfra/base.go b/pkg/sql/execinfra/base.go index 40c01409cbc8..48b8df07b638 100644 --- a/pkg/sql/execinfra/base.go +++ b/pkg/sql/execinfra/base.go @@ -246,6 +246,17 @@ func GetTraceData(ctx context.Context) []tracingpb.RecordedSpan { return nil } +// GetTraceDataAsMetadata returns the trace data as execinfrapb.ProducerMetadata +// object. +func GetTraceDataAsMetadata(span *tracing.Span) *execinfrapb.ProducerMetadata { + if trace := span.GetRecording(); len(trace) > 0 { + meta := execinfrapb.GetProducerMeta() + meta.TraceData = trace + return meta + } + return nil +} + // SendTraceData collects the tracing information from the ctx and pushes it to // dst. The ConsumerStatus returned by dst is ignored. // diff --git a/pkg/sql/sem/tree/eval_test/eval_test.go b/pkg/sql/sem/tree/eval_test/eval_test.go index 8efd7785e570..a0c1aa5df8c2 100644 --- a/pkg/sql/sem/tree/eval_test/eval_test.go +++ b/pkg/sql/sem/tree/eval_test/eval_test.go @@ -203,7 +203,6 @@ func TestEval(t *testing.T) { result.OpWithMetaInfo, []*types.T{typedExpr.ResolvedType()}, nil, /* output */ - nil, /* statsCollectors */ nil, /* cancelFlow */ ) require.NoError(t, err) From ec2c445d0a95ea5e33f015e94d07863a9bce5e06 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 19 Mar 2021 12:04:20 -0700 Subject: [PATCH 5/5] colflow: enforce stats collection before drain meta This commit introduces a dummy vectorized stats collector that is also a metadata source that simply checks that `GetStats` has been called before `DrainMeta` and returns an error as a metadata if not. This stats invariants checker is now planned whenever `util.CrdbTestBuild` is true (which is when `TAGS=crdb_test` was specified). Release note: None --- pkg/sql/colflow/stats.go | 44 ++++++++++++++++++++++ pkg/sql/colflow/vectorized_flow.go | 60 ++++++++++++++---------------- 2 files changed, 72 insertions(+), 32 deletions(-) diff --git a/pkg/sql/colflow/stats.go b/pkg/sql/colflow/stats.go index 8410a0006c6f..3808b5d06f20 100644 --- a/pkg/sql/colflow/stats.go +++ b/pkg/sql/colflow/stats.go @@ -15,10 +15,12 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -237,3 +239,45 @@ func (nvsc *networkVectorizedStatsCollectorImpl) GetStats() *execinfrapb.Compone return s } + +// maybeAddStatsInvariantChecker will add a statsInvariantChecker to both +// StatsCollectors and MetadataSources of op if crdb_test build tag is +// specified. See comment on statsInvariantChecker for the kind of invariants +// checked. +func maybeAddStatsInvariantChecker(op *colexecargs.OpWithMetaInfo) { + if util.CrdbTestBuild { + c := &statsInvariantChecker{} + op.StatsCollectors = append(op.StatsCollectors, c) + op.MetadataSources = append(op.MetadataSources, c) + } +} + +// statsInvariantChecker is a dummy colexecop.VectorizedStatsCollector as well +// as colexecop.MetadataSource which asserts that GetStats is called before +// DrainMeta. It should only be used in the test environment. +type statsInvariantChecker struct { + colexecop.ZeroInputNode + + statsRetrieved bool +} + +var _ colexecop.VectorizedStatsCollector = &statsInvariantChecker{} +var _ colexecop.MetadataSource = &statsInvariantChecker{} + +func (i *statsInvariantChecker) Init() {} + +func (i *statsInvariantChecker) Next(context.Context) coldata.Batch { + return coldata.ZeroBatch +} + +func (i *statsInvariantChecker) GetStats() *execinfrapb.ComponentStats { + i.statsRetrieved = true + return &execinfrapb.ComponentStats{} +} + +func (i *statsInvariantChecker) DrainMeta(context.Context) []execinfrapb.ProducerMetadata { + if !i.statsRetrieved { + return []execinfrapb.ProducerMetadata{{Err: errors.New("GetStats wasn't called before DrainMeta")}} + } + return nil +} diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 24135ab43b58..7c7caac9ec66 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -315,13 +315,13 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { // created wrapper with those corresponding to operators in inputs (the latter // must have already been wrapped). func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( - op colexecop.Operator, + op *colexecargs.OpWithMetaInfo, kvReader colexecop.KVReader, columnarizer colexecop.VectorizedStatsCollector, inputs []colexecargs.OpWithMetaInfo, component execinfrapb.ComponentID, monitors []*mon.BytesMonitor, -) (colexecop.VectorizedStatsCollector, error) { +) error { inputWatch := timeutil.NewStopWatch() var memMonitors, diskMonitors []*mon.BytesMonitor for _, m := range monitors { @@ -335,26 +335,33 @@ func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( for i, input := range inputs { sc, ok := input.Root.(childStatsCollector) if !ok { - return nil, errors.New("unexpectedly an input is not collecting stats") + return errors.New("unexpectedly an input is not collecting stats") } inputStatsCollectors[i] = sc } - return newVectorizedStatsCollector( - op, kvReader, columnarizer, component, inputWatch, + vsc := newVectorizedStatsCollector( + op.Root, kvReader, columnarizer, component, inputWatch, memMonitors, diskMonitors, inputStatsCollectors, - ), nil + ) + op.Root = vsc + op.StatsCollectors = append(op.StatsCollectors, vsc) + maybeAddStatsInvariantChecker(op) + return nil } // wrapWithNetworkVectorizedStatsCollector creates a new -// colexec.NetworkVectorizedStatsCollector that wraps op. +// colexecop.VectorizedStatsCollector that wraps op. func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( - op colexecop.Operator, + op *colexecargs.OpWithMetaInfo, inbox *colrpc.Inbox, component execinfrapb.ComponentID, latency time.Duration, -) colexecop.VectorizedStatsCollector { +) { inputWatch := timeutil.NewStopWatch() - return newNetworkVectorizedStatsCollector(op, component, inputWatch, inbox, latency) + nvsc := newNetworkVectorizedStatsCollector(op.Root, component, inputWatch, inbox, latency) + op.Root = nvsc + op.StatsCollectors = []colexecop.VectorizedStatsCollector{nvsc} + maybeAddStatsInvariantChecker(op) } // makeGetStatsFnForOutbox creates a function that will retrieve all execution @@ -768,15 +775,12 @@ func (s *vectorizedFlowCreator) setupRouter( // Wrap local outputs with vectorized stats collectors when recording // stats. This is mostly for compatibility but will provide some useful // information (e.g. output stall time). - vsc, err := s.wrapWithVectorizedStatsCollectorBase( - op, nil /* kvReader */, nil /* columnarizer */, nil, /* inputs */ - flowCtx.StreamComponentID(stream.StreamID), mons, - ) - if err != nil { + if err := s.wrapWithVectorizedStatsCollectorBase( + &opWithMetaInfo, nil /* kvReader */, nil, /* columnarizer */ + nil /* inputs */, flowCtx.StreamComponentID(stream.StreamID), mons, + ); err != nil { return err } - opWithMetaInfo.Root = vsc - opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} } s.streamIDToInputOp[stream.StreamID] = opWithMetaInfo } @@ -860,9 +864,7 @@ func (s *vectorizedFlowCreator) setupInput( compID := execinfrapb.StreamComponentID( base.SQLInstanceID(inputStream.OriginNodeID), flowCtx.ID, inputStream.StreamID, ) - vsc := s.wrapWithNetworkVectorizedStatsCollector(op, inbox, compID, latency) - opWithMetaInfo.Root = vsc - opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} + s.wrapWithNetworkVectorizedStatsCollector(&opWithMetaInfo, inbox, compID, latency) } inputStreamOps = append(inputStreamOps, opWithMetaInfo) default: @@ -920,15 +922,12 @@ func (s *vectorizedFlowCreator) setupInput( } // TODO(asubiotto): Once we have IDs for synchronizers, plumb them into // this stats collector to display stats. - vsc, err := s.wrapWithVectorizedStatsCollectorBase( - opWithMetaInfo.Root, nil /* kvReader */, nil, /* columnarizer */ + if err := s.wrapWithVectorizedStatsCollectorBase( + &opWithMetaInfo, nil /* kvReader */, nil, /* columnarizer */ statsInputsAsOps, execinfrapb.ComponentID{}, nil, /* monitors */ - ) - if err != nil { + ); err != nil { return colexecargs.OpWithMetaInfo{}, err } - opWithMetaInfo.Root = vsc - opWithMetaInfo.StatsCollectors = []colexecop.VectorizedStatsCollector{vsc} } } return opWithMetaInfo, nil @@ -1113,15 +1112,12 @@ func (s *vectorizedFlowCreator) setupFlow( } if s.recordingStats { - vsc, err := s.wrapWithVectorizedStatsCollectorBase( - result.Root, result.KVReader, result.Columnarizer, inputs, + if err := s.wrapWithVectorizedStatsCollectorBase( + &result.OpWithMetaInfo, result.KVReader, result.Columnarizer, inputs, flowCtx.ProcessorComponentID(pspec.ProcessorID), result.OpMonitors, - ) - if err != nil { + ); err != nil { return } - result.Root = vsc - result.StatsCollectors = append(result.StatsCollectors, vsc) } if err = s.setupOutput(