You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2017/03/01 04:10:49 UTC

[2/9] beam git commit: BEAM-1420 GroupByKey should comply with PTransform style guide

BEAM-1420 GroupByKey should comply with PTransform style guide


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

Branch: refs/heads/master
Commit: a7c60cc0b9d59d19398f788b84a17d4686ea3f82
Parents: 63b63f0
Author: Eugene Kirpichov <ki...@google.com>
Authored: Tue Feb 7 16:17:21 2017 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Feb 28 20:10:36 2017 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/transforms/Combine.java   | 9 +++++----
 .../java/org/apache/beam/sdk/transforms/GroupByKey.java     | 8 ++++----
 .../java/org/apache/beam/sdk/transforms/GroupByKeyTest.java | 2 +-
 3 files changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a7c60cc0/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index 51c5e71..b4626e7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -43,7 +43,6 @@ import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn;
@@ -1877,9 +1876,11 @@ public class Combine {
     @Override
     public PCollection<KV<K, OutputT>> expand(PCollection<KV<K, InputT>> input) {
       return input
-          .apply(GroupByKey.<K, InputT>create(fewKeys))
-          .apply(Combine.<K, InputT, OutputT>groupedValues(fn, fnDisplayData)
-              .withSideInputs(sideInputs));
+          .apply(
+              fewKeys ? GroupByKey.<K, InputT>createWithFewKeys() : GroupByKey.<K, InputT>create())
+          .apply(
+              Combine.<K, InputT, OutputT>groupedValues(fn, fnDisplayData)
+                  .withSideInputs(sideInputs));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/a7c60cc0/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
index 1541059..adf189b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java
@@ -142,17 +142,17 @@ public class GroupByKey<K, V>
   }
 
   /**
-   * Returns a {@code GroupByKey<K, V>} {@code PTransform}.
+   * Returns a {@code GroupByKey<K, V>} {@code PTransform} that assumes it will be grouping
+   * a small number of keys.
    *
    * @param <K> the type of the keys of the input and output
    * {@code PCollection}s
    * @param <V> the type of the values of the input {@code PCollection}
    * and the elements of the {@code Iterable}s in the output
    * {@code PCollection}
-   * @param fewKeys whether it groups just few keys.
    */
-  static <K, V> GroupByKey<K, V> create(boolean fewKeys) {
-    return new GroupByKey<>(fewKeys);
+  static <K, V> GroupByKey<K, V> createWithFewKeys() {
+    return new GroupByKey<>(true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a7c60cc0/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
index f4bec3a..73cedfd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
@@ -374,7 +374,7 @@ public class GroupByKeyTest {
   @Test
   public void testDisplayData() {
     GroupByKey<String, String> groupByKey = GroupByKey.create();
-    GroupByKey<String, String> groupByFewKeys = GroupByKey.create(true);
+    GroupByKey<String, String> groupByFewKeys = GroupByKey.createWithFewKeys();
 
     DisplayData gbkDisplayData = DisplayData.from(groupByKey);
     DisplayData fewKeysDisplayData = DisplayData.from(groupByFewKeys);