From ff28ee37bc2d6f2c11dd411ec3f23500a6d0910f Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 21 Dec 2016 13:53:48 -0800 Subject: [PATCH 1/2] Add a Test for Flatten with Heterogeneous Coders Add a category, and suppress in the Flink and Apex runners --- runners/apex/pom.xml | 1 + runners/flink/runner/pom.xml | 1 + .../FlattenWithHeterogeneousCoders.java | 29 +++++++++++++++++++ .../beam/sdk/transforms/FlattenTest.java | 24 +++++++++++++++ 4 files changed, 55 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FlattenWithHeterogeneousCoders.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index e8020a187290..47139a6942e0 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -184,6 +184,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index eadbe6eb639d..1146e97bd4e1 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -54,6 +54,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, 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 000000000000..31872650b589 --- /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 {} 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 48251bc0247c..956cad7bf59c 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.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; @@ -146,6 +151,25 @@ public Long apply(Long input) { p.run(); } + @Test + @Category({RunnableOnService.class, FlattenWithHeterogeneousCoders.class}) + public void testFlattenMultipleCoders() throws CannotProvideCoderException { + PCollection bigEndianLongs = + p.apply("BigEndianLongs", CountingInput.upTo(10L)) + .setCoder(NullableCoder.of(BigEndianLongCoder.of())); + PCollection varLongs = + p.apply("VarLengthLongs", CountingInput.upTo(5L)).setCoder(VarLongCoder.of()); + + PCollection flattened = + PCollectionList.of(bigEndianLongs) + .and(varLongs) + .apply(Flatten.pCollections()) + .setCoder(BigEndianLongCoder.of()); + PAssert.that(flattened) + .containsInAnyOrder(0L, 0L, 1L, 1L, 2L, 3L, 2L, 4L, 5L, 3L, 6L, 7L, 4L, 8L, 9L); + p.run(); + } + @Test @Category(RunnableOnService.class) public void testEmptyFlattenAsSideInput() { From 00f5c9f57bca483b241b8d8aab8a57b8ada7213e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 4 Jan 2017 13:55:26 -0800 Subject: [PATCH 2/2] fixup! Add a Test for Flatten with Heterogeneous Coders --- .../org/apache/beam/sdk/transforms/FlattenTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 956cad7bf59c..3b5011bf3308 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 @@ -155,8 +155,10 @@ public Long apply(Long input) { @Category({RunnableOnService.class, FlattenWithHeterogeneousCoders.class}) public void testFlattenMultipleCoders() throws CannotProvideCoderException { PCollection bigEndianLongs = - p.apply("BigEndianLongs", CountingInput.upTo(10L)) - .setCoder(NullableCoder.of(BigEndianLongCoder.of())); + p.apply( + "BigEndianLongs", + Create.of(0L, 1L, 2L, 3L, null, 4L, 5L, null, 6L, 7L, 8L, null, 9L) + .withCoder(NullableCoder.of(BigEndianLongCoder.of()))); PCollection varLongs = p.apply("VarLengthLongs", CountingInput.upTo(5L)).setCoder(VarLongCoder.of()); @@ -164,9 +166,10 @@ public void testFlattenMultipleCoders() throws CannotProvideCoderException { PCollectionList.of(bigEndianLongs) .and(varLongs) .apply(Flatten.pCollections()) - .setCoder(BigEndianLongCoder.of()); + .setCoder(NullableCoder.of(VarLongCoder.of())); PAssert.that(flattened) - .containsInAnyOrder(0L, 0L, 1L, 1L, 2L, 3L, 2L, 4L, 5L, 3L, 6L, 7L, 4L, 8L, 9L); + .containsInAnyOrder( + 0L, 0L, 1L, 1L, 2L, 3L, 2L, 4L, 5L, 3L, 6L, 7L, 4L, 8L, 9L, null, null, null); p.run(); }