private <T> void flattenHelper( Flatten.PCollections<T> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "Flatten"); List<OutputReference> inputs = new ArrayList<>(); for (PValue input : context.getInputs(transform).values()) { inputs.add(context.asOutputReference(input, context.getProducer(input))); } stepContext.addInput(PropertyNames.INPUTS, inputs); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); } });
private static void translateSideInputs( StepTranslationContext stepContext, List<PCollectionView<?>> sideInputs, TranslationContext context) { Map<String, Object> nonParInputs = new HashMap<>(); for (PCollectionView<?> view : sideInputs) { nonParInputs.put( view.getTagInternal().getId(), context.asOutputReference(view, context.getProducer(view))); } stepContext.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs); }
outputCoders); stepContext.addInput( PropertyNames.RESTRICTION_CODER, translateCoder(transform.getRestrictionCoder(), context));
PubsubUnboundedSource overriddenTransform = transform.getOverriddenTransform(); StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "pubsub"); if (overriddenTransform.getTopicProvider() != null) { if (overriddenTransform.getTopicProvider().isAccessible()) { stepContext.addInput( PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); } else { stepContext.addInput( PropertyNames.PUBSUB_TOPIC_OVERRIDE, ((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName()); stepContext.addInput( PropertyNames.PUBSUB_SUBSCRIPTION, overriddenTransform.getSubscription().getV1Beta1Path()); } else { stepContext.addInput( PropertyNames.PUBSUB_SUBSCRIPTION_OVERRIDE, ((NestedValueProvider) overriddenTransform.getSubscriptionProvider()).propertyName()); stepContext.addInput( PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, overriddenTransform.getTimestampAttribute()); stepContext.addInput( PropertyNames.PUBSUB_ID_ATTRIBUTE, overriddenTransform.getIdAttribute()); stepContext.addInput( PropertyNames.PUBSUB_SERIALIZED_ATTRIBUTES_FN, byteArrayToJsonString(serializeToByteArray(new IdentityMessageFn())));
PubsubUnboundedSink overriddenTransform = transform.getOverriddenTransform(); StepTranslationContext stepContext = context.addStep(transform, "ParallelWrite"); stepContext.addInput(PropertyNames.FORMAT, "pubsub"); if (overriddenTransform.getTopicProvider().isAccessible()) { stepContext.addInput( PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); } else { stepContext.addInput( PropertyNames.PUBSUB_TOPIC_OVERRIDE, ((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName()); stepContext.addInput( PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, overriddenTransform.getTimestampAttribute()); stepContext.addInput( PropertyNames.PUBSUB_ID_ATTRIBUTE, overriddenTransform.getIdAttribute()); stepContext.addInput( PropertyNames.PUBSUB_SERIALIZED_ATTRIBUTES_FN, byteArrayToJsonString(serializeToByteArray(new IdentityMessageFn()))); stepContext.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
private <K, V> void groupByKeyHelper( GroupByKey<K, V> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "GroupByKey"); PCollection<KV<K, V>> input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy(); boolean isStreaming = context.getPipelineOptions().as(StreamingOptions.class).isStreaming(); boolean allowCombinerLifting = windowingStrategy.getWindowFn().isNonMerging() && windowingStrategy.getWindowFn().assignsToOneWindow(); if (isStreaming) { allowCombinerLifting &= transform.fewKeys(); // TODO: Allow combiner lifting on the non-default trigger, as appropriate. allowCombinerLifting &= (windowingStrategy.getTrigger() instanceof DefaultTrigger); } stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, !allowCombinerLifting); stepContext.addInput( PropertyNames.SERIALIZED_FN, byteArrayToJsonString(serializeWindowingStrategy(windowingStrategy))); stepContext.addInput( PropertyNames.IS_MERGING_WINDOW_FN, !windowingStrategy.getWindowFn().isNonMerging()); } });
@Override public void translate(Impulse transform, TranslationContext context) { if (context.getPipelineOptions().isStreaming()) { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "pubsub"); stepContext.addInput(PropertyNames.PUBSUB_SUBSCRIPTION, "_starting_signal/"); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); } else { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "impulse"); WindowedValue.FullWindowedValueCoder<byte[]> coder = WindowedValue.getFullCoder( context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); byte[] encodedImpulse; try { encodedImpulse = encodeToByteArray(coder, WindowedValue.valueInGlobalWindow(new byte[0])); } catch (Exception e) { throw new RuntimeException(e); } stepContext.addInput(PropertyNames.IMPULSE_ELEMENT, byteArrayToJsonString(encodedImpulse)); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); } } }
private <ElemT, ViewT> void translateTyped( View.CreatePCollectionView<ElemT, ViewT> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "CollectionToSingleton"); PCollection<ElemT> input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy(); stepContext.addInput( PropertyNames.WINDOWING_STRATEGY, byteArrayToJsonString(serializeWindowingStrategy(windowingStrategy))); stepContext.addInput( PropertyNames.IS_MERGING_WINDOW_FN, !windowingStrategy.getWindowFn().isNonMerging()); stepContext.addCollectionToSingletonOutput( input, PropertyNames.OUTPUT, transform.getView()); } });
public static <T> void translateReadHelper( Source<T> source, PTransform<?, ? extends PCollection<?>> transform, TranslationContext context) { try { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT); stepContext.addInput( PropertyNames.SOURCE_STEP_INPUT, cloudSourceToDictionary( CustomSources.serializeToCloudSource(source, context.getPipelineOptions()))); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); } catch (Exception e) { throw new RuntimeException(e); } }
private <T> void translateHelper(Window.Assign<T> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "Bucket"); PCollection<T> input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy(); byte[] serializedBytes = serializeWindowingStrategy(strategy); String serializedJson = byteArrayToJsonString(serializedBytes); stepContext.addInput(PropertyNames.SERIALIZED_FN, serializedJson); } });
private static void translateInputs( StepTranslationContext stepContext, PCollection<?> input, List<PCollectionView<?>> sideInputs, TranslationContext context) { stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); translateSideInputs(stepContext, sideInputs, context); }
private <ElemT, ViewT> void translateTyped( CreateDataflowView<ElemT, ViewT> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "CollectionToSingleton"); PCollection<ElemT> input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); stepContext.addCollectionToSingletonOutput( input, PropertyNames.OUTPUT, transform.getView()); } });
private <K1, K2, V> void groupByKeyAndSortValuesHelper( GroupByKeyAndSortValuesOnly<K1, K2, V> transform, TranslationContext context) { StepTranslationContext stepContext = context.addStep(transform, "GroupByKey"); PCollection<KV<K1, KV<K2, V>>> input = context.getInput(transform); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform)); stepContext.addInput(PropertyNames.SORT_VALUES, true); // TODO: Add support for combiner lifting once the need arises. stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, true); } });