/** * Expands a list of {@link PCollectionView} into the form needed for {@link * PTransform#getAdditionalInputs()}. */ public static Map<TupleTag<?>, PValue> toAdditionalInputs(Iterable<PCollectionView<?>> views) { ImmutableMap.Builder<TupleTag<?>, PValue> additionalInputs = ImmutableMap.builder(); for (PCollectionView<?> view : views) { additionalInputs.put(view.getTagInternal(), view.getPCollection()); } return additionalInputs.build(); }
@Override public PCollection<ElemT> expand(PCollection<ElemT> input) { if (streaming) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } return (PCollection) view.getPCollection(); }
addVertex(sideInputTransformVertex); final Coder viewCoder = getCoderForView(view, this); final Coder windowCoder = view.getPCollection().getWindowingStrategy().getWindowFn().windowCoder();
@Test public void getMainInputSingleOutputSideInputs() { AppliedPTransform<PCollection<Long>, ?, ?> application = AppliedPTransform.of( "application", ImmutableMap.<TupleTag<?>, PValue>builder() .put(new TupleTag<Long>(), mainInput) .put(sideInput.getTagInternal(), sideInput.getPCollection()) .build(), Collections.singletonMap(new TupleTag<Long>(), output), ParDo.of(new TestDoFn()).withSideInputs(sideInput), pipeline); PCollection<Long> input = PTransformReplacements.getSingletonMainInput(application); assertThat(input, equalTo(mainInput)); }
@Test public void getMainInputNoMainInputsThrows() { ImmutableMap<TupleTag<?>, PValue> inputs = ImmutableMap.<TupleTag<?>, PValue>builder() .put(sideInput.getTagInternal(), sideInput.getPCollection()) .build(); AppliedPTransform<PCollection<Long>, ?, ?> application = AppliedPTransform.of( "application", inputs, Collections.singletonMap(new TupleTag<Long>(), output), ParDo.of(new TestDoFn()).withSideInputs(sideInput), pipeline); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("No main input"); PTransformReplacements.getSingletonMainInput(application); }
sideInput, view.getTagInternal().getId(), view.getPCollection(), protoTransform, rehydratedComponents);
@Test public void getMainInputExtraMainInputsThrows() { PCollection<Long> notInParDo = pipeline.apply("otherPCollection", Create.of(1L, 2L, 3L)); ImmutableMap<TupleTag<?>, PValue> inputs = ImmutableMap.<TupleTag<?>, PValue>builder() .putAll(mainInput.expand()) // Not represnted as an input .put(new TupleTag<Long>(), notInParDo) .put(sideInput.getTagInternal(), sideInput.getPCollection()) .build(); AppliedPTransform<PCollection<Long>, ?, ?> application = AppliedPTransform.of( "application", inputs, Collections.singletonMap(new TupleTag<Long>(), output), ParDo.of(new TestDoFn()).withSideInputs(sideInput), pipeline); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("multiple inputs"); thrown.expectMessage("not additional inputs"); thrown.expectMessage(mainInput.toString()); thrown.expectMessage(notInParDo.toString()); PTransformReplacements.getSingletonMainInput(application); }