@Override public PCollection<T> expand(PBegin input) { PCollectionList<T> empty = PCollectionList.empty(input.getPipeline()); return empty.apply(Flatten.pCollections()); } }
@Override public PCollection<T> expand(PCollection<T> input) { applicableTo(input); WindowingStrategy<?, ?> outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); if (getWindowFn() == null) { // A new PCollection must be created in case input is reused in a different location as the // two PCollections will, in general, have a different windowing strategy. return PCollectionList.of(input) .apply(Flatten.pCollections()) .setWindowingStrategyInternal(outputStrategy); } else { // This is the AssignWindows primitive return input.apply(new Assign<>(this, outputStrategy)); } }
@Test @Category(ValidatesRunner.class) public void testFlattenPCollectionsSingletonList() { PCollection<String> input = p.apply(Create.of(LINES)); PCollection<String> output = PCollectionList.of(input).apply(Flatten.pCollections()); assertThat(output, not(equalTo(input))); PAssert.that(output).containsInAnyOrder(LINES); p.run(); }
@Test @Category(ValidatesRunner.class) public void testFlattenPCollections() { List<List<String>> inputs = Arrays.asList(LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES); PCollection<String> output = makePCollectionListOfStrings(p, inputs).apply(Flatten.pCollections()); PAssert.that(output).containsInAnyOrder(flattenLists(inputs)); p.run(); }
@Test @Category(NeedsRunner.class) public void testFlattenNoListsNoCoder() { // not ValidatesRunner because it should fail at pipeline construction time anyhow. thrown.expect(IllegalStateException.class); thrown.expectMessage("Unable to return a default Coder"); PCollectionList.<ClassWithoutCoder>empty(p).apply(Flatten.pCollections()); p.run(); }
@Test @Category(ValidatesRunner.class) public void testFlattenPCollectionsThenParDo() { List<List<String>> inputs = Arrays.asList(LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES); PCollection<String> output = makePCollectionListOfStrings(p, inputs) .apply(Flatten.pCollections()) .apply(ParDo.of(new IdentityFn<>())); PAssert.that(output).containsInAnyOrder(flattenLists(inputs)); p.run(); }
@Test public void duplicatesInsertsMultipliers() { PTransform<PCollectionList<String>, PCollection<String>> replacement = new DeduplicatedFlattenFactory.FlattenWithoutDuplicateInputs<>(); final PCollectionList<String> inputList = PCollectionList.of(first).and(second).and(first).and(first); inputList.apply(replacement); pipeline.traverseTopologically( new Defaults() { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (node.getTransform() instanceof Flatten.PCollections) { assertThat(node.getInputs(), not(equalTo(inputList.expand()))); } } }); }
@Test public void outputMapping() { final PCollectionList<String> inputList = PCollectionList.of(first).and(second).and(first).and(first); PCollection<String> original = inputList.apply(Flatten.pCollections()); PCollection<String> replacement = inputList.apply(new FlattenWithoutDuplicateInputs<>()); assertThat( factory.mapOutputs(original.expand(), replacement), Matchers.hasEntry( replacement, ReplacementOutput.of( TaggedPValue.ofExpandedValue(original), TaggedPValue.ofExpandedValue(replacement)))); } }
@Test @Category(ValidatesRunner.class) public void testFlattenPCollectionsEmpty() { PCollection<String> output = PCollectionList.<String>empty(p) .apply(Flatten.pCollections()) .setCoder(StringUtf8Coder.of()); PAssert.that(output).empty(); p.run(); }
@Test @Category(NeedsRunner.class) public void testDroppedPartition() { // Compute the set of integers either 1 or 2 mod 3, the hard way. PCollectionList<Integer> outputs = pipeline .apply(Create.of(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) .apply(Partition.of(3, new ModFn())); List<PCollection<Integer>> outputsList = new ArrayList<>(outputs.getAll()); outputsList.remove(0); outputs = PCollectionList.of(outputsList); assertTrue(outputs.size() == 2); PCollection<Integer> output = outputs.apply(Flatten.pCollections()); PAssert.that(output).containsInAnyOrder(2, 4, 5, 7, 8, 10, 11); pipeline.run(); }
@Test public void testFlattenSingleCollection() { ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); Pipeline p = Pipeline.create(); PCollection<String> single = p.apply(Create.of(Collections.singletonList("1"))); PCollectionList.of(single) .apply(Flatten.pCollections()) .apply(ParDo.of(new EmbeddedCollector())); DAG dag = TestApexRunner.translate(p, options); Assert.assertNotNull( dag.getOperatorMeta("ParDo(EmbeddedCollector)/ParMultiDo(EmbeddedCollector)")); } }
@Test @Category(NeedsRunner.class) public void testOverride() { final PCollectionList<String> inputList = PCollectionList.of(first).and(second).and(first).and(first); PTransform<PCollectionList<String>, PCollection<String>> replacement = new FlattenWithoutDuplicateInputs<>(); PCollection<String> flattened = inputList.apply(replacement); PAssert.that(flattened).containsInAnyOrder("one", "two", "one", "one"); pipeline.run(); }
@Test public void getValueToConsumersWithDuplicateInputSucceeds() { PCollection<String> created = p.apply(Create.of("1", "2", "3")); PCollection<String> flattened = PCollectionList.of(created).and(created).apply(Flatten.pCollections()); p.traverseTopologically(visitor); DirectGraph graph = visitor.getGraph(); AppliedPTransform<?, ?, ?> flattenedProducer = graph.getProducer(flattened); assertThat( graph.getPerElementConsumers(created), Matchers.containsInAnyOrder(new Object[] {flattenedProducer, flattenedProducer})); assertThat(graph.getPerElementConsumers(flattened), emptyIterable()); }
@Test @Category(ValidatesRunner.class) public void testFlattenPCollectionsEmptyThenParDo() { PCollection<String> output = PCollectionList.<String>empty(p) .apply(Flatten.pCollections()) .setCoder(StringUtf8Coder.of()) .apply(ParDo.of(new IdentityFn<>())); PAssert.that(output).empty(); p.run(); }
@Test @Category({ValidatesRunner.class, UsesParDoLifecycle.class}) public void testFnCallSequence() { PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) .apply(ParDo.of(new CallSequenceEnforcingFn<>())); p.run(); }
@Test @Category(ValidatesRunner.class) public void testMultipleApply() { PTransform<PCollection<? extends String>, PCollection<String>> myTransform = addSuffix("+"); PCollection<String> input = pipeline.apply(Create.of(ImmutableList.of("a", "b"))); PCollection<String> left = input.apply("Left1", myTransform).apply("Left2", myTransform); PCollection<String> right = input.apply("Right", myTransform); PCollection<String> both = PCollectionList.of(left).and(right).apply(Flatten.pCollections()); PAssert.that(both).containsInAnyOrder("a++", "b++", "a+", "b+"); pipeline.run(); }
@Test @Category({ValidatesRunner.class, UsesParDoLifecycle.class}) public void testFnCallSequenceMulti() { PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) .apply( ParDo.of(new CallSequenceEnforcingFn<Integer>()) .withOutputTags(new TupleTag<Integer>() {}, TupleTagList.empty())); p.run(); }
@Test @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesParDoLifecycle.class}) public void testFnCallSequenceStateful() { PCollectionList.of(p.apply("Impolite", Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 4)))) .and( p.apply( "Polite", Create.of(KV.of("b", 3), KV.of("a", 5), KV.of("c", 6), KV.of("c", 7)))) .apply(Flatten.pCollections()) .apply( ParDo.of(new CallSequenceEnforcingStatefulFn<String, Integer>()) .withOutputTags(new TupleTag<KV<String, Integer>>() {}, TupleTagList.empty())); p.run(); }
@Test @Category(NeedsRunner.class) public void testEqualWindowFnPropagation() { PCollection<String> input1 = p.apply("CreateInput1", Create.of("Input1")) .apply("Window1", Window.into(FixedWindows.of(Duration.standardMinutes(1)))); PCollection<String> input2 = p.apply("CreateInput2", Create.of("Input2")) .apply("Window2", Window.into(FixedWindows.of(Duration.standardMinutes(1)))); PCollection<String> output = PCollectionList.of(input1).and(input2).apply(Flatten.pCollections()); p.run(); Assert.assertTrue( output .getWindowingStrategy() .getWindowFn() .isCompatible(FixedWindows.of(Duration.standardMinutes(1)))); }
@Test @Category(NeedsRunner.class) public void testCompatibleWindowFnPropagation() { PCollection<String> input1 = p.apply("CreateInput1", Create.of("Input1")) .apply("Window1", Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))); PCollection<String> input2 = p.apply("CreateInput2", Create.of("Input2")) .apply("Window2", Window.into(Sessions.withGapDuration(Duration.standardMinutes(2)))); PCollection<String> output = PCollectionList.of(input1).and(input2).apply(Flatten.pCollections()); p.run(); Assert.assertTrue( output .getWindowingStrategy() .getWindowFn() .isCompatible(Sessions.withGapDuration(Duration.standardMinutes(2)))); }