@Override @SuppressWarnings("unchecked") public void translateNode( Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) { List<TaggedPValue> allInputs = context.getInputs(transform); DataSet<WindowedValue<T>> result = null; if (allInputs.isEmpty()) { // create an empty dummy source to satisfy downstream operations // we cannot create an empty source in Flink, therefore we have to // add the flatMap that simply never forwards the single element DataSource<String> dummySource = context.getExecutionEnvironment().fromElements("dummy"); result = dummySource .flatMap( new FlatMapFunction<String, WindowedValue<T>>() { @Override public void flatMap(String s, Collector<WindowedValue<T>> collector) throws Exception { // never return anything } }) .returns( new CoderTypeInformation<>( WindowedValue.getFullCoder( (Coder<T>) VoidCoder.of(), GlobalWindow.Coder.INSTANCE))); } else { for (TaggedPValue taggedPc : allInputs) { checkArgument( taggedPc.getValue() instanceof PCollection, "Got non-PCollection input to flatten: %s of type %s", taggedPc.getValue(), taggedPc.getValue().getClass().getSimpleName()); PCollection<T> collection = (PCollection<T>) taggedPc.getValue(); DataSet<WindowedValue<T>> current = context.getInputDataSet(collection); if (result == null) { result = current; } else { result = result.union(current); } } } // insert a dummy filter, there seems to be a bug in Flink // that produces duplicate elements after the union in some cases // if we don't result = result .filter( new FilterFunction<WindowedValue<T>>() { @Override public boolean filter(WindowedValue<T> tWindowedValue) throws Exception { return true; } }) .name("UnionFixFilter"); context.setOutputDataSet(context.getOutput(transform), result); }
@Override public void visitValue(PValue value, TransformHierarchy.Node producer) { boolean inputsAreKeyed = true; for (TaggedPValue input : producer.getInputs()) { inputsAreKeyed = inputsAreKeyed && keyedValues.contains(input.getValue()); } if (PRODUCES_KEYED_OUTPUTS.contains(producer.getTransform().getClass()) || (isKeyPreserving(producer.getTransform()) && inputsAreKeyed)) { keyedValues.add(value); } }
@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())) { List<TaggedPValue> outputs = node.getOutputs(); for (TaggedPValue output : outputs) { keyedValues.add(output.getValue()); } } }
@Override public void translateNode( ParDo.BoundMulti<InputT, OutputT> transform, FlinkBatchTranslationContext context) { DoFn<InputT, OutputT> doFn = transform.getFn(); rejectStateAndTimers(doFn); DataSet<WindowedValue<InputT>> inputDataSet = context.getInputDataSet(context.getInput(transform)); List<TaggedPValue> outputs = context.getOutputs(transform); Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap(); // put the main output at index 0, FlinkMultiOutputDoFnFunction expects this outputMap.put(transform.getMainOutputTag(), 0); int count = 1; for (TaggedPValue taggedValue : outputs) { if (!outputMap.containsKey(taggedValue.getTag())) { outputMap.put(taggedValue.getTag(), count++); } } // assume that the windowing strategy is the same for all outputs WindowingStrategy<?, ?> windowingStrategy = null; // collect all output Coders and create a UnionCoder for our tagged outputs List<Coder<?>> outputCoders = Lists.newArrayList(); for (TaggedPValue taggedValue : outputs) { checkState( taggedValue.getValue() instanceof PCollection, "Within ParDo, got a non-PCollection output %s of type %s", taggedValue.getValue(), taggedValue.getValue().getClass().getSimpleName()); PCollection<?> coll = (PCollection<?>) taggedValue.getValue(); outputCoders.add(coll.getCoder()); windowingStrategy = coll.getWindowingStrategy(); } if (windowingStrategy == null) { throw new IllegalStateException("No outputs defined."); } UnionCoder unionCoder = UnionCoder.of(outputCoders); TypeInformation<WindowedValue<RawUnionValue>> typeInformation = new CoderTypeInformation<>( WindowedValue.getFullCoder( unionCoder, windowingStrategy.getWindowFn().windowCoder())); List<PCollectionView<?>> sideInputs = transform.getSideInputs(); // construct a map from side input to WindowingStrategy so that // the OldDoFn runner can map main-input windows to side input windows Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>(); for (PCollectionView<?> sideInput : sideInputs) { sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); } @SuppressWarnings("unchecked") FlinkMultiOutputDoFnFunction<InputT, OutputT> doFnWrapper = new FlinkMultiOutputDoFnFunction( doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), outputMap); MapPartitionOperator<WindowedValue<InputT>, WindowedValue<RawUnionValue>> taggedDataSet = new MapPartitionOperator<>( inputDataSet, typeInformation, doFnWrapper, transform.getName()); transformSideInputs(sideInputs, taggedDataSet, context); for (TaggedPValue output : outputs) { pruneOutput( taggedDataSet, context, outputMap.get(output.getTag()), (PCollection) output.getValue()); } }