public static <UserT, DestinationT> List<PCollectionView<?>> getDynamicDestinationSideInputs( AppliedPTransform< PCollection<UserT>, WriteFilesResult<DestinationT>, ? extends PTransform<PCollection<UserT>, WriteFilesResult<DestinationT>>> transform) throws IOException { SdkComponents sdkComponents = SdkComponents.create(transform.getPipeline().getOptions()); RunnerApi.PTransform transformProto = PTransformTranslation.toProto(transform, sdkComponents); List<PCollectionView<?>> views = Lists.newArrayList(); Map<String, SideInput> sideInputs = getWriteFilesPayload(transform).getSideInputsMap(); for (Map.Entry<String, SideInput> entry : sideInputs.entrySet()) { PCollection<?> originalPCollection = checkNotNull( (PCollection<?>) transform.getInputs().get(new TupleTag<>(entry.getKey())), "no input with tag %s", entry.getKey()); views.add( PCollectionViewTranslation.viewFromProto( entry.getValue(), entry.getKey(), originalPCollection, transformProto, RehydratedComponents.forComponents(sdkComponents.toComponents()))); } return views; }
private static <T, DestinationT> WriteFilesPayload getWriteFilesPayload( AppliedPTransform< PCollection<T>, WriteFilesResult<DestinationT>, ? extends PTransform<PCollection<T>, WriteFilesResult<DestinationT>>> transform) throws IOException { SdkComponents components = SdkComponents.create(transform.getPipeline().getOptions()); return WriteFilesPayload.parseFrom( PTransformTranslation.toProto(transform, Collections.emptyList(), components) .getSpec() .getPayload()); }
@Override public Map<TupleTag<?>, PValue> getAdditionalInputs() { Map<TupleTag<?>, PValue> additionalInputs = new HashMap<>(); for (Map.Entry<String, SideInput> sideInputEntry : payload.getSideInputsMap().entrySet()) { try { additionalInputs.put( new TupleTag<>(sideInputEntry.getKey()), rehydratedComponents.getPCollection( protoTransform.getInputsOrThrow(sideInputEntry.getKey()))); } catch (IOException exc) { throw new IllegalStateException( String.format( "Could not find input with name %s for %s transform", sideInputEntry.getKey(), WriteFiles.class.getSimpleName())); } } return additionalInputs; }
@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())); }
@Override public boolean isWindowedWrites() { return payload.getWindowedWrites(); }
public static <T, DestinationT> boolean isRunnerDeterminedSharding( AppliedPTransform< PCollection<T>, WriteFilesResult<DestinationT>, ? extends PTransform<PCollection<T>, WriteFilesResult<DestinationT>>> transform) throws IOException { return getWriteFilesPayload(transform).getRunnerDeterminedSharding(); }
public static <UserT, DestinationT, OutputT> FileBasedSink<UserT, DestinationT, OutputT> getSink( AppliedPTransform< PCollection<UserT>, WriteFilesResult<DestinationT>, ? extends PTransform<PCollection<UserT>, WriteFilesResult<DestinationT>>> transform) throws IOException { return (FileBasedSink<UserT, DestinationT, OutputT>) sinkFromProto(getWriteFilesPayload(transform).getSink()); }
@Override public SdkFunctionSpec translateSink(SdkComponents newComponents) { // TODO: re-register the environment with the new components return payload.getSink(); }
public RawWriteFiles( RunnerApi.PTransform protoTransform, RehydratedComponents rehydratedComponents) throws IOException { this.rehydratedComponents = rehydratedComponents; this.protoTransform = protoTransform; this.spec = protoTransform.getSpec(); this.payload = RunnerApi.WriteFilesPayload.parseFrom(spec.getPayload()); }
@Override public FunctionSpec migrate(SdkComponents components) throws IOException { return FunctionSpec.newBuilder() .setUrn(WRITE_FILES_TRANSFORM_URN) .setPayload(payloadForWriteFilesLike(this, components).toByteString()) .build(); }
@Override public Map<String, SideInput> translateSideInputs(SdkComponents components) { // TODO: re-register the PCollections and UDF environments return MoreObjects.firstNonNull( payload.getSideInputsMap(), Collections.<String, SideInput>emptyMap()); }
public static WriteFilesPayload payloadForWriteFilesLike( WriteFilesLike writeFiles, SdkComponents components) throws IOException { return WriteFilesPayload.newBuilder() .setSink(writeFiles.translateSink(components)) .putAllSideInputs(writeFiles.translateSideInputs(components)) .setWindowedWrites(writeFiles.isWindowedWrites()) .setRunnerDeterminedSharding(writeFiles.isRunnerDeterminedSharding()) .build(); } }
@Override public boolean isRunnerDeterminedSharding() { return payload.getRunnerDeterminedSharding(); } }
@Override public FunctionSpec translate( AppliedPTransform<?, ?, WriteFiles<?, ?, ?>> transform, SdkComponents components) throws IOException { return FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) .setPayload(payloadForWriteFiles(transform.getTransform(), components).toByteString()) .build(); } }
public static <T, DestinationT> boolean isWindowedWrites( AppliedPTransform< PCollection<T>, WriteFilesResult<DestinationT>, ? extends PTransform<PCollection<T>, WriteFilesResult<DestinationT>>> transform) throws IOException { return getWriteFilesPayload(transform).getWindowedWrites(); }