/** * Translates a composite {@link AppliedPTransform} into a runner API proto with no component * transforms. * * <p>This should not be used when translating a {@link Pipeline}. * * <p>Does not register the {@code appliedPTransform} within the provided {@link SdkComponents}. */ static RunnerApi.PTransform toProto( AppliedPTransform<?, ?, ?> appliedPTransform, SdkComponents components) throws IOException { return toProto(appliedPTransform, Collections.emptyList(), components); }
private static <T> ReadPayload getReadPayload( AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> transform) throws IOException { SdkComponents components = SdkComponents.create(transform.getPipeline().getOptions()); return ReadPayload.parseFrom( PTransformTranslation.toProto(transform, Collections.emptyList(), components) .getSpec() .getPayload()); }
private static <T> ParDoPayload getParDoPayload(AppliedPTransform<?, ?, ?> transform) throws IOException { SdkComponents components = SdkComponents.create(transform.getPipeline().getOptions()); RunnerApi.PTransform parDoPTransform = PTransformTranslation.toProto(transform, Collections.emptyList(), components); return ParDoPayload.parseFrom(parDoPTransform.getSpec().getPayload()); }
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()); }
public static PCollection.IsBounded sourceIsBounded(AppliedPTransform<?, ?, ?> transform) { try { SdkComponents components = SdkComponents.create(transform.getPipeline().getOptions()); return PCollectionTranslation.fromProto( ReadPayload.parseFrom( PTransformTranslation.toProto(transform, Collections.emptyList(), components) .getSpec() .getPayload()) .getIsBounded()); } catch (IOException e) { throw new RuntimeException("Internal error determining boundedness of Read", e); } }
/** * Registers the provided {@link AppliedPTransform} into this {@link SdkComponents}, returning a * unique ID for the {@link AppliedPTransform}. Multiple registrations of the same {@link * AppliedPTransform} will return the same unique ID. * * <p>All of the children must already be registered within this {@link SdkComponents}. */ public String registerPTransform( AppliedPTransform<?, ?, ?> appliedPTransform, List<AppliedPTransform<?, ?, ?>> children) throws IOException { String name = getApplicationName(appliedPTransform); // If this transform is present in the components, nothing to do. return the existing name. // Otherwise the transform must be translated and added to the components. if (componentsBuilder.getTransformsOrDefault(name, null) != null) { return name; } checkNotNull(children, "child nodes may not be null"); componentsBuilder.putTransforms( name, PTransformTranslation.toProto(appliedPTransform, children, this)); return name; }
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; }
taggedInput.getValue().getClass()); transformBuilder.putInputs( toProto(taggedInput.getKey()), components.registerPCollection((PCollection<?>) taggedInput.getValue())); taggedOutput.getValue().getClass()); transformBuilder.putOutputs( toProto(taggedOutput.getKey()), components.registerPCollection((PCollection<?>) taggedOutput.getValue()));
public static WindowIntoPayload getWindowIntoPayload(AppliedPTransform<?, ?, ?> application) { RunnerApi.PTransform transformProto; try { SdkComponents components = SdkComponents.create(application.getPipeline().getOptions()); transformProto = PTransformTranslation.toProto(application, Collections.emptyList(), components); } catch (IOException exc) { throw new RuntimeException(exc); } checkArgument( PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN.equals( transformProto.getSpec().getUrn()), "Illegal attempt to extract %s from transform %s with name \"%s\" and URN \"%s\"", Window.Assign.class.getSimpleName(), application.getTransform(), application.getFullName(), transformProto.getSpec().getUrn()); try { return WindowIntoPayload.parseFrom(transformProto.getSpec().getPayload()); } catch (InvalidProtocolBufferException exc) { throw new IllegalStateException( String.format( "%s translated %s with URN '%s' but payload was not a %s", PTransformTranslation.class.getSimpleName(), application, PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, WindowIntoPayload.class.getSimpleName()), exc); } }
private static Optional<CombinePayload> getCombinePayload( AppliedPTransform<?, ?, ?> transform, SdkComponents components) throws IOException { RunnerApi.PTransform proto = PTransformTranslation.toProto(transform, Collections.emptyList(), components); // Even if the proto has no spec, calling getSpec still returns a blank spec, which we want to // avoid. It should be clear to the caller whether or not there was a spec in the transform. if (proto.hasSpec()) { return Optional.of(CombinePayload.parseFrom(proto.getSpec().getPayload())); } else { return Optional.empty(); } } }
/** Extracts an {@link ExecutableStagePayload} from the given transform. */ public static ExecutableStagePayload getExecutableStagePayload( AppliedPTransform<?, ?, ?> appliedTransform) throws IOException { RunnerApi.PTransform transform = PTransformTranslation.toProto( appliedTransform, SdkComponents.create(appliedTransform.getPipeline().getOptions())); checkArgument(ExecutableStage.URN.equals(transform.getSpec().getUrn())); return ExecutableStagePayload.parseFrom(transform.getSpec().getPayload()); } }
public static List<PCollectionView<?>> getSideInputs(AppliedPTransform<?, ?, ?> application) throws IOException { PTransform<?, ?> transform = application.getTransform(); if (transform instanceof ParDo.MultiOutput) { return ((ParDo.MultiOutput<?, ?>) transform).getSideInputs(); } SdkComponents sdkComponents = SdkComponents.create(application.getPipeline().getOptions()); RunnerApi.PTransform parDoProto = PTransformTranslation.toProto(application, sdkComponents); ParDoPayload payload = ParDoPayload.parseFrom(parDoProto.getSpec().getPayload()); List<PCollectionView<?>> views = new ArrayList<>(); RehydratedComponents components = RehydratedComponents.forComponents(sdkComponents.toComponents()); for (Map.Entry<String, SideInput> sideInputEntry : payload.getSideInputsMap().entrySet()) { String sideInputTag = sideInputEntry.getKey(); RunnerApi.SideInput sideInput = sideInputEntry.getValue(); PCollection<?> originalPCollection = checkNotNull( (PCollection<?>) application.getInputs().get(new TupleTag<>(sideInputTag)), "no input with tag %s", sideInputTag); views.add( PCollectionViewTranslation.viewFromProto( sideInput, sideInputTag, originalPCollection, parDoProto, components)); } return views; }
public static TupleTagList getAdditionalOutputTags(AppliedPTransform<?, ?, ?> application) throws IOException { PTransform<?, ?> transform = application.getTransform(); if (transform instanceof ParDo.MultiOutput) { return ((ParDo.MultiOutput<?, ?>) transform).getAdditionalOutputTags(); } RunnerApi.PTransform protoTransform = PTransformTranslation.toProto( application, SdkComponents.create(application.getPipeline().getOptions())); ParDoPayload payload = ParDoPayload.parseFrom(protoTransform.getSpec().getPayload()); TupleTag<?> mainOutputTag = getMainOutputTag(payload); Set<String> outputTags = Sets.difference( protoTransform.getOutputsMap().keySet(), Collections.singleton(mainOutputTag.getId())); ArrayList<TupleTag<?>> additionalOutputTags = new ArrayList<>(); for (String outputTag : outputTags) { additionalOutputTags.add(new TupleTag<>(outputTag)); } return TupleTagList.of(additionalOutputTags); }
/** * @deprecated Since {@link CreatePCollectionView} is not a part of the Beam model, there is no * SDK-agnostic specification. Using this method means your runner is tied to Java. */ @Deprecated public static <ElemT, ViewT> PCollectionView<ViewT> getView( AppliedPTransform< PCollection<ElemT>, PCollection<ElemT>, PTransform<PCollection<ElemT>, PCollection<ElemT>>> application) throws IOException { RunnerApi.PTransform transformProto = PTransformTranslation.toProto( application, Collections.emptyList(), SdkComponents.create(application.getPipeline().getOptions())); checkArgument( PTransformTranslation.CREATE_VIEW_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), "Illegal attempt to extract %s from transform %s with name \"%s\" and URN \"%s\"", PCollectionView.class.getSimpleName(), application.getTransform(), application.getFullName(), transformProto.getSpec().getUrn()); return (PCollectionView<ViewT>) SerializableUtils.deserializeFromByteArray( transformProto.getSpec().getPayload().toByteArray(), PCollectionView.class.getSimpleName()); }
/** * Converts an {@link AppliedPTransform}, which may be a rehydrated transform or an original * {@link TestStream}, to a {@link TestStream}. */ public static <T> TestStream<T> getTestStream( AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> application) throws IOException { // For robustness, we don't take this shortcut: // if (application.getTransform() instanceof TestStream) { // return application.getTransform() // } SdkComponents sdkComponents = SdkComponents.create(application.getPipeline().getOptions()); RunnerApi.PTransform transformProto = PTransformTranslation.toProto(application, sdkComponents); checkArgument( TEST_STREAM_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), "Attempt to get %s from a transform with wrong URN %s", TestStream.class.getSimpleName(), transformProto.getSpec().getUrn()); RunnerApi.TestStreamPayload testStreamPayload = RunnerApi.TestStreamPayload.parseFrom(transformProto.getSpec().getPayload()); return (TestStream<T>) testStreamFromProtoPayload( testStreamPayload, RehydratedComponents.forComponents(sdkComponents.toComponents())); }
private RunnerApi.PTransform convert(ToAndFromProtoSpec spec, SdkComponents components) throws IOException { List<AppliedPTransform<?, ?, ?>> childTransforms = new ArrayList<>(); for (ToAndFromProtoSpec child : spec.getChildren()) { childTransforms.add(child.getTransform()); System.out.println("Converting child " + child); convert(child, components); // Sanity call components.getExistingPTransformId(child.getTransform()); } RunnerApi.PTransform convert = PTransformTranslation.toProto(spec.getTransform(), childTransforms, components); // Make sure the converted transform is registered. Convert it independently, but if this is a // child spec, the child must be in the components. components.registerPTransform(spec.getTransform(), childTransforms); return convert; }
@Test public void testEncodedProto() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); components.registerPCollection(testPCollection); AppliedPTransform<?, ?, ?> appliedPTransform = AppliedPTransform.of( "foo", testPCollection.expand(), createViewTransform.getView().expand(), createViewTransform, p); FunctionSpec payload = PTransformTranslation.toProto(appliedPTransform, components).getSpec(); // Checks that the payload is what it should be PCollectionView<?> deserializedView = (PCollectionView<?>) SerializableUtils.deserializeFromByteArray( payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat(deserializedView, Matchers.equalTo(createViewTransform.getView())); }
@Test public void testExtractionDirectFromTransform() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); components.registerPCollection(testPCollection); AppliedPTransform<?, ?, ?> appliedPTransform = AppliedPTransform.of( "foo", testPCollection.expand(), createViewTransform.getView().expand(), createViewTransform, p); CreatePCollectionViewTranslation.getView((AppliedPTransform) appliedPTransform); FunctionSpec payload = PTransformTranslation.toProto(appliedPTransform, components).getSpec(); // Checks that the payload is what it should be PCollectionView<?> deserializedView = (PCollectionView<?>) SerializableUtils.deserializeFromByteArray( payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat(deserializedView, Matchers.equalTo(createViewTransform.getView())); } }
@Test public void testRegistrarEncodedProto() throws Exception { PCollection<String> output = p.apply(testStream); AppliedPTransform<PBegin, PCollection<String>, TestStream<String>> appliedTestStream = AppliedPTransform.of("fakeName", PBegin.in(p).expand(), output.expand(), testStream, p); SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.FunctionSpec spec = PTransformTranslation.toProto(appliedTestStream, components).getSpec(); assertThat(spec.getUrn(), equalTo(TEST_STREAM_TRANSFORM_URN)); RunnerApi.TestStreamPayload payload = TestStreamPayload.parseFrom(spec.getPayload()); verifyTestStreamEncoding( testStream, payload, RehydratedComponents.forComponents(components.toComponents())); }
PTransformTranslation.toProto( AppliedPTransform.<PCollection<KV<Long, String>>, PCollection<Void>, MultiOutput>of( "foo", inputs, output.expand(), parDo, p),