@Override public boolean isRunnerDeterminedSharding() { return transform.getNumShardsProvider() == null && transform.getComputeNumShards() == null; } },
@Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .add(DisplayData.item("sink", getSink().getClass()).withLabel("WriteFiles Sink")) .include("sink", getSink()); if (getComputeNumShards() != null) { builder.include("sharding", getComputeNumShards()); } else { builder.addIfNotNull( DisplayData.item("numShards", getNumShardsProvider()) .withLabel("Fixed Number of Shards")); } }
@Test public void testBuildWrite() { SimpleSink<Void> sink = makeSimpleSink(); WriteFiles<String, ?, String> write = WriteFiles.to(sink).withNumShards(3); assertThat((SimpleSink<Void>) write.getSink(), is(sink)); PTransform<PCollection<String>, PCollectionView<Integer>> originalSharding = write.getComputeNumShards(); assertThat(write.getComputeNumShards(), is(nullValue())); assertThat(write.getNumShardsProvider(), instanceOf(StaticValueProvider.class)); assertThat(write.getNumShardsProvider().get(), equalTo(3)); assertThat(write.getComputeNumShards(), equalTo(originalSharding)); WriteFiles<String, ?, ?> write2 = write.withSharding(SHARDING_TRANSFORM); assertThat((SimpleSink<Void>) write2.getSink(), is(sink)); assertThat(write2.getComputeNumShards(), equalTo(SHARDING_TRANSFORM)); // original unchanged WriteFiles<String, ?, ?> writeUnsharded = write2.withRunnerDeterminedSharding(); assertThat(writeUnsharded.getComputeNumShards(), nullValue()); assertThat(write.getComputeNumShards(), equalTo(originalSharding)); }
@Test public void testEncodedProto() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.WriteFilesPayload payload = WriteFilesTranslation.payloadForWriteFiles(writeFiles, components); assertThat( payload.getRunnerDeterminedSharding(), equalTo( writeFiles.getNumShardsProvider() == null && writeFiles.getComputeNumShards() == null)); assertThat(payload.getWindowedWrites(), equalTo(writeFiles.getWindowedWrites())); assertThat( (FileBasedSink<String, Void, String>) WriteFilesTranslation.sinkFromProto(payload.getSink()), equalTo(writeFiles.getSink())); }
@Test public void testExtractionDirectFromTransform() throws Exception { PCollection<String> input = p.apply(Create.of("hello")); WriteFilesResult<Void> output = input.apply(writeFiles); AppliedPTransform<PCollection<String>, WriteFilesResult<Void>, WriteFiles<String, Void, String>> appliedPTransform = AppliedPTransform.of("foo", input.expand(), output.expand(), writeFiles, p); assertThat( WriteFilesTranslation.isRunnerDeterminedSharding(appliedPTransform), equalTo( writeFiles.getNumShardsProvider() == null && writeFiles.getComputeNumShards() == null)); assertThat( WriteFilesTranslation.isWindowedWrites(appliedPTransform), equalTo(writeFiles.getWindowedWrites())); assertThat( WriteFilesTranslation.<String, Void, String>getSink(appliedPTransform), equalTo(writeFiles.getSink())); }
@Test public void testRunnerDeterminedSharding() { FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); options.setRunner(TestFlinkRunner.class); options.setFlinkMaster("[auto]"); options.setParallelism(5); TestPipeline p = TestPipeline.fromOptions(options); StreamingShardedWriteFactory<Object, Void, Object> factory = new StreamingShardedWriteFactory<>(p.getOptions()); WriteFiles<Object, Void, Object> original = WriteFiles.to(new TestSink(tmpFolder.toString())); @SuppressWarnings("unchecked") PCollection<Object> objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); AppliedPTransform<PCollection<Object>, WriteFilesResult<Void>, WriteFiles<Object, Void, Object>> originalApplication = AppliedPTransform.of("writefiles", objs.expand(), Collections.emptyMap(), original, p); WriteFiles<Object, Void, Object> replacement = (WriteFiles<Object, Void, Object>) factory.getReplacementTransform(originalApplication).getTransform(); assertThat(replacement, not(equalTo((Object) original))); assertThat(replacement.getNumShardsProvider().get(), is(10)); }
private void testStreamingWriteOverride(PipelineOptions options, int expectedNumShards) { TestPipeline p = TestPipeline.fromOptions(options); StreamingShardedWriteFactory<Object, Void, Object> factory = new StreamingShardedWriteFactory<>(p.getOptions()); WriteFiles<Object, Void, Object> original = WriteFiles.to(new TestSink(tmpFolder.toString())); PCollection<Object> objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); AppliedPTransform<PCollection<Object>, WriteFilesResult<Void>, WriteFiles<Object, Void, Object>> originalApplication = AppliedPTransform.of("writefiles", objs.expand(), Collections.emptyMap(), original, p); WriteFiles<Object, Void, Object> replacement = (WriteFiles<Object, Void, Object>) factory.getReplacementTransform(originalApplication).getTransform(); assertThat(replacement, not(equalTo((Object) original))); assertThat(replacement.getNumShardsProvider().get(), equalTo(expectedNumShards)); WriteFilesResult<Void> originalResult = objs.apply(original); WriteFilesResult<Void> replacementResult = objs.apply(replacement); Map<PValue, ReplacementOutput> res = factory.mapOutputs(originalResult.expand(), replacementResult); assertEquals(1, res.size()); assertEquals( originalResult.getPerDestinationOutputFilenames(), res.get(replacementResult.getPerDestinationOutputFilenames()).getOriginal().getValue()); }
(write.getNumShardsProvider() != null && !write.getWindowedWrites()) ? Optional.of(write.getNumShardsProvider().get()) : Optional.absent(); checkFileContents(baseName, inputs, numShards, !write.getWindowedWrites());
getComputeNumShards() != null || getNumShardsProvider() != null, "When applying %s to an unbounded PCollection, " + "must specify number of output shards explicitly", (getComputeNumShards() == null && getNumShardsProvider() == null) ? input.apply( "WriteUnshardedBundlesToTempFiles",