-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
execplan.go
2301 lines (2201 loc) · 86.6 KB
/
execplan.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 colbuilder
import (
"context"
"fmt"
"math"
"reflect"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldataext"
"github.com/cockroachdb/cockroach/pkg/col/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/colcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/colconv"
"github.com/cockroachdb/cockroach/pkg/sql/colexec"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colfetcher"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"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/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/errors"
)
func checkNumIn(inputs []colexecop.Operator, numIn int) error {
if len(inputs) != numIn {
return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs))
}
return nil
}
// wrapRowSources, given input Operators, integrates toWrap into a columnar
// execution flow and returns toWrap's output as an Operator.
func wrapRowSources(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
inputs []colexecop.Operator,
inputTypes [][]*types.T,
args *colexecargs.NewColOperatorArgs,
newToWrap func([]execinfra.RowSource) (execinfra.RowSource, error),
factory coldata.ColumnFactory,
) (*colexec.Columnarizer, error) {
var toWrapInputs []execinfra.RowSource
for i, input := range inputs {
// 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 {
// 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 execinfrapb.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
}
// 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,
inputTypes[i],
nil, /* output */
metadataSources,
nil, /* toClose */
nil, /* getStats */
nil, /* cancelFlow */
)
if err != nil {
return nil, err
}
toWrapInputs = append(toWrapInputs, toWrapInput)
}
}
toWrap, err := newToWrap(toWrapInputs)
if err != nil {
return nil, err
}
if _, mustBeStreaming := toWrap.(execinfra.StreamingProcessor); mustBeStreaming {
return colexec.NewStreamingColumnarizer(
ctx, colmem.NewAllocator(ctx, args.StreamingMemAccount, factory), flowCtx, args.Spec.ProcessorID, toWrap,
)
}
return colexec.NewBufferingColumnarizer(
ctx, colmem.NewAllocator(ctx, args.StreamingMemAccount, factory), flowCtx, args.Spec.ProcessorID, toWrap,
)
}
type opResult struct {
*colexecargs.NewColOperatorResult
}
// resetToState resets r to the state specified in arg. arg may be a shallow
// copy made at a given point in time.
func (r *opResult) resetToState(ctx context.Context, arg colexecargs.NewColOperatorResult) {
// MetadataSources are left untouched since there is no need to do any
// cleaning there.
// Close BoundAccounts that are not present in arg.OpAccounts.
accs := make(map[*mon.BoundAccount]struct{})
for _, a := range arg.OpAccounts {
accs[a] = struct{}{}
}
for _, a := range r.OpAccounts {
if _, ok := accs[a]; !ok {
a.Close(ctx)
}
}
// Stop BytesMonitors that are not present in arg.OpMonitors.
mons := make(map[*mon.BytesMonitor]struct{})
for _, m := range arg.OpMonitors {
mons[m] = struct{}{}
}
for _, m := range r.OpMonitors {
if _, ok := mons[m]; !ok {
m.Stop(ctx)
}
}
// Shallow copy over the rest.
*r.NewColOperatorResult = arg
}
func needHashAggregator(aggSpec *execinfrapb.AggregatorSpec) (bool, error) {
var groupCols, orderedCols util.FastIntSet
for _, col := range aggSpec.OrderedGroupCols {
orderedCols.Add(int(col))
}
for _, col := range aggSpec.GroupCols {
if !orderedCols.Contains(int(col)) {
return true, nil
}
groupCols.Add(int(col))
}
if !orderedCols.SubsetOf(groupCols) {
return false, errors.AssertionFailedf("ordered cols must be a subset of grouping cols")
}
return false, nil
}
// IsSupported returns an error if the given spec is not supported by the
// vectorized engine (neither natively nor by wrapping the corresponding row
// execution processor).
func IsSupported(mode sessiondatapb.VectorizeExecMode, spec *execinfrapb.ProcessorSpec) error {
err := supportedNatively(spec)
if err != nil {
if wrapErr := canWrap(mode, spec); wrapErr == nil {
// We don't support this spec natively, but we can wrap the row
// execution processor.
return nil
}
}
return err
}
// supportedNatively checks whether we have a columnar operator equivalent to a
// processor described by spec. Note that it doesn't perform any other checks
// (like validity of the number of inputs).
func supportedNatively(spec *execinfrapb.ProcessorSpec) error {
switch {
case spec.Core.Noop != nil:
return nil
case spec.Core.Values != nil:
if spec.Core.Values.NumRows != 0 && len(spec.Core.Values.Columns) != 0 {
return errors.Newf("values core is supported only with zero rows or zero columns")
}
return nil
case spec.Core.TableReader != nil:
if spec.Core.TableReader.IsCheck {
return errors.Newf("scrub table reader is unsupported in vectorized")
}
return nil
case spec.Core.Filterer != nil:
return nil
case spec.Core.Aggregator != nil:
for _, agg := range spec.Core.Aggregator.Aggregations {
if agg.FilterColIdx != nil {
return errors.Newf("filtering aggregation not supported")
}
}
return nil
case spec.Core.Distinct != nil:
if spec.Core.Distinct.NullsAreDistinct {
return errors.Newf("distinct with unique nulls not supported")
}
if spec.Core.Distinct.ErrorOnDup != "" {
return errors.Newf("distinct with error on duplicates not supported")
}
return nil
case spec.Core.Ordinality != nil:
return nil
case spec.Core.HashJoiner != nil:
if !spec.Core.HashJoiner.OnExpr.Empty() && spec.Core.HashJoiner.Type != descpb.InnerJoin {
return errors.Newf("can't plan vectorized non-inner hash joins with ON expressions")
}
return nil
case spec.Core.MergeJoiner != nil:
if !spec.Core.MergeJoiner.OnExpr.Empty() && spec.Core.MergeJoiner.Type != descpb.InnerJoin {
return errors.Errorf("can't plan non-inner merge join with ON expressions")
}
return nil
case spec.Core.Sorter != nil:
return nil
case spec.Core.Windower != nil:
for _, wf := range spec.Core.Windower.WindowFns {
if wf.Frame != nil {
frame, err := wf.Frame.ConvertToAST()
if err != nil {
return err
}
if !frame.IsDefaultFrame() {
return errors.Newf("window functions with non-default window frames are not supported")
}
}
if wf.FilterColIdx != tree.NoColumnIdx {
return errors.Newf("window functions with FILTER clause are not supported")
}
if wf.Func.AggregateFunc != nil {
return errors.Newf("aggregate functions used as window functions are not supported")
}
if _, supported := colexecwindow.SupportedWindowFns[*wf.Func.WindowFunc]; !supported {
return errors.Newf("window function %s is not supported", wf.String())
}
}
return nil
default:
return errCoreUnsupportedNatively
}
}
var (
errCoreUnsupportedNatively = errors.New("unsupported processor core")
errMetadataTestSenderWrap = errors.New("core.MetadataTestSender is not supported")
errMetadataTestReceiverWrap = errors.New("core.MetadataTestReceiver is not supported")
errChangeAggregatorWrap = errors.New("core.ChangeAggregator is not supported")
errChangeFrontierWrap = errors.New("core.ChangeFrontier is not supported")
errBackfillerWrap = errors.New("core.Backfiller is not supported (not an execinfra.RowSource)")
errCSVWriterWrap = errors.New("core.CSVWriter is not supported (not an execinfra.RowSource)")
errSamplerWrap = errors.New("core.Sampler is not supported (not an execinfra.RowSource)")
errSampleAggregatorWrap = errors.New("core.SampleAggregator is not supported (not an execinfra.RowSource)")
errExperimentalWrappingProhibited = errors.New("wrapping for non-JoinReader and non-LocalPlanNode cores is prohibited in vectorize=experimental_always")
)
func canWrap(mode sessiondatapb.VectorizeExecMode, spec *execinfrapb.ProcessorSpec) error {
if mode == sessiondatapb.VectorizeExperimentalAlways && spec.Core.JoinReader == nil && spec.Core.LocalPlanNode == nil {
return errExperimentalWrappingProhibited
}
switch {
case spec.Core.Noop != nil:
case spec.Core.TableReader != nil:
case spec.Core.JoinReader != nil:
case spec.Core.Sorter != nil:
case spec.Core.Aggregator != nil:
case spec.Core.Distinct != nil:
case spec.Core.MergeJoiner != nil:
case spec.Core.HashJoiner != nil:
case spec.Core.Values != nil:
case spec.Core.Backfiller != nil:
return errBackfillerWrap
case spec.Core.ReadImport != nil:
case spec.Core.CSVWriter != nil:
return errCSVWriterWrap
case spec.Core.Sampler != nil:
return errSamplerWrap
case spec.Core.SampleAggregator != nil:
return errSampleAggregatorWrap
case spec.Core.MetadataTestSender != nil:
// We do not wrap MetadataTestSender because of the way metadata is
// propagated through the vectorized flow - it is drained at the flow
// shutdown unlike these test processors expect.
return errMetadataTestSenderWrap
case spec.Core.MetadataTestReceiver != nil:
// We do not wrap MetadataTestReceiver because of the way metadata is
// propagated through the vectorized flow - it is drained at the flow
// shutdown unlike these test processors expect.
return errMetadataTestReceiverWrap
case spec.Core.ZigzagJoiner != nil:
case spec.Core.ProjectSet != nil:
case spec.Core.Windower != nil:
case spec.Core.LocalPlanNode != nil:
case spec.Core.ChangeAggregator != nil:
// Currently, there is an issue with cleaning up the changefeed flows
// (#55408), so we fallback to the row-by-row engine.
return errChangeAggregatorWrap
case spec.Core.ChangeFrontier != nil:
// Currently, there is an issue with cleaning up the changefeed flows
// (#55408), so we fallback to the row-by-row engine.
return errChangeFrontierWrap
case spec.Core.Ordinality != nil:
case spec.Core.BulkRowWriter != nil:
case spec.Core.InvertedFilterer != nil:
case spec.Core.InvertedJoiner != nil:
case spec.Core.BackupData != nil:
case spec.Core.SplitAndScatter != nil:
case spec.Core.RestoreData != nil:
case spec.Core.Filterer != nil:
case spec.Core.StreamIngestionData != nil:
case spec.Core.StreamIngestionFrontier != nil:
default:
return errors.AssertionFailedf("unexpected processor core %q", spec.Core)
}
return nil
}
// createDiskBackedSort creates a new disk-backed operator that sorts the input
// according to ordering.
// - matchLen specifies the length of the prefix of ordering columns the input
// is already ordered on.
// - maxNumberPartitions (when non-zero) overrides the semi-dynamically
// computed maximum number of partitions that the external sorter will have
// at once.
// - processorID is the ProcessorID of the processor core that requested
// creation of this operator. It is used only to distinguish memory monitors.
// - post describes the post-processing spec of the processor. It will be used
// to determine whether top K sort can be planned. If you want the general sort
// operator, then pass in empty struct.
func (r opResult) createDiskBackedSort(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
args *colexecargs.NewColOperatorArgs,
input colexecop.Operator,
inputTypes []*types.T,
ordering execinfrapb.Ordering,
matchLen uint32,
maxNumberPartitions int,
processorID int32,
post *execinfrapb.PostProcessSpec,
memMonitorNamePrefix string,
factory coldata.ColumnFactory,
) (colexecop.Operator, error) {
streamingMemAccount := args.StreamingMemAccount
useStreamingMemAccountForBuffering := args.TestingKnobs.UseStreamingMemAccountForBuffering
var (
sorterMemMonitorName string
inMemorySorter colexecop.Operator
err error
)
if len(ordering.Columns) == int(matchLen) {
// The input is already fully ordered, so there is nothing to sort.
return input, nil
}
if matchLen > 0 {
// The input is already partially ordered. Use a chunks sorter to avoid
// loading all the rows into memory.
sorterMemMonitorName = fmt.Sprintf("%ssort-chunks-%d", memMonitorNamePrefix, processorID)
var sortChunksMemAccount *mon.BoundAccount
if useStreamingMemAccountForBuffering {
sortChunksMemAccount = streamingMemAccount
} else {
sortChunksMemAccount = r.createMemAccountForSpillStrategy(
ctx, flowCtx, sorterMemMonitorName,
)
}
inMemorySorter, err = colexec.NewSortChunks(
colmem.NewAllocator(ctx, sortChunksMemAccount, factory), input, inputTypes,
ordering.Columns, int(matchLen),
)
} else if post.Limit != 0 && post.Limit < math.MaxUint64-post.Offset {
// There is a limit specified, so we know exactly how many rows the
// sorter should output. The last part of the condition is making sure
// there is no overflow.
//
// Choose a top K sorter, which uses a heap to avoid storing more rows
// than necessary.
//
// TODO(radu): we should not choose this processor when K is very large
// - it is slower unless we get significantly more rows than the limit.
sorterMemMonitorName = fmt.Sprintf("%stopk-sort-%d", memMonitorNamePrefix, processorID)
var topKSorterMemAccount *mon.BoundAccount
if useStreamingMemAccountForBuffering {
topKSorterMemAccount = streamingMemAccount
} else {
topKSorterMemAccount = r.createMemAccountForSpillStrategy(
ctx, flowCtx, sorterMemMonitorName,
)
}
k := post.Limit + post.Offset
inMemorySorter = colexec.NewTopKSorter(
colmem.NewAllocator(ctx, topKSorterMemAccount, factory), input, inputTypes,
ordering.Columns, k,
)
} else {
// No optimizations possible. Default to the standard sort operator.
sorterMemMonitorName = fmt.Sprintf("%ssort-all-%d", memMonitorNamePrefix, processorID)
var sorterMemAccount *mon.BoundAccount
if useStreamingMemAccountForBuffering {
sorterMemAccount = streamingMemAccount
} else {
sorterMemAccount = r.createMemAccountForSpillStrategy(
ctx, flowCtx, sorterMemMonitorName,
)
}
inMemorySorter, err = colexec.NewSorter(
colmem.NewAllocator(ctx, sorterMemAccount, factory), input, inputTypes, ordering.Columns,
)
}
if err != nil {
return nil, err
}
if inMemorySorter == nil {
return nil, errors.AssertionFailedf("unexpectedly inMemorySorter is nil")
}
// NOTE: when spilling to disk, we're using the same general external
// sorter regardless of which sorter variant we have instantiated (i.e.
// we don't take advantage of the limits and of partial ordering). We
// could improve this.
return colexec.NewOneInputDiskSpiller(
input, inMemorySorter.(colexecop.BufferingInMemoryOperator),
sorterMemMonitorName,
func(input colexecop.Operator) colexecop.Operator {
monitorNamePrefix := fmt.Sprintf("%sexternal-sorter", memMonitorNamePrefix)
// We are using unlimited memory monitors here because external
// sort itself is responsible for making sure that we stay within
// the memory limit.
sortUnlimitedAllocator := colmem.NewAllocator(
ctx, r.createBufferingUnlimitedMemAccount(
ctx, flowCtx, monitorNamePrefix+"-sort",
), factory)
mergeUnlimitedAllocator := colmem.NewAllocator(
ctx, r.createBufferingUnlimitedMemAccount(
ctx, flowCtx, monitorNamePrefix+"-merge",
), factory)
outputUnlimitedAllocator := colmem.NewAllocator(
ctx, r.createBufferingUnlimitedMemAccount(
ctx, flowCtx, monitorNamePrefix+"-output",
), factory)
diskAccount := r.createDiskAccount(ctx, flowCtx, monitorNamePrefix)
es := colexec.NewExternalSorter(
sortUnlimitedAllocator,
mergeUnlimitedAllocator,
outputUnlimitedAllocator,
input, inputTypes, ordering,
execinfra.GetWorkMemLimit(flowCtx.Cfg),
maxNumberPartitions,
args.TestingKnobs.NumForcedRepartitions,
args.TestingKnobs.DelegateFDAcquisitions,
args.DiskQueueCfg,
args.FDSemaphore,
diskAccount,
)
r.ToClose = append(r.ToClose, es.(colexecop.Closer))
return es
},
args.TestingKnobs.SpillingCallbackFn,
), nil
}
// makeDistBackedSorterConstructors creates a DiskBackedSorterConstructor that
// can be used by the hash-based partitioner.
// NOTE: unless DelegateFDAcquisitions testing knob is set to true, it is up to
// the caller to acquire the necessary file descriptors up front.
func (r opResult) makeDiskBackedSorterConstructor(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
args *colexecargs.NewColOperatorArgs,
monitorNamePrefix string,
factory coldata.ColumnFactory,
) colexec.DiskBackedSorterConstructor {
return func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecop.Operator {
if maxNumberPartitions < colexecop.ExternalSorterMinPartitions {
colexecerror.InternalError(errors.AssertionFailedf(
"external sorter is attempted to be created with %d partitions, minimum %d required",
maxNumberPartitions, colexecop.ExternalSorterMinPartitions,
))
}
sortArgs := *args
if !args.TestingKnobs.DelegateFDAcquisitions {
// Set the FDSemaphore to nil. This indicates that no FDs should be
// acquired. The hash-based partitioner will do this up front.
sortArgs.FDSemaphore = nil
}
sorter, err := r.createDiskBackedSort(
ctx, flowCtx, &sortArgs, input, inputTypes,
execinfrapb.Ordering{Columns: orderingCols},
0 /* matchLen */, maxNumberPartitions, args.Spec.ProcessorID,
&execinfrapb.PostProcessSpec{}, monitorNamePrefix+"-", factory,
)
if err != nil {
colexecerror.InternalError(err)
}
return sorter
}
}
// createAndWrapRowSource takes a processor spec, creating the row source and
// wrapping it using wrapRowSources. Note that the post process spec is included
// in the processor creation, so make sure to clear it if it will be inspected
// again. NewColOperatorResult is updated with the new OutputTypes and the
// resulting Columnarizer if there is no error. The result is also annotated as
// streaming because the resulting operator is not a buffering operator (even if
// it is a buffering processor). This is not a problem for memory accounting
// because each processor does that on its own, so the used memory will be
// accounted for.
// - causeToWrap is an error that prompted us to wrap a processor core into the
// vectorized plan (for example, it could be an unsupported processor core, an
// unsupported function, etc).
func (r opResult) createAndWrapRowSource(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
args *colexecargs.NewColOperatorArgs,
inputs []colexecop.Operator,
inputTypes [][]*types.T,
spec *execinfrapb.ProcessorSpec,
factory coldata.ColumnFactory,
causeToWrap error,
) error {
if args.ProcessorConstructor == nil {
return errors.New("processorConstructor is nil")
}
if err := canWrap(flowCtx.EvalCtx.SessionData.VectorizeMode, spec); err != nil {
return causeToWrap
}
c, err := wrapRowSources(
ctx,
flowCtx,
inputs,
inputTypes,
args,
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
// here because when wrapping the processor, the materializer will
// be its output, and it will be set up in wrapRowSources.
proc, err := args.ProcessorConstructor(
ctx, flowCtx, spec.ProcessorID, &spec.Core, &spec.Post, inputs,
[]execinfra.RowReceiver{nil} /* outputs */, args.LocalProcessors,
)
if err != nil {
return nil, err
}
var (
rs execinfra.RowSource
ok bool
)
if rs, ok = proc.(execinfra.RowSource); !ok {
return nil, errors.AssertionFailedf(
"processor %s is not an execinfra.RowSource", spec.Core.String(),
)
}
r.ColumnTypes = rs.OutputTypes()
return rs, nil
},
factory,
)
if err != nil {
return err
}
r.Op = c
if args.TestingKnobs.PlanInvariantsCheckers {
r.Op = colexec.NewInvariantsChecker(r.Op)
}
r.MetadataSources = append(r.MetadataSources, r.Op.(execinfrapb.MetadataSource))
r.ToClose = append(r.ToClose, c)
return nil
}
// NOTE: throughout this file we do not append an output type of a projecting
// operator to the passed-in type schema - we, instead, always allocate a new
// type slice and copy over the old schema and set the output column of a
// projecting operator in the next slot. We attempt to enforce this by a linter
// rule, and such behavior prevents the type schema corruption scenario as
// described below.
//
// Without explicit new allocations, it is possible that planSelectionOperators
// (and other planning functions) reuse the same array for filterColumnTypes as
// result.ColumnTypes is using because there was enough capacity to do so.
// As an example, consider the following scenario in the context of
// planFilterExpr method:
// 1. r.ColumnTypes={types.Bool} with len=1 and cap=4
// 2. planSelectionOperators adds another types.Int column, so
// filterColumnTypes={types.Bool, types.Int} with len=2 and cap=4
// Crucially, it uses exact same underlying array as r.ColumnTypes
// uses.
// 3. we project out second column, so r.ColumnTypes={types.Bool}
// 4. later, we add another types.Float column, so
// r.ColumnTypes={types.Bool, types.Float}, but there is enough
// capacity in the array, so we simply overwrite the second slot
// with the new type which corrupts filterColumnTypes to become
// {types.Bool, types.Float}, and we can get into a runtime type
// mismatch situation.
// NewColOperator creates a new columnar operator according to the given spec.
func NewColOperator(
ctx context.Context, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs,
) (_ *colexecargs.NewColOperatorResult, err error) {
result := opResult{NewColOperatorResult: colexecargs.GetNewColOperatorResult()}
r := result.NewColOperatorResult
// Make sure that we clean up memory monitoring infrastructure in case of an
// error or a panic.
defer func() {
returnedErr := err
panicErr := recover()
if returnedErr != nil || panicErr != nil {
for _, acc := range result.OpAccounts {
acc.Close(ctx)
}
result.OpAccounts = result.OpAccounts[:0]
for _, mon := range result.OpMonitors {
mon.Stop(ctx)
}
result.OpMonitors = result.OpMonitors[:0]
}
if panicErr != nil {
colexecerror.InternalError(logcrash.PanicAsError(0, panicErr))
}
}()
spec := args.Spec
inputs := args.Inputs
evalCtx := flowCtx.NewEvalCtx()
factory := args.Factory
if factory == nil {
factory = coldataext.NewExtendedColumnFactory(evalCtx)
}
streamingMemAccount := args.StreamingMemAccount
streamingAllocator := colmem.NewAllocator(ctx, streamingMemAccount, factory)
useStreamingMemAccountForBuffering := args.TestingKnobs.UseStreamingMemAccountForBuffering
if args.ExprHelper == nil {
args.ExprHelper = colexecargs.NewExprHelper()
}
if log.V(2) {
log.Infof(ctx, "planning col operator for spec %q", spec)
}
core := &spec.Core
post := &spec.Post
estimatedRowCount := spec.EstimatedRowCount
// resultPreSpecPlanningStateShallowCopy is a shallow copy of the result
// before any specs are planned. Used if there is a need to backtrack.
resultPreSpecPlanningStateShallowCopy := *r
if err = supportedNatively(spec); err != nil {
if err := canWrap(flowCtx.EvalCtx.SessionData.VectorizeMode, spec); err != nil {
return r, err
}
if log.V(1) {
log.Infof(ctx, "planning a wrapped processor because %s", err.Error())
}
inputTypes := make([][]*types.T, len(spec.Input))
for inputIdx, input := range spec.Input {
inputTypes[inputIdx] = make([]*types.T, len(input.ColumnTypes))
copy(inputTypes[inputIdx], input.ColumnTypes)
}
err = result.createAndWrapRowSource(ctx, flowCtx, args, inputs, inputTypes, spec, factory, err)
// The wrapped processors need to be passed the post-process specs,
// since they inspect them to figure out information about needed
// columns. This means that we'll let those processors do any renders
// or filters, which isn't ideal. We could improve this.
post = &execinfrapb.PostProcessSpec{}
} else {
switch {
case core.Noop != nil:
if err := checkNumIn(inputs, 1); err != nil {
return r, err
}
result.Op = colexecop.NewNoop(inputs[0])
result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes))
copy(result.ColumnTypes, spec.Input[0].ColumnTypes)
case core.Values != nil:
if err := checkNumIn(inputs, 0); err != nil {
return r, err
}
if core.Values.NumRows != 0 && len(core.Values.Columns) != 0 {
return r, errors.AssertionFailedf(
"values core is supported only with zero rows or zero columns, %d rows, %d columns given",
core.Values.NumRows, len(core.Values.Columns),
)
}
result.Op = 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
}
case core.TableReader != nil:
if err := checkNumIn(inputs, 0); err != nil {
return r, err
}
scanOp, err := colfetcher.NewColBatchScan(
ctx, streamingAllocator, flowCtx, evalCtx, core.TableReader, post, estimatedRowCount,
)
if err != nil {
return r, err
}
// colBatchScan is wrapped with a cancel checker below, so we need
// to log its creation separately.
if log.V(1) {
log.Infof(ctx, "made op %T\n", scanOp)
}
result.Op = scanOp
if args.TestingKnobs.PlanInvariantsCheckers {
result.Op = colexec.NewInvariantsChecker(result.Op)
}
result.KVReader = scanOp
result.MetadataSources = append(result.MetadataSources, result.Op.(execinfrapb.MetadataSource))
result.Releasables = append(result.Releasables, scanOp)
// We want to check for cancellation once per input batch, and
// wrapping only colBatchScan with a CancelChecker allows us to do
// just that. It's sufficient for most of the operators since they
// 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.ColumnTypes = scanOp.ResultTypes
result.ToClose = append(result.ToClose, scanOp)
case core.Filterer != nil:
if err := checkNumIn(inputs, 1); err != nil {
return r, err
}
result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes))
copy(result.ColumnTypes, spec.Input[0].ColumnTypes)
result.Op = inputs[0]
if err := result.planAndMaybeWrapFilter(ctx, flowCtx, evalCtx, args, core.Filterer.Filter, factory); err != nil {
return r, err
}
case core.Aggregator != nil:
if err := checkNumIn(inputs, 1); err != nil {
return r, err
}
aggSpec := core.Aggregator
if len(aggSpec.Aggregations) == 0 {
// We can get an aggregator when no aggregate functions are
// present if HAVING clause is present, for example, with a
// query as follows: SELECT 1 FROM t HAVING true. In this case,
// we plan a special operator that outputs a batch of length 1
// without actual columns once and then zero-length batches. The
// actual "data" will be added by projections below.
// TODO(solon): The distsql plan for this case includes a
// 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
// 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.ColumnTypes = []*types.T{types.Int}
break
}
var needHash bool
needHash, err = needHashAggregator(aggSpec)
if err != nil {
return r, err
}
inputTypes := make([]*types.T, len(spec.Input[0].ColumnTypes))
copy(inputTypes, spec.Input[0].ColumnTypes)
newAggArgs := &colexecagg.NewAggregatorArgs{
Input: inputs[0],
InputTypes: inputTypes,
Spec: aggSpec,
EvalCtx: evalCtx,
}
semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn)
newAggArgs.Constructors, newAggArgs.ConstArguments, newAggArgs.OutputTypes, err = colexecagg.ProcessAggregations(
evalCtx, semaCtx, aggSpec.Aggregations, inputTypes,
)
if err != nil {
return r, err
}
result.ColumnTypes = newAggArgs.OutputTypes
if needHash {
// We have separate unit tests that instantiate the in-memory
// hash aggregators, so we don't need to look at
// args.TestingKnobs.DiskSpillingDisabled and always instantiate
// a disk-backed one here.
hashAggregatorMemMonitorName := fmt.Sprintf("hash-aggregator-%d", spec.ProcessorID)
diskSpillingDisabled := !colexec.HashAggregationDiskSpillingEnabled.Get(&flowCtx.Cfg.Settings.SV)
if diskSpillingDisabled {
// The disk spilling is disabled by the cluster setting, so
// we give an unlimited memory account to the in-memory
// hash aggregator and don't set up the disk spiller.
hashAggregatorUnlimitedMemAccount := result.createBufferingUnlimitedMemAccount(
ctx, flowCtx, hashAggregatorMemMonitorName,
)
newAggArgs.Allocator = colmem.NewAllocator(
ctx, hashAggregatorUnlimitedMemAccount, factory,
)
newAggArgs.MemAccount = hashAggregatorUnlimitedMemAccount
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 */)
} else {
// We will divide the available memory equally between the
// two usages - the hash aggregation itself and the input
// tuples tracking.
totalMemLimit := execinfra.GetWorkMemLimit(flowCtx.Cfg)
hashAggregatorMemAccount := result.createMemAccountForSpillStrategyWithLimit(
ctx, flowCtx, hashAggregatorMemMonitorName, totalMemLimit/2,
)
spillingQueueMemMonitorName := hashAggregatorMemMonitorName + "-spilling-queue"
// We need to create a separate memory account for the
// spilling queue because it looks at how much memory it has
// already used in order to decide when to spill to disk.
spillingQueueMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, spillingQueueMemMonitorName)
spillingQueueCfg := args.DiskQueueCfg
spillingQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeReuseCache
spillingQueueCfg.SetDefaultBufferSizeBytesForCacheMode()
newAggArgs.Allocator = colmem.NewAllocator(ctx, hashAggregatorMemAccount, factory)
newAggArgs.MemAccount = hashAggregatorMemAccount
var inMemoryHashAggregator colexecop.Operator
inMemoryHashAggregator, err = colexec.NewHashAggregator(
newAggArgs,
&colexecutils.NewSpillingQueueArgs{
UnlimitedAllocator: colmem.NewAllocator(ctx, spillingQueueMemAccount, factory),
Types: inputTypes,
MemoryLimit: totalMemLimit / 2,
DiskQueueCfg: spillingQueueCfg,
FDSemaphore: args.FDSemaphore,
DiskAcc: result.createDiskAccount(ctx, flowCtx, spillingQueueMemMonitorName),
},
)
if err != nil {
return r, err
}
ehaMonitorNamePrefix := fmt.Sprintf("external-hash-aggregator-%d", spec.ProcessorID)
ehaMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, ehaMonitorNamePrefix)
// Note that we will use an unlimited memory account here
// even for the in-memory hash aggregator since it is easier
// to do so than to try to replace the memory account if the
// spilling to disk occurs (if we don't replace it in such
// 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),
hashAggregatorMemMonitorName,
func(input colexecop.Operator) colexecop.Operator {
newAggArgs := *newAggArgs
// Note that the hash-based partitioner will make
// sure that partitions to process using the
// in-memory hash aggregator fit under the limit, so
// we use an unlimited allocator.
newAggArgs.Allocator = colmem.NewAllocator(ctx, ehaMemAccount, factory)
newAggArgs.MemAccount = ehaMemAccount
newAggArgs.Input = input
return colexec.NewExternalHashAggregator(
flowCtx,
args,
&newAggArgs,
result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, ehaMonitorNamePrefix, factory),
result.createDiskAccount(ctx, flowCtx, ehaMonitorNamePrefix),
)
},
args.TestingKnobs.SpillingCallbackFn,
)
}
} else {
evalCtx.SingleDatumAggMemAccount = streamingMemAccount
newAggArgs.Allocator = streamingAllocator
newAggArgs.MemAccount = streamingMemAccount
result.Op, err = colexec.NewOrderedAggregator(newAggArgs)
}
result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer))
case core.Distinct != nil:
if err := checkNumIn(inputs, 1); err != nil {
return r, err
}
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)
} else {
// We have separate unit tests that instantiate in-memory
// distinct operators, so we don't need to look at
// args.TestingKnobs.DiskSpillingDisabled and always instantiate
// a disk-backed one here.
distinctMemMonitorName := fmt.Sprintf("distinct-%d", spec.ProcessorID)
distinctMemAccount := result.createMemAccountForSpillStrategy(
ctx, flowCtx, distinctMemMonitorName,
)
// TODO(yuzefovich): we have an implementation of partially
// ordered distinct, and we should plan it when we have
// non-empty ordered columns and we think that the probability
// 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,
)
diskAccount := result.createDiskAccount(ctx, flowCtx, distinctMemMonitorName)
result.Op = colexec.NewOneInputDiskSpiller(
inputs[0], inMemoryUnorderedDistinct.(colexecop.BufferingInMemoryOperator),
distinctMemMonitorName,
func(input colexecop.Operator) colexecop.Operator {
monitorNamePrefix := fmt.Sprintf("external-distinct-%d", spec.ProcessorID)
unlimitedAllocator := colmem.NewAllocator(
ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory,
)
return colexec.NewExternalDistinct(
unlimitedAllocator,
flowCtx,
args,
input,
result.ColumnTypes,
result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory),
inMemoryUnorderedDistinct,
diskAccount,
)
},
args.TestingKnobs.SpillingCallbackFn,
)
result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer))
}
case core.Ordinality != nil:
if err := checkNumIn(inputs, 1); err != nil {
return r, err
}
outputIdx := len(spec.Input[0].ColumnTypes)
result.Op = colexecbase.NewOrdinalityOp(streamingAllocator, inputs[0], outputIdx)
result.ColumnTypes = appendOneType(spec.Input[0].ColumnTypes, types.Int)
case core.HashJoiner != nil:
if err := checkNumIn(inputs, 2); err != nil {
return r, err
}
leftTypes := make([]*types.T, len(spec.Input[0].ColumnTypes))
copy(leftTypes, spec.Input[0].ColumnTypes)
rightTypes := make([]*types.T, len(spec.Input[1].ColumnTypes))
copy(rightTypes, spec.Input[1].ColumnTypes)
memoryLimit := execinfra.GetWorkMemLimit(flowCtx.Cfg)
if len(core.HashJoiner.LeftEqColumns) == 0 {
// We are performing a cross-join, so we need to plan a
// specialized operator.
crossJoinerMemMonitorName := fmt.Sprintf("cross-joiner-%d", spec.ProcessorID)
crossJoinerMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, crossJoinerMemMonitorName)
crossJoinerDiskAcc := result.createDiskAccount(ctx, flowCtx, crossJoinerMemMonitorName)
unlimitedAllocator := colmem.NewAllocator(ctx, crossJoinerMemAccount, factory)