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 2016/08/04 01:26:10 UTC

[02/19] incubator-beam git commit: Rename DoFnWithContext to DoFn

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bcb6f46/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 0a6eab0..7fe053c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -168,17 +168,17 @@ public class ParDoTest implements Serializable {
     public void processElement(OldDoFn<Integer, String>.ProcessContext c) throws Exception {}
   }
 
-  static class TestDoFnWithContext extends DoFnWithContext<Integer, String> {
+  static class TestDoFn extends DoFn<Integer, String> {
     enum State { UNSTARTED, STARTED, PROCESSING, FINISHED }
     State state = State.UNSTARTED;
 
     final List<PCollectionView<Integer>> sideInputViews = new ArrayList<>();
     final List<TupleTag<String>> sideOutputTupleTags = new ArrayList<>();
 
-    public TestDoFnWithContext() {
+    public TestDoFn() {
     }
 
-    public TestDoFnWithContext(List<PCollectionView<Integer>> sideInputViews,
+    public TestDoFn(List<PCollectionView<Integer>> sideInputViews,
                     List<TupleTag<String>> sideOutputTupleTags) {
       this.sideInputViews.addAll(sideInputViews);
       this.sideOutputTupleTags.addAll(sideOutputTupleTags);
@@ -362,7 +362,7 @@ public class ParDoTest implements Serializable {
 
     PCollection<String> output = pipeline
         .apply(Create.of(inputs))
-        .apply(ParDo.of(new TestDoFnWithContext()));
+        .apply(ParDo.of(new TestDoFn()));
 
     PAssert.that(output)
         .satisfies(ParDoTest.HasExpectedOutput.forInput(inputs));
@@ -1426,7 +1426,7 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testDoFnWithContextDisplayData() {
-    DoFnWithContext<String, String> fn = new DoFnWithContext<String, String>() {
+    DoFn<String, String> fn = new DoFn<String, String>() {
       @ProcessElement
       public void proccessElement(ProcessContext c) {}
 
@@ -1445,7 +1445,7 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testWithOutputTagsDisplayData() {
-    DoFnWithContext<String, String> fn = new DoFnWithContext<String, String>() {
+    DoFn<String, String> fn = new DoFn<String, String>() {
       @ProcessElement
       public void proccessElement(ProcessContext c) {}
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bcb6f46/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java
index 5ff2bf1..90fba12 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.transforms.dofnreflector;
 
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnReflectorTest.Invocations;
-import org.apache.beam.sdk.transforms.DoFnWithContext;
 
 /**
  * Test helper for DoFnReflectorTest, which needs to test package-private access
@@ -26,7 +26,7 @@ import org.apache.beam.sdk.transforms.DoFnWithContext;
  */
 public class DoFnReflectorTestHelper {
 
-  private static class StaticPrivateDoFn extends DoFnWithContext<String, String> {
+  private static class StaticPrivateDoFn extends DoFn<String, String> {
     final Invocations invocations;
 
     public StaticPrivateDoFn(Invocations invocations) {
@@ -39,7 +39,7 @@ public class DoFnReflectorTestHelper {
     }
   }
 
-  private class InnerPrivateDoFn extends DoFnWithContext<String, String> {
+  private class InnerPrivateDoFn extends DoFn<String, String> {
     final Invocations invocations;
 
     public InnerPrivateDoFn(Invocations invocations) {
@@ -52,7 +52,7 @@ public class DoFnReflectorTestHelper {
     }
   }
 
-  static class StaticPackagePrivateDoFn extends DoFnWithContext<String, String> {
+  static class StaticPackagePrivateDoFn extends DoFn<String, String> {
     final Invocations invocations;
 
     public StaticPackagePrivateDoFn(Invocations invocations) {
@@ -65,7 +65,7 @@ public class DoFnReflectorTestHelper {
     }
   }
 
-  class InnerPackagePrivateDoFn extends DoFnWithContext<String, String> {
+  class InnerPackagePrivateDoFn extends DoFn<String, String> {
     final Invocations invocations;
 
     public InnerPackagePrivateDoFn(Invocations invocations) {
@@ -78,25 +78,25 @@ public class DoFnReflectorTestHelper {
     }
   }
 
-  public static DoFnWithContext<String, String> newStaticPackagePrivateDoFn(
+  public static DoFn<String, String> newStaticPackagePrivateDoFn(
       Invocations invocations) {
     return new StaticPackagePrivateDoFn(invocations);
   }
 
-  public DoFnWithContext<String, String> newInnerPackagePrivateDoFn(Invocations invocations) {
+  public DoFn<String, String> newInnerPackagePrivateDoFn(Invocations invocations) {
     return new InnerPackagePrivateDoFn(invocations);
   }
 
-  public static DoFnWithContext<String, String> newStaticPrivateDoFn(Invocations invocations) {
+  public static DoFn<String, String> newStaticPrivateDoFn(Invocations invocations) {
     return new StaticPrivateDoFn(invocations);
   }
 
-  public DoFnWithContext<String, String> newInnerPrivateDoFn(Invocations invocations) {
+  public DoFn<String, String> newInnerPrivateDoFn(Invocations invocations) {
     return new InnerPrivateDoFn(invocations);
   }
 
-  public DoFnWithContext<String, String> newInnerAnonymousDoFn(final Invocations invocations) {
-    return new DoFnWithContext<String, String>() {
+  public DoFn<String, String> newInnerAnonymousDoFn(final Invocations invocations) {
+    return new DoFn<String, String>() {
       @ProcessElement
       public void process(ProcessContext c) {
         invocations.wasProcessElementInvoked = true;
@@ -104,9 +104,9 @@ public class DoFnReflectorTestHelper {
     };
   }
 
-  public static DoFnWithContext<String, String> newStaticAnonymousDoFn(
+  public static DoFn<String, String> newStaticAnonymousDoFn(
       final Invocations invocations) {
-    return new DoFnWithContext<String, String>() {
+    return new DoFn<String, String>() {
       @ProcessElement
       public void process(ProcessContext c) {
         invocations.wasProcessElementInvoked = true;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bcb6f46/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
index 0da75f4..fd75e95 100644
--- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
@@ -20,10 +20,10 @@ package org.apache.beam.sdk.microbenchmarks.transforms;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory;
 import org.apache.beam.sdk.transforms.DoFnReflector;
 import org.apache.beam.sdk.transforms.DoFnReflector.DoFnInvoker;
-import org.apache.beam.sdk.transforms.DoFnWithContext;
-import org.apache.beam.sdk.transforms.DoFnWithContext.ExtraContextFactory;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
@@ -40,7 +40,7 @@ import org.openjdk.jmh.annotations.State;
 import org.openjdk.jmh.annotations.Warmup;
 
 /**
- * Benchmarks for {@link OldDoFn} and {@link DoFnWithContext} invocations, specifically
+ * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically
  * for measuring the overhead of {@link DoFnReflector}.
  */
 @State(Scope.Benchmark)
@@ -50,12 +50,12 @@ public class DoFnReflectorBenchmark {
 
   private static final String ELEMENT = "some string to use for testing";
 
-  private OldDoFn<String, String> doFn = new UpperCaseDoFn();
-  private DoFnWithContext<String, String> doFnWithContext = new UpperCaseDoFnWithContext();
+  private OldDoFn<String, String> oldDoFn = new UpperCaseOldDoFn();
+  private DoFn<String, String> doFn = new UpperCaseDoFn();
 
-  private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(doFn, ELEMENT);
+  private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(oldDoFn, ELEMENT);
   private StubDoFnWithContextProcessContext stubDoFnWithContextContext =
-      new StubDoFnWithContextProcessContext(doFnWithContext, ELEMENT);
+      new StubDoFnWithContextProcessContext(doFn, ELEMENT);
   private ExtraContextFactory<String, String> extraContextFactory =
       new ExtraContextFactory<String, String>() {
 
@@ -77,14 +77,14 @@ public class DoFnReflectorBenchmark {
 
   @Setup
   public void setUp() {
-    doFnReflector = DoFnReflector.of(doFnWithContext.getClass());
-    adaptedDoFnWithContext = doFnReflector.toDoFn(doFnWithContext);
-    invoker = doFnReflector.bindInvoker(doFnWithContext);
+    doFnReflector = DoFnReflector.of(doFn.getClass());
+    adaptedDoFnWithContext = doFnReflector.toDoFn(doFn);
+    invoker = doFnReflector.bindInvoker(doFn);
   }
 
   @Benchmark
   public String invokeDoFn() throws Exception {
-    doFn.processElement(stubDoFnContext);
+    oldDoFn.processElement(stubDoFnContext);
     return stubDoFnContext.output;
   }
 
@@ -100,7 +100,7 @@ public class DoFnReflectorBenchmark {
     return stubDoFnWithContextContext.output;
   }
 
-  private static class UpperCaseDoFn extends OldDoFn<String, String> {
+  private static class UpperCaseOldDoFn extends OldDoFn<String, String> {
 
     @Override
     public void processElement(ProcessContext c) throws Exception {
@@ -108,7 +108,7 @@ public class DoFnReflectorBenchmark {
     }
   }
 
-  private static class UpperCaseDoFnWithContext extends DoFnWithContext<String, String> {
+  private static class UpperCaseDoFn extends DoFn<String, String> {
 
     @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
@@ -187,11 +187,11 @@ public class DoFnReflectorBenchmark {
   }
 
   private static class StubDoFnWithContextProcessContext
-      extends DoFnWithContext<String, String>.ProcessContext {
+      extends DoFn<String, String>.ProcessContext {
     private final String element;
     private  String output;
 
-    public StubDoFnWithContextProcessContext(DoFnWithContext<String, String> fn, String element) {
+    public StubDoFnWithContextProcessContext(DoFn<String, String> fn, String element) {
       fn.super();
       this.element = element;
     }