@Override public boolean matches(AppliedPTransform<?, ?, ?> application) { PTransform<?, ?> transform = application.getTransform(); if (transform instanceof SplittableParDo.ProcessKeyedElements) { DoFn<?, ?> fn = ((SplittableParDo.ProcessKeyedElements) transform).getFn(); DoFnSignature signature = DoFnSignatures.signatureForDoFn(fn); return signature.processElement().isSplittable() && signature.isBoundedPerElement() == IsBounded.BOUNDED; } return false; }
@Override public boolean matches(AppliedPTransform<?, ?, ?> application) { PTransform<?, ?> transform = application.getTransform(); if (transform instanceof SplittableParDo.ProcessKeyedElements) { DoFn<?, ?> fn = ((SplittableParDo.ProcessKeyedElements) transform).getFn(); DoFnSignature signature = DoFnSignatures.signatureForDoFn(fn); return signature.processElement().isSplittable() && signature.isBoundedPerElement() == IsBounded.UNBOUNDED; } return false; }
@Test public void testSplittableRespectsBoundednessAnnotation() throws Exception { @BoundedPerElement class BoundedFnWithContinuation extends BaseFnWithContinuation {} assertEquals( PCollection.IsBounded.BOUNDED, DoFnSignatures.getSignature(BoundedFnWithContinuation.class).isBoundedPerElement()); @UnboundedPerElement class UnboundedFnWithContinuation extends BaseFnWithContinuation {} assertEquals( PCollection.IsBounded.UNBOUNDED, DoFnSignatures.getSignature(UnboundedFnWithContinuation.class).isBoundedPerElement()); }
@Test public void testSplittableBoundednessInferredFromReturnValue() throws Exception { assertEquals( PCollection.IsBounded.BOUNDED, DoFnSignatures.getSignature(BaseFnWithoutContinuation.class).isBoundedPerElement()); assertEquals( PCollection.IsBounded.UNBOUNDED, DoFnSignatures.getSignature(BaseFnWithContinuation.class).isBoundedPerElement()); }
@Test public void testUnsplittableIsBounded() throws Exception { class UnsplittableFn extends DoFn<Integer, String> { @ProcessElement public void process(ProcessContext context) {} } assertEquals( PCollection.IsBounded.BOUNDED, DoFnSignatures.getSignature(UnsplittableFn.class).isBoundedPerElement()); }
@Test public void testInfersBoundednessFromAnnotation() throws Exception { class BaseSplittableFn extends DoFn<Integer, String> { @ProcessElement public void processElement( ProcessContext context, RestrictionTracker<SomeRestriction, Void> tracker) {} @GetInitialRestriction public SomeRestriction getInitialRestriction(Integer element) { return null; } } @BoundedPerElement class BoundedSplittableFn extends BaseSplittableFn {} @UnboundedPerElement class UnboundedSplittableFn extends BaseSplittableFn {} assertEquals( PCollection.IsBounded.BOUNDED, DoFnSignatures.getSignature(BaseSplittableFn.class).isBoundedPerElement()); assertEquals( PCollection.IsBounded.BOUNDED, DoFnSignatures.getSignature(BoundedSplittableFn.class).isBoundedPerElement()); assertEquals( PCollection.IsBounded.UNBOUNDED, DoFnSignatures.getSignature(UnboundedSplittableFn.class).isBoundedPerElement()); }
public static <OutputT> PCollectionTuple createPrimitiveOutputFor( PCollection<?> input, DoFn<?, OutputT> fn, TupleTag<OutputT> mainOutputTag, TupleTagList additionalOutputTags, Map<TupleTag<?>, Coder<?>> outputTagsToCoders, WindowingStrategy<?, ?> windowingStrategy) { DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), outputTagsToCoders, windowingStrategy, input.isBounded().and(signature.isBoundedPerElement())); // Set output type descriptor similarly to how ParDo.MultiOutput does it. outputs.get(mainOutputTag).setTypeDescriptor(fn.getOutputTypeDescriptor()); return outputs; }
@Override public PTransformReplacement< PCollection<KV<byte[], KV<InputT, RestrictionT>>>, PCollectionTuple> getReplacementTransform( AppliedPTransform< PCollection<KV<byte[], KV<InputT, RestrictionT>>>, PCollectionTuple, ProcessKeyedElements<InputT, OutputT, RestrictionT>> transform) { checkArgument( DoFnSignatures.signatureForDoFn(transform.getTransform().getFn()).isBoundedPerElement() == IsBounded.BOUNDED, "Expecting a bounded-per-element splittable DoFn"); return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), new SplittableProcessNaive<>(transform.getTransform())); }
input.isBounded().and(signature.isBoundedPerElement())); @SuppressWarnings("unchecked") Coder<InputT> inputCoder = ((PCollection<InputT>) input).getCoder();