@Override public SdkFunctionSpec translateSink(SdkComponents newComponents) { // TODO: register the environment return toProto(transform.getSink()); }
@Override public void validate(PipelineOptions options) { getSink().validate(options); }
@Override public Map<String, SideInput> translateSideInputs(SdkComponents components) { Map<String, SideInput> sideInputs = new HashMap<>(); for (PCollectionView<?> view : transform.getSink().getDynamicDestinations().getSideInputs()) { sideInputs.put( view.getTagInternal().getId(), ParDoTranslation.translateView(view, components)); } return sideInputs; }
@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 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 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())); }
WriteFiles.class.getSimpleName()); this.writeOperation = getSink().createWriteOperation(); this.writeOperation.setWindowedWrites(getWindowedWrites());