protected Pipeline(PipelineOptions options) { this(new TransformHierarchy(), options); }
/** * Applies a {@link PTransform} to the given {@link PInput}. * * @see Pipeline#apply */ private <InputT extends PInput, OutputT extends POutput> OutputT applyInternal( String name, InputT input, PTransform<? super InputT, OutputT> transform) { String namePrefix = transforms.getCurrent().getFullName(); String uniqueName = uniquifyInternal(namePrefix, name); final String builtName = buildName(namePrefix, name); instancePerName.put(builtName, transform); LOG.debug("Adding {} to {}", transform, this); transforms.pushNode(uniqueName, input, transform); try { transforms.finishSpecifyingInput(); OutputT output = transform.expand(input); transforms.setOutput(output); return output; } finally { transforms.popNode(); } }
/** * Finish specifying any remaining nodes within the {@link TransformHierarchy}. These are {@link * PValue PValues} that are produced as output of some {@link PTransform} but are never consumed * as input. These values must still be finished specifying. */ private void finishSpecifying() { for (Entry<PValue, PInput> producerInputEntry : producerInput.entrySet()) { PValue value = producerInputEntry.getKey(); value.finishSpecifying(producerInputEntry.getValue(), getProducer(value).getTransform()); } producerInput.clear(); }
@Test public void pushThenPopSucceeds() { TransformHierarchy.Node root = hierarchy.getCurrent(); TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); assertThat(hierarchy.getCurrent(), equalTo(node)); hierarchy.popNode(); assertThat(node.finishedSpecifying, is(true)); assertThat(hierarchy.getCurrent(), equalTo(root)); }
private < InputT extends PInput, OutputT extends POutput, TransformT extends PTransform<? super InputT, OutputT>> void applyReplacement( Node original, PTransformOverrideFactory<InputT, OutputT, TransformT> replacementFactory) { PTransformReplacement<InputT, OutputT> replacement = replacementFactory.getReplacementTransform( (AppliedPTransform<InputT, OutputT, TransformT>) original.toAppliedPTransform(this)); if (replacement.getTransform() == original.getTransform()) { return; } InputT originalInput = replacement.getInput(); LOG.debug("Replacing {} with {}", original, replacement); transforms.replaceNode(original, originalInput, replacement.getTransform()); try { OutputT newOutput = replacement.getTransform().expand(originalInput); Map<PValue, ReplacementOutput> originalToReplacement = replacementFactory.mapOutputs(original.getOutputs(), newOutput); // Ensure the internal TransformHierarchy data structures are consistent. transforms.setOutput(newOutput); transforms.replaceOutputs(originalToReplacement); } finally { transforms.popNode(); } }
Node root = hierarchy.getCurrent(); final SingleOutput<Long, Long> originalParDo = ParDo.of( PCollection<Long> upstream = pipeline.apply(genUpstream); PCollection<Long> output = upstream.apply("Original", originalParDo); Node upstreamNode = hierarchy.pushNode("Upstream", pipeline.begin(), genUpstream); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(upstream); hierarchy.popNode(); Node original = hierarchy.pushNode("Original", upstream, originalParDo); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(output); hierarchy.popNode(); Node compositeNode = hierarchy.replaceNode(original, upstream, replacementComposite); Node replacementParNode = hierarchy.pushNode("Original/Contained", upstream, replacementParDo); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(replacementOutput); hierarchy.popNode(); hierarchy.setOutput(replacementOutput.get(longs)); hierarchy.replaceOutputs( Collections.singletonMap( replacementOutput.get(longs), TaggedPValue.ofExpandedValue(output), TaggedPValue.of(replacementLongs.getKey(), replacementLongs.getValue()))));
@Test public void visitVisitsAllPushed() { TransformHierarchy.Node root = hierarchy.getCurrent(); PBegin begin = PBegin.in(pipeline); TransformHierarchy.Node compositeNode = hierarchy.pushNode("Create", begin, create); hierarchy.finishSpecifyingInput(); assertThat(hierarchy.getCurrent(), equalTo(compositeNode)); assertThat(compositeNode.getInputs().entrySet(), Matchers.empty()); assertThat(compositeNode.getTransform(), equalTo(create)); TransformHierarchy.Node primitiveNode = hierarchy.pushNode("Create/Read", begin, read); assertThat(hierarchy.getCurrent(), equalTo(primitiveNode)); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(created); hierarchy.popNode(); assertThat(primitiveNode.getOutputs().values(), containsInAnyOrder(created)); assertThat(primitiveNode.getInputs().entrySet(), Matchers.emptyIterable()); hierarchy.setOutput(created); assertThat(hierarchy.getProducer(created), equalTo(primitiveNode)); hierarchy.popNode(); TransformHierarchy.Node otherPrimitive = hierarchy.pushNode("ParDo", created, pardo); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(mapped); hierarchy.popNode(); hierarchy.visit(
hierarchy.pushNode("Enclosing", PBegin.in(pipeline), enclosingPT); hierarchy.pushNode("Create", PBegin.in(pipeline), originalTransform); assertThat(hierarchy.getCurrent(), equalTo(original)); PCollection<Long> originalOutput = pipeline.apply(originalTransform); hierarchy.setOutput(originalOutput); hierarchy.popNode(); assertThat(original.finishedSpecifying, is(true)); hierarchy.setOutput(PDone.in(pipeline)); hierarchy.popNode(); assertThat(hierarchy.getCurrent(), not(equalTo(enclosing))); Read.Bounded<Long> replacementTransform = Read.from(CountingSource.upTo(1L)); PCollection<Long> replacementOutput = pipeline.apply(replacementTransform); Node replacement = hierarchy.replaceNode(original, PBegin.in(pipeline), replacementTransform); assertThat(hierarchy.getCurrent(), equalTo(replacement)); hierarchy.setOutput(replacementOutput); replacementOutput, ReplacementOutput.of(TaggedPValue.ofExpandedValue(originalOutput), taggedReplacement)); hierarchy.replaceOutputs(replacementOutputs); hierarchy.popNode();
PCollection<Long> upstream = pipeline.apply(genUpstream); PCollection<Long> output = upstream.apply("Original", originalParDo); hierarchy.pushNode("Upstream", pipeline.begin(), genUpstream); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(upstream); hierarchy.popNode(); TransformHierarchy.Node original = hierarchy.pushNode("Original", upstream, originalParDo); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(output); hierarchy.popNode(); Node compositeNode = hierarchy.replaceNode(original, upstream, replacementComposite); Node replacementParNode = hierarchy.pushNode("Original/Contained", upstream, replacementParDo); hierarchy.finishSpecifyingInput(); hierarchy.setOutput(replacementOutput); hierarchy.popNode(); hierarchy.setOutput(replacementOutput.get(longs)); hierarchy.replaceOutputs( Collections.singletonMap( replacementOutput.get(longs), equalTo(replacementOutput.get(longs).expand().keySet())); assertThat(compositeNode.getOutputs().values(), Matchers.contains(output)); hierarchy.popNode();
hierarchy.pushNode("consumes_both", one, multiConsumer); hierarchy.setOutput(done); hierarchy.popNode(); hierarchy.pushNode( "encloses_producer", PBegin.in(pipeline), hierarchy.pushNode("creates_one_and_two", PBegin.in(pipeline), producer); hierarchy.setOutput(oneAndTwo); hierarchy.popNode(); hierarchy.setOutput(oneAndTwo); hierarchy.popNode(); hierarchy.pushNode("second_copy_of_consumes_both", one, multiConsumer); hierarchy.setOutput(done); hierarchy.popNode(); hierarchy.visit( new PipelineVisitor.Defaults() {
@Test public void emptyCompositeSucceeds() { PCollection<Long> created = PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); hierarchy.setOutput(created); hierarchy.popNode(); PCollectionList<Long> pcList = PCollectionList.of(created); TransformHierarchy.Node emptyTransform = hierarchy.pushNode( "Extract", pcList, new PTransform<PCollectionList<Long>, PCollection<Long>>() { @Override public PCollection<Long> expand(PCollectionList<Long> input) { return input.get(0); } }); hierarchy.setOutput(created); hierarchy.popNode(); assertThat(hierarchy.getProducer(created), equalTo(node)); assertThat( "A Transform that produces non-primitive output should be composite", emptyTransform.isCompositeNode(), is(true)); }
@Test public void producingOwnOutputWithCompositeFails() { final PCollection<Long> comp = PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); PTransform<PBegin, PCollection<Long>> root = new PTransform<PBegin, PCollection<Long>>() { @Override public PCollection<Long> expand(PBegin input) { return comp; } }; hierarchy.pushNode("Composite", PBegin.in(pipeline), root); Create.Values<Integer> create = Create.of(1); hierarchy.pushNode("Create", PBegin.in(pipeline), create); hierarchy.setOutput(pipeline.apply(create)); hierarchy.popNode(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("contains a primitive POutput produced by it"); thrown.expectMessage("primitive transforms are permitted to produce"); thrown.expectMessage("Composite"); hierarchy.setOutput(comp); }
/** * <b><i>For internal use only; no backwards-compatibility guarantees.</i></b> * * <p>Invokes the {@link PipelineVisitor PipelineVisitor's} {@link * PipelineVisitor#visitPrimitiveTransform} and {@link PipelineVisitor#visitValue} operations on * each of this {@link Pipeline Pipeline's} transform and value nodes, in forward topological * order. * * <p>Traversal of the {@link Pipeline} causes {@link PTransform PTransforms} and {@link PValue * PValues} owned by the {@link Pipeline} to be marked as finished, at which point they may no * longer be modified. * * <p>Typically invoked by {@link PipelineRunner} subclasses. */ @Internal public void traverseTopologically(PipelineVisitor visitor) { visitor.enterPipeline(this); transforms.visit(visitor); visitor.leavePipeline(this); }
@Test public void pushWithoutPushFails() { thrown.expect(IllegalStateException.class); hierarchy.popNode(); }
@Test public void getCurrentNoPushReturnsRoot() { assertThat(hierarchy.getCurrent().isRootNode(), is(true)); }
final PCollectionTuple oneAndTwo = PCollectionTuple.of(oneTag, one).and(twoTag, two); hierarchy.pushNode( "consumes_both", one, hierarchy.setOutput(done); hierarchy.popNode(); hierarchy.pushNode( "encloses_producer", PBegin.in(pipeline), Node enclosed = hierarchy.pushNode("creates_one_and_two", PBegin.in(pipeline), producer); hierarchy.setOutput(oneAndTwo); hierarchy.popNode(); hierarchy.setOutput(oneAndTwo); hierarchy.popNode(); hierarchy.visit( new PipelineVisitor.Defaults() { @Override
PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); hierarchy.setOutput(created); hierarchy.popNode(); PCollectionList<Long> pcList = PCollectionList.of(created); VarLongCoder.of()) .setName("prim")); hierarchy.pushNode( "AddPc", pcList, thrown.expectMessage("Create"); thrown.expectMessage(appended.expand().toString()); hierarchy.setOutput(appended);
private boolean returnsOthersOutput() { PTransform<?, ?> transform = getTransform(); if (outputs != null) { for (PValue outputValue : outputs.values()) { if (!getProducer(outputValue).getTransform().equals(transform)) { return true; } } } return false; }
@Before public void setup() { hierarchy = new TransformHierarchy(); }
/** * Finish specifying all of the input {@link PValue PValues} of the current {@link Node}. Ensures * that all of the inputs to the current node have been fully specified, and have been produced by * a node in this graph. */ public void finishSpecifyingInput() { // Inputs must be completely specified before they are consumed by a transform. for (PValue inputValue : current.getInputs().values()) { Node producerNode = getProducer(inputValue); PInput input = producerInput.remove(inputValue); inputValue.finishSpecifying(input, producerNode.getTransform()); } }