-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
vectorized_flow.go
1429 lines (1327 loc) · 49.6 KB
/
vectorized_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
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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 colflow
import (
"context"
"path/filepath"
"sync"
"sync/atomic"
"time"
"unsafe"
"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/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/colcontainer"
"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/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"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/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/optional"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"github.com/marusama/semaphore"
)
// fdCountingSemaphore is a semaphore that keeps track of the number of file
// descriptors currently used by the vectorized engine.
//
// Note that it effectively implements the execreleasable.Releasable interface
// but due to the method name conflict doesn't.
type fdCountingSemaphore struct {
semaphore.Semaphore
globalCount *metric.Gauge
count int64
acquireMaxRetries int
}
var fdCountingSemaphorePool = sync.Pool{
New: func() interface{} {
return &fdCountingSemaphore{}
},
}
func newFDCountingSemaphore(
sem semaphore.Semaphore, globalCount *metric.Gauge, sv *settings.Values,
) *fdCountingSemaphore {
s := fdCountingSemaphorePool.Get().(*fdCountingSemaphore)
s.Semaphore = sem
s.globalCount = globalCount
s.acquireMaxRetries = int(fdCountingSemaphoreMaxRetries.Get(sv))
return s
}
var errAcquireTimeout = pgerror.New(
pgcode.ConfigurationLimitExceeded,
"acquiring of file descriptors timed out, consider increasing "+
"COCKROACH_VEC_MAX_OPEN_FDS environment variable",
)
var fdCountingSemaphoreMaxRetries = settings.RegisterIntSetting(
settings.TenantWritable,
"sql.distsql.acquire_vec_fds.max_retries",
"determines the number of retries performed during the acquisition of "+
"file descriptors needed for disk-spilling operations, set to 0 for "+
"unlimited retries",
8,
settings.NonNegativeInt,
)
func (s *fdCountingSemaphore) Acquire(ctx context.Context, n int) error {
if s.TryAcquire(n) {
return nil
}
// Currently there is not enough capacity in the semaphore to acquire the
// desired number, so we set up a retry loop that exponentially backs off,
// until either the semaphore opens up or we time out (most likely due to a
// deadlock).
//
// The latter situation is possible when multiple queries already hold some
// of the quota and each of them needs more to proceed resulting in a
// deadlock. We get out of such a deadlock by randomly erroring out one of
// the queries (which would release some quota back to the semaphore) making
// it possible for other queries to proceed.
//
// Note that we've already tried to acquire the quota above (which failed),
// so the initial backoff time of 100ms seems ok (we are spilling to disk
// after all, so the query is likely to experience significant latency). The
// current choice of options is such that we'll spend on the order of 25s
// in the retry loop before timing out with the default value of the
// 'sql.distsql.acquire_vec_fds.max_retries' cluster settings.
opts := retry.Options{
InitialBackoff: 100 * time.Millisecond,
Multiplier: 2.0,
RandomizationFactor: 0.25,
MaxRetries: s.acquireMaxRetries,
}
for r := retry.StartWithCtx(ctx, opts); r.Next(); {
if s.TryAcquire(n) {
return nil
}
}
if ctx.Err() != nil {
return ctx.Err()
}
log.Warning(ctx, "acquiring of file descriptors for disk-spilling timed out")
return errAcquireTimeout
}
func (s *fdCountingSemaphore) TryAcquire(n int) bool {
success := s.Semaphore.TryAcquire(n)
if !success {
return false
}
atomic.AddInt64(&s.count, int64(n))
s.globalCount.Inc(int64(n))
return success
}
func (s *fdCountingSemaphore) Release(n int) int {
atomic.AddInt64(&s.count, int64(-n))
s.globalCount.Dec(int64(n))
return s.Semaphore.Release(n)
}
// ReleaseToPool should be named Release and should implement the
// execinfra.Releasable interface, but that would lead to a conflict with
// semaphore.Semaphore.Release method.
func (s *fdCountingSemaphore) ReleaseToPool() {
if unreleased := atomic.LoadInt64(&s.count); unreleased != 0 {
colexecerror.InternalError(errors.Newf("unexpectedly %d count on the semaphore when releasing it to the pool", unreleased))
}
*s = fdCountingSemaphore{}
fdCountingSemaphorePool.Put(s)
}
type vectorizedFlow struct {
*flowinfra.FlowBase
// creator is the object that created this flow. It must be cleaned up in
// order to shut down the memory monitoring infrastructure and should be
// released back to the pool.
creator *vectorizedFlowCreator
// batchFlowCoordinator will be set if the flow is pushing coldata.Batches
// to the consumer.
batchFlowCoordinator *BatchFlowCoordinator
// countingSemaphore is a wrapper over a semaphore.Semaphore that keeps track
// of the number of resources held in a semaphore.Semaphore requested from the
// context of this flow so that these can be released unconditionally upon
// Cleanup.
countingSemaphore *fdCountingSemaphore
tempStorage struct {
syncutil.Mutex
// path is the path to this flow's temporary storage directory. If
// it is an empty string, then it hasn't been computed yet nor the
// directory has been created.
path string
}
testingKnobs struct {
// onSetupFlow is a testing knob that is called before calling
// creator.setupFlow with the given creator.
onSetupFlow func(*vectorizedFlowCreator)
}
}
var _ flowinfra.Flow = &vectorizedFlow{}
var _ execreleasable.Releasable = &vectorizedFlow{}
var vectorizedFlowPool = sync.Pool{
New: func() interface{} {
return &vectorizedFlow{}
},
}
// NewVectorizedFlow creates a new vectorized flow given the flow base.
func NewVectorizedFlow(base *flowinfra.FlowBase) flowinfra.Flow {
vf := vectorizedFlowPool.Get().(*vectorizedFlow)
vf.FlowBase = base
return vf
}
// Setup is part of the flowinfra.Flow interface.
func (f *vectorizedFlow) Setup(
ctx context.Context, spec *execinfrapb.FlowSpec, opt flowinfra.FuseOpt,
) (context.Context, execopnode.OpChains, error) {
var err error
ctx, _, err = f.FlowBase.Setup(ctx, spec, opt)
if err != nil {
return ctx, nil, err
}
log.VEvent(ctx, 2, "setting up vectorized flow")
recordingStats := false
if execstats.ShouldCollectStats(ctx, f.FlowCtx.CollectStats) {
recordingStats = true
}
diskQueueCfg := colcontainer.DiskQueueCfg{
FS: f.Cfg.TempFS,
GetPather: f,
SpilledBytesWritten: f.Cfg.Metrics.SpilledBytesWritten,
SpilledBytesRead: f.Cfg.Metrics.SpilledBytesRead,
}
if err := diskQueueCfg.EnsureDefaults(); err != nil {
return ctx, nil, err
}
flowCtx := f.GetFlowCtx()
f.countingSemaphore = newFDCountingSemaphore(
f.Cfg.VecFDSemaphore, f.Cfg.Metrics.VecOpenFDs, &flowCtx.EvalCtx.Settings.SV,
)
f.creator = newVectorizedFlowCreator(
f.FlowBase,
nil, /* helper */
nil, /* componentCreator */
recordingStats,
f.Gateway,
f.GetWaitGroup(),
f.GetRowSyncFlowConsumer(),
f.GetBatchSyncFlowConsumer(),
flowCtx.Cfg.PodNodeDialer,
diskQueueCfg,
f.countingSemaphore,
flowCtx.NewTypeResolver(flowCtx.Txn),
f.FlowBase.GetAdmissionInfo(),
)
if f.testingKnobs.onSetupFlow != nil {
f.testingKnobs.onSetupFlow(f.creator)
}
opChains, batchFlowCoordinator, err := f.creator.setupFlow(ctx, flowCtx, spec.Processors, f.GetLocalProcessors(), f.GetLocalVectorSources(), opt)
if err != nil {
// It is (theoretically) possible that some of the memory monitoring
// infrastructure was created even in case of an error, and we need to
// clean that up.
f.creator.cleanup(ctx)
f.creator.Release()
log.VEventf(ctx, 1, "failed to vectorize: %v", err)
return ctx, nil, err
}
f.batchFlowCoordinator = batchFlowCoordinator
f.SetStartedGoroutines(f.creator.operatorConcurrency)
log.VEventf(ctx, 2, "vectorized flow setup succeeded")
if !f.IsLocal() {
// For distributed flows set opChains to nil, per the contract of
// flowinfra.Flow.Setup.
opChains = nil
}
return ctx, opChains, nil
}
// Resume is part of the Flow interface.
func (f *vectorizedFlow) Resume(recv execinfra.RowReceiver) {
if f.batchFlowCoordinator != nil {
recv.Push(
nil, /* row */
&execinfrapb.ProducerMetadata{
Err: errors.AssertionFailedf(
"batchFlowCoordinator should be nil for vectorizedFlow",
)})
recv.ProducerDone()
return
}
f.FlowBase.Resume(recv)
}
// Run is part of the Flow interface.
func (f *vectorizedFlow) Run(ctx context.Context, noWait bool) {
if f.batchFlowCoordinator == nil {
// If we didn't create a BatchFlowCoordinator, then we have a processor
// as the root, so we run this flow with the default implementation.
f.FlowBase.Run(ctx, noWait)
return
}
if !noWait {
defer f.Wait()
}
if err := f.StartInternal(ctx, nil /* processors */, nil /* outputs */); err != nil {
f.GetRowSyncFlowConsumer().Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: err})
f.GetRowSyncFlowConsumer().ProducerDone()
return
}
log.VEvent(ctx, 1, "running the batch flow coordinator in the flow's goroutine")
f.batchFlowCoordinator.Run(ctx)
}
var _ colcontainer.GetPather = &vectorizedFlow{}
// GetPath returns the path of the temporary directory for
// disk-spilling components of the flow. The directory is created on the first
// call to this method.
func (f *vectorizedFlow) GetPath(ctx context.Context) string {
f.tempStorage.Lock()
defer f.tempStorage.Unlock()
if f.tempStorage.path != "" {
// The temporary directory has already been created.
return f.tempStorage.path
}
// We haven't created this flow's temporary directory yet, so we do so now.
// The directory name is the flow's ID.
tempDirName := f.GetID().String()
f.tempStorage.path = filepath.Join(f.Cfg.TempStoragePath, tempDirName)
log.VEventf(ctx, 1, "flow %s spilled to disk, stack trace: %s", f.ID, util.GetSmallTrace(2))
if err := f.Cfg.TempFS.MkdirAll(f.tempStorage.path); err != nil {
colexecerror.InternalError(errors.Wrap(err, "unable to create temporary storage directory"))
}
// We have just created the temporary directory which will be used for all
// disk-spilling operations of this flow, thus, it is a convenient place to
// increment the counter of the number of queries spilled - this code won't
// be executed for this flow in the future since we short-circuit above.
f.Cfg.Metrics.QueriesSpilled.Inc(1)
return f.tempStorage.path
}
// ConcurrentTxnUse is part of the flowinfra.Flow interface. It is conservative
// in that it returns that there is concurrent txn use as soon as any operator
// concurrency is detected. This should be inconsequential for local flows that
// use the RootTxn (which are the cases in which we care about this return
// value), because only unordered synchronizers introduce operator concurrency
// at the time of writing.
func (f *vectorizedFlow) ConcurrentTxnUse() bool {
return f.creator.operatorConcurrency || f.FlowBase.ConcurrentTxnUse()
}
// Release implements the execinfra.Releasable interface.
func (f *vectorizedFlow) Release() {
f.creator.Release()
f.countingSemaphore.ReleaseToPool()
*f = vectorizedFlow{}
vectorizedFlowPool.Put(f)
}
const (
vectorizedFlowOverhead = int64(unsafe.Sizeof(vectorizedFlow{}))
vectorizedFlowCreatorOverhead = int64(unsafe.Sizeof(vectorizedFlowCreator{}))
fdCountingSemaphoreOverhead = int64(unsafe.Sizeof(fdCountingSemaphore{}))
)
// MemUsage is part of the flowinfra.Flow interface.
func (f *vectorizedFlow) MemUsage() int64 {
return f.FlowBase.MemUsage() + vectorizedFlowOverhead +
vectorizedFlowCreatorOverhead + fdCountingSemaphoreOverhead
}
// Cleanup is part of the flowinfra.Flow interface.
func (f *vectorizedFlow) Cleanup(ctx context.Context) {
startCleanup, endCleanup := f.FlowBase.GetOnCleanupFns()
startCleanup()
defer endCleanup()
// This cleans up all the memory and disk monitoring of the vectorized flow
// as well as closes all the closers.
f.creator.cleanup(ctx)
f.tempStorage.Lock()
created := f.tempStorage.path != ""
f.tempStorage.Unlock()
if created {
if err := f.Cfg.TempFS.RemoveAll(f.GetPath(ctx)); err != nil {
// Log error as a Warning but keep on going to close the memory
// infrastructure.
log.Warningf(
ctx,
"unable to remove flow %s's temporary directory at %s, files may be left over: %v",
f.GetID().Short(),
f.GetPath(ctx),
err,
)
}
}
// Release any leftover temporary storage file descriptors from this flow.
if unreleased := atomic.LoadInt64(&f.countingSemaphore.count); unreleased > 0 {
f.countingSemaphore.Release(int(unreleased))
}
f.FlowBase.Cleanup(ctx)
f.Release()
}
// wrapWithVectorizedStatsCollectorBase creates a new
// 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 *colexecargs.OpWithMetaInfo,
kvReader colexecop.KVReader,
columnarizer colexecop.VectorizedStatsCollector,
inputs []colexecargs.OpWithMetaInfo,
component execinfrapb.ComponentID,
monitors []*mon.BytesMonitor,
) error {
inputWatch := timeutil.NewStopWatchWithCPU()
var memMonitors, diskMonitors []*mon.BytesMonitor
for _, m := range monitors {
if m.Resource() == mon.DiskResource {
diskMonitors = append(diskMonitors, m)
} else {
memMonitors = append(memMonitors, m)
}
}
inputStatsCollectors := make([]childStatsCollector, len(inputs))
for i, input := range inputs {
sc, ok := input.Root.(childStatsCollector)
if !ok {
return errors.New("unexpectedly an input is not collecting stats")
}
inputStatsCollectors[i] = sc
}
vsc := newVectorizedStatsCollector(
op.Root, kvReader, columnarizer, component, inputWatch,
memMonitors, diskMonitors, inputStatsCollectors,
)
op.Root = vsc
op.StatsCollectors = append(op.StatsCollectors, vsc)
maybeAddStatsInvariantChecker(op)
return nil
}
// wrapWithNetworkVectorizedStatsCollector creates a new
// colexecop.VectorizedStatsCollector that wraps op.
func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector(
op *colexecargs.OpWithMetaInfo,
inbox *colrpc.Inbox,
component execinfrapb.ComponentID,
latency time.Duration,
) {
inputWatch := timeutil.NewStopWatch()
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
// statistics that the outbox is responsible for, nil is returned if stats are
// not being collected.
func (s *vectorizedFlowCreator) makeGetStatsFnForOutbox(
flowCtx *execinfra.FlowCtx, statsCollectors []colexecop.VectorizedStatsCollector,
) func(context.Context) []*execinfrapb.ComponentStats {
if !s.recordingStats {
return nil
}
return func(ctx context.Context) []*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: flowCtx.FlowComponentID(),
FlowStats: execinfrapb.FlowStats{
MaxMemUsage: optional.MakeUint(uint64(flowCtx.Mon.MaximumBytes())),
MaxDiskUsage: optional.MakeUint(uint64(flowCtx.DiskMonitor.MaximumBytes())),
ConsumedRU: optional.MakeUint(uint64(flowCtx.TenantCPUMonitor.EndCollection(ctx))),
},
})
}
return result
}
}
type runFn func(_ context.Context, flowCtxCancel context.CancelFunc)
// flowCreatorHelper contains all the logic needed to add the vectorized
// infrastructure to be run asynchronously as well as to perform some sanity
// checks.
type flowCreatorHelper interface {
// addStreamEndpoint stores information about an inbound stream.
addStreamEndpoint(execinfrapb.StreamID, *colrpc.Inbox, *sync.WaitGroup)
// checkInboundStreamID checks that the provided stream ID has not been seen
// yet.
checkInboundStreamID(execinfrapb.StreamID) error
// accumulateAsyncComponent stores a component (either a router or an outbox)
// to be run asynchronously.
accumulateAsyncComponent(runFn)
// addFlowCoordinator adds the FlowCoordinator to the flow. This is only
// done on the gateway node.
addFlowCoordinator(coordinator *FlowCoordinator, output execinfra.RowReceiver)
// getCtxDone returns done channel of the context of this flow.
getFlowCtxDone() <-chan struct{}
// getCancelFlowFn returns a flow cancellation function.
getCancelFlowFn() context.CancelFunc
}
type admissionOptions struct {
admissionQ *admission.WorkQueue
admissionInfo admission.WorkInfo
}
// remoteComponentCreator is an interface that abstracts the constructors for
// several components in a remote flow. Mostly for testing purposes.
type remoteComponentCreator interface {
newOutbox(
flowCtx *execinfra.FlowCtx,
processorID int32,
allocator *colmem.Allocator,
converterMemAcc *mon.BoundAccount,
input colexecargs.OpWithMetaInfo,
typs []*types.T,
getStats func(context.Context) []*execinfrapb.ComponentStats,
) (*colrpc.Outbox, error)
newInbox(
allocator *colmem.Allocator,
typs []*types.T,
streamID execinfrapb.StreamID,
flowCtxDone <-chan struct{},
admissionOpts admissionOptions,
) (*colrpc.Inbox, error)
}
type vectorizedRemoteComponentCreator struct{}
func (vectorizedRemoteComponentCreator) newOutbox(
flowCtx *execinfra.FlowCtx,
processorID int32,
allocator *colmem.Allocator,
converterMemAcc *mon.BoundAccount,
input colexecargs.OpWithMetaInfo,
typs []*types.T,
getStats func(context.Context) []*execinfrapb.ComponentStats,
) (*colrpc.Outbox, error) {
return colrpc.NewOutbox(flowCtx, processorID, allocator, converterMemAcc, input, typs, getStats)
}
func (vectorizedRemoteComponentCreator) newInbox(
allocator *colmem.Allocator,
typs []*types.T,
streamID execinfrapb.StreamID,
flowCtxDone <-chan struct{},
admissionOpts admissionOptions,
) (*colrpc.Inbox, error) {
return colrpc.NewInboxWithAdmissionControl(
allocator, typs, streamID, flowCtxDone,
admissionOpts.admissionQ, admissionOpts.admissionInfo,
)
}
// vectorizedFlowCreator performs all the setup of vectorized flows. Depending
// on embedded flowCreatorHelper, it can either do the actual setup in order
// to run the flow or do the setup needed to check that the flow is supported
// through the vectorized engine.
type vectorizedFlowCreator struct {
flowCreatorHelper
remoteComponentCreator
// These two fields should not be accessed directly - instead, the embedded
// interfaces above should be used. These two structs are embedded in order
// to avoid allocations on the main code path.
fcHelper vectorizedFlowCreatorHelper
rcCreator vectorizedRemoteComponentCreator
// rowReceiver is always set.
rowReceiver execinfra.RowReceiver
// batchReceiver might be set if the consumer supports pushing of
// coldata.Batches.
batchReceiver execinfra.BatchReceiver
// batchFlowCoordinator, if set, indicates that the vectorized flow should
// not use the default FlowBase.Run implementation.
batchFlowCoordinator *BatchFlowCoordinator
streamIDToInputOp map[execinfrapb.StreamID]colexecargs.OpWithMetaInfo
streamIDToSpecIdx map[execinfrapb.StreamID]int
recordingStats bool
isGatewayNode bool
waitGroup *sync.WaitGroup
podNodeDialer *nodedialer.Dialer
exprHelper *colexecargs.ExprHelper
typeResolver descs.DistSQLTypeResolver
admissionInfo admission.WorkInfo
// numOutboxes counts how many colrpc.Outbox'es have been set up on this
// node. It must be accessed atomically.
numOutboxes int32
// numOutboxesDrained is an atomic that keeps track of how many outboxes
// have been drained. When numOutboxesDrained equals numOutboxes, flow-level
// metadata is added to a flow-level span on the non-gateway nodes.
numOutboxesDrained int32
// procIdxQueue is a queue of indices into processorSpecs (the argument to
// setupFlow), for topologically ordered processing.
procIdxQueue []int
// opChains accumulates all operators that have no further outputs on the
// current node, for the purposes of EXPLAIN output.
opChains execopnode.OpChains
// operatorConcurrency is set if any operators are executed in parallel.
operatorConcurrency bool
// closers will be closed during the flow cleanup. It is safe to do so in
// the main flow goroutine since all other goroutines that might have used
// these objects must have exited by the time Cleanup() is called -
// Flow.Wait() ensures that.
closers colexecop.Closers
// releasables contains all components that should be released back to their
// pools during the flow cleanup.
releasables []execreleasable.Releasable
monitorRegistry colexecargs.MonitorRegistry
diskQueueCfg colcontainer.DiskQueueCfg
fdSemaphore semaphore.Semaphore
}
var _ execreleasable.Releasable = &vectorizedFlowCreator{}
var vectorizedFlowCreatorPool = sync.Pool{
New: func() interface{} {
return &vectorizedFlowCreator{
streamIDToInputOp: make(map[execinfrapb.StreamID]colexecargs.OpWithMetaInfo),
streamIDToSpecIdx: make(map[execinfrapb.StreamID]int),
exprHelper: colexecargs.NewExprHelper(),
}
},
}
// newVectorizedFlowCreator returns a new vectorizedFlowCreator.
//
// Only one of flowBase and helper should be set, if the former, then the
// embedded vectorizedFlowCreatorHelper is updated with the flowBase is used.
//
// componentCreator can be nil in which case the embedded
// vectorizedRemoteComponentCreator is used.
func newVectorizedFlowCreator(
flowBase *flowinfra.FlowBase,
helper flowCreatorHelper,
componentCreator remoteComponentCreator,
recordingStats bool,
isGatewayNode bool,
waitGroup *sync.WaitGroup,
rowSyncFlowConsumer execinfra.RowReceiver,
batchSyncFlowConsumer execinfra.BatchReceiver,
podNodeDialer *nodedialer.Dialer,
diskQueueCfg colcontainer.DiskQueueCfg,
fdSemaphore semaphore.Semaphore,
typeResolver descs.DistSQLTypeResolver,
admissionInfo admission.WorkInfo,
) *vectorizedFlowCreator {
creator := vectorizedFlowCreatorPool.Get().(*vectorizedFlowCreator)
*creator = vectorizedFlowCreator{
streamIDToInputOp: creator.streamIDToInputOp,
streamIDToSpecIdx: creator.streamIDToSpecIdx,
recordingStats: recordingStats,
isGatewayNode: isGatewayNode,
waitGroup: waitGroup,
rowReceiver: rowSyncFlowConsumer,
batchReceiver: batchSyncFlowConsumer,
podNodeDialer: podNodeDialer,
exprHelper: creator.exprHelper,
typeResolver: typeResolver,
admissionInfo: admissionInfo,
procIdxQueue: creator.procIdxQueue,
opChains: creator.opChains,
releasables: creator.releasables,
monitorRegistry: creator.monitorRegistry,
diskQueueCfg: diskQueueCfg,
fdSemaphore: fdSemaphore,
}
if flowBase != nil {
// On the main code path, update the embedded helper with the provided
// FlowBase and use it.
creator.fcHelper.f = flowBase
creator.flowCreatorHelper = &creator.fcHelper
} else {
creator.flowCreatorHelper = helper
}
if componentCreator == nil {
// On the main code path, use the embedded component creator.
creator.remoteComponentCreator = creator.rcCreator
} else {
creator.remoteComponentCreator = componentCreator
}
return creator
}
func (s *vectorizedFlowCreator) cleanup(ctx context.Context) {
if err := colexecerror.CatchVectorizedRuntimeError(func() {
for _, closer := range s.closers {
if err := closer.Close(ctx); err != nil && log.V(1) {
log.Infof(ctx, "error closing Closer: %v", err)
}
}
}); err != nil && log.V(1) {
log.Infof(ctx, "runtime error closing the closers: %v", err)
}
s.monitorRegistry.Close(ctx)
}
// Release implements the execinfra.Releasable interface.
func (s *vectorizedFlowCreator) Release() {
for k := range s.streamIDToInputOp {
delete(s.streamIDToInputOp, k)
}
for k := range s.streamIDToSpecIdx {
delete(s.streamIDToSpecIdx, k)
}
for _, r := range s.releasables {
r.Release()
}
// Deeply reset slices that might point to the objects of non-trivial size
// so that the old references don't interfere with the objects being
// garbage-collected.
for i := range s.opChains {
s.opChains[i] = nil
}
for i := range s.closers {
s.closers[i] = nil
}
for i := range s.releasables {
s.releasables[i] = nil
}
if s.exprHelper != nil {
s.exprHelper.SemaCtx = nil
}
s.monitorRegistry.Reset()
*s = vectorizedFlowCreator{
streamIDToInputOp: s.streamIDToInputOp,
streamIDToSpecIdx: s.streamIDToSpecIdx,
exprHelper: s.exprHelper,
// procIdxQueue is a slice of ints, so it's ok to just slice up to 0 to
// prime it for reuse.
procIdxQueue: s.procIdxQueue[:0],
opChains: s.opChains[:0],
closers: s.closers[:0],
releasables: s.releasables[:0],
monitorRegistry: s.monitorRegistry,
}
vectorizedFlowCreatorPool.Put(s)
}
// setupRemoteOutputStream sets up a colrpc.Outbox that will operate according
// to the given execinfrapb.StreamEndpointSpec. It will also drain all
// MetadataSources in op.
// NOTE: The caller must not reuse the metadata sources.
func (s *vectorizedFlowCreator) setupRemoteOutputStream(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
processorID int32,
op colexecargs.OpWithMetaInfo,
outputTyps []*types.T,
stream *execinfrapb.StreamEndpointSpec,
factory coldata.ColumnFactory,
getStats func(context.Context) []*execinfrapb.ComponentStats,
) (execopnode.OpNode, error) {
outbox, err := s.remoteComponentCreator.newOutbox(
flowCtx, processorID, colmem.NewAllocator(ctx, s.monitorRegistry.NewStreamingMemAccount(flowCtx), factory),
s.monitorRegistry.NewStreamingMemAccount(flowCtx),
op, outputTyps, getStats,
)
if err != nil {
return nil, err
}
atomic.AddInt32(&s.numOutboxes, 1)
run := func(ctx context.Context, flowCtxCancel context.CancelFunc) {
outbox.Run(
ctx,
s.podNodeDialer,
stream.TargetNodeID,
stream.StreamID,
flowCtxCancel,
flowinfra.SettingFlowStreamTimeout.Get(&flowCtx.Cfg.Settings.SV),
)
}
s.accumulateAsyncComponent(run)
return outbox, nil
}
// setupRouter sets up a vectorized hash router according to the output router
// spec. The router takes the responsibility of draining the metadata sources
// from input.MetadataSources.
// NOTE: This method supports only BY_HASH routers. Callers should handle
// PASS_THROUGH routers separately.
// NOTE: The caller must not reuse the metadata sources.
func (s *vectorizedFlowCreator) setupRouter(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
processorID int32,
input colexecargs.OpWithMetaInfo,
outputTyps []*types.T,
output *execinfrapb.OutputRouterSpec,
factory coldata.ColumnFactory,
) error {
if output.Type != execinfrapb.OutputRouterSpec_BY_HASH {
return errors.Errorf("vectorized output router type %s unsupported", output.Type)
}
// HashRouter memory monitor names are the concatenated output stream IDs.
var streamIDs redact.RedactableString
for i, s := range output.Streams {
if i > 0 {
streamIDs = streamIDs + ","
}
streamIDs = redact.Sprintf("%s%d", streamIDs, s.StreamID)
}
mmName := "hash-router-[" + streamIDs + "]"
numOutputs := len(output.Streams)
// We need to create two memory accounts for each output (one for the
// allocator and another one for the converter).
hashRouterMemMonitor, accounts := s.monitorRegistry.CreateUnlimitedMemAccountsWithName(
ctx, flowCtx, mmName, 2*numOutputs,
)
allocatorAccounts, converterAccounts := accounts[:numOutputs], accounts[numOutputs:]
allocators := make([]*colmem.Allocator, numOutputs)
for i := range allocators {
allocators[i] = colmem.NewAllocator(ctx, allocatorAccounts[i], factory)
}
diskMon, diskAccounts := s.monitorRegistry.CreateDiskAccounts(ctx, flowCtx, mmName, numOutputs)
router, outputs := NewHashRouter(
flowCtx, processorID, allocators, input, outputTyps, output.HashColumns, execinfra.GetWorkMemLimit(flowCtx),
s.diskQueueCfg, s.fdSemaphore, diskAccounts, converterAccounts,
)
runRouter := func(ctx context.Context, _ context.CancelFunc) {
router.Run(logtags.AddTag(ctx, "hashRouterID", streamIDs))
}
s.accumulateAsyncComponent(runRouter)
foundLocalOutput := false
for i, op := range outputs {
s.closers = append(s.closers, op)
if buildutil.CrdbTestBuild {
op = colexec.NewInvariantsChecker(op)
}
stream := &output.Streams[i]
switch stream.Type {
case execinfrapb.StreamEndpointSpec_SYNC_RESPONSE:
return errors.Errorf("unexpected sync response output when setting up router")
case execinfrapb.StreamEndpointSpec_REMOTE:
if _, err := s.setupRemoteOutputStream(
ctx, flowCtx, processorID, colexecargs.OpWithMetaInfo{
Root: op,
MetadataSources: colexecop.MetadataSources{op},
}, outputTyps, stream, factory, nil, /* getStats */
); err != nil {
return err
}
case execinfrapb.StreamEndpointSpec_LOCAL:
foundLocalOutput = true
opWithMetaInfo := colexecargs.OpWithMetaInfo{
Root: op,
MetadataSources: colexecop.MetadataSources{op},
}
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).
if err := s.wrapWithVectorizedStatsCollectorBase(
&opWithMetaInfo, nil /* kvReader */, nil, /* columnarizer */
nil /* inputs */, flowCtx.StreamComponentID(stream.StreamID), mons,
); err != nil {
return err
}
}
s.streamIDToInputOp[stream.StreamID] = opWithMetaInfo
}
}
if !foundLocalOutput {
// No local output means that our router is a root of its operator
// chain.
s.opChains = append(s.opChains, router)
}
return nil
}
// setupInput sets up one or more input operators (local or remote) and a
// synchronizer to expose these separate streams as one exec.Operator which is
// returned. If s.recordingStats is true, these inputs and synchronizer are
// wrapped in stats collectors if not done so, although these stats are not
// exposed as of yet. Inboxes that are created are also returned as
// []colexecop.MetadataSource so that any remote metadata can be read through
// calling DrainMeta.
func (s *vectorizedFlowCreator) setupInput(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
processorID int32,
input execinfrapb.InputSyncSpec,
opt flowinfra.FuseOpt,
factory coldata.ColumnFactory,
) (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{}, err
}
for _, inputStream := range input.Streams {
switch inputStream.Type {
case execinfrapb.StreamEndpointSpec_LOCAL:
in := s.streamIDToInputOp[inputStream.StreamID]
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{}, err
}
// Retrieve the latency from the origin node (the one that has the
// outbox).
latency, err := s.podNodeDialer.Latency(roachpb.NodeID(inputStream.OriginNodeID))
if err != nil {
// If an error occurred, latency's nil value of 0 is used. If latency is
// 0, it is not included in the displayed stats for EXPLAIN ANALYZE
// diagrams.
latency = 0
log.VEventf(ctx, 1, "an error occurred during vectorized planning while getting latency: %v", err)
}
inbox, err := s.remoteComponentCreator.newInbox(
colmem.NewAllocator(ctx, s.monitorRegistry.NewStreamingMemAccount(flowCtx), factory),
input.ColumnTypes,
inputStream.StreamID,
s.flowCreatorHelper.getFlowCtxDone(),
admissionOptions{
admissionQ: flowCtx.Cfg.SQLSQLResponseAdmissionQ,
admissionInfo: s.admissionInfo,
})
if err != nil {
return colexecargs.OpWithMetaInfo{}, err
}
s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup)
op := colexecop.Operator(inbox)
ms := colexecop.MetadataSource(inbox)
if buildutil.CrdbTestBuild {
op = colexec.NewInvariantsChecker(op)
ms = op.(colexecop.MetadataSource)
}
opWithMetaInfo := colexecargs.OpWithMetaInfo{
Root: op,
MetadataSources: colexecop.MetadataSources{ms},
}
if s.recordingStats {
// Note: we can't use flowCtx.StreamComponentID because the stream does
// not originate from this node (we are the target node).
compID := execinfrapb.StreamComponentID(
inputStream.OriginNodeID, flowCtx.ID, inputStream.StreamID,
)
s.wrapWithNetworkVectorizedStatsCollector(&opWithMetaInfo, inbox, compID, latency)
}
inputStreamOps = append(inputStreamOps, opWithMetaInfo)
default:
return colexecargs.OpWithMetaInfo{}, errors.Errorf("unsupported input stream type %s", inputStream.Type)
}
}
opWithMetaInfo := inputStreamOps[0]
if len(inputStreamOps) > 1 {
statsInputs := inputStreamOps
if input.Type == execinfrapb.InputSyncSpec_ORDERED {
os := colexec.NewOrderedSynchronizer(
flowCtx,
processorID,
colmem.NewAllocator(ctx, s.monitorRegistry.NewStreamingMemAccount(flowCtx), factory),