@Override public void leaveCompositeTransform(TransformHierarchy.Node node) { checkState( !finalized, "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)", KeyedPValueTrackingVisitor.class.getSimpleName(), node); if (node.isRootNode()) { finalized = true; } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) { Map<TupleTag<?>, PValue> outputs = node.getOutputs(); for (PValue output : outputs.values()) { keyedValues.add(output); } } }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(ParDo.MultiOutput.class) private static void parDoMultiOutputTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final ParDo.MultiOutput<?, ?> transform) { final Map<Integer, PCollectionView<?>> sideInputMap = getSideInputMap(transform.getSideInputs()); final AbstractDoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode, sideInputMap); final IRVertex vertex = new OperatorVertex(doFnTransform); ctx.addVertex(vertex); beamNode.getInputs().values().stream() .filter(input -> !transform.getAdditionalInputs().values().contains(input)) .forEach(input -> ctx.addEdgeTo(vertex, input)); ctx.addSideInputEdges(vertex, sideInputMap); beamNode.getOutputs().entrySet().stream() .filter(pValueWithTupleTag -> pValueWithTupleTag.getKey().equals(transform.getMainOutputTag())) .forEach(pValueWithTupleTag -> ctx.registerMainOutputFrom(beamNode, vertex, pValueWithTupleTag.getValue())); beamNode.getOutputs().entrySet().stream() .filter(pValueWithTupleTag -> !pValueWithTupleTag.getKey().equals(transform.getMainOutputTag())) .forEach(pValueWithTupleTag -> ctx.registerAdditionalOutputFrom(beamNode, vertex, pValueWithTupleTag.getValue(), pValueWithTupleTag.getKey())); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(ParDo.SingleOutput.class) private static void parDoSingleOutputTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final ParDo.SingleOutput<?, ?> transform) { final Map<Integer, PCollectionView<?>> sideInputMap = getSideInputMap(transform.getSideInputs()); final AbstractDoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode, sideInputMap); final IRVertex vertex = new OperatorVertex(doFnTransform); ctx.addVertex(vertex); beamNode.getInputs().values().stream() .filter(input -> !transform.getAdditionalInputs().values().contains(input)) .forEach(input -> ctx.addEdgeTo(vertex, input)); ctx.addSideInputEdges(vertex, sideInputMap); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
assertThat(compositeNode.getTransform(), equalTo(create)); assertThat(compositeNode.getOutputs().entrySet(), Matchers.emptyIterable()); assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true)); hierarchy.setOutput(created); hierarchy.popNode(); assertThat(primitiveNode.getOutputs().values(), containsInAnyOrder(created)); assertThat(primitiveNode.getInputs().entrySet(), Matchers.emptyIterable()); assertThat(primitiveNode.getTransform(), equalTo(read)); assertThat(compositeNode.getOutputs().values(), containsInAnyOrder(created));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, finalCombine, output));
replacementParNode.getOutputs().keySet(), Matchers.contains(replacementLongs.getKey())); assertThat(replacementParNode.getOutputs().values(), Matchers.contains(output)); assertThat( compositeNode.getOutputs().keySet(), equalTo(replacementOutput.get(longs).expand().keySet())); assertThat(compositeNode.getOutputs().values(), Matchers.contains(output)); hierarchy.popNode();
assertThat(replacement.getTransform(), equalTo(replacementTransform)); assertThat(replacement.getOutputs().keySet(), Matchers.contains(taggedReplacement.getTag())); assertThat(replacement.getOutputs().values(), Matchers.contains(originalOutput)); hierarchy.popNode();
final KvCoder<?, ?> inputKVCoder = (KvCoder) src.getOutputs().values().stream() .filter(v -> v instanceof PCollection) .map(v -> (PCollection) v)
Node replacement = new Node(existing.getEnclosingNode(), transform, existing.getFullName(), input); for (PValue output : existing.getOutputs().values()) { Node producer = producers.get(output); boolean producedInExisting = false;
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(); } }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator({Window.class, Window.Assign.class}) private static void windowTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final PTransform<?, ?> transform) { final WindowFn windowFn; if (transform instanceof Window) { windowFn = ((Window) transform).getWindowFn(); } else if (transform instanceof Window.Assign) { windowFn = ((Window.Assign) transform).getWindowFn(); } else { throw new UnsupportedOperationException(String.format("%s is not supported", transform)); } final IRVertex vertex = new OperatorVertex( new WindowFnTransform(windowFn, DisplayData.from(beamNode.getTransform()))); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
/** * Determine if this Node belongs to a Bounded branch of the pipeline, or Unbounded, and * translate with the proper translator. */ protected <TransformT extends PTransform<? super PInput, POutput>> TransformEvaluator<TransformT> translate( TransformHierarchy.Node node, TransformT transform) { // --- determine if node is bounded/unbounded. // usually, the input determines if the PCollection to apply the next transformation to // is BOUNDED or UNBOUNDED, meaning RDD/DStream. Map<TupleTag<?>, PValue> pValues; if (node.getInputs().isEmpty()) { // in case of a PBegin, it's the output. pValues = node.getOutputs(); } else { pValues = node.getInputs(); } PCollection.IsBounded isNodeBounded = isBoundedCollection(pValues.values()); // translate accordingly. LOG.debug("Translating {} as {}", transform, isNodeBounded); return isNodeBounded.equals(PCollection.IsBounded.BOUNDED) ? translator.translateBounded(transform) : translator.translateUnbounded(transform); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(GroupByKey.class) private static void groupByKeyTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final GroupByKey<?, ?> transform) { final IRVertex vertex = new OperatorVertex(createGBKTransform(ctx, beamNode)); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(Read.Bounded.class) private static void boundedReadTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final Read.Bounded<?> transform) { final IRVertex vertex = new BeamBoundedSourceVertex<>(transform.getSource(), DisplayData.from(transform)); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(Flatten.PCollections.class) private static void flattenTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final Flatten.PCollections<?> transform) { final IRVertex vertex = new OperatorVertex(new FlattenTransform()); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(Read.Unbounded.class) private static void unboundedReadTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final Read.Unbounded<?> transform) { final IRVertex vertex = new BeamUnboundedSourceVertex<>(transform.getSource(), DisplayData.from(transform)); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }
/** * @param ctx provides translation context * @param beamNode the beam node to be translated * @param transform transform which can be obtained from {@code beamNode} */ @PrimitiveTransformTranslator(View.CreatePCollectionView.class) private static void createPCollectionViewTranslator(final PipelineTranslationContext ctx, final TransformHierarchy.Node beamNode, final View.CreatePCollectionView<?, ?> transform) { final IRVertex vertex = new OperatorVertex(new CreateViewTransform(transform.getView().getViewFn())); ctx.addVertex(vertex); beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input)); ctx.registerMainOutputFrom(beamNode, vertex, transform.getView()); beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output)); }