-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
column_exec_setup.go
1629 lines (1522 loc) · 57.7 KB
/
column_exec_setup.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 2018 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 distsqlrun
import (
"context"
"fmt"
"math"
"reflect"
"sync"
"sync/atomic"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlpb"
"github.com/cockroachdb/cockroach/pkg/sql/exec"
"github.com/cockroachdb/cockroach/pkg/sql/exec/colrpc"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types/conv"
"github.com/cockroachdb/cockroach/pkg/sql/exec/vecbuiltins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
semtypes "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/mon"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
opentracing "github.com/opentracing/opentracing-go"
)
func checkNumIn(inputs []exec.Operator, numIn int) error {
if len(inputs) != numIn {
return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs))
}
return nil
}
// wrapRowSource, given an input exec.Operator, integrates toWrap into a
// columnar execution flow and returns toWrap's output as an exec.Operator.
func wrapRowSource(
ctx context.Context,
flowCtx *FlowCtx,
input exec.Operator,
inputTypes []semtypes.T,
newToWrap func(RowSource) (RowSource, error),
) (*columnarizer, error) {
var (
toWrapInput RowSource
// TODO(asubiotto): Plumb proper processorIDs once we have stats.
processorID int32
)
// Optimization: if the input is a columnarizer, its input is necessarily a
// RowSource, so remove the unnecessary conversion.
if c, ok := input.(*columnarizer); ok {
// TODO(asubiotto): We might need to do some extra work to remove references
// to this operator (e.g. streamIDToOp).
toWrapInput = c.input
} else {
var err error
toWrapInput, err = newMaterializer(
flowCtx,
processorID,
input,
inputTypes,
&distsqlpb.PostProcessSpec{},
nil, /* output */
nil, /* metadataSourcesQueue */
nil, /* outputStatsToTrace */
nil, /* cancelFlow */
)
if err != nil {
return nil, err
}
}
toWrap, err := newToWrap(toWrapInput)
if err != nil {
return nil, err
}
return newColumnarizer(ctx, flowCtx, processorID, toWrap)
}
type newColOperatorResult struct {
op exec.Operator
outputTypes []types.T
memUsage int
metadataSources []distsqlpb.MetadataSource
isStreaming bool
}
// newColOperator creates a new columnar operator according to the given spec.
func newColOperator(
ctx context.Context, flowCtx *FlowCtx, spec *distsqlpb.ProcessorSpec, inputs []exec.Operator,
) (result newColOperatorResult, err error) {
log.VEventf(ctx, 2, "planning col operator for spec %q", spec)
core := &spec.Core
post := &spec.Post
// Planning additional operators for the PostProcessSpec (filters and render
// expressions) requires knowing the operator's output column types. Currently
// this must be set for any core spec which might require post-processing. In
// the future we may want to make these column types part of the Operator
// interface.
var columnTypes []semtypes.T
// By default, we safely assume that an operator is not streaming. Note that
// projections, renders, filters, limits, offsets as well as all internal
// operators (like stats collectors and cancel checkers) are streaming, so in
// order to determine whether the resulting chain of operators is streaming,
// it is sufficient to look only at the "core" operator.
result.isStreaming = false
switch {
case core.Noop != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
result.op, result.isStreaming = exec.NewNoop(inputs[0]), true
columnTypes = spec.Input[0].ColumnTypes
case core.TableReader != nil:
if err := checkNumIn(inputs, 0); err != nil {
return result, err
}
if core.TableReader.IsCheck {
return result, errors.Newf("scrub table reader is unsupported in vectorized")
}
var scanOp *colBatchScan
scanOp, err = newColBatchScan(flowCtx, core.TableReader, post)
if err != nil {
return result, err
}
result.op, result.isStreaming = scanOp, true
result.metadataSources = append(result.metadataSources, scanOp)
// colBatchScan is wrapped with a cancel checker below, so we need to
// account for its static memory usage here. We also need to log its
// creation separately.
result.memUsage += scanOp.EstimateStaticMemoryUsage()
log.VEventf(ctx, 1, "made op %T\n", result.op)
// We want to check for cancellation once per input batch, and wrapping
// only colBatchScan with an exec.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 = exec.NewCancelChecker(result.op)
returnMutations := core.TableReader.Visibility == distsqlpb.ScanVisibility_PUBLIC_AND_NOT_PUBLIC
columnTypes = core.TableReader.Table.ColumnTypesWithMutations(returnMutations)
case core.Aggregator != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, 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, result.isStreaming, err = exec.NewSingleTupleNoInputOp(), true, nil
// We make columnTypes non-nil so that sanity check doesn't panic.
columnTypes = make([]semtypes.T, 0)
break
}
if len(aggSpec.GroupCols) == 0 &&
len(aggSpec.Aggregations) == 1 &&
aggSpec.Aggregations[0].FilterColIdx == nil &&
aggSpec.Aggregations[0].Func == distsqlpb.AggregatorSpec_COUNT_ROWS &&
!aggSpec.Aggregations[0].Distinct {
result.op, result.isStreaming, err = exec.NewCountOp(inputs[0]), true, nil
columnTypes = []semtypes.T{*semtypes.Int}
break
}
var groupCols, orderedCols util.FastIntSet
for _, col := range aggSpec.OrderedGroupCols {
orderedCols.Add(int(col))
}
needHash := false
for _, col := range aggSpec.GroupCols {
if !orderedCols.Contains(int(col)) {
needHash = true
}
groupCols.Add(int(col))
}
if !orderedCols.SubsetOf(groupCols) {
return result, errors.AssertionFailedf("ordered cols must be a subset of grouping cols")
}
aggTyps := make([][]semtypes.T, len(aggSpec.Aggregations))
aggCols := make([][]uint32, len(aggSpec.Aggregations))
aggFns := make([]distsqlpb.AggregatorSpec_Func, len(aggSpec.Aggregations))
columnTypes = make([]semtypes.T, len(aggSpec.Aggregations))
for i, agg := range aggSpec.Aggregations {
if agg.Distinct {
return result, errors.Newf("distinct aggregation not supported")
}
if agg.FilterColIdx != nil {
return result, errors.Newf("filtering aggregation not supported")
}
if len(agg.Arguments) > 0 {
return result, errors.Newf("aggregates with arguments not supported")
}
aggTyps[i] = make([]semtypes.T, len(agg.ColIdx))
for j, colIdx := range agg.ColIdx {
aggTyps[i][j] = spec.Input[0].ColumnTypes[colIdx]
}
aggCols[i] = agg.ColIdx
aggFns[i] = agg.Func
switch agg.Func {
case distsqlpb.AggregatorSpec_SUM:
switch aggTyps[i][0].Family() {
case semtypes.IntFamily:
// TODO(alfonso): plan ordinary SUM on integer types by casting to DECIMAL
// at the end, mod issues with overflow. Perhaps to avoid the overflow
// issues, at first, we could plan SUM for all types besides Int64.
return result, errors.Newf("sum on int cols not supported (use sum_int)")
}
case distsqlpb.AggregatorSpec_SUM_INT:
// TODO(yuzefovich): support this case through vectorize.
if aggTyps[i][0].Width() != 64 {
return result, errors.Newf("sum_int is only supported on Int64 through vectorized")
}
}
_, retType, err := GetAggregateInfo(agg.Func, aggTyps[i]...)
if err != nil {
return result, err
}
columnTypes[i] = *retType
}
var typs []types.T
typs, err = conv.FromColumnTypes(spec.Input[0].ColumnTypes)
if err != nil {
return result, err
}
if needHash {
result.op, err = exec.NewHashAggregator(
inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, isScalarAggregate(aggSpec),
)
} else {
result.op, err = exec.NewOrderedAggregator(
inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, isScalarAggregate(aggSpec),
)
result.isStreaming = true
}
case core.Distinct != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
var distinctCols, orderedCols util.FastIntSet
for _, col := range core.Distinct.OrderedColumns {
orderedCols.Add(int(col))
}
for _, col := range core.Distinct.DistinctColumns {
if !orderedCols.Contains(int(col)) {
return result, errors.Newf("unsorted distinct not supported")
}
distinctCols.Add(int(col))
}
if !orderedCols.SubsetOf(distinctCols) {
return result, errors.AssertionFailedf("ordered cols must be a subset of distinct cols")
}
columnTypes = spec.Input[0].ColumnTypes
var typs []types.T
typs, err = conv.FromColumnTypes(columnTypes)
if err != nil {
return result, err
}
result.op, err = exec.NewOrderedDistinct(inputs[0], core.Distinct.OrderedColumns, typs)
result.isStreaming = true
case core.Ordinality != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
columnTypes = append(spec.Input[0].ColumnTypes, *semtypes.Int)
result.op, result.isStreaming = exec.NewOrdinalityOp(inputs[0]), true
case core.HashJoiner != nil:
if err := checkNumIn(inputs, 2); err != nil {
return result, err
}
var leftTypes, rightTypes []types.T
leftTypes, err = conv.FromColumnTypes(spec.Input[0].ColumnTypes)
if err != nil {
return result, err
}
rightTypes, err = conv.FromColumnTypes(spec.Input[1].ColumnTypes)
if err != nil {
return result, err
}
nLeftCols := uint32(len(leftTypes))
nRightCols := uint32(len(rightTypes))
leftOutCols := make([]uint32, 0)
rightOutCols := make([]uint32, 0)
// Note that we do not need a special treatment in case of LEFT SEMI and
// LEFT ANTI joins when setting up outCols because in such cases there will
// be a projection with post.OutputColumns already projecting out the right
// side.
if post.Projection {
for _, col := range post.OutputColumns {
if col < nLeftCols {
leftOutCols = append(leftOutCols, col)
} else {
rightOutCols = append(rightOutCols, col-nLeftCols)
}
}
} else {
for i := uint32(0); i < nLeftCols; i++ {
leftOutCols = append(leftOutCols, i)
}
for i := uint32(0); i < nRightCols; i++ {
rightOutCols = append(rightOutCols, i)
}
}
result.op, err = exec.NewEqHashJoinerOp(
inputs[0],
inputs[1],
core.HashJoiner.LeftEqColumns,
core.HashJoiner.RightEqColumns,
leftOutCols,
rightOutCols,
leftTypes,
rightTypes,
core.HashJoiner.RightEqColumnsAreKey,
core.HashJoiner.LeftEqColumnsAreKey || core.HashJoiner.RightEqColumnsAreKey,
core.HashJoiner.Type,
)
if err != nil {
return result, err
}
columnTypes = make([]semtypes.T, nLeftCols+nRightCols)
copy(columnTypes, spec.Input[0].ColumnTypes)
if core.HashJoiner.Type != sqlbase.JoinType_LEFT_SEMI {
// TODO(yuzefovich): update this conditional once LEFT ANTI is supported.
copy(columnTypes[nLeftCols:], spec.Input[1].ColumnTypes)
} else {
columnTypes = columnTypes[:nLeftCols]
}
if !core.HashJoiner.OnExpr.Empty() {
if core.HashJoiner.Type != sqlbase.JoinType_INNER {
return result, errors.Newf("can't plan non-inner hash join with on expressions")
}
columnTypes, err = result.planFilterExpr(flowCtx, core.HashJoiner.OnExpr, columnTypes)
}
case core.MergeJoiner != nil:
// TODO(yuzefovich): merge joiner is streaming when both input sources are
// unique. We probably need to propagate that information from the
// optimizer.
if err := checkNumIn(inputs, 2); err != nil {
return result, err
}
if core.MergeJoiner.Type.IsSetOpJoin() {
return result, errors.AssertionFailedf("unexpectedly %s merge join was planned", core.MergeJoiner.Type.String())
}
var leftTypes, rightTypes []types.T
leftTypes, err = conv.FromColumnTypes(spec.Input[0].ColumnTypes)
if err != nil {
return result, err
}
rightTypes, err = conv.FromColumnTypes(spec.Input[1].ColumnTypes)
if err != nil {
return result, err
}
nLeftCols := uint32(len(leftTypes))
nRightCols := uint32(len(rightTypes))
leftOutCols := make([]uint32, 0, nLeftCols)
rightOutCols := make([]uint32, 0, nRightCols)
// Note that we do not need a special treatment in case of LEFT SEMI and
// LEFT ANTI joins when setting up outCols because in such cases there will
// be a projection with post.OutputColumns already projecting out the right
// side.
if post.Projection {
for _, col := range post.OutputColumns {
if col < nLeftCols {
leftOutCols = append(leftOutCols, col)
} else {
rightOutCols = append(rightOutCols, col-nLeftCols)
}
}
} else {
for i := uint32(0); i < nLeftCols; i++ {
leftOutCols = append(leftOutCols, i)
}
for i := uint32(0); i < nRightCols; i++ {
rightOutCols = append(rightOutCols, i)
}
}
result.op, err = exec.NewMergeJoinOp(
core.MergeJoiner.Type,
inputs[0],
inputs[1],
leftOutCols,
rightOutCols,
leftTypes,
rightTypes,
core.MergeJoiner.LeftOrdering.Columns,
core.MergeJoiner.RightOrdering.Columns,
)
if err != nil {
return result, err
}
columnTypes = make([]semtypes.T, nLeftCols+nRightCols)
copy(columnTypes, spec.Input[0].ColumnTypes)
if core.MergeJoiner.Type != sqlbase.JoinType_LEFT_SEMI &&
core.MergeJoiner.Type != sqlbase.JoinType_LEFT_ANTI {
copy(columnTypes[nLeftCols:], spec.Input[1].ColumnTypes)
} else {
columnTypes = columnTypes[:nLeftCols]
}
if !core.MergeJoiner.OnExpr.Empty() {
if core.MergeJoiner.Type != sqlbase.JoinType_INNER {
return result, errors.Errorf("can't plan non-inner merge joins with on expressions")
}
columnTypes, err = result.planFilterExpr(flowCtx, core.MergeJoiner.OnExpr, columnTypes)
}
case core.JoinReader != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
var c *columnarizer
c, err = wrapRowSource(ctx, flowCtx, inputs[0], spec.Input[0].ColumnTypes, func(input RowSource) (RowSource, error) {
var (
jr RowSource
err error
)
// The lookup and index joiners 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.
if len(core.JoinReader.LookupColumns) == 0 {
jr, err = newIndexJoiner(
flowCtx, spec.ProcessorID, core.JoinReader, input, post, nil, /* output */
)
} else {
jr, err = newJoinReader(
flowCtx, spec.ProcessorID, core.JoinReader, input, post, nil, /* output */
)
}
post = &distsqlpb.PostProcessSpec{}
if err != nil {
return nil, err
}
columnTypes = jr.OutputTypes()
return jr, nil
})
result.op, result.isStreaming = c, true
result.metadataSources = append(result.metadataSources, c)
case core.Sorter != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
input := inputs[0]
var inputTypes []types.T
inputTypes, err = conv.FromColumnTypes(spec.Input[0].ColumnTypes)
if err != nil {
return result, err
}
orderingCols := core.Sorter.OutputOrdering.Columns
matchLen := core.Sorter.OrderingMatchLen
if matchLen > 0 {
// The input is already partially ordered. Use a chunks sorter to avoid
// loading all the rows into memory.
result.op, err = exec.NewSortChunks(input, inputTypes, orderingCols, int(matchLen))
} else if post.Limit != 0 && post.Filter.Empty() && post.Limit+post.Offset < math.MaxUint16 {
// There is a limit specified with no post-process filter, so we know
// exactly how many rows the sorter should output. Choose a top K sorter,
// which uses a heap to avoid storing more rows than necessary.
k := uint16(post.Limit + post.Offset)
result.op, result.isStreaming = exec.NewTopKSorter(input, inputTypes, orderingCols, k), true
} else {
// No optimizations possible. Default to the standard sort operator.
result.op, err = exec.NewSorter(input, inputTypes, orderingCols)
}
columnTypes = spec.Input[0].ColumnTypes
case core.Windower != nil:
if err := checkNumIn(inputs, 1); err != nil {
return result, err
}
if len(core.Windower.WindowFns) != 1 {
return result, errors.Newf("only a single window function is currently supported")
}
wf := core.Windower.WindowFns[0]
if wf.Frame != nil &&
(wf.Frame.Mode != distsqlpb.WindowerSpec_Frame_RANGE ||
wf.Frame.Bounds.Start.BoundType != distsqlpb.WindowerSpec_Frame_UNBOUNDED_PRECEDING ||
(wf.Frame.Bounds.End != nil && wf.Frame.Bounds.End.BoundType != distsqlpb.WindowerSpec_Frame_CURRENT_ROW)) {
return result, errors.Newf("window functions with non-default window frames are not supported")
}
if wf.Func.AggregateFunc != nil {
return result, errors.Newf("aggregate functions used as window functions are not supported")
}
input := inputs[0]
var typs []types.T
typs, err = conv.FromColumnTypes(spec.Input[0].ColumnTypes)
if err != nil {
return result, err
}
tempPartitionColOffset, partitionColIdx := 0, -1
if len(core.Windower.PartitionBy) > 0 {
// TODO(yuzefovich): add support for hashing partitioner (probably by
// leveraging hash routers once we can distribute). The decision about
// which kind of partitioner to use should come from the optimizer.
input, err = exec.NewWindowSortingPartitioner(input, typs, core.Windower.PartitionBy, wf.Ordering.Columns, int(wf.OutputColIdx))
tempPartitionColOffset, partitionColIdx = 1, int(wf.OutputColIdx)
} else {
if len(wf.Ordering.Columns) > 0 {
input, err = exec.NewSorter(input, typs, wf.Ordering.Columns)
}
// TODO(yuzefovich): when both PARTITION BY and ORDER BY clauses are
// omitted, the window function operator is actually streaming.
}
if err != nil {
return result, err
}
orderingCols := make([]uint32, len(wf.Ordering.Columns))
for i, col := range wf.Ordering.Columns {
orderingCols[i] = col.ColIdx
}
switch *wf.Func.WindowFunc {
case distsqlpb.WindowerSpec_ROW_NUMBER:
result.op = vecbuiltins.NewRowNumberOperator(input, int(wf.OutputColIdx)+tempPartitionColOffset, partitionColIdx)
case distsqlpb.WindowerSpec_RANK:
result.op, err = vecbuiltins.NewRankOperator(input, typs, false /* dense */, orderingCols, int(wf.OutputColIdx)+tempPartitionColOffset, partitionColIdx)
case distsqlpb.WindowerSpec_DENSE_RANK:
result.op, err = vecbuiltins.NewRankOperator(input, typs, true /* dense */, orderingCols, int(wf.OutputColIdx)+tempPartitionColOffset, partitionColIdx)
default:
return result, errors.Newf("window function %s is not supported", wf.String())
}
if partitionColIdx != -1 {
// Window partitioner will append a temporary column to the batch which
// we want to project out.
projection := make([]uint32, 0, wf.OutputColIdx+1)
for i := uint32(0); i < wf.OutputColIdx; i++ {
projection = append(projection, i)
}
projection = append(projection, wf.OutputColIdx+1)
result.op = exec.NewSimpleProjectOp(result.op, int(wf.OutputColIdx+1), projection)
}
columnTypes = append(spec.Input[0].ColumnTypes, *semtypes.Int)
default:
return result, errors.Newf("unsupported processor core %q", core)
}
if err != nil {
return result, err
}
// After constructing the base operator, calculate the memory usage
// of the operator.
if sMemOp, ok := result.op.(exec.StaticMemoryOperator); ok {
result.memUsage += sMemOp.EstimateStaticMemoryUsage()
}
log.VEventf(ctx, 1, "made op %T\n", result.op)
if columnTypes == nil {
return result, errors.AssertionFailedf("output columnTypes unset after planning %T", result.op)
}
if !post.Filter.Empty() {
if columnTypes, err = result.planFilterExpr(flowCtx, post.Filter, columnTypes); err != nil {
return result, err
}
}
if post.Projection {
result.op = exec.NewSimpleProjectOp(result.op, len(columnTypes), post.OutputColumns)
// Update output columnTypes.
newTypes := make([]semtypes.T, 0, len(post.OutputColumns))
for _, j := range post.OutputColumns {
newTypes = append(newTypes, columnTypes[j])
}
columnTypes = newTypes
} else if post.RenderExprs != nil {
log.VEventf(ctx, 2, "planning render expressions %+v", post.RenderExprs)
var renderedCols []uint32
for _, expr := range post.RenderExprs {
var (
helper exprHelper
renderMem int
)
err := helper.init(expr, columnTypes, flowCtx.EvalCtx)
if err != nil {
return result, err
}
var outputIdx int
result.op, outputIdx, columnTypes, renderMem, err = planProjectionOperators(
flowCtx.NewEvalCtx(), helper.expr, columnTypes, result.op)
if err != nil {
return result, errors.Wrapf(err, "unable to columnarize render expression %q", expr)
}
if outputIdx < 0 {
return result, errors.AssertionFailedf("missing outputIdx")
}
result.memUsage += renderMem
renderedCols = append(renderedCols, uint32(outputIdx))
}
result.op = exec.NewSimpleProjectOp(result.op, len(columnTypes), renderedCols)
newTypes := make([]semtypes.T, 0, len(renderedCols))
for _, j := range renderedCols {
newTypes = append(newTypes, columnTypes[j])
}
columnTypes = newTypes
}
if post.Offset != 0 {
result.op = exec.NewOffsetOp(result.op, post.Offset)
}
if post.Limit != 0 {
result.op = exec.NewLimitOp(result.op, post.Limit)
}
if err != nil {
return result, err
}
result.outputTypes, err = conv.FromColumnTypes(columnTypes)
return result, err
}
func (r *newColOperatorResult) planFilterExpr(
flowCtx *FlowCtx, filter distsqlpb.Expression, columnTypes []semtypes.T,
) ([]semtypes.T, error) {
var (
helper exprHelper
selectionMem int
)
err := helper.init(filter, columnTypes, flowCtx.EvalCtx)
if err != nil {
return columnTypes, err
}
var filterColumnTypes []semtypes.T
r.op, _, filterColumnTypes, selectionMem, err = planSelectionOperators(flowCtx.NewEvalCtx(), helper.expr, columnTypes, r.op)
if err != nil {
return columnTypes, errors.Wrapf(err, "unable to columnarize filter expression %q", filter.Expr)
}
r.memUsage += selectionMem
if len(filterColumnTypes) > len(columnTypes) {
// Additional columns were appended to store projections while evaluating
// the filter. Project them away.
var outputColumns []uint32
for i := range columnTypes {
outputColumns = append(outputColumns, uint32(i))
}
r.op = exec.NewSimpleProjectOp(r.op, len(filterColumnTypes), outputColumns)
}
return columnTypes, nil
}
func planSelectionOperators(
ctx *tree.EvalContext, expr tree.TypedExpr, columnTypes []semtypes.T, input exec.Operator,
) (op exec.Operator, resultIdx int, ct []semtypes.T, memUsed int, err error) {
if err := assertHomogeneousTypes(expr); err != nil {
return op, resultIdx, ct, memUsed, err
}
switch t := expr.(type) {
case *tree.IndexedVar:
return exec.NewBoolVecToSelOp(input, t.Idx), -1, columnTypes, memUsed, nil
case *tree.AndExpr:
leftOp, _, ct, memUsage, err := planSelectionOperators(ctx, t.TypedLeft(), columnTypes, input)
if err != nil {
return nil, resultIdx, ct, memUsage, err
}
return planSelectionOperators(ctx, t.TypedRight(), ct, leftOp)
case *tree.ComparisonExpr:
cmpOp := t.Operator
leftOp, leftIdx, ct, memUsageLeft, err := planProjectionOperators(ctx, t.TypedLeft(), columnTypes, input)
if err != nil {
return nil, resultIdx, ct, memUsageLeft, err
}
typ := &ct[leftIdx]
if constArg, ok := t.Right.(tree.Datum); ok {
if t.Operator == tree.Like || t.Operator == tree.NotLike {
negate := t.Operator == tree.NotLike
op, err := exec.GetLikeOperator(
ctx, leftOp, leftIdx, string(tree.MustBeDString(constArg)), negate)
return op, resultIdx, ct, memUsageLeft, err
}
if t.Operator == tree.In || t.Operator == tree.NotIn {
negate := t.Operator == tree.NotIn
datumTuple, ok := tree.AsDTuple(constArg)
if !ok {
err = errors.Errorf("IN is only supported for constant expressions")
return nil, resultIdx, ct, memUsed, err
}
op, err := exec.GetInOperator(typ, leftOp, leftIdx, datumTuple, negate)
return op, resultIdx, ct, memUsageLeft, err
}
op, err := exec.GetSelectionConstOperator(typ, cmpOp, leftOp, leftIdx, constArg)
return op, resultIdx, ct, memUsageLeft, err
}
rightOp, rightIdx, ct, memUsageRight, err := planProjectionOperators(ctx, t.TypedRight(), ct, leftOp)
if err != nil {
return nil, resultIdx, ct, memUsageLeft + memUsageRight, err
}
op, err := exec.GetSelectionOperator(typ, cmpOp, rightOp, leftIdx, rightIdx)
return op, resultIdx, ct, memUsageLeft + memUsageRight, err
default:
return nil, resultIdx, nil, memUsed, errors.Errorf("unhandled selection expression type: %s", reflect.TypeOf(t))
}
}
// planProjectionOperators plans a chain of operators to execute the provided
// expression. It returns the tail of the chain, as well as the column index
// of the expression's result (if any, otherwise -1) and the column types of the
// resulting batches.
func planProjectionOperators(
ctx *tree.EvalContext, expr tree.TypedExpr, columnTypes []semtypes.T, input exec.Operator,
) (op exec.Operator, resultIdx int, ct []semtypes.T, memUsed int, err error) {
if err := assertHomogeneousTypes(expr); err != nil {
return op, resultIdx, ct, memUsed, err
}
resultIdx = -1
switch t := expr.(type) {
case *tree.IndexedVar:
return input, t.Idx, columnTypes, memUsed, nil
case *tree.ComparisonExpr:
return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input)
case *tree.BinaryExpr:
return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input)
case *tree.FuncExpr:
var (
inputCols []int
projectionMem int
)
ct = columnTypes
op = input
for _, e := range t.Exprs {
var err error
// TODO(rohany): This could be done better, especially in the case of
// constant arguments, because the vectorized engine right now
// creates a new column full of the constant value.
op, resultIdx, ct, projectionMem, err = planProjectionOperators(ctx, e.(tree.TypedExpr), ct, op)
if err != nil {
return nil, resultIdx, nil, memUsed, err
}
inputCols = append(inputCols, resultIdx)
memUsed += projectionMem
}
funcOutputType := t.ResolvedType()
resultIdx = len(ct)
ct = append(ct, *funcOutputType)
op = exec.NewBuiltinFunctionOperator(ctx, t, ct, inputCols, resultIdx, op)
return op, resultIdx, ct, memUsed, nil
case tree.Datum:
datumType := t.ResolvedType()
ct := columnTypes
resultIdx = len(ct)
ct = append(ct, *datumType)
if datumType.Family() == semtypes.UnknownFamily {
return exec.NewConstNullOp(input, resultIdx), resultIdx, ct, memUsed, nil
}
typ := conv.FromColumnType(datumType)
constVal, err := conv.GetDatumToPhysicalFn(datumType)(t)
if err != nil {
return nil, resultIdx, ct, memUsed, err
}
op, err := exec.NewConstOp(input, typ, constVal, resultIdx)
if err != nil {
return nil, resultIdx, ct, memUsed, err
}
return op, resultIdx, ct, memUsed, nil
default:
return nil, resultIdx, nil, memUsed, errors.Errorf("unhandled projection expression type: %s", reflect.TypeOf(t))
}
}
func planProjectionExpr(
ctx *tree.EvalContext,
binOp tree.Operator,
outputType *semtypes.T,
left, right tree.TypedExpr,
columnTypes []semtypes.T,
input exec.Operator,
) (op exec.Operator, resultIdx int, ct []semtypes.T, memUsed int, err error) {
resultIdx = -1
// There are 3 cases. Either the left is constant, the right is constant,
// or neither are constant.
lConstArg, lConst := left.(tree.Datum)
if lConst {
// Case one: The left is constant.
// Normally, the optimizer normalizes binary exprs so that the constant
// argument is on the right side. This doesn't happen for non-commutative
// operators such as - and /, though, so we still need this case.
var rightOp exec.Operator
var rightIdx int
rightOp, rightIdx, ct, memUsed, err = planProjectionOperators(ctx, right, columnTypes, input)
if err != nil {
return nil, resultIdx, ct, memUsed, err
}
resultIdx = len(ct)
// The projection result will be outputted to a new column which is appended
// to the input batch.
op, err = exec.GetProjectionLConstOperator(&ct[rightIdx], binOp, rightOp, rightIdx, lConstArg, resultIdx)
ct = append(ct, *outputType)
if sMem, ok := op.(exec.StaticMemoryOperator); ok {
memUsed += sMem.EstimateStaticMemoryUsage()
}
return op, resultIdx, ct, memUsed, err
}
leftOp, leftIdx, ct, leftMem, err := planProjectionOperators(ctx, left, columnTypes, input)
if err != nil {
return nil, resultIdx, ct, leftMem, err
}
if rConstArg, rConst := right.(tree.Datum); rConst {
// Case 2: The right is constant.
// The projection result will be outputted to a new column which is appended
// to the input batch.
resultIdx = len(ct)
if binOp == tree.Like || binOp == tree.NotLike {
negate := binOp == tree.NotLike
op, err = exec.GetLikeProjectionOperator(
ctx, leftOp, leftIdx, resultIdx, string(tree.MustBeDString(rConstArg)), negate)
} else if binOp == tree.In || binOp == tree.NotIn {
negate := binOp == tree.NotIn
datumTuple, ok := tree.AsDTuple(rConstArg)
if !ok {
err = errors.Errorf("IN operator supported only on constant expressions")
return nil, resultIdx, ct, leftMem, err
}
op, err = exec.GetInProjectionOperator(&ct[leftIdx], leftOp, leftIdx, resultIdx, datumTuple, negate)
} else {
op, err = exec.GetProjectionRConstOperator(&ct[leftIdx], binOp, leftOp, leftIdx, rConstArg, resultIdx)
}
ct = append(ct, *outputType)
if sMem, ok := op.(exec.StaticMemoryOperator); ok {
memUsed += sMem.EstimateStaticMemoryUsage()
}
return op, resultIdx, ct, leftMem + memUsed, err
}
// Case 3: neither are constant.
rightOp, rightIdx, ct, rightMem, err := planProjectionOperators(ctx, right, ct, leftOp)
if err != nil {
return nil, resultIdx, nil, leftMem + rightMem, err
}
resultIdx = len(ct)
op, err = exec.GetProjectionOperator(&ct[leftIdx], binOp, rightOp, leftIdx, rightIdx, resultIdx)
ct = append(ct, *outputType)
if sMem, ok := op.(exec.StaticMemoryOperator); ok {
memUsed += sMem.EstimateStaticMemoryUsage()
}
return op, resultIdx, ct, leftMem + rightMem + memUsed, err
}
// assertHomogeneousTypes checks that the left and right sides of an expression
// have identical types. (Vectorized execution does not yet handle mixed types.)
// For BinaryExprs, it also checks that the result type matches, since this is
// not the case for certain operations like integer division.
func assertHomogeneousTypes(expr tree.TypedExpr) error {
switch t := expr.(type) {
case *tree.BinaryExpr:
left := t.TypedLeft().ResolvedType()
right := t.TypedRight().ResolvedType()
result := t.ResolvedType()
if !left.Identical(right) {
return errors.Errorf("BinaryExpr on %s and %s is unhandled", left, right)
}
if !left.Identical(result) {
return errors.Errorf("BinaryExpr on %s with %s result is unhandled", left, result)
}
case *tree.ComparisonExpr:
left := t.TypedLeft().ResolvedType()
right := t.TypedRight().ResolvedType()
// Special rules for IN and NOT IN expressions. The type checker
// handles invalid types for the IN and NOT IN operations at this point,
// and we allow a comparison between t and t tuple.
if t.Operator == tree.In || t.Operator == tree.NotIn {
return nil
}
if !left.Identical(right) {
return errors.Errorf("ComparisonExpr on %s and %s is unhandled", left, right)
}
}
return nil
}
// wrapWithVectorizedStatsCollector creates a new exec.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 wrapWithVectorizedStatsCollector(
op exec.Operator, inputs []exec.Operator, pspec *distsqlpb.ProcessorSpec,
) (*exec.VectorizedStatsCollector, error) {
inputWatch := timeutil.NewStopWatch()
vsc := exec.NewVectorizedStatsCollector(op, pspec.ProcessorID, len(inputs) == 0, inputWatch)
for _, input := range inputs {
sc, ok := input.(*exec.VectorizedStatsCollector)
if !ok {
return nil, errors.New("unexpectedly an input is not collecting stats")
}
sc.SetOutputWatch(inputWatch)
}
return vsc, nil
}
// finishVectorizedStatsCollectors finishes the given stats collectors and
// outputs their stats to the trace contained in the ctx's span.
func finishVectorizedStatsCollectors(
ctx context.Context,
deterministicStats bool,
vectorizedStatsCollectors []*exec.VectorizedStatsCollector,
procIDs []int32,
) {
spansByProcID := make(map[int32]opentracing.Span)
for _, pid := range procIDs {
// We're creating a new span for every processor setting the
// appropriate tag so that it is displayed correctly on the flow
// diagram.
// TODO(yuzefovich): these spans are created and finished right
// away which is not the way they are supposed to be used, so this
// should be fixed.
_, spansByProcID[pid] = tracing.ChildSpan(ctx, fmt.Sprintf("operator for processor %d", pid))
spansByProcID[pid].SetTag(distsqlpb.ProcessorIDTagKey, pid)
}
for _, vsc := range vectorizedStatsCollectors {
// TODO(yuzefovich): I'm not sure whether there are cases when
// multiple operators correspond to a single processor. We might
// need to do some aggregation here in that case.
vsc.FinalizeStats()
if deterministicStats {
vsc.VectorizedStats.Time = 0
}
if vsc.ID < 0 {
// Ignore stats collectors not associated with a processor.
continue
}
tracing.SetSpanStats(spansByProcID[vsc.ID], &vsc.VectorizedStats)
}
for _, sp := range spansByProcID {
sp.Finish()
}
}
type runFn func(context.Context, 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(distsqlpb.StreamID, *colrpc.Inbox, *sync.WaitGroup)
// checkInboundStreamID checks that the provided stream ID has not been seen
// yet.
checkInboundStreamID(distsqlpb.StreamID) error
// accumulateAsyncComponent stores a component (either a router or an outbox)
// to be run asynchronously.
accumulateAsyncComponent(runFn)
// addMaterializer adds a materializer to the flow.
addMaterializer(*materializer)
// getCancelFlowFn returns a flow cancellation function.
getCancelFlowFn() context.CancelFunc
}
// opDAGWithMetaSources is a helper struct that stores an operator DAG as well
// as the metadataSources in this DAG that need to be drained.