public static <ElemT, ViewT> CreateDataflowView<ElemT, ViewT> forBatch( PCollectionView<ViewT> view) { return new CreateDataflowView<>(view, false); }
static <T, FinalT, ViewT, W extends BoundedWindow> PCollection<?> applyForSingleton( DataflowRunner runner, PCollection<T> input, DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>, IsmRecord<WindowedValue<FinalT>>> doFn, Coder<FinalT> defaultValueCoder, PCollectionView<ViewT> view) { @SuppressWarnings("unchecked") Coder<W> windowCoder = (Coder<W>) input.getWindowingStrategy().getWindowFn().windowCoder(); IsmRecordCoder<WindowedValue<FinalT>> ismCoder = coderForSingleton(windowCoder, defaultValueCoder); PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder)) .apply(ParDo.of(doFn)); reifiedPerWindowAndSorted.setCoder(ismCoder); runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted); reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view)); return reifiedPerWindowAndSorted; }
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()); } });
@Override public PCollection<ElemT> expand(PCollection<ElemT> input) { return input .apply(Combine.globally(new Concatenate<ElemT>()).withoutDefaults()) .apply(ParDo.of(StreamingPCollectionViewWriterFn.create(view, input.getCoder()))) .apply(CreateDataflowView.forStreaming(view)); } }
static <T, W extends BoundedWindow, ViewT> PCollection<?> applyForIterableLike( DataflowRunner runner, PCollection<T> input, PCollectionView<ViewT> view) { @SuppressWarnings("unchecked") Coder<W> windowCoder = (Coder<W>) input.getWindowingStrategy().getWindowFn().windowCoder(); IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder()); // If we are working in the global window, we do not need to do a GBK using the window // as the key since all the elements of the input PCollection are already such. // We just reify the windowed value while converting them to IsmRecords and generating // an index based upon where we are within the bundle. Each bundle // maps to one file exactly. if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) { PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<>())); reifiedPerWindowAndSorted.setCoder(ismCoder); runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted); reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view)); return reifiedPerWindowAndSorted; } PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder)) .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<>(windowCoder))); reifiedPerWindowAndSorted.setCoder(ismCoder); runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted); reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view)); return reifiedPerWindowAndSorted; }
@Override public void visitValue(PValue value, TransformHierarchy.Node producer) { LOG.debug("Checking translation of {}", value); // Primitive transforms are the only ones assigned step names. if (producer.getTransform() instanceof CreateDataflowView && !hasExperiment(options, "beam_fn_api")) { // CreateDataflowView produces a dummy output (as it must be a primitive transform) // but in the Dataflow Job graph produces only the view and not the output PCollection. asOutputReference( ((CreateDataflowView) producer.getTransform()).getView(), producer.toAppliedPTransform(getPipeline())); return; } else if (producer.getTransform() instanceof View.CreatePCollectionView && hasExperiment(options, "beam_fn_api")) { // View.CreatePCollectionView produces a dummy output (as it must be a primitive transform) // but in the Dataflow Job graph produces only the view and not the output PCollection. asOutputReference( ((View.CreatePCollectionView) producer.getTransform()).getView(), producer.toAppliedPTransform(getPipeline())); return; } asOutputReference(value, producer.toAppliedPTransform(getPipeline())); }
flattenedOutputs.apply(CreateDataflowView.forBatch(view)); return flattenedOutputs;
public static <ElemT, ViewT> CreateDataflowView<ElemT, ViewT> forStreaming( PCollectionView<ViewT> view) { return new CreateDataflowView<>(view, true); }