-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathrouters_test.go
982 lines (870 loc) · 28.2 KB
/
routers_test.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
// 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 colexec
import (
"context"
"fmt"
"math/rand"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/sql/colcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
)
// memoryTestCase is a helper struct for a test with memory limits.
type memoryTestCase struct {
// bytes is the memory limit.
bytes int64
// skipExpSpillCheck specifies whether expSpill should be checked to assert
// that expected spilling behavior happened. This is true if bytes was
// randomly generated.
skipExpSpillCheck bool
// expSpill specifies whether a spill is expected or not. Should be ignored if
// skipExpSpillCheck is true.
expSpill bool
}
// getDiskqueueCfgAndMemoryTestCases is a test helper that creates an in-memory
// DiskQueueCfg that can be used to create a new DiskQueue. A cleanup function
// is also returned as well as some default memory limits that are useful to
// test with: 0 for an immediate spill, a random memory limit up to 64 MiB, and
// 1GiB, which shouldn't result in a spill.
// Note that not all tests will check for a spill, it is enough that some
// deterministic tests do so for the simple cases.
// TODO(asubiotto): We might want to also return a verify() function that will
// check for leftover files.
func getDiskQueueCfgAndMemoryTestCases(
t *testing.T, rng *rand.Rand,
) (colcontainer.DiskQueueCfg, func(), []memoryTestCase) {
t.Helper()
queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */)
return queueCfg, cleanup, []memoryTestCase{
{bytes: 0, expSpill: true},
{bytes: 1 + rng.Int63n(64<<20 /* 64 MiB */), skipExpSpillCheck: true},
{bytes: 1 << 30 /* 1 GiB */, expSpill: false},
}
}
// getDataAndFullSelection is a test helper that generates tuples representing
// a one-column coltypes.Int64 batch where each element is its ordinal and an
// accompanying selection vector that selects every index in tuples.
func getDataAndFullSelection() (tuples, []uint16) {
data := make(tuples, coldata.BatchSize())
fullSelection := make([]uint16, coldata.BatchSize())
for i := range data {
data[i] = tuple{i}
fullSelection[i] = uint16(i)
}
return data, fullSelection
}
func TestRouterOutputAddBatch(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
data, fullSelection := getDataAndFullSelection()
// Since the actual data doesn't matter, we will just be reusing data for each
// test case.
testCases := []struct {
inputBatchSize uint16
outputBatchSize int
blockedThreshold int
// selection determines which indices to add to the router output as well
// as how many elements from data are compared to the output.
selection []uint16
name string
}{
{
inputBatchSize: coldata.BatchSize(),
outputBatchSize: int(coldata.BatchSize()),
blockedThreshold: getDefaultRouterOutputBlockedThreshold(),
selection: fullSelection,
name: "OneBatch",
},
{
inputBatchSize: coldata.BatchSize(),
outputBatchSize: 4,
blockedThreshold: getDefaultRouterOutputBlockedThreshold(),
selection: fullSelection,
name: "OneBatchGTOutputSize",
},
{
inputBatchSize: 4,
outputBatchSize: int(coldata.BatchSize()),
blockedThreshold: getDefaultRouterOutputBlockedThreshold(),
selection: fullSelection,
name: "MultipleInputBatchesLTOutputSize",
},
{
inputBatchSize: coldata.BatchSize(),
outputBatchSize: int(coldata.BatchSize()),
blockedThreshold: getDefaultRouterOutputBlockedThreshold(),
selection: fullSelection[:len(fullSelection)/4],
name: "QuarterSelection",
},
}
// unblockEventsChan is purposefully unbuffered; the router output should never write to it
// in this test.
unblockEventsChan := make(chan struct{})
rng, _ := randutil.NewPseudoRand()
queueCfg, cleanup, memoryTestCases := getDiskQueueCfgAndMemoryTestCases(t, rng)
defer cleanup()
for _, tc := range testCases {
for _, mtc := range memoryTestCases {
t.Run(fmt.Sprintf("%s/memoryLimit=%s", tc.name, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
// Clear the testAllocator for use.
testAllocator.Clear()
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
testAllocator, []coltypes.T{coltypes.Int64}, unblockEventsChan, mtc.bytes, queueCfg, tc.blockedThreshold, tc.outputBatchSize,
)
in := newOpTestInput(tc.inputBatchSize, data, nil /* typs */)
out := newOpTestOutput(o, data[:len(tc.selection)])
in.Init()
for {
b := in.Next(ctx)
o.addBatch(b, tc.selection)
if b.Length() == 0 {
break
}
}
if err := out.Verify(); err != nil {
t.Fatal(err)
}
// The output should never block. This assumes test cases never send more
// than defaultRouterOutputBlockedThreshold values.
select {
case b := <-unblockEventsChan:
t.Fatalf("unexpected output state change blocked: %t", b)
default:
}
if !mtc.skipExpSpillCheck {
require.Equal(t, mtc.expSpill, o.mu.data.spilled())
}
})
}
}
}
func TestRouterOutputNext(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
data, fullSelection := getDataAndFullSelection()
testCases := []struct {
unblockEvent func(in Operator, o *routerOutputOp)
expected tuples
name string
}{
{
// ReaderWaitsForData verifies that a reader blocks in Next(ctx) until there
// is data available.
unblockEvent: func(in Operator, o *routerOutputOp) {
for {
b := in.Next(ctx)
o.addBatch(b, fullSelection)
if b.Length() == 0 {
break
}
}
},
expected: data,
name: "ReaderWaitsForData",
},
{
// ReaderWaitsForZeroBatch verifies that a reader blocking on Next will
// also get unblocked with no data other than the zero batch.
unblockEvent: func(_ Operator, o *routerOutputOp) {
o.addBatch(coldata.ZeroBatch, nil /* selection */)
},
expected: tuples{},
name: "ReaderWaitsForZeroBatch",
},
{
// CancelUnblocksReader verifies that calling cancel on an output unblocks
// a reader.
unblockEvent: func(_ Operator, o *routerOutputOp) {
o.cancel(ctx)
},
expected: tuples{},
name: "CancelUnblocksReader",
},
}
// unblockedEventsChan is purposefully unbuffered; the router output should
// never write to it in this test.
unblockedEventsChan := make(chan struct{})
rng, _ := randutil.NewPseudoRand()
queueCfg, cleanup, memoryTestCases := getDiskQueueCfgAndMemoryTestCases(t, rng)
defer cleanup()
for _, mtc := range memoryTestCases {
for _, tc := range testCases {
t.Run(fmt.Sprintf("%s/memoryLimit=%s", tc.name, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
var wg sync.WaitGroup
batchChan := make(chan coldata.Batch)
if queueCfg.FS == nil {
t.Fatal("FS was nil")
}
o := newRouterOutputOp(testAllocator, []coltypes.T{coltypes.Int64}, unblockedEventsChan, mtc.bytes, queueCfg)
in := newOpTestInput(coldata.BatchSize(), data, nil /* typs */)
in.Init()
wg.Add(1)
go func() {
for {
b := o.Next(ctx)
batchChan <- b
if b.Length() == 0 {
break
}
}
wg.Done()
}()
// Sleep a long enough amount of time to make sure that if Next didn't block
// above, we have a good chance of reading a batch.
time.Sleep(time.Millisecond)
select {
case <-batchChan:
t.Fatal("expected reader goroutine to block when no data ready")
default:
}
tc.unblockEvent(in, o)
// Should have data available, pushed by our reader goroutine.
batches := NewBatchBuffer()
out := newOpTestOutput(batches, tc.expected)
for {
b := <-batchChan
batches.Add(b)
if b.Length() == 0 {
break
}
}
if err := out.Verify(); err != nil {
t.Fatal(err)
}
wg.Wait()
select {
case <-unblockedEventsChan:
t.Fatal("unexpected output state change")
default:
}
})
}
t.Run(fmt.Sprintf("NextAfterZeroBatchDoesntBlock/memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
o := newRouterOutputOp(testAllocator, []coltypes.T{coltypes.Int64}, unblockedEventsChan, mtc.bytes, queueCfg)
o.addBatch(coldata.ZeroBatch, fullSelection)
o.Next(ctx)
o.Next(ctx)
select {
case <-unblockedEventsChan:
t.Fatal("unexpected output state change")
default:
}
})
t.Run(fmt.Sprintf("AddBatchDoesntBlockWhenOutputIsBlocked/memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
var (
smallBatchSize = 8
blockThreshold = smallBatchSize / 2
)
if len(fullSelection) <= smallBatchSize {
// If a full batch is smaller than our small batch size, reduce it, since
// this test relies on multiple batches returned from the input.
smallBatchSize = 2
if smallBatchSize >= coldata.MinBatchSize {
// Sanity check.
t.Fatalf("smallBatchSize=%d still too large (must be less than MinBatchSize=%d)", smallBatchSize, coldata.MinBatchSize)
}
blockThreshold = 1
}
// Use a smaller selection than the batch size; it increases test coverage.
selection := fullSelection[:blockThreshold]
expected := make(tuples, 0, len(data))
for i := 0; i < len(data); i += smallBatchSize {
for k := 0; k < blockThreshold && i+k < len(data); k++ {
expected = append(expected, data[i+k])
}
}
ch := make(chan struct{}, 2)
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
testAllocator, []coltypes.T{coltypes.Int64}, ch, mtc.bytes, queueCfg, blockThreshold, int(coldata.BatchSize()),
)
in := newOpTestInput(uint16(smallBatchSize), data, nil /* typs */)
out := newOpTestOutput(o, expected)
in.Init()
b := in.Next(ctx)
// Make sure the output doesn't consider itself blocked. We're right at the
// limit but not over.
if o.addBatch(b, selection) {
t.Fatal("unexpectedly blocked")
}
b = in.Next(ctx)
// This addBatch call should now block the output.
if !o.addBatch(b, selection) {
t.Fatal("unexpectedly still unblocked")
}
// Add the rest of the data.
for {
b = in.Next(ctx)
if o.addBatch(b, selection) {
t.Fatal("should only return true when switching from unblocked to blocked")
}
if b.Length() == 0 {
break
}
}
// Unblock the output.
if err := out.Verify(); err != nil {
t.Fatal(err)
}
// Verify that an unblock event is sent on the channel. This test will fail
// with a timeout on a channel read if not.
<-ch
})
}
}
func TestRouterOutputRandom(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
rng, _ := randutil.NewPseudoRand()
var (
maxValues = int(coldata.BatchSize()) * 4
blockedThreshold = 1 + rng.Intn(maxValues-1)
outputSize = 1 + rng.Intn(maxValues-1)
)
typs := []coltypes.T{coltypes.Int64, coltypes.Int64}
dataLen := 1 + rng.Intn(maxValues-1)
data := make(tuples, dataLen)
for i := range data {
data[i] = make(tuple, len(typs))
for j := range typs {
data[i][j] = rng.Int63()
}
}
queueCfg, cleanup, memoryTestCases := getDiskQueueCfgAndMemoryTestCases(t, rng)
defer cleanup()
testName := fmt.Sprintf(
"blockedThreshold=%d/outputSize=%d/totalInputSize=%d", blockedThreshold, outputSize, len(data),
)
for _, mtc := range memoryTestCases {
t.Run(fmt.Sprintf("%s/memoryLimit=%s", testName, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []Operator) {
var wg sync.WaitGroup
unblockedEventsChans := make(chan struct{}, 2)
o := newRouterOutputOpWithBlockedThresholdAndBatchSize(
testAllocator, typs, unblockedEventsChans, mtc.bytes, queueCfg, blockedThreshold, outputSize,
)
inputs[0].Init()
expected := make(tuples, 0, len(data))
// Producer.
errCh := make(chan error)
go func() {
lastBlockedState := false
for {
b := inputs[0].Next(ctx)
selection := b.Selection()
if selection == nil {
selection = randomSel(rng, b.Length(), rng.Float64())
}
selection = selection[:b.Length()]
for _, i := range selection {
expected = append(expected, make(tuple, len(typs)))
for j := range typs {
expected[len(expected)-1][j] = b.ColVec(j).Int64()[i]
}
}
if o.addBatch(b, selection) {
if lastBlockedState {
// We might have missed an unblock event during the last loop.
select {
case <-unblockedEventsChans:
default:
errCh <- errors.New("output returned state change to blocked when already blocked")
}
}
lastBlockedState = true
}
// Read any state changes.
for moreToRead := true; moreToRead; {
select {
case <-unblockedEventsChans:
if !lastBlockedState {
errCh <- errors.New("received unblocked state change when output is already unblocked")
}
lastBlockedState = false
default:
moreToRead = false
}
}
if b.Length() == 0 {
errCh <- nil
return
}
}
}()
actual := NewBatchBuffer()
// Consumer.
wg.Add(1)
go func() {
// Create a new allocator to copy the resulting batches. We need
// a separate allocator to testAllocator since this is a separate
// goroutine and allocators may not be used concurrently.
acc := testMemMonitor.MakeBoundAccount()
allocator := NewAllocator(ctx, &acc)
defer acc.Close(ctx)
for {
b := o.Next(ctx)
actual.Add(CopyBatch(allocator, b))
if b.Length() == 0 {
wg.Done()
return
}
}
}()
if err := <-errCh; err != nil {
t.Fatal(err)
}
wg.Wait()
if err := newOpTestOutput(actual, expected).Verify(); err != nil {
t.Fatal(err)
}
})
})
}
}
type callbackRouterOutput struct {
ZeroInputNode
addBatchCb func(coldata.Batch, []uint16) bool
cancelCb func()
}
var _ routerOutput = callbackRouterOutput{}
func (o callbackRouterOutput) addBatch(batch coldata.Batch, selection []uint16) bool {
if o.addBatchCb != nil {
return o.addBatchCb(batch, selection)
}
return false
}
func (o callbackRouterOutput) cancel(context.Context) {
if o.cancelCb != nil {
o.cancelCb()
}
}
func TestHashRouterComputesDestination(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
// We have precomputed expectedNumVals only for the default batch size, so we
// will override it if a different value is set.
const expectedBatchSize = 1024
batchSize := coldata.BatchSize()
if batchSize != expectedBatchSize {
coldata.SetBatchSizeForTests(expectedBatchSize)
defer func(batchSize uint16) { coldata.SetBatchSizeForTests(batchSize) }(batchSize)
batchSize = expectedBatchSize
}
data := make(tuples, batchSize)
valsYetToSee := make(map[int64]struct{})
for i := range data {
data[i] = tuple{i}
valsYetToSee[int64(i)] = struct{}{}
}
in := newOpTestInput(batchSize, data, nil /* typs */)
in.Init()
var (
// expectedNumVals is the number of expected values the output at the
// corresponding index in outputs receives. This should not change between
// runs of tests unless the underlying hash algorithm changes. If it does,
// distributed hash routing will not produce correct results.
expectedNumVals = []int{273, 252, 287, 212}
numOutputs = 4
valsPushed = make([]int, numOutputs)
)
outputs := make([]routerOutput, numOutputs)
for i := range outputs {
// Capture the index.
outputIdx := i
outputs[i] = callbackRouterOutput{
addBatchCb: func(batch coldata.Batch, sel []uint16) bool {
for _, j := range sel {
key := batch.ColVec(0).Int64()[j]
if _, ok := valsYetToSee[key]; !ok {
t.Fatalf("pushed alread seen value to router output: %d", key)
}
delete(valsYetToSee, key)
valsPushed[outputIdx]++
}
return false
},
cancelCb: func() {
t.Fatalf(
"output %d canceled, outputs should not be canceled during normal operation", outputIdx,
)
},
}
}
r := newHashRouterWithOutputs(in, []coltypes.T{coltypes.Int64}, []uint32{0}, nil /* ch */, outputs)
for r.processNextBatch(ctx) {
}
if len(valsYetToSee) != 0 {
t.Fatalf("hash router failed to push values: %v", valsYetToSee)
}
for i, expected := range expectedNumVals {
if valsPushed[i] != expected {
t.Fatalf("num val slices differ at output %d, expected: %v actual: %v", i, expectedNumVals, valsPushed)
}
}
}
func TestHashRouterCancellation(t *testing.T) {
defer leaktest.AfterTest(t)()
outputs := make([]routerOutput, 4)
numCancels := int64(0)
numAddBatches := int64(0)
for i := range outputs {
// We'll just be checking canceled.
outputs[i] = callbackRouterOutput{
addBatchCb: func(_ coldata.Batch, _ []uint16) bool {
atomic.AddInt64(&numAddBatches, 1)
return false
},
cancelCb: func() { atomic.AddInt64(&numCancels, 1) },
}
}
// Never-ending input of 0s.
batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64})
batch.SetLength(coldata.BatchSize())
in := NewRepeatableBatchSource(testAllocator, batch)
unbufferedCh := make(chan struct{})
r := newHashRouterWithOutputs(in, []coltypes.T{coltypes.Int64}, []uint32{0}, unbufferedCh, outputs)
t.Run("BeforeRun", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
cancel()
r.Run(ctx)
if numCancels != int64(len(outputs)) {
t.Fatalf("expected %d canceled outputs, actual %d", len(outputs), numCancels)
}
if numAddBatches != 0 {
t.Fatalf("detected %d addBatch calls but expected 0", numAddBatches)
}
meta := r.DrainMeta(ctx)
require.Equal(t, 1, len(meta))
require.True(t, testutils.IsError(meta[0].Err, "context canceled"), meta[0].Err)
})
testCases := []struct {
blocked bool
name string
}{
{
blocked: false,
name: "DuringRun",
},
{
blocked: true,
name: "WhileWaitingForUnblock",
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
numCancels = 0
numAddBatches = 0
ctx, cancel := context.WithCancel(context.Background())
if tc.blocked {
r.numBlockedOutputs = len(outputs)
defer func() {
r.numBlockedOutputs = 0
}()
}
routerMeta := make(chan []execinfrapb.ProducerMetadata)
go func() {
r.Run(ctx)
routerMeta <- r.DrainMeta(ctx)
close(routerMeta)
}()
time.Sleep(time.Millisecond)
if tc.blocked {
// Make sure no addBatches happened.
if n := atomic.LoadInt64(&numAddBatches); n != 0 {
t.Fatalf("expected router to be blocked, but detected %d addBatch calls", n)
}
}
select {
case <-routerMeta:
t.Fatal("hash router goroutine unexpectedly done")
default:
}
cancel()
meta := <-routerMeta
require.Equal(t, 1, len(meta))
require.True(t, testutils.IsError(meta[0].Err, "canceled"), meta[0].Err)
if numCancels != int64(len(outputs)) {
t.Fatalf("expected %d canceled outputs, actual %d", len(outputs), numCancels)
}
})
}
}
func TestHashRouterOneOutput(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
rng, _ := randutil.NewPseudoRand()
sel := randomSel(rng, coldata.BatchSize(), rng.Float64())
data, _ := getDataAndFullSelection()
typs := []coltypes.T{coltypes.Int64}
expected := make(tuples, 0, len(data))
for _, i := range sel {
expected = append(expected, data[i])
}
queueCfg, cleanup, memoryTestCases := getDiskQueueCfgAndMemoryTestCases(t, rng)
defer cleanup()
for _, mtc := range memoryTestCases {
t.Run(fmt.Sprintf("memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
// Clear the testAllocator for use.
testAllocator.Clear()
r, routerOutputs := NewHashRouter([]*Allocator{testAllocator}, newOpFixedSelTestInput(sel, uint16(len(sel)), data), typs, []uint32{0}, mtc.bytes, queueCfg)
if len(routerOutputs) != 1 {
t.Fatalf("expected 1 router output but got %d", len(routerOutputs))
}
o := newOpTestOutput(routerOutputs[0], expected)
ro := routerOutputs[0].(*routerOutputOp)
// Set alwaysFlush so that data is always flushed to the spillingQueue.
ro.testingKnobs.alwaysFlush = true
var wg sync.WaitGroup
wg.Add(1)
go func() {
r.Run(ctx)
wg.Done()
}()
if err := o.Verify(); err != nil {
t.Fatal(err)
}
wg.Wait()
// Expect no metadata, this should be a successful run.
unexpectedMetadata := r.DrainMeta(ctx)
if len(unexpectedMetadata) != 0 {
t.Fatalf("unexpected metadata when draining HashRouter: %+v", unexpectedMetadata)
}
if !mtc.skipExpSpillCheck {
// If len(sel) == 0, no items will have been enqueued so override an
// expected spill if this is the case.
mtc.expSpill = mtc.expSpill && len(sel) != 0
require.Equal(t, mtc.expSpill, ro.mu.data.spilled())
}
})
}
}
func TestHashRouterRandom(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
rng, _ := randutil.NewPseudoRand()
var (
maxValues = int(coldata.BatchSize()) * 4
maxOutputs = 128
blockedThreshold = 1 + rng.Intn(maxValues-1)
outputSize = 1 + rng.Intn(maxValues-1)
numOutputs = 1 + rng.Intn(maxOutputs-1)
)
typs := []coltypes.T{coltypes.Int64, coltypes.Int64}
dataLen := 1 + rng.Intn(maxValues-1)
data := make(tuples, dataLen)
for i := range data {
data[i] = make(tuple, len(typs))
for j := range typs {
data[i][j] = rng.Int63()
}
}
hashCols := make([]uint32, 0, len(typs))
hashCols = append(hashCols, 0)
for i := 1; i < cap(hashCols); i++ {
if rng.Float64() < 0.5 {
hashCols = append(hashCols, uint32(i))
}
}
// cancel determines whether we test cancellation.
cancel := false
if rng.Float64() < 0.25 {
cancel = true
}
testName := fmt.Sprintf(
"numOutputs=%d/blockedThreshold=%d/outputSize=%d/totalInputSize=%d/hashCols=%v/cancel=%t",
numOutputs,
blockedThreshold,
outputSize,
len(data),
hashCols,
cancel,
)
queueCfg, cleanup, memoryTestCases := getDiskQueueCfgAndMemoryTestCases(t, rng)
defer cleanup()
// expectedDistribution is set after the first run and used to verify that the
// distribution of results does not change between runs, as we are sending the
// same data to the same number of outputs.
var expectedDistribution []int
for _, mtc := range memoryTestCases {
t.Run(fmt.Sprintf(testName+"/memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) {
runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []Operator) {
unblockEventsChan := make(chan struct{}, 2*numOutputs)
outputs := make([]routerOutput, numOutputs)
outputsAsOps := make([]Operator, numOutputs)
memoryLimitPerOutput := mtc.bytes / int64(len(outputs))
for i := range outputs {
acc := testMemMonitor.MakeBoundAccount()
defer acc.Close(ctx)
// Create a separate allocator for each output as a single allocator
// may not be used concurrently.
allocator := NewAllocator(ctx, &acc)
op := newRouterOutputOpWithBlockedThresholdAndBatchSize(
allocator, typs, unblockEventsChan, memoryLimitPerOutput, queueCfg, blockedThreshold, outputSize,
)
outputs[i] = op
outputsAsOps[i] = op
}
r := newHashRouterWithOutputs(
inputs[0], typs, hashCols, unblockEventsChan, outputs,
)
var (
results uint64
wg sync.WaitGroup
)
resultsByOp := make([]int, len(outputsAsOps))
wg.Add(len(outputsAsOps))
for i := range outputsAsOps {
go func(i int) {
for {
b := outputsAsOps[i].Next(ctx)
if b.Length() == 0 {
break
}
atomic.AddUint64(&results, uint64(b.Length()))
resultsByOp[i] += int(b.Length())
}
wg.Done()
}(i)
}
ctx, cancelFunc := context.WithCancel(context.Background())
wg.Add(1)
go func() {
r.Run(ctx)
wg.Done()
}()
if cancel {
// Sleep between 0 and ~5 milliseconds.
time.Sleep(time.Microsecond * time.Duration(rng.Intn(5000)))
cancelFunc()
} else {
// Satisfy linter context leak error.
defer cancelFunc()
}
// Ensure all goroutines end. If a test fails with a hang here it is most
// likely due to a cancellation bug.
wg.Wait()
if !cancel {
// Expect no metadata, this should be a successful run.
unexpectedMetadata := r.DrainMeta(ctx)
if len(unexpectedMetadata) != 0 {
t.Fatalf("unexpected metadata when draining HashRouter: %+v", unexpectedMetadata)
}
// Only do output verification if no cancellation happened.
if actualTotal := atomic.LoadUint64(&results); actualTotal != uint64(len(data)) {
t.Fatalf("unexpected number of results %d, expected %d", actualTotal, len(data))
}
if expectedDistribution == nil {
expectedDistribution = resultsByOp
return
}
for i, numVals := range expectedDistribution {
if numVals != resultsByOp[i] {
t.Fatalf(
"distribution of results changed compared to first run at output %d. expected: %v, actual: %v",
i,
expectedDistribution,
resultsByOp,
)
}
}
}
})
})
}
}
func BenchmarkHashRouter(b *testing.B) {
defer leaktest.AfterTest(b)()
ctx := context.Background()
types := []coltypes.T{coltypes.Int64}
// Use only one type. Note: the more types you use, the more you inflate the
// numbers.
batch := testAllocator.NewMemBatch(types)
batch.SetLength(coldata.BatchSize())
input := NewRepeatableBatchSource(testAllocator, batch)
queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, true /* inMem */)
defer cleanup()
var wg sync.WaitGroup
for _, numOutputs := range []int{2, 4, 8, 16} {
for _, numInputBatches := range []int{2, 4, 8, 16} {
b.Run(fmt.Sprintf("numOutputs=%d/numInputBatches=%d", numOutputs, numInputBatches), func(b *testing.B) {
allocators := make([]*Allocator, numOutputs)
for i := range allocators {
acc := testMemMonitor.MakeBoundAccount()
allocators[i] = NewAllocator(ctx, &acc)
defer acc.Close(ctx)
}
r, outputs := NewHashRouter(allocators, input, types, []uint32{0}, 64<<20 /* 64 MiB */, queueCfg)
b.SetBytes(8 * int64(coldata.BatchSize()) * int64(numInputBatches))
// We expect distribution to not change. This is a sanity check that
// we're resetting properly.
var expectedDistribution []int
actualDistribution := make([]int, len(outputs))
// zeroDistribution just allows us to reset actualDistribution with a
// copy.
zeroDistribution := make([]int, len(outputs))
b.ResetTimer()
for i := 0; i < b.N; i++ {
input.ResetBatchesToReturn(numInputBatches)
r.reset()
wg.Add(len(outputs))
for j := range outputs {
go func(j int) {
for {
oBatch := outputs[j].Next(ctx)
actualDistribution[j] += int(oBatch.Length())
if oBatch.Length() == 0 {
break
}
}
wg.Done()
}(j)
}
r.Run(ctx)
wg.Wait()
// sum sanity checks that we are actually pushing as many values as we
// expect.
sum := 0
for i := range actualDistribution {
sum += actualDistribution[i]
}
if sum != numInputBatches*int(coldata.BatchSize()) {
b.Fatalf("unexpected sum %d, expected %d", sum, numInputBatches*int(coldata.BatchSize()))
}
if expectedDistribution == nil {
expectedDistribution = make([]int, len(actualDistribution))
copy(expectedDistribution, actualDistribution)
} else {
for j := range expectedDistribution {
if expectedDistribution[j] != actualDistribution[j] {
b.Fatalf(
"not resetting properly expected distribution: %v, actual distribution: %v",
expectedDistribution,
actualDistribution,
)
}
}
}
copy(actualDistribution, zeroDistribution)
}
})
}
}
}