forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
UnboundedSourceWrapperTest.java
827 lines (691 loc) · 33.9 KB
/
UnboundedSourceWrapperTest.java
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
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
import java.util.stream.LongStream;
import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.flink.streaming.StreamSources;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.io.CountingSource;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.ValueWithRecordId;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.OutputTag;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.junit.runners.Parameterized;
import org.mockito.Mockito;
import org.powermock.reflect.Whitebox;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Tests for {@link UnboundedSourceWrapper}. */
@RunWith(Enclosed.class)
public class UnboundedSourceWrapperTest {
private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceWrapperTest.class);
/** Parameterized tests. */
@RunWith(Parameterized.class)
public static class ParameterizedUnboundedSourceWrapperTest {
private final int numTasks;
private final int numSplits;
public ParameterizedUnboundedSourceWrapperTest(int numTasks, int numSplits) {
this.numTasks = numTasks;
this.numSplits = numSplits;
}
@Parameterized.Parameters(name = "numTasks = {0}; numSplits={1}")
public static Collection<Object[]> data() {
/*
* Parameters for initializing the tests:
* {numTasks, numSplits}
* The test currently assumes powers of two for some assertions.
*/
return Arrays.asList(
new Object[][] {
{1, 1}, {1, 2}, {1, 4},
{2, 1}, {2, 2}, {2, 4},
{4, 1}, {4, 2}, {4, 4}
});
}
/**
* Creates a {@link UnboundedSourceWrapper} that has one or multiple readers per source. If
* numSplits > numTasks the source has one source will manage multiple readers.
*/
@Test(timeout = 30_000)
public void testValueEmission() throws Exception {
final int numElementsPerShard = 20;
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
options.setShutdownSourcesOnFinalWatermark(true);
final long[] numElementsReceived = {0L};
final int[] numWatermarksReceived = {0};
// this source will emit exactly NUM_ELEMENTS for each parallel reader,
// afterwards it will stall. We check whether we also receive NUM_ELEMENTS
// elements later.
TestCountingSource source =
new TestCountingSource(numElementsPerShard).withFixedNumSplits(numSplits);
for (int subtaskIndex = 0; subtaskIndex < numTasks; subtaskIndex++) {
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, source, numTasks);
// the source wrapper will only request as many splits as there are tasks and the source
// will create at most numSplits splits
assertEquals(numSplits, flinkWrapper.getSplitSources().size());
StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
sourceOperator = new StreamSource<>(flinkWrapper);
AbstractStreamOperatorTestHarness<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
testHarness =
new AbstractStreamOperatorTestHarness<>(
sourceOperator,
numTasks /* max parallelism */,
numTasks /* parallelism */,
subtaskIndex /* subtask index */);
// The testing timer service is synchronous, so we must configure a watermark interval
// > 0, otherwise we can get loop infinitely due to a timer always becoming ready after
// it has been set.
testHarness.getExecutionConfig().setAutoWatermarkInterval(10L);
testHarness.setProcessingTime(System.currentTimeMillis());
testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
// start a thread that advances processing time, so that we eventually get the final
// watermark which is only updated via a processing-time trigger
Thread processingTimeUpdateThread =
new Thread() {
@Override
public void run() {
while (true) {
try {
// Need to advance this so that the watermark timers in the source wrapper fire
// Synchronize is necessary because this can interfere with updating the
// PriorityQueue of the ProcessingTimeService which is accessed when setting
// timers in UnboundedSourceWrapper.
synchronized (testHarness.getCheckpointLock()) {
testHarness.setProcessingTime(System.currentTimeMillis());
}
Thread.sleep(100);
} catch (InterruptedException e) {
// this is ok
break;
} catch (Exception e) {
LOG.error("Unexpected error advancing processing time", e);
break;
}
}
}
};
processingTimeUpdateThread.start();
try {
testHarness.open();
StreamSources.run(
sourceOperator,
testHarness.getCheckpointLock(),
new TestStreamStatusMaintainer(),
new Output<StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>>() {
private boolean hasSeenMaxWatermark = false;
@Override
public void emitWatermark(Watermark watermark) {
// we get this when there is no more data
// it can happen that we get the max watermark several times, so guard against
// this
if (!hasSeenMaxWatermark
&& watermark.getTimestamp()
>= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
numWatermarksReceived[0]++;
hasSeenMaxWatermark = true;
}
}
@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> streamRecord) {
collect((StreamRecord) streamRecord);
}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {}
@Override
public void collect(
StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
windowedValueStreamRecord) {
numElementsReceived[0]++;
}
@Override
public void close() {}
});
} finally {
processingTimeUpdateThread.interrupt();
processingTimeUpdateThread.join();
}
}
// verify that we get the expected count across all subtasks
assertEquals(numElementsPerShard * numSplits, numElementsReceived[0]);
// and that we get as many final watermarks as there are subtasks
assertEquals(numTasks, numWatermarksReceived[0]);
}
/**
* Creates a {@link UnboundedSourceWrapper} that has one or multiple readers per source. If
* numSplits > numTasks the source will manage multiple readers.
*
* <p>This test verifies that watermarks are correctly forwarded.
*/
@Test(timeout = 30_000)
@Ignore("https://issues.apache.org/jira/browse/BEAM-9164")
public void testWatermarkEmission() throws Exception {
final int numElements = 500;
PipelineOptions options = PipelineOptionsFactory.create();
// this source will emit exactly NUM_ELEMENTS across all parallel readers,
// afterwards it will stall. We check whether we also receive NUM_ELEMENTS
// elements later.
TestCountingSource source = new TestCountingSource(numElements);
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, source, numSplits);
assertEquals(numSplits, flinkWrapper.getSplitSources().size());
final StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
sourceOperator = new StreamSource<>(flinkWrapper);
final AbstractStreamOperatorTestHarness<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
testHarness =
new AbstractStreamOperatorTestHarness<>(
sourceOperator,
numTasks /* max parallelism */,
numTasks /* parallelism */,
0 /* subtask index */);
testHarness.getExecutionConfig().setLatencyTrackingInterval(0);
testHarness.getExecutionConfig().setAutoWatermarkInterval(1);
testHarness.setProcessingTime(Long.MIN_VALUE);
testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
final ConcurrentLinkedQueue<Object> caughtExceptions = new ConcurrentLinkedQueue<>();
// We test emission of two watermarks here, one intermediate, one final
final CountDownLatch seenWatermarks = new CountDownLatch(2);
final int minElementsPerReader = numElements / numSplits;
final CountDownLatch minElementsCountdown = new CountDownLatch(minElementsPerReader);
// first halt the source to test auto watermark emission
source.haltEmission();
testHarness.open();
Thread sourceThread =
new Thread(
() -> {
try {
StreamSources.run(
sourceOperator,
testHarness.getCheckpointLock(),
new TestStreamStatusMaintainer(),
new Output<
StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>>() {
@Override
public void emitWatermark(Watermark watermark) {
seenWatermarks.countDown();
}
@Override
public <X> void collect(
OutputTag<X> outputTag, StreamRecord<X> streamRecord) {}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {}
@Override
public void collect(
StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
windowedValueStreamRecord) {
minElementsCountdown.countDown();
}
@Override
public void close() {}
});
} catch (Exception e) {
LOG.info("Caught exception:", e);
caughtExceptions.add(e);
}
});
sourceThread.start();
while (flinkWrapper.getLocalReaders().stream()
.anyMatch(reader -> reader.getWatermark().getMillis() == 0)) {
// readers haven't been initialized
Thread.sleep(50);
}
// Need to advance this so that the watermark timers in the source wrapper fire
// Synchronize is necessary because this can interfere with updating the PriorityQueue
// of the ProcessingTimeService which is also accessed through UnboundedSourceWrapper.
synchronized (testHarness.getCheckpointLock()) {
testHarness.setProcessingTime(0);
}
// now read the elements
source.continueEmission();
// ..and await elements
minElementsCountdown.await();
// Need to advance this so that the watermark timers in the source wrapper fire
// Synchronize is necessary because this can interfere with updating the PriorityQueue
// of the ProcessingTimeService which is also accessed through UnboundedSourceWrapper.
synchronized (testHarness.getCheckpointLock()) {
testHarness.setProcessingTime(Long.MAX_VALUE);
}
seenWatermarks.await();
if (!caughtExceptions.isEmpty()) {
fail("Caught exception(s): " + Joiner.on(",").join(caughtExceptions));
}
sourceOperator.cancel();
sourceThread.join();
}
/**
* Verify that snapshot/restore work as expected. We bring up a source and cancel after seeing a
* certain number of elements. Then we snapshot that source, bring up a completely new source
* that we restore from the snapshot and verify that we see all expected elements in the end.
*/
@Test
public void testRestore() throws Exception {
final int numElements = 20;
final Object checkpointLock = new Object();
PipelineOptions options = PipelineOptionsFactory.create();
// this source will emit exactly NUM_ELEMENTS across all parallel readers,
// afterwards it will stall. We check whether we also receive NUM_ELEMENTS
// elements later.
TestCountingSource source = new TestCountingSource(numElements);
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, source, numSplits);
assertEquals(numSplits, flinkWrapper.getSplitSources().size());
StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
sourceOperator = new StreamSource<>(flinkWrapper);
AbstractStreamOperatorTestHarness<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
testHarness =
new AbstractStreamOperatorTestHarness<>(
sourceOperator,
numTasks /* max parallelism */,
numTasks /* parallelism */,
0 /* subtask index */);
testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
final Set<KV<Integer, Integer>> emittedElements = new HashSet<>();
boolean readFirstBatchOfElements = false;
try {
testHarness.open();
StreamSources.run(
sourceOperator,
checkpointLock,
new TestStreamStatusMaintainer(),
new Output<StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>>() {
private int count = 0;
@Override
public void emitWatermark(Watermark watermark) {}
@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> streamRecord) {
collect((StreamRecord) streamRecord);
}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {}
@Override
public void collect(
StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
windowedValueStreamRecord) {
emittedElements.add(windowedValueStreamRecord.getValue().getValue().getValue());
count++;
if (count >= numElements / 2) {
throw new SuccessException();
}
}
@Override
public void close() {}
});
} catch (SuccessException e) {
// success
readFirstBatchOfElements = true;
}
assertTrue("Did not successfully read first batch of elements.", readFirstBatchOfElements);
// draw a snapshot
OperatorSubtaskState snapshot = testHarness.snapshot(0, 0);
// test that finalizeCheckpoint on CheckpointMark is called
final ArrayList<Integer> finalizeList = new ArrayList<>();
TestCountingSource.setFinalizeTracker(finalizeList);
testHarness.notifyOfCompletedCheckpoint(0);
assertEquals(flinkWrapper.getLocalSplitSources().size(), finalizeList.size());
// create a completely new source but restore from the snapshot
TestCountingSource restoredSource = new TestCountingSource(numElements);
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>
restoredFlinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, restoredSource, numSplits);
assertEquals(numSplits, restoredFlinkWrapper.getSplitSources().size());
StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
restoredSourceOperator = new StreamSource<>(restoredFlinkWrapper);
// set parallelism to 1 to ensure that our testing operator gets all checkpointed state
AbstractStreamOperatorTestHarness<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
restoredTestHarness =
new AbstractStreamOperatorTestHarness<>(
restoredSourceOperator,
numTasks /* max parallelism */,
1 /* parallelism */,
0 /* subtask index */);
restoredTestHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
// restore snapshot
restoredTestHarness.initializeState(snapshot);
boolean readSecondBatchOfElements = false;
// run again and verify that we see the other elements
try {
restoredTestHarness.open();
StreamSources.run(
restoredSourceOperator,
checkpointLock,
new TestStreamStatusMaintainer(),
new Output<StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>>() {
private int count = 0;
@Override
public void emitWatermark(Watermark watermark) {}
@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> streamRecord) {
collect((StreamRecord) streamRecord);
}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {}
@Override
public void collect(
StreamRecord<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
windowedValueStreamRecord) {
emittedElements.add(windowedValueStreamRecord.getValue().getValue().getValue());
count++;
if (count >= numElements / 2) {
throw new SuccessException();
}
}
@Override
public void close() {}
});
} catch (SuccessException e) {
// success
readSecondBatchOfElements = true;
}
assertEquals(
Math.max(1, numSplits / numTasks), restoredFlinkWrapper.getLocalSplitSources().size());
assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements);
// verify that we saw all NUM_ELEMENTS elements
assertTrue(emittedElements.size() == numElements);
}
@Test
public void testNullCheckpoint() throws Exception {
final int numElements = 20;
PipelineOptions options = PipelineOptionsFactory.create();
TestCountingSource source =
new TestCountingSource(numElements) {
@Override
public Coder<CounterMark> getCheckpointMarkCoder() {
return null;
}
};
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, source, numSplits);
StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
sourceOperator = new StreamSource<>(flinkWrapper);
AbstractStreamOperatorTestHarness<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
testHarness =
new AbstractStreamOperatorTestHarness<>(
sourceOperator,
numTasks /* max parallelism */,
numTasks /* parallelism */,
0 /* subtask index */);
testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
testHarness.open();
OperatorSubtaskState snapshot = testHarness.snapshot(0, 0);
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>
restoredFlinkWrapper =
new UnboundedSourceWrapper<>(
"stepName", options, new TestCountingSource(numElements), numSplits);
StreamSource<
WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>,
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark>>
restoredSourceOperator = new StreamSource<>(restoredFlinkWrapper);
// set parallelism to 1 to ensure that our testing operator gets all checkpointed state
AbstractStreamOperatorTestHarness<WindowedValue<ValueWithRecordId<KV<Integer, Integer>>>>
restoredTestHarness =
new AbstractStreamOperatorTestHarness<>(
restoredSourceOperator,
numTasks /* max parallelism */,
1 /* parallelism */,
0 /* subtask index */);
restoredTestHarness.setup();
restoredTestHarness.initializeState(snapshot);
restoredTestHarness.open();
// when the source checkpointed a null we don't re-initialize the splits, that is we
// will have no splits.
assertEquals(0, restoredFlinkWrapper.getLocalSplitSources().size());
}
/** A special {@link RuntimeException} that we throw to signal that the test was successful. */
private static class SuccessException extends RuntimeException {}
}
/** Not parameterized tests. */
@RunWith(JUnit4.class)
public static class BasicTest {
/** Check serialization a {@link UnboundedSourceWrapper}. */
@Test
public void testSerialization() throws Exception {
final int parallelism = 1;
final int numElements = 20;
PipelineOptions options = PipelineOptionsFactory.create();
TestCountingSource source = new TestCountingSource(numElements);
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
new UnboundedSourceWrapper<>("stepName", options, source, parallelism);
InstantiationUtil.serializeObject(flinkWrapper);
}
@Test(timeout = 10_000)
public void testSourceWithNoReaderDoesNotShutdown() throws Exception {
testSourceDoesNotShutdown(false);
}
@Test(timeout = 10_000)
public void testSourceWithReadersDoesNotShutdown() throws Exception {
testSourceDoesNotShutdown(true);
}
private static void testSourceDoesNotShutdown(boolean shouldHaveReaders) throws Exception {
final int parallelism = 2;
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
TestCountingSource source = new TestCountingSource(20).withoutSplitting();
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> sourceWrapper =
new UnboundedSourceWrapper<>("noReader", options, source, parallelism);
StreamingRuntimeContext mock = Mockito.mock(StreamingRuntimeContext.class);
if (shouldHaveReaders) {
// Since the source can't be split, the first subtask index will read everything
Mockito.when(mock.getIndexOfThisSubtask()).thenReturn(0);
} else {
// Set up the RuntimeContext such that this instance won't receive any readers
Mockito.when(mock.getIndexOfThisSubtask()).thenReturn(parallelism - 1);
}
Mockito.when(mock.getNumberOfParallelSubtasks()).thenReturn(parallelism);
Mockito.when(mock.getExecutionConfig()).thenReturn(new ExecutionConfig());
ProcessingTimeService timerService = Mockito.mock(ProcessingTimeService.class);
Mockito.when(timerService.getCurrentProcessingTime()).thenReturn(Long.MAX_VALUE);
Mockito.when(mock.getProcessingTimeService()).thenReturn(timerService);
sourceWrapper.setRuntimeContext(mock);
sourceWrapper.open(new Configuration());
SourceFunction.SourceContext sourceContext = Mockito.mock(SourceFunction.SourceContext.class);
Object checkpointLock = new Object();
Mockito.when(sourceContext.getCheckpointLock()).thenReturn(checkpointLock);
// Initialize source context early to avoid concurrency issues with its initialization in the
// run
// method and the onProcessingTime call on the wrapper.
sourceWrapper.setSourceContext(sourceContext);
sourceWrapper.open(new Configuration());
assertThat(sourceWrapper.getLocalReaders().isEmpty(), is(!shouldHaveReaders));
Thread thread =
new Thread(
() -> {
try {
sourceWrapper.run(sourceContext);
} catch (Exception e) {
LOG.error("Error while running UnboundedSourceWrapper", e);
}
});
try {
thread.start();
// Wait to see if the wrapper shuts down immediately in case it doesn't have readers
if (!shouldHaveReaders) {
// The expected state is for finalizeSource to sleep instead of exiting
while (true) {
StackTraceElement[] callStack = thread.getStackTrace();
if (callStack.length >= 2
&& "sleep".equals(callStack[0].getMethodName())
&& "finalizeSource".equals(callStack[1].getMethodName())) {
break;
}
Thread.sleep(10);
}
}
// Source should still be running even if there are no readers
assertThat(sourceWrapper.isRunning(), is(true));
synchronized (checkpointLock) {
// Trigger emission of the watermark by updating processing time.
// The actual processing time value does not matter.
sourceWrapper.onProcessingTime(42);
}
// Source should still be running even when watermark is at max
assertThat(sourceWrapper.isRunning(), is(true));
assertThat(thread.isAlive(), is(true));
sourceWrapper.cancel();
} finally {
thread.interrupt();
// try to join but also don't mask exceptions with test timeout
thread.join(1000);
}
assertThat(thread.isAlive(), is(false));
}
@Test
public void testSequentialReadingFromBoundedSource() throws Exception {
UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter<Long> source =
new UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter<>(
CountingSource.upTo(1000));
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
options.setShutdownSourcesOnFinalWatermark(true);
UnboundedSourceWrapper<
Long, UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint<Long>>
sourceWrapper = new UnboundedSourceWrapper<>("sequentialRead", options, source, 4);
StreamingRuntimeContext runtimeContextMock = Mockito.mock(StreamingRuntimeContext.class);
Mockito.when(runtimeContextMock.getIndexOfThisSubtask()).thenReturn(0);
when(runtimeContextMock.getNumberOfParallelSubtasks()).thenReturn(2);
when(runtimeContextMock.getExecutionConfig()).thenReturn(new ExecutionConfig());
TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
processingTimeService.setCurrentTime(0);
when(runtimeContextMock.getProcessingTimeService()).thenReturn(processingTimeService);
when(runtimeContextMock.getMetricGroup()).thenReturn(new UnregisteredMetricsGroup());
sourceWrapper.setRuntimeContext(runtimeContextMock);
sourceWrapper.open(new Configuration());
assertThat(sourceWrapper.getLocalReaders().size(), is(2));
List<Long> integers = new ArrayList<>();
sourceWrapper.run(
new SourceFunction.SourceContext<WindowedValue<ValueWithRecordId<Long>>>() {
private final Object checkpointLock = new Object();
@Override
public void collect(WindowedValue<ValueWithRecordId<Long>> element) {
integers.add(element.getValue().getValue());
}
@Override
public void collectWithTimestamp(
WindowedValue<ValueWithRecordId<Long>> element, long timestamp) {
throw new IllegalStateException("Should not collect with timestamp");
}
@Override
public void emitWatermark(Watermark mark) {}
@Override
public void markAsTemporarilyIdle() {}
@Override
public Object getCheckpointLock() {
return checkpointLock;
}
@Override
public void close() {}
});
// The source is effectively split into two parts: The initial splitting is performed with a
// parallelism of 4, but there are 2 parallel subtasks. This instances taskes 2 out of 4
// partitions.
assertThat(integers.size(), is(500));
assertThat(
integers,
contains(
LongStream.concat(LongStream.range(0, 250), LongStream.range(500, 750))
.boxed()
.toArray()));
}
@Test
public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
TestCountingSource source = new TestCountingSource(20).withoutSplitting();
UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> sourceWrapper =
new UnboundedSourceWrapper<>("noReader", options, source, 2);
StreamingRuntimeContext mock = Mockito.mock(StreamingRuntimeContext.class);
Mockito.when(mock.getNumberOfParallelSubtasks()).thenReturn(1);
Mockito.when(mock.getExecutionConfig()).thenReturn(new ExecutionConfig());
Mockito.when(mock.getIndexOfThisSubtask()).thenReturn(0);
sourceWrapper.setRuntimeContext(mock);
sourceWrapper.open(new Configuration());
String metricContainerFieldName = "metricContainer";
FlinkMetricContainer monitoredContainer =
Mockito.spy(
(FlinkMetricContainer)
Whitebox.getInternalState(sourceWrapper, metricContainerFieldName));
Whitebox.setInternalState(sourceWrapper, metricContainerFieldName, monitoredContainer);
sourceWrapper.close();
Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
}
}
private static final class TestStreamStatusMaintainer implements StreamStatusMaintainer {
StreamStatus currentStreamStatus = StreamStatus.ACTIVE;
@Override
public void toggleStreamStatus(StreamStatus streamStatus) {
if (!currentStreamStatus.equals(streamStatus)) {
currentStreamStatus = streamStatus;
}
}
@Override
public StreamStatus getStreamStatus() {
return currentStreamStatus;
}
}
}