Repository: beam Updated Branches: refs/heads/master 13767de88 -> 83c9831f4
Add a Test for Flatten with Heterogeneous Coders Add a category, and suppress in the Flink and Apex runners Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/01c0e475 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/01c0e475 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/01c0e475 Branch: refs/heads/master Commit: 01c0e475453ac7715bfcaa414eb4185c79eb4ddf Parents: 13767de Author: Thomas Groh <[email protected]> Authored: Wed Dec 21 13:53:48 2016 -0800 Committer: Thomas Groh <[email protected]> Committed: Thu Jan 5 13:47:26 2017 -0800 ---------------------------------------------------------------------- runners/apex/pom.xml | 1 + runners/flink/runner/pom.xml | 1 + .../testing/FlattenWithHeterogeneousCoders.java | 29 ++++++++++++++++++++ .../apache/beam/sdk/transforms/FlattenTest.java | 27 ++++++++++++++++++ 4 files changed, 58 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/01c0e475/runners/apex/pom.xml ---------------------------------------------------------------------- diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index e8020a1..47139a6 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -184,6 +184,7 @@ <configuration> <groups>org.apache.beam.sdk.testing.RunnableOnService</groups> <excludedGroups> + org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, http://git-wip-us.apache.org/repos/asf/beam/blob/01c0e475/runners/flink/runner/pom.xml ---------------------------------------------------------------------- diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index eadbe6e..1146e97 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -54,6 +54,7 @@ <configuration> <groups>org.apache.beam.sdk.testing.RunnableOnService</groups> <excludedGroups> + org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, http://git-wip-us.apache.org/repos/asf/beam/blob/01c0e475/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FlattenWithHeterogeneousCoders.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FlattenWithHeterogeneousCoders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FlattenWithHeterogeneousCoders.java new file mode 100644 index 0000000..3187265 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FlattenWithHeterogeneousCoders.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; + +/** + * Category tag for tests that use a {@link Flatten} where the input {@link PCollectionList} + * contains {@link PCollection PCollections} heterogeneous {@link Coder coders}. + */ +public interface FlattenWithHeterogeneousCoders {} http://git-wip-us.apache.org/repos/asf/beam/blob/01c0e475/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 48251bc..3b5011b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -31,14 +31,19 @@ import java.util.Collection; import java.util.List; import java.util.Set; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CollectionCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -147,6 +152,28 @@ public class FlattenTest implements Serializable { } @Test + @Category({RunnableOnService.class, FlattenWithHeterogeneousCoders.class}) + public void testFlattenMultipleCoders() throws CannotProvideCoderException { + PCollection<Long> bigEndianLongs = + p.apply( + "BigEndianLongs", + Create.of(0L, 1L, 2L, 3L, null, 4L, 5L, null, 6L, 7L, 8L, null, 9L) + .withCoder(NullableCoder.of(BigEndianLongCoder.of()))); + PCollection<Long> varLongs = + p.apply("VarLengthLongs", CountingInput.upTo(5L)).setCoder(VarLongCoder.of()); + + PCollection<Long> flattened = + PCollectionList.of(bigEndianLongs) + .and(varLongs) + .apply(Flatten.<Long>pCollections()) + .setCoder(NullableCoder.of(VarLongCoder.of())); + PAssert.that(flattened) + .containsInAnyOrder( + 0L, 0L, 1L, 1L, 2L, 3L, 2L, 4L, 5L, 3L, 6L, 7L, 4L, 8L, 9L, null, null, null); + p.run(); + } + + @Test @Category(RunnableOnService.class) public void testEmptyFlattenAsSideInput() { final PCollectionView<Iterable<String>> view =
