try { List<T> items = readNItemsFromUnstartedReader(reader, numItemsToReadBeforeSplitting); unblockSplitter.countDown(); items.addAll(readRemainingFromReader(reader, numItemsToReadBeforeSplitting > 0));
try (BoundedSource.BoundedReader<T> reader = source.createReader(options)) { BoundedSource<T> originalSource = reader.getCurrentSource(); List<T> currentItems = readNItemsFromUnstartedReader(reader, numItemsToReadBeforeSplit); BoundedSource<T> residual = reader.splitAtFraction(splitFraction); if (residual != null) {
@Test public void generatesInitialSplits() throws Exception { when(context.createRootBundle()).thenAnswer(invocation -> bundleFactory.createRootBundle()); int numSplits = 5; Collection<CommittedBundle<?>> initialInputs = new UnboundedReadEvaluatorFactory.InputProvider(context, options) .getInitialInputs(graph.getProducer(longs), numSplits); // CountingSource.unbounded has very good splitting behavior assertThat(initialInputs, hasSize(numSplits)); int readPerSplit = 100; int totalSize = numSplits * readPerSplit; Set<Long> expectedOutputs = ContiguousSet.create(Range.closedOpen(0L, (long) totalSize), DiscreteDomain.longs()); Collection<Long> readItems = new ArrayList<>(totalSize); for (CommittedBundle<?> initialInput : initialInputs) { CommittedBundle<UnboundedSourceShard<Long, ?>> shardBundle = (CommittedBundle<UnboundedSourceShard<Long, ?>>) initialInput; WindowedValue<UnboundedSourceShard<Long, ?>> shard = Iterables.getOnlyElement(shardBundle.getElements()); assertThat(shard.getTimestamp(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); assertThat(shard.getWindows(), Matchers.contains(GlobalWindow.INSTANCE)); UnboundedSource<Long, ?> shardSource = shard.getValue().getSource(); readItems.addAll( SourceTestUtils.readNItemsFromUnstartedReader( shardSource.createReader( PipelineOptionsFactory.create(), null /* No starting checkpoint */), readPerSplit)); } assertThat(readItems, containsInAnyOrder(expectedOutputs.toArray(new Long[0]))); }
@Test public void testToUnsplittableSource() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); BoundedSource<Long> baseSource = CountingSource.upTo(100); BoundedSource<Long> unsplittableSource = SourceTestUtils.toUnsplittableSource(baseSource); List<?> splits = unsplittableSource.split(1, options); assertEquals(1, splits.size()); assertEquals(unsplittableSource, splits.get(0)); BoundedReader<Long> unsplittableReader = unsplittableSource.createReader(options); assertEquals(0, unsplittableReader.getFractionConsumed(), 1e-15); Set<Long> expected = Sets.newHashSet(SourceTestUtils.readFromSource(baseSource, options)); Set<Long> actual = Sets.newHashSet(); actual.addAll(SourceTestUtils.readNItemsFromUnstartedReader(unsplittableReader, 40)); assertNull(unsplittableReader.splitAtFraction(0.5)); actual.addAll(SourceTestUtils.readRemainingFromReader(unsplittableReader, true /* started */)); assertEquals(1, unsplittableReader.getFractionConsumed(), 1e-15); assertEquals(100, actual.size()); assertEquals(Sets.newHashSet(expected), Sets.newHashSet(actual)); } }