Repository: beam
Updated Branches:
  refs/heads/master f9bc76364 -> 66d7b6f84


[BEAM-2989] Fixed error when using Void type in WithKeys.

Also added error messages to WithKeys.of and VoidCoder.encode.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/bf0182d2
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/bf0182d2
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/bf0182d2

Branch: refs/heads/master
Commit: bf0182d2edd1d4436305bb1c8dc414b012a878a9
Parents: f9bc763
Author: Daniel Oliveira <daniel.o.program...@gmail.com>
Authored: Fri Sep 29 14:10:00 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Tue Oct 3 11:38:44 2017 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/transforms/WithKeys.java    |  6 +++-
 .../beam/sdk/transforms/WithKeysTest.java       | 30 ++++++++++++++++++--
 2 files changed, 32 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/bf0182d2/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
index c66d1b1..79cac99 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -64,6 +66,8 @@ public class WithKeys<K, V> extends PTransform<PCollection<V>,
    * be called on the result {@link PTransform}.
    */
   public static <K, V> WithKeys<K, V> of(SerializableFunction<V, K> fn) {
+    checkNotNull(fn,
+        "WithKeys constructed with null function. Did you mean 
WithKeys.of((Void) null)?");
     return new WithKeys<>(fn, null);
   }
 
@@ -82,7 +86,7 @@ public class WithKeys<K, V> extends PTransform<PCollection<V>,
             return key;
           }
         },
-        (Class<K>) (key == null ? null : key.getClass()));
+        (Class<K>) (key == null ? Void.class : key.getClass()));
   }
 
 

http://git-wip-us.apache.org/repos/asf/beam/blob/bf0182d2/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
index 8abbf1a..444979e 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
@@ -39,7 +39,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class WithKeysTest {
-  static final String[] COLLECTION = new String[] {
+  private static final String[] COLLECTION = new String[] {
     "a",
     "aa",
     "b",
@@ -47,7 +47,7 @@ public class WithKeysTest {
     "bbb"
   };
 
-  static final List<KV<Integer, String>> WITH_KEYS = Arrays.asList(
+  private static final List<KV<Integer, String>> WITH_KEYS = Arrays.asList(
     KV.of(1, "a"),
     KV.of(2, "aa"),
     KV.of(1, "b"),
@@ -55,7 +55,7 @@ public class WithKeysTest {
     KV.of(3, "bbb")
   );
 
-  static final List<KV<Integer, String>> WITH_CONST_KEYS = Arrays.asList(
+  private static final List<KV<Integer, String>> WITH_CONST_KEYS = 
Arrays.asList(
     KV.of(100, "a"),
     KV.of(100, "aa"),
     KV.of(100, "b"),
@@ -63,6 +63,14 @@ public class WithKeysTest {
     KV.of(100, "bbb")
   );
 
+  private static final List<KV<Void, String>> WITH_CONST_NULL_KEYS = 
Arrays.asList(
+      KV.of((Void) null, "a"),
+      KV.of((Void) null, "aa"),
+      KV.of((Void) null, "b"),
+      KV.of((Void) null, "bb"),
+      KV.of((Void) null, "bbb")
+  );
+
   @Rule
   public final TestPipeline p = TestPipeline.create();
 
@@ -99,6 +107,22 @@ public class WithKeysTest {
   }
 
   @Test
+  @Category(NeedsRunner.class)
+  public void testConstantVoidKeys() {
+
+    PCollection<String> input =
+        p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder(
+            StringUtf8Coder.of()));
+
+    PCollection<KV<Void, String>> output =
+        input.apply(WithKeys.<Void, String>of((Void) null));
+    PAssert.that(output)
+        .containsInAnyOrder(WITH_CONST_NULL_KEYS);
+
+    p.run();
+  }
+
+  @Test
   public void testWithKeysGetName() {
     assertEquals("WithKeys", WithKeys.<Integer, String>of(100).getName());
   }

Reply via email to