/** * @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()); }
private static RunnerApi.Coder toKnownCoder(Coder<?> coder, SdkComponents components) throws IOException { CoderTranslator translator = KNOWN_TRANSLATORS.get(coder.getClass()); List<String> componentIds = registerComponents(coder, translator, components); return RunnerApi.Coder.newBuilder() .addAllComponentCoderIds(componentIds) .setSpec( SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(KNOWN_CODER_URNS.get(coder.getClass())) .setPayload(ByteString.copyFrom(translator.getPayload(coder))))) .build(); }
PTransform.newBuilder() .addSubtransforms("sub_first") .setSpec(FunctionSpec.newBuilder().setUrn("beam:composite")) .build()) .putTransforms( "sub_first", PTransform.newBuilder() .setSpec(FunctionSpec.newBuilder().setUrn("beam:inner")) .build())) .build(); .addSubtransforms("bar") .setSpec( FunctionSpec.getDefaultInstance() .newBuilderForType() .setUrn("beam:composite")) .build(), Components.getDefaultInstance())); assertThat( pipeline.getComponents().getTransformsOrThrow("root").getSpec().getUrn(), equalTo("beam:composite")); assertThat(
PTransform.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN) .setPayload(payload.toByteString())
@Override public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput( String transformId, String sideInputId, RunnerApi.FunctionSpec accessPattern, Coder<T> elementCoder, Coder<W> windowCoder) { PCollectionView collectionNode = sideInputToCollection.get( SideInputId.newBuilder().setTransformId(transformId).setLocalName(sideInputId).build()); checkArgument(collectionNode != null, "No side input for %s/%s", transformId, sideInputId); if (PTransformTranslation.ITERABLE_SIDE_INPUT.equals(accessPattern.getUrn())) { @SuppressWarnings("unchecked") // T == V Coder<V> outputCoder = (Coder<V>) elementCoder; return forIterableSideInput(collectionNode, outputCoder); } else if (PTransformTranslation.MULTIMAP_SIDE_INPUT.equals(accessPattern.getUrn()) || Materializations.MULTIMAP_MATERIALIZATION_URN.equals(accessPattern.getUrn())) { // TODO: Remove non standard URN. // Using non standard version of multimap urn as dataflow uses the non standard urn. @SuppressWarnings("unchecked") // T == KV<?, V> KvCoder<?, V> kvCoder = (KvCoder<?, V>) elementCoder; return forMultimapSideInput(collectionNode, kvCoder.getKeyCoder(), kvCoder.getValueCoder()); } else { throw new IllegalArgumentException( String.format("Unknown side input access pattern: %s", accessPattern)); } }
@Test public void getEnvironmentCombine() throws IOException { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); CombinePayload payload = CombinePayload.newBuilder() .setCombineFn(CombineTranslation.toProto(Sum.ofLongs(), components)) .build(); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components.toComponents()); PTransform builder = PTransform.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN) .setPayload(payload.toByteString()) .build()) .build(); Environment env = Environments.getEnvironment(builder, rehydratedComponents).get(); assertThat( env, equalTo( components .toComponents() .getEnvironmentsOrThrow(payload.getCombineFn().getEnvironmentId()))); } }
/** * Update all composites present in the {@code originalPipeline} with an URN equal to the provided * {@code urn} using the provided {@link TransformReplacement}. */ public static Pipeline updateTransform( String urn, Pipeline originalPipeline, TransformReplacement compositeBuilder) { Components.Builder resultComponents = originalPipeline.getComponents().toBuilder(); for (Map.Entry<String, PTransform> pt : originalPipeline.getComponents().getTransformsMap().entrySet()) { if (pt.getValue().getSpec() != null && urn.equals(pt.getValue().getSpec().getUrn())) { MessageWithComponents updated = compositeBuilder.getReplacement(pt.getKey(), originalPipeline.getComponents()); checkArgument( updated.getPtransform().getOutputsMap().equals(pt.getValue().getOutputsMap()), "A %s must produce all of the outputs of the original %s", TransformReplacement.class.getSimpleName(), PTransform.class.getSimpleName()); removeSubtransforms(pt.getValue(), resultComponents); resultComponents .mergeFrom(updated.getComponents()) .putTransforms(pt.getKey(), updated.getPtransform()); } } return originalPipeline.toBuilder().setComponents(resultComponents).build(); }
@Test public void getEnvironmentRead() throws IOException { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); ReadPayload payload = ReadTranslation.toProto(Read.from(CountingSource.unbounded()), components); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components.toComponents()); PTransform builder = PTransform.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN) .setPayload(payload.toByteString()) .build()) .build(); Environment env = Environments.getEnvironment(builder, rehydratedComponents).get(); assertThat( env, equalTo( components .toComponents() .getEnvironmentsOrThrow(payload.getSource().getEnvironmentId()))); }
@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())); } }
private void translateStreamingImpulse( String id, RunnerApi.Pipeline pipeline, StreamingTranslationContext context) { RunnerApi.PTransform pTransform = pipeline.getComponents().getTransformsOrThrow(id); TypeInformation<WindowedValue<byte[]>> typeInfo = new CoderTypeInformation<>( WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE)); ObjectMapper objectMapper = new ObjectMapper(); final int intervalMillis; final int messageCount; try { JsonNode config = objectMapper.readTree(pTransform.getSpec().getPayload().toByteArray()); intervalMillis = config.path("interval_ms").asInt(100); messageCount = config.path("message_count").asInt(0); } catch (IOException e) { throw new RuntimeException("Failed to parse configuration for streaming impulse", e); } SingleOutputStreamOperator<WindowedValue<byte[]>> source = context .getExecutionEnvironment() .addSource( new StreamingImpulseSource(intervalMillis, messageCount), StreamingImpulseSource.class.getSimpleName()) .returns(typeInfo); context.addDataStream(Iterables.getOnlyElement(pTransform.getOutputsMap().values()), source); }
@Test public void noEnvironmentThrows() { // (impulse.out) -> runnerTransform -> gbk.out // runnerTransform can't be executed in an environment, so trying to construct it should fail PTransform gbkTransform = PTransform.newBuilder() .putInputs("input", "impulse.out") .setSpec( FunctionSpec.newBuilder().setUrn(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN)) .putOutputs("output", "gbk.out") .build(); QueryablePipeline p = QueryablePipeline.forPrimitivesIn( partialComponents .toBuilder() .putTransforms("runnerTransform", gbkTransform) .putPcollections( "gbk.out", PCollection.newBuilder().setUniqueName("gbk.out").build()) .build()); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Environment must be populated"); GreedyStageFuser.forGrpcPortRead( p, impulseOutputNode, ImmutableSet.of(PipelineNode.pTransform("runnerTransform", gbkTransform))); }
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; }
/** * 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())); }
/** * Tests that {@link QueryablePipeline#getPerElementConsumers(PCollectionNode)} returns a * transform that consumes the node more than once. */ @Test public void perElementConsumersWithConsumingMultipleTimes() { Pipeline p = Pipeline.create(); PCollection<Long> longs = p.apply("BoundedRead", Read.from(CountingSource.upTo(100L))); PCollectionList.of(longs).and(longs).and(longs).apply("flatten", Flatten.pCollections()); Components components = PipelineTranslation.toProto(p).getComponents(); // This breaks if the way that IDs are assigned to PTransforms changes in PipelineTranslation String readOutput = getOnlyElement(components.getTransformsOrThrow("BoundedRead").getOutputsMap().values()); QueryablePipeline qp = QueryablePipeline.forPrimitivesIn(components); Set<PTransformNode> consumers = qp.getPerElementConsumers( PipelineNode.pCollection(readOutput, components.getPcollectionsOrThrow(readOutput))); assertThat(consumers.size(), equalTo(1)); assertThat( getOnlyElement(consumers).getTransform().getSpec().getUrn(), equalTo(PTransformTranslation.FLATTEN_TRANSFORM_URN)); }
private void translateStreamingImpulse( String id, RunnerApi.Pipeline pipeline, StreamingTranslationContext context) { RunnerApi.PTransform pTransform = pipeline.getComponents().getTransformsOrThrow(id); TypeInformation<WindowedValue<byte[]>> typeInfo = new CoderTypeInformation<>( WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE)); ObjectMapper objectMapper = new ObjectMapper(); final int intervalMillis; final int messageCount; try { JsonNode config = objectMapper.readTree(pTransform.getSpec().getPayload().toByteArray()); intervalMillis = config.path("interval_ms").asInt(100); messageCount = config.path("message_count").asInt(0); } catch (IOException e) { throw new RuntimeException("Failed to parse configuration for streaming impulse", e); } SingleOutputStreamOperator<WindowedValue<byte[]>> source = context .getExecutionEnvironment() .addSource(new StreamingImpulseSource(intervalMillis, messageCount)) .returns(typeInfo); context.addDataStream(Iterables.getOnlyElement(pTransform.getOutputsMap().values()), source); }
@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())); }
public static Optional<Environment> getEnvironment( PTransform ptransform, RehydratedComponents components) { try { String envId = KNOWN_URN_SPEC_EXTRACTORS .getOrDefault(ptransform.getSpec().getUrn(), DEFAULT_SPEC_EXTRACTOR) .getEnvironmentId(ptransform); if (!Strings.isNullOrEmpty(envId)) { // Some PTransform payloads may have an empty (default) Environment ID, for example a // WindowIntoPayload with a known WindowFn. Others will never have an Environment ID, such // as a GroupByKeyPayload, and the Default extractor returns null in this case. return Optional.of(components.getEnvironment(envId)); } else { return Optional.empty(); } } catch (IOException e) { throw new RuntimeException(e); } }
@Before public void setup() { partialComponents = Components.newBuilder() .putTransforms( "impulse", PTransform.newBuilder() .setUniqueName("Impulse") .putOutputs("output", "impulse.out") .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN)) .build()) .putPcollections("impulse.out", pc("impulse.out")) .putEnvironments("go", Environments.createDockerEnvironment("go")) .putEnvironments("py", Environments.createDockerEnvironment("py")) .putCoders("coder", Coder.newBuilder().build()) .putCoders("windowCoder", Coder.newBuilder().build()) .putWindowingStrategies( "ws", WindowingStrategy.newBuilder().setWindowCoderId("windowCoder").build()) .build(); }
@Override public FlinkPortablePipelineTranslator.Executor translate( BatchTranslationContext context, RunnerApi.Pipeline pipeline) { // Use a QueryablePipeline to traverse transforms topologically. QueryablePipeline p = QueryablePipeline.forTransforms( pipeline.getRootTransformIdsList(), pipeline.getComponents()); for (PipelineNode.PTransformNode transform : p.getTopologicallyOrderedTransforms()) { urnToTransformTranslator .getOrDefault( transform.getTransform().getSpec().getUrn(), FlinkBatchPortablePipelineTranslator::urnNotFound) .translate(transform, pipeline, context); } // Ensure that side effects are performed for unconsumed DataSets. for (DataSet<?> dataSet : context.getDanglingDataSets()) { dataSet.output(new DiscardingOutputFormat<>()).name("DiscardingOutput"); } return context; }
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); }