beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dhalp...@apache.org
Subject [6/9] beam git commit: BEAM-1426 SortValues should comply with PTransform style guide
Date Wed, 01 Mar 2017 04:10:53 GMT
BEAM-1426 SortValues 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/fffd2c55
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/fffd2c55
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/fffd2c55

Branch: refs/heads/master
Commit: fffd2c55895e4eb03ae0796d3ce9c4f75b9b34c3
Parents: 4f34caa
Author: Eugene Kirpichov <kirpichov@google.com>
Authored: Tue Feb 7 16:39:33 2017 -0800
Committer: Dan Halperin <dhalperi@google.com>
Committed: Tue Feb 28 20:10:37 2017 -0800

----------------------------------------------------------------------
 .../sorter/BufferedExternalSorter.java          | 23 ++++++----
 .../sorter/BufferedExternalSorterTest.java      | 46 ++++++++++----------
 .../sdk/extensions/sorter/SortValuesTest.java   |  2 +-
 3 files changed, 39 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/fffd2c55/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
index 1a16511..efa251a 100644
--- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
+++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
@@ -29,19 +29,27 @@ import org.apache.beam.sdk.values.KV;
  * then fall back to external sorting.
  */
 public class BufferedExternalSorter implements Sorter {
+  public static Options options() {
+    return new Options("/tmp", 100);
+  }
+
   /** Contains configuration for the sorter. */
   public static class Options implements Serializable {
-    private String tempLocation = "/tmp";
-    private int memoryMB = 100;
+    private final String tempLocation;
+    private final int memoryMB;
+
+    private Options(String tempLocation, int memoryMB) {
+      this.tempLocation = tempLocation;
+      this.memoryMB = memoryMB;
+    }
 
     /** Sets the path to a temporary location where the sorter writes intermediate files.
*/
-    public Options setTempLocation(String tempLocation) {
+    public Options withTempLocation(String tempLocation) {
       checkArgument(
           !tempLocation.startsWith("gs://"),
           "BufferedExternalSorter does not support GCS temporary location");
 
-      this.tempLocation = tempLocation;
-      return this;
+      return new Options(tempLocation, memoryMB);
     }
 
     /** Returns the configured temporary location. */
@@ -54,13 +62,12 @@ public class BufferedExternalSorter implements Sorter {
      * memory sorting and the buffer used when external sorting. Must be greater than zero
and less
      * than 2048.
      */
-    public Options setMemoryMB(int memoryMB) {
+    public Options withMemoryMB(int memoryMB) {
       checkArgument(memoryMB > 0, "memoryMB must be greater than zero");
       // Hadoop's external sort stores the number of available memory bytes in an int, this
prevents
       // overflow
       checkArgument(memoryMB < 2048, "memoryMB must be less than 2048");
-      this.memoryMB = memoryMB;
-      return this;
+      return new Options(tempLocation, memoryMB);
     }
 
     /** Returns the configured size of the memory buffer. */

http://git-wip-us.apache.org/repos/asf/beam/blob/fffd2c55/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java
b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java
index 3d63b1a..f2be1f5 100644
--- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java
+++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java
@@ -137,29 +137,29 @@ public class BufferedExternalSorterTest {
 
   @Test
   public void testEmpty() throws Exception {
-    SorterTestUtils.testEmpty(BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-        .setTempLocation(tmpLocation.toString())));
+    SorterTestUtils.testEmpty(BufferedExternalSorter.create(BufferedExternalSorter.options()
+        .withTempLocation(tmpLocation.toString())));
   }
 
   @Test
   public void testSingleElement() throws Exception {
     SorterTestUtils.testSingleElement(
-        BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-            .setTempLocation(tmpLocation.toString())));
+        BufferedExternalSorter.create(BufferedExternalSorter.options()
+            .withTempLocation(tmpLocation.toString())));
   }
 
   @Test
   public void testEmptyKeyValueElement() throws Exception {
     SorterTestUtils.testEmptyKeyValueElement(
-        BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-            .setTempLocation(tmpLocation.toString())));
+        BufferedExternalSorter.create(BufferedExternalSorter.options()
+            .withTempLocation(tmpLocation.toString())));
   }
 
   @Test
   public void testMultipleIterations() throws Exception {
     SorterTestUtils.testMultipleIterations(
-        BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-            .setTempLocation(tmpLocation.toString())));
+        BufferedExternalSorter.create(BufferedExternalSorter.options()
+            .withTempLocation(tmpLocation.toString())));
   }
 
   @Test
@@ -168,8 +168,8 @@ public class BufferedExternalSorterTest {
         new SorterGenerator() {
           @Override
           public Sorter generateSorter() throws Exception {
-            return BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-                .setTempLocation(tmpLocation.toString()));
+            return BufferedExternalSorter.create(BufferedExternalSorter.options()
+                .withTempLocation(tmpLocation.toString()));
           }
         },
         1000000,
@@ -182,8 +182,8 @@ public class BufferedExternalSorterTest {
         new SorterGenerator() {
           @Override
           public Sorter generateSorter() throws Exception {
-            return BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-                .setTempLocation(tmpLocation.toString()));
+            return BufferedExternalSorter.create(BufferedExternalSorter.options()
+                .withTempLocation(tmpLocation.toString()));
           }
         },
         1,
@@ -193,16 +193,16 @@ public class BufferedExternalSorterTest {
   @Test
   public void testAddAfterSort() throws Exception {
     SorterTestUtils.testAddAfterSort(
-        BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-              .setTempLocation(tmpLocation.toString())), thrown);
+        BufferedExternalSorter.create(BufferedExternalSorter.options()
+              .withTempLocation(tmpLocation.toString())), thrown);
     fail();
   }
 
   @Test
   public void testSortTwice() throws Exception {
     SorterTestUtils.testSortTwice(
-        BufferedExternalSorter.create(new BufferedExternalSorter.Options()
-            .setTempLocation(tmpLocation.toString())), thrown);
+        BufferedExternalSorter.create(BufferedExternalSorter.options()
+            .withTempLocation(tmpLocation.toString())), thrown);
     fail();
   }
 
@@ -210,24 +210,24 @@ public class BufferedExternalSorterTest {
   public void testNegativeMemory() throws Exception {
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("memoryMB must be greater than zero");
-    BufferedExternalSorter.Options options = new BufferedExternalSorter.Options()
-        .setTempLocation(tmpLocation.toString());
-    options.setMemoryMB(-1);
+    BufferedExternalSorter.Options options = BufferedExternalSorter.options()
+        .withTempLocation(tmpLocation.toString());
+    options.withMemoryMB(-1);
   }
 
   @Test
   public void testZeroMemory() throws Exception {
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("memoryMB must be greater than zero");
-    BufferedExternalSorter.Options options = new BufferedExternalSorter.Options();
-    options.setMemoryMB(0);
+    BufferedExternalSorter.Options options = BufferedExternalSorter.options();
+    options.withMemoryMB(0);
   }
 
   @Test
   public void testMemoryTooLarge() throws Exception {
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("memoryMB must be less than 2048");
-    BufferedExternalSorter.Options options = new BufferedExternalSorter.Options();
-    options.setMemoryMB(2048);
+    BufferedExternalSorter.Options options = BufferedExternalSorter.options();
+    options.withMemoryMB(2048);
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/fffd2c55/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
index 4f77100..96d6b6a 100644
--- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
+++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
@@ -66,7 +66,7 @@ public class SortValuesTest {
     // For every Key, sort the iterable of <SecondaryKey, Value> pairs by SecondaryKey.
     PCollection<KV<String, Iterable<KV<String, Integer>>>> groupedAndSorted
=
         grouped.apply(
-            SortValues.<String, String, Integer>create(new BufferedExternalSorter.Options()));
+            SortValues.<String, String, Integer>create(BufferedExternalSorter.options()));
 
     PAssert.that(groupedAndSorted)
         .satisfies(new AssertThatHasExpectedContentsForTestSecondaryKeySorting());


Mime
View raw message