-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
flow.go
682 lines (592 loc) · 22.7 KB
/
flow.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
// Copyright 2019 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 flowinfra
import (
"context"
"sync"
"unsafe"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/multitenant"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra/execreleasable"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/cancelchecker"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/optional"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)
type flowStatus int
// Flow status indicators.
const (
flowNotStarted flowStatus = iota
flowRunning
flowFinished
)
// Startable is any component that can be started (a router or an outbox).
type Startable interface {
Start(ctx context.Context, wg *sync.WaitGroup, flowCtxCancel context.CancelFunc)
}
// StartableFn is an adapter when a customer function (i.e. a custom goroutine)
// needs to become Startable.
type StartableFn func(context.Context, *sync.WaitGroup, context.CancelFunc)
// Start is a part of the Startable interface.
func (f StartableFn) Start(
ctx context.Context, wg *sync.WaitGroup, flowCtxCancel context.CancelFunc,
) {
f(ctx, wg, flowCtxCancel)
}
// FuseOpt specifies options for processor fusing at Flow.Setup() time.
type FuseOpt bool
const (
// FuseNormally means fuse what you can, but don't serialize unordered input
// synchronizers.
FuseNormally FuseOpt = false
// FuseAggressively means serialize unordered input synchronizers.
// This is useful for flows that might have mutations which can't have any
// concurrency.
FuseAggressively = true
)
// Flow represents a flow which consists of processors and streams.
type Flow interface {
// Setup sets up all the infrastructure for the flow as defined by the flow
// spec. The flow will then need to be started or run. A new context (along
// with a context cancellation function) is derived. The new context must be
// used when running the flow so that all components running in their own
// goroutines could listen for a cancellation on the same context.
//
// The second return argument contains all operator chains planned on the
// gateway node if the flow is vectorized and the physical plan is fully
// local (in all other cases the second return argument is nil).
Setup(ctx context.Context, spec *execinfrapb.FlowSpec, opt FuseOpt) (context.Context, execopnode.OpChains, error)
// SetTxn is used to provide the transaction in which the flow will run.
// It needs to be called after Setup() and before Start/Run.
SetTxn(*kv.Txn)
// Start starts the flow. Processors run asynchronously in their own
// goroutines. Wait() needs to be called to wait for the flow to finish.
// See Run() for a synchronous version.
//
// If errors are encountered during the setup part, they're returned.
Start(context.Context) error
// Run runs the flow to completion. The last processor is run in the current
// goroutine; others may run in different goroutines depending on how the
// flow was configured.
//
// f.Wait() is called internally, so the call blocks until all the flow's
// goroutines are done.
//
// It is assumed that rowSyncFlowConsumer is set, so all errors encountered
// when running this flow are sent to it.
//
// noWait is set true when the flow is bound to a pausable portal. With it set,
// the function returns without waiting the all goroutines to finish. For a
// pausable portal we will persist this flow and reuse it when re-executing
// the portal. The flow will be cleaned when the portal is closed, rather than
// when each portal execution finishes.
//
// The caller needs to call f.Cleanup().
Run(ctx context.Context, noWait bool)
// Resume continue running the headproc saved in the FlowBase, but with
// a new output receiver. It is called when resuming a paused portal.
// The lifecycle of a flow for a pausable portal is:
// flow.Run(ctx, true /* noWait */) (only once), flow.Resume() (for all
// re-executions of the portal), flow.Cleanup() (only once)
// TODO(janexing): confirm the comment is accurate.
Resume(ctx context.Context, recv execinfra.RowReceiver)
// Wait waits for all the goroutines for this flow to exit. If the context gets
// canceled before all goroutines exit, it calls f.cancel().
Wait()
// IsLocal returns whether this flow is being run as part of a local-only
// query.
IsLocal() bool
// StatementSQL is the SQL statement for which this flow is executing. It is
// populated on a best effort basis (only available for user-issued queries
// that are also not like BulkIO/CDC related).
StatementSQL() string
// GetFlowCtx returns the flow context of this flow.
GetFlowCtx() *execinfra.FlowCtx
// AddStartable accumulates a Startable object.
AddStartable(Startable)
// GetID returns the flow ID.
GetID() execinfrapb.FlowID
// MemUsage returns the estimated memory footprint of this Flow object. Note
// that this ignores all the memory usage of the components that are created
// on behalf of this Flow.
MemUsage() int64
// Cancel cancels the flow by canceling its context. Safe to be called from
// any goroutine but **cannot** be called after (or concurrently with)
// Cleanup.
Cancel()
// AddOnCleanupStart adds a callback to be executed at the very beginning of
// Cleanup.
AddOnCleanupStart(fn func())
// GetOnCleanupFns returns a couple of functions that should be called at
// the very beginning and the very end of Cleanup, respectively. Both will
// be non-nil.
GetOnCleanupFns() (startCleanup, endCleanup func())
// Cleanup must be called whenever the flow is done (meaning it either
// completes gracefully after all processors and mailboxes exited or an
// error is encountered that stops the flow from making progress). The
// implementations must be safe to execute in case the Flow is never Run()
// or Start()ed.
Cleanup(context.Context)
// ConcurrentTxnUse returns true if multiple processors/operators in the flow
// will execute concurrently (i.e. if not all of them have been fused) and
// more than one goroutine will be using a txn.
// Can only be called after Setup().
ConcurrentTxnUse() bool
}
// FlowBase is the shared logic between row based and vectorized flows. It
// implements Flow interface for convenience and for usage in tests, but if
// FlowBase.Setup is called, it'll panic.
type FlowBase struct {
execinfra.FlowCtx
flowRegistry *FlowRegistry
// processors contains a subset of the processors in the flow - the ones that
// run in their own goroutines. Some processors that implement RowSource are
// scheduled to run in their consumer's goroutine; those are not present here.
processors []execinfra.Processor
// outputs contains an output for each execinfra.Processor in processors.
outputs []execinfra.RowReceiver
// startables are entities that must be started when the flow starts;
// currently these are outboxes and routers.
startables []Startable
// rowSyncFlowConsumer is a special execinfra.RowReceiver which, instead of
// sending rows to another host (as the outboxes do), returns them directly
// (to the local host). It is always set.
rowSyncFlowConsumer execinfra.RowReceiver
// batchSyncFlowConsumer, if set, provides an alternative interface for
// pushing coldata.Batches to locally.
batchSyncFlowConsumer execinfra.BatchReceiver
localProcessors []execinfra.LocalProcessor
// startedGoroutines specifies whether this flow started any goroutines. This
// is used in Wait() to avoid the overhead of waiting for non-existent
// goroutines.
startedGoroutines bool
// inboundStreams are streams that receive data from other hosts; this map
// is to be passed to FlowRegistry.RegisterFlow. This map is populated in
// Flow.Setup(), so it is safe to lookup into concurrently later.
inboundStreams map[execinfrapb.StreamID]*InboundStreamInfo
// waitGroup is used to wait for async components of the flow:
// - processors
// - inbound streams
// - outboxes
waitGroup sync.WaitGroup
// onCleanupStart and onCleanupEnd will be called in the very beginning and
// the very end of Cleanup(), respectively.
onCleanupStart func()
onCleanupEnd func()
statementSQL string
status flowStatus
// Cancel function for ctx. Call this to cancel the flow (safe to be called
// multiple times).
ctxCancel context.CancelFunc
ctxDone <-chan struct{}
// sp is the span that this Flow runs in. Can be nil if no span was created
// for the flow. Flow.Cleanup() finishes it.
sp *tracing.Span
// spec is the request that produced this flow. Only used for debugging.
spec *execinfrapb.FlowSpec
admissionInfo admission.WorkInfo
}
// Setup is part of the Flow interface.
func (f *FlowBase) Setup(
ctx context.Context, spec *execinfrapb.FlowSpec, _ FuseOpt,
) (context.Context, execopnode.OpChains, error) {
ctx, f.ctxCancel = contextutil.WithCancel(ctx)
f.ctxDone = ctx.Done()
f.spec = spec
return ctx, nil, nil
}
// SetTxn is part of the Flow interface.
func (f *FlowBase) SetTxn(txn *kv.Txn) {
f.FlowCtx.Txn = txn
f.EvalCtx.Txn = txn
}
// ConcurrentTxnUse is part of the Flow interface.
func (f *FlowBase) ConcurrentTxnUse() bool {
numProcessorsThatMightUseTxn := 0
for _, proc := range f.processors {
if txnUser, ok := proc.(execinfra.DoesNotUseTxn); !ok || !txnUser.DoesNotUseTxn() {
numProcessorsThatMightUseTxn++
if numProcessorsThatMightUseTxn > 1 {
return true
}
}
}
return false
}
// SetStartedGoroutines sets FlowBase.startedGoroutines to the passed in value.
// This allows notifying the FlowBase about the concurrent goroutines which are
// started outside of the FlowBase.StartInternal machinery.
func (f *FlowBase) SetStartedGoroutines(val bool) {
f.startedGoroutines = val
}
// Started returns true if f has either been Run() or Start()ed.
func (f *FlowBase) Started() bool {
return f.status != flowNotStarted
}
var _ Flow = &FlowBase{}
// NewFlowBase creates a new FlowBase.
//
// sp, if not nil, is the Span corresponding to the flow. The flow takes
// ownership; Cleanup() will finish it.
func NewFlowBase(
flowCtx execinfra.FlowCtx,
sp *tracing.Span,
flowReg *FlowRegistry,
rowSyncFlowConsumer execinfra.RowReceiver,
batchSyncFlowConsumer execinfra.BatchReceiver,
localProcessors []execinfra.LocalProcessor,
onFlowCleanupEnd func(),
statementSQL string,
) *FlowBase {
// We are either in a single tenant cluster, or a SQL node in a multi-tenant
// cluster, where the SQL node is single tenant. The tenant below is used
// within SQL (not KV), so using an arbitrary tenant is ok -- we choose to
// use SystemTenantID since it is already defined.
admissionInfo := admission.WorkInfo{TenantID: roachpb.SystemTenantID}
if flowCtx.Txn == nil {
admissionInfo.Priority = admissionpb.NormalPri
admissionInfo.CreateTime = timeutil.Now().UnixNano()
} else {
h := flowCtx.Txn.AdmissionHeader()
admissionInfo.Priority = admissionpb.WorkPriority(h.Priority)
admissionInfo.CreateTime = h.CreateTime
}
return &FlowBase{
FlowCtx: flowCtx,
sp: sp,
flowRegistry: flowReg,
rowSyncFlowConsumer: rowSyncFlowConsumer,
batchSyncFlowConsumer: batchSyncFlowConsumer,
localProcessors: localProcessors,
admissionInfo: admissionInfo,
onCleanupEnd: onFlowCleanupEnd,
status: flowNotStarted,
statementSQL: statementSQL,
}
}
// StatementSQL is part of the Flow interface.
func (f *FlowBase) StatementSQL() string {
return f.statementSQL
}
// GetFlowCtx is part of the Flow interface.
func (f *FlowBase) GetFlowCtx() *execinfra.FlowCtx {
return &f.FlowCtx
}
// AddStartable is part of the Flow interface.
func (f *FlowBase) AddStartable(s Startable) {
f.startables = append(f.startables, s)
}
// GetID is part of the Flow interface.
func (f *FlowBase) GetID() execinfrapb.FlowID {
return f.ID
}
// CheckInboundStreamID takes a stream ID and returns an error if an inbound
// stream already exists with that ID in the inbound streams map, creating the
// inbound streams map if it is nil.
func (f *FlowBase) CheckInboundStreamID(sid execinfrapb.StreamID) error {
if _, found := f.inboundStreams[sid]; found {
return errors.Errorf("inbound stream %d already exists in map", sid)
}
if f.inboundStreams == nil {
f.inboundStreams = make(map[execinfrapb.StreamID]*InboundStreamInfo)
}
return nil
}
// GetWaitGroup returns the wait group of this flow.
func (f *FlowBase) GetWaitGroup() *sync.WaitGroup {
return &f.waitGroup
}
// GetCtxDone returns done channel of the context of this flow.
func (f *FlowBase) GetCtxDone() <-chan struct{} {
return f.ctxDone
}
// GetCancelFlowFn returns the context cancellation function of the context of
// this flow.
func (f *FlowBase) GetCancelFlowFn() context.CancelFunc {
return f.ctxCancel
}
// SetProcessorsAndOutputs overrides the current f.processors and f.outputs with
// the provided slices.
func (f *FlowBase) SetProcessorsAndOutputs(
processors []execinfra.Processor, outputs []execinfra.RowReceiver,
) error {
if len(processors) != len(outputs) {
return errors.AssertionFailedf(
"processors and outputs don't match: %d processors, %d outputs",
len(processors), len(outputs),
)
}
f.processors = processors
f.outputs = outputs
return nil
}
// AddRemoteStream adds a remote stream to this flow.
func (f *FlowBase) AddRemoteStream(streamID execinfrapb.StreamID, streamInfo *InboundStreamInfo) {
f.inboundStreams[streamID] = streamInfo
}
// GetRowSyncFlowConsumer returns the special rowSyncFlowConsumer outbox.
func (f *FlowBase) GetRowSyncFlowConsumer() execinfra.RowReceiver {
return f.rowSyncFlowConsumer
}
// GetBatchSyncFlowConsumer returns the special batchSyncFlowConsumer outbox.
// Will return nil if the consumer cannot receive batches.
func (f *FlowBase) GetBatchSyncFlowConsumer() execinfra.BatchReceiver {
return f.batchSyncFlowConsumer
}
// GetLocalProcessors return the execinfra.LocalProcessors of this flow.
func (f *FlowBase) GetLocalProcessors() []execinfra.LocalProcessor {
return f.localProcessors
}
// GetAdmissionInfo returns the information to use for admission control on
// responses received from a remote flow.
func (f *FlowBase) GetAdmissionInfo() admission.WorkInfo {
return f.admissionInfo
}
// StartInternal starts the flow. All processors are started, each in their own
// goroutine. The caller must forward any returned error to rowSyncFlowConsumer if
// set.
func (f *FlowBase) StartInternal(
ctx context.Context, processors []execinfra.Processor, outputs []execinfra.RowReceiver,
) error {
log.VEventf(
ctx, 1, "starting (%d processors, %d startables) asynchronously", len(processors), len(f.startables),
)
// Only register the flow if it is a part of the distributed plan. This is
// needed to satisfy two different use cases:
// 1. there are inbound stream connections that need to look up this flow in
// the flow registry. This can only happen if the plan is not fully local
// (since those inbound streams originate on different nodes).
// 2. when the node is draining, the flow registry can cancel all running
// non-fully local flows if they don't finish on their own during the grace
// period. Cancellation of local flows occurs by cancelling the connections
// that the local flows were spinned up for.
if !f.IsLocal() {
// Once we call RegisterFlow, the inbound streams become accessible; we
// must set up the WaitGroup counter before.
// The counter will be further incremented below to account for the
// processors.
f.waitGroup.Add(len(f.inboundStreams))
if err := f.flowRegistry.RegisterFlow(
ctx, f.ID, f, f.inboundStreams, SettingFlowStreamTimeout.Get(&f.FlowCtx.Cfg.Settings.SV),
); err != nil {
return err
}
}
f.status = flowRunning
if multitenant.TenantRUEstimateEnabled.Get(&f.Cfg.Settings.SV) &&
!f.Gateway && f.CollectStats {
// Remote flows begin collecting CPU usage here, and finish when the last
// outbox finishes. Gateway flows are handled by the connExecutor.
f.FlowCtx.TenantCPUMonitor.StartCollection(ctx, f.Cfg.TenantCostController)
}
if log.V(1) {
log.Infof(ctx, "registered flow %s", f.ID.Short())
}
for _, s := range f.startables {
s.Start(ctx, &f.waitGroup, f.ctxCancel)
}
for i := 0; i < len(processors); i++ {
f.waitGroup.Add(1)
go func(i int) {
processors[i].Run(ctx, outputs[i])
f.waitGroup.Done()
}(i)
}
// Note that we might have already set f.startedGoroutines to true if it is
// a vectorized flow with a parallel unordered synchronizer. That component
// starts goroutines on its own, so we need to preserve that fact so that we
// correctly wait in Wait().
f.startedGoroutines = f.startedGoroutines || len(f.startables) > 0 || len(processors) > 0 || len(f.inboundStreams) > 0
return nil
}
// IsLocal returns whether this flow is being run as part of a local-only query.
func (f *FlowBase) IsLocal() bool {
return f.Local
}
// Start is part of the Flow interface.
func (f *FlowBase) Start(ctx context.Context) error {
return f.StartInternal(ctx, f.processors, f.outputs)
}
// Resume is part of the Flow interface.
func (f *FlowBase) Resume(ctx context.Context, recv execinfra.RowReceiver) {
if len(f.processors) != 1 || len(f.outputs) != 1 {
f.rowSyncFlowConsumer.Push(
nil, /* row */
&execinfrapb.ProducerMetadata{
Err: errors.AssertionFailedf(
"length of both the processor and the output must be 1",
)})
f.rowSyncFlowConsumer.ProducerDone()
return
}
f.outputs[0] = recv
log.VEventf(ctx, 1, "resuming %T in the flow's goroutine", f.processors[0])
f.processors[0].Resume(recv)
}
// Run is part of the Flow interface.
func (f *FlowBase) Run(ctx context.Context, noWait bool) {
if !noWait {
defer f.Wait()
}
if len(f.processors) == 0 {
f.rowSyncFlowConsumer.Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: errors.AssertionFailedf("no processors in flow")})
f.rowSyncFlowConsumer.ProducerDone()
return
}
// We'll take care of the last processor in particular.
headProc := f.processors[len(f.processors)-1]
headOutput := f.outputs[len(f.outputs)-1]
otherProcs := f.processors[:len(f.processors)-1]
otherOutputs := f.outputs[:len(f.outputs)-1]
var err error
if err = f.StartInternal(ctx, otherProcs, otherOutputs); err != nil {
f.rowSyncFlowConsumer.Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: err})
f.rowSyncFlowConsumer.ProducerDone()
return
}
log.VEventf(ctx, 1, "running %T in the flow's goroutine", headProc)
headProc.Run(ctx, headOutput)
}
// Wait is part of the Flow interface.
func (f *FlowBase) Wait() {
if !f.startedGoroutines {
return
}
var panicVal interface{}
if panicVal = recover(); panicVal != nil {
// If Wait is called as part of stack unwinding during a panic, the flow
// context must be canceled to ensure that all asynchronous goroutines get
// the message that they must exit (otherwise we will wait indefinitely).
f.ctxCancel()
}
waitChan := make(chan struct{})
go func() {
f.waitGroup.Wait()
close(waitChan)
}()
select {
case <-f.ctxDone:
f.cancel()
<-waitChan
case <-waitChan:
// Exit normally
}
if panicVal != nil {
panic(panicVal)
}
}
const flowBaseOverhead = int64(unsafe.Sizeof(FlowBase{}))
// MemUsage is part of the Flow interface.
func (f *FlowBase) MemUsage() int64 {
return flowBaseOverhead + int64(len(f.statementSQL))
}
// Cancel is part of the Flow interface.
func (f *FlowBase) Cancel() {
f.ctxCancel()
}
// AddOnCleanupStart is part of the Flow interface.
func (f *FlowBase) AddOnCleanupStart(fn func()) {
if f.onCleanupStart != nil {
oldOnCleanupStart := f.onCleanupStart
f.onCleanupStart = func() {
fn()
oldOnCleanupStart()
}
} else {
f.onCleanupStart = fn
}
}
var noopFn = func() {}
// GetOnCleanupFns is part of the Flow interface.
func (f *FlowBase) GetOnCleanupFns() (startCleanup, endCleanup func()) {
onCleanupStart, onCleanupEnd := f.onCleanupStart, f.onCleanupEnd
if onCleanupStart == nil {
onCleanupStart = noopFn
}
if onCleanupEnd == nil {
onCleanupEnd = noopFn
}
return onCleanupStart, onCleanupEnd
}
// Cleanup is part of the Flow interface.
// NOTE: this implements only the shared clean up logic between row-based and
// vectorized flows.
func (f *FlowBase) Cleanup(ctx context.Context) {
if f.status == flowFinished {
panic("flow cleanup called twice")
}
// Release any descriptors accessed by this flow.
if f.Descriptors != nil && f.IsDescriptorsCleanupRequired {
f.Descriptors.ReleaseAll(ctx)
}
if f.sp != nil {
defer f.sp.Finish()
if f.Gateway && f.CollectStats {
// If this is the gateway node and we're collecting execution stats,
// output the maximum memory usage to the flow span. Note that
// non-gateway nodes use the last outbox to send this information
// over.
f.sp.RecordStructured(&execinfrapb.ComponentStats{
Component: execinfrapb.FlowComponentID(f.NodeID.SQLInstanceID(), f.FlowCtx.ID),
FlowStats: execinfrapb.FlowStats{
MaxMemUsage: optional.MakeUint(uint64(f.FlowCtx.Mon.MaximumBytes())),
MaxDiskUsage: optional.MakeUint(uint64(f.FlowCtx.DiskMonitor.MaximumBytes())),
},
})
}
}
// This closes the disk monitor opened in newFlowContext as well as the
// memory monitor opened in ServerImpl.setupFlow.
if r := recover(); r != nil {
f.DiskMonitor.EmergencyStop(ctx)
f.Mon.EmergencyStop(ctx)
panic(r)
} else {
f.DiskMonitor.Stop(ctx)
f.Mon.Stop(ctx)
}
for _, p := range f.processors {
if d, ok := p.(execreleasable.Releasable); ok {
d.Release()
}
}
if log.V(1) {
log.Infof(ctx, "cleaning up")
}
// Local flows do not get registered.
if !f.IsLocal() && f.Started() {
f.flowRegistry.UnregisterFlow(f.ID)
}
f.status = flowFinished
f.ctxCancel()
}
// cancel cancels all unconnected streams of this flow. This function is called
// in Wait() after the associated context has been canceled. In order to cancel
// a flow, call f.ctxCancel() instead of this function.
//
// For a detailed description of the distsql query cancellation mechanism,
// read docs/RFCS/query_cancellation.md.
func (f *FlowBase) cancel() {
if len(f.inboundStreams) == 0 {
return
}
// Pending streams have yet to be started; send an error to its receivers
// and prevent them from being connected.
f.flowRegistry.cancelPendingStreams(f.ID, cancelchecker.QueryCanceledError)
}