private DescendantConsumers getRootConsumers(PTransformNode rootNode) { checkArgument( rootNode.getTransform().getInputsCount() == 0, "Transform %s is not at the root of the graph (consumes %s)", rootNode.getId(), rootNode.getTransform().getInputsMap()); checkArgument( !pipeline.getEnvironment(rootNode).isPresent(), "%s requires all root nodes to be runner-implemented %s primitives, " + "but transform %s executes in environment %s", GreedyPipelineFuser.class.getSimpleName(), PTransformTranslation.IMPULSE_TRANSFORM_URN, rootNode.getId(), pipeline.getEnvironment(rootNode)); Set<PTransformNode> unfused = new HashSet<>(); unfused.add(rootNode); NavigableSet<CollectionConsumer> environmentNodes = new TreeSet<>(); // Walk down until the first environments are found, and fuse them as appropriate. for (PCollectionNode output : pipeline.getOutputPCollections(rootNode)) { DescendantConsumers descendants = getDescendantConsumers(output); unfused.addAll(descendants.getUnfusedNodes()); environmentNodes.addAll(descendants.getFusibleConsumers()); } return DescendantConsumers.of(unfused, environmentNodes); }
@Test public void toAndFromProto() throws IOException { SdkComponents components = SdkComponents.create(spec.getTransform().getPipeline().getOptions()); RunnerApi.PTransform converted = convert(spec, components); Components protoComponents = components.toComponents(); // Sanity checks assertThat(converted.getInputsCount(), equalTo(spec.getTransform().getInputs().size())); assertThat(converted.getOutputsCount(), equalTo(spec.getTransform().getOutputs().size())); assertThat(converted.getSubtransformsCount(), equalTo(spec.getChildren().size())); assertThat(converted.getUniqueName(), equalTo(spec.getTransform().getFullName())); for (PValue inputValue : spec.getTransform().getInputs().values()) { PCollection<?> inputPc = (PCollection<?>) inputValue; protoComponents.getPcollectionsOrThrow(components.registerPCollection(inputPc)); } for (PValue outputValue : spec.getTransform().getOutputs().values()) { PCollection<?> outputPc = (PCollection<?>) outputValue; protoComponents.getPcollectionsOrThrow(components.registerPCollection(outputPc)); } }
@Test public void rootTransforms() { Pipeline p = Pipeline.create(); p.apply("UnboundedRead", Read.from(CountingSource.unbounded())) .apply(Window.into(FixedWindows.of(Duration.millis(5L)))) .apply(Count.perElement()); p.apply("BoundedRead", Read.from(CountingSource.upTo(100L))); Components components = PipelineTranslation.toProto(p).getComponents(); QueryablePipeline qp = QueryablePipeline.forPrimitivesIn(components); assertThat(qp.getRootTransforms(), hasSize(2)); for (PTransformNode rootTransform : qp.getRootTransforms()) { assertThat( "Root transforms should have no inputs", rootTransform.getTransform().getInputsCount(), equalTo(0)); assertThat( "Only added source reads to the pipeline", rootTransform.getTransform().getSpec().getUrn(), equalTo(PTransformTranslation.READ_TRANSFORM_URN)); } }
assertThat( stagePTransform.getInputsMap(), allOf(hasValue("input.out"), hasValue("sideInput.in"))); assertThat(stagePTransform.getInputsCount(), equalTo(2));
assertThat( stagePTransform.getInputsMap(), allOf(hasValue("input.out"), hasValue("sideInput.in"))); assertThat(stagePTransform.getInputsCount(), equalTo(2));