UnboundedSource<T, ?> adaptedRawSource = new BoundedToUnboundedSourceAdapter<>(rawSource); DataStream<WindowedValue<T>> source; try {
private <T> void testBoundedToUnboundedSourceAdapterCheckpoint( BoundedSource<T> boundedSource, List<T> expectedElements) throws Exception { BoundedToUnboundedSourceAdapter<T> unboundedSource = new BoundedToUnboundedSourceAdapter<>(boundedSource); PipelineOptions options = PipelineOptionsFactory.create(); BoundedToUnboundedSourceAdapter<T>.Reader reader = unboundedSource.createReader(options, null); List<T> actual = Lists.newArrayList(); for (boolean hasNext = reader.start(); hasNext; hasNext = reader.advance()) { actual.add(reader.getCurrent()); // checkpoint every 9 elements if (actual.size() % 9 == 0) { Checkpoint<T> checkpoint = reader.getCheckpointMark(); checkpoint.finalizeCheckpoint(); } } Checkpoint<T> checkpointDone = reader.getCheckpointMark(); assertTrue( checkpointDone.getResidualElements() == null || checkpointDone.getResidualElements().isEmpty()); assertEquals(expectedElements.size(), actual.size()); assertEquals(Sets.newHashSet(expectedElements), Sets.newHashSet(actual)); }
new BoundedToUnboundedSourceAdapter<>(source); UnboundedSourceWrapper<Long, Checkpoint<Long>> flinkWrapper = new UnboundedSourceWrapper<>("stepName", options, unboundedSource, numSplits); new BoundedToUnboundedSourceAdapter<>(restoredSource); UnboundedSourceWrapper<Long, Checkpoint<Long>> restoredFlinkWrapper = new UnboundedSourceWrapper<>("stepName", options, restoredUnboundedSource, numSplits);
new BoundedToUnboundedSourceAdapter<>(source); UnboundedSourceWrapper<Long, Checkpoint<Long>> flinkWrapper = new UnboundedSourceWrapper<>("stepName", options, unboundedSource, numSplits); new BoundedToUnboundedSourceAdapter<>(restoredSource); UnboundedSourceWrapper<Long, Checkpoint<Long>> restoredFlinkWrapper = new UnboundedSourceWrapper<>("stepName", options, restoredUnboundedSource, numSplits);
UnboundedSource<T, ?> adaptedRawSource = new BoundedToUnboundedSourceAdapter<>(rawSource); DataStream<WindowedValue<T>> source; try {
private <T> void testBoundedToUnboundedSourceAdapterCheckpointRestart( BoundedSource<T> boundedSource, List<T> expectedElements) throws Exception { BoundedToUnboundedSourceAdapter<T> unboundedSource = new BoundedToUnboundedSourceAdapter<>(boundedSource); BoundedToUnboundedSourceAdapter<T>.Reader reader = unboundedSource.createReader(options, null); Coder<Checkpoint<T>> checkpointCoder = unboundedSource.getCheckpointMarkCoder(); Checkpoint<T> decodedCheckpoint = CoderUtils.decodeFromByteArray( unboundedSource.createReader(options, decodedCheckpoint); reader = restarted; hasNext = reader.start();
@Override public List<BoundedToUnboundedSourceAdapter<T>> split( int desiredNumSplits, PipelineOptions options) throws Exception { try { long desiredBundleSize = boundedSource.getEstimatedSizeBytes(options) / desiredNumSplits; if (desiredBundleSize <= 0) { LOG.warn( "BoundedSource {} cannot estimate its size, skips the initial splits.", boundedSource); return ImmutableList.of(this); } List<? extends BoundedSource<T>> splits = boundedSource.split(desiredBundleSize, options); return splits .stream() .map(input -> new BoundedToUnboundedSourceAdapter<>(input)) .collect(Collectors.toList()); } catch (Exception e) { LOG.warn("Exception while splitting {}, skips the initial splits.", boundedSource, e); return ImmutableList.of(this); } }
@Test @Category(NeedsRunner.class) public void testBoundedToUnboundedSourceAdapter() throws Exception { long numElements = 100; BoundedSource<Long> boundedSource = CountingSource.upTo(numElements); UnboundedSource<Long, Checkpoint<Long>> unboundedSource = new BoundedToUnboundedSourceAdapter<>(boundedSource); PCollection<Long> output = p.apply(Read.from(unboundedSource).withMaxNumRecords(numElements)); // Count == numElements PAssert.thatSingleton(output.apply("Count", Count.globally())).isEqualTo(numElements); // Unique count == numElements PAssert.thatSingleton(output.apply(Distinct.create()).apply("UniqueCount", Count.globally())) .isEqualTo(numElements); // Min == 0 PAssert.thatSingleton(output.apply("Min", Min.globally())).isEqualTo(0L); // Max == numElements-1 PAssert.thatSingleton(output.apply("Max", Max.globally())).isEqualTo(numElements - 1); p.run(); }
@Override public PCollection<T> expand(PBegin input) { return input.getPipeline().apply(Read.from(new BoundedToUnboundedSourceAdapter<>(source))); }
@Override public void translate(Read.Bounded<T> transform, TranslationContext context) { // TODO: adapter is visibleForTesting BoundedToUnboundedSourceAdapter unboundedSource = new BoundedToUnboundedSourceAdapter<>(transform.getSource()); ApexReadUnboundedInputOperator<T, ?> operator = new ApexReadUnboundedInputOperator<>(unboundedSource, true, context.getPipelineOptions()); context.addOperator(operator, operator.output); } }
@Test public void testReadFromCheckpointBeforeStart() throws Exception { thrown.expect(NoSuchElementException.class); BoundedSource<Long> countingSource = CountingSource.upTo(100); BoundedToUnboundedSourceAdapter<Long> unboundedSource = new BoundedToUnboundedSourceAdapter<>(countingSource); PipelineOptions options = PipelineOptionsFactory.create(); List<TimestampedValue<Long>> elements = ImmutableList.of(TimestampedValue.of(1L, new Instant(1L))); Checkpoint<Long> checkpoint = new Checkpoint<>(elements, countingSource); unboundedSource.createReader(options, checkpoint).getCurrent(); }
@Test public void testReadBeforeStart() throws Exception { thrown.expect(NoSuchElementException.class); BoundedSource<Long> countingSource = CountingSource.upTo(100); BoundedToUnboundedSourceAdapter<Long> unboundedSource = new BoundedToUnboundedSourceAdapter<>(countingSource); PipelineOptions options = PipelineOptionsFactory.create(); unboundedSource.createReader(options, null).getCurrent(); }