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/10/26 16:43:39 UTC

[01/50] [abbrv] incubator-beam git commit: Port direct runner to use new DoFn directly

Repository: incubator-beam
Updated Branches:
  refs/heads/gearpump-runner 8f4334cf8 -> 3933b5577


Port direct runner to use new DoFn directly


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

Branch: refs/heads/gearpump-runner
Commit: 1919d8b3a850bd146137652546da851ee461cd28
Parents: f0c8d30
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 20 20:55:00 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/DoFnLifecycleManager.java    | 42 +++++++++--------
 .../beam/runners/direct/ParDoEvaluator.java     |  3 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |  6 +--
 .../direct/ParDoSingleEvaluatorFactory.java     |  5 +-
 ...leManagerRemovingTransformEvaluatorTest.java | 16 +++----
 .../direct/DoFnLifecycleManagerTest.java        | 12 ++---
 .../direct/DoFnLifecycleManagersTest.java       | 48 ++++++++++++++++----
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 11 +++++
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 11 +++++
 .../beam/runners/direct/SplittableDoFnTest.java |  8 +++-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 23 ++++++++--
 11 files changed, 130 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
index 0e15c18..23460b6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
@@ -26,7 +26,9 @@ import java.util.Collection;
 import java.util.Iterator;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn.Setup;
+import org.apache.beam.sdk.transforms.DoFn.Teardown;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,49 +37,49 @@ import org.slf4j.LoggerFactory;
  * Manages {@link DoFn} setup, teardown, and serialization.
  *
  * <p>{@link DoFnLifecycleManager} is similar to a {@link ThreadLocal} storing a {@link DoFn}, but
- * calls the {@link DoFn} {@link Setup} the first time the {@link DoFn} is obtained and {@link
- * Teardown} whenever the {@link DoFn} is removed, and provides a method for clearing all cached
- * {@link DoFn DoFns}.
+ * calls the {@link DoFn} {@link Setup @Setup} method the first time the {@link DoFn} is obtained
+ * and {@link Teardown @Teardown} whenever the {@link DoFn} is removed, and provides a method for
+ * clearing all cached {@link DoFn DoFns}.
  */
 class DoFnLifecycleManager {
   private static final Logger LOG = LoggerFactory.getLogger(DoFnLifecycleManager.class);
 
-  public static DoFnLifecycleManager of(OldDoFn<?, ?> original) {
+  public static DoFnLifecycleManager of(DoFn<?, ?> original) {
     return new DoFnLifecycleManager(original);
   }
 
-  private final LoadingCache<Thread, OldDoFn<?, ?>> outstanding;
+  private final LoadingCache<Thread, DoFn<?, ?>> outstanding;
 
-  private DoFnLifecycleManager(OldDoFn<?, ?> original) {
+  private DoFnLifecycleManager(DoFn<?, ?> original) {
     this.outstanding = CacheBuilder.newBuilder().build(new DeserializingCacheLoader(original));
   }
 
-  public OldDoFn<?, ?> get() throws Exception {
+  public DoFn<?, ?> get() throws Exception {
     Thread currentThread = Thread.currentThread();
     return outstanding.get(currentThread);
   }
 
   public void remove() throws Exception {
     Thread currentThread = Thread.currentThread();
-    OldDoFn<?, ?> fn = outstanding.asMap().remove(currentThread);
-    fn.teardown();
+    DoFn<?, ?> fn = outstanding.asMap().remove(currentThread);
+    DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
   }
 
   /**
    * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. Returns all exceptions
    * that were thrown while calling the remove methods.
    *
-   * <p>If the returned Collection is nonempty, an exception was thrown from at least one
-   * {@link DoFn#teardown()} method, and the {@link PipelineRunner} should throw an exception.
+   * <p>If the returned Collection is nonempty, an exception was thrown from at least one {@link
+   * DoFn.Teardown @Teardown} method, and the {@link PipelineRunner} should throw an exception.
    */
   public Collection<Exception> removeAll() throws Exception {
-    Iterator<OldDoFn<?, ?>> fns = outstanding.asMap().values().iterator();
+    Iterator<DoFn<?, ?>> fns = outstanding.asMap().values().iterator();
     Collection<Exception> thrown = new ArrayList<>();
     while (fns.hasNext()) {
-      OldDoFn<?, ?> fn = fns.next();
+      DoFn<?, ?> fn = fns.next();
       fns.remove();
       try {
-        fn.teardown();
+        DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
       } catch (Exception e) {
         thrown.add(e);
       }
@@ -85,18 +87,18 @@ class DoFnLifecycleManager {
     return thrown;
   }
 
-  private class DeserializingCacheLoader extends CacheLoader<Thread, OldDoFn<?, ?>> {
+  private class DeserializingCacheLoader extends CacheLoader<Thread, DoFn<?, ?>> {
     private final byte[] original;
 
-    public DeserializingCacheLoader(OldDoFn<?, ?> original) {
+    public DeserializingCacheLoader(DoFn<?, ?> original) {
       this.original = SerializableUtils.serializeToByteArray(original);
     }
 
     @Override
-    public OldDoFn<?, ?> load(Thread key) throws Exception {
-      OldDoFn<?, ?> fn = (OldDoFn<?, ?>) SerializableUtils.deserializeFromByteArray(original,
+    public DoFn<?, ?> load(Thread key) throws Exception {
+      DoFn<?, ?> fn = (DoFn<?, ?>) SerializableUtils.deserializeFromByteArray(original,
           "DoFn Copy in thread " + key.getName());
-      fn.setup();
+      DoFnInvokers.INSTANCE.invokerFor(fn).invokeSetup();
       return fn;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
index a59fb4d..b524dfa 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.direct;
 
 import com.google.common.collect.ImmutableList;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -45,7 +46,7 @@ class ParDoEvaluator<T> implements TransformEvaluator<T> {
       DirectStepContext stepContext,
       CommittedBundle<InputT> inputBundle,
       AppliedPTransform<PCollection<InputT>, ?, ?> application,
-      Object fn,
+      Serializable fn, // may be OldDoFn or DoFn
       List<PCollectionView<?>> sideInputs,
       TupleTag<OutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
index d909e8b..02469ff 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -24,7 +24,7 @@ import java.util.Map;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.values.PCollection;
@@ -50,7 +50,7 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
           public DoFnLifecycleManager load(AppliedPTransform<?, ?, ?> key)
               throws Exception {
             BoundMulti<?, ?> bound = (BoundMulti<?, ?>) key.getTransform();
-            return DoFnLifecycleManager.of(bound.getFn());
+            return DoFnLifecycleManager.of(bound.getNewFn());
           }
         });
   }
@@ -87,7 +87,7 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
               stepContext,
               inputBundle,
               application,
-              (OldDoFn) fnLocal.get(),
+              (DoFn) fnLocal.get(),
               application.getTransform().getSideInputs(),
               application.getTransform().getMainOutputTag(),
               application.getTransform().getSideOutputTags().getAll(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
index 1a06ea6..0584e41 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo.Bound;
 import org.apache.beam.sdk.values.PCollection;
@@ -52,7 +51,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
                   public DoFnLifecycleManager load(AppliedPTransform<?, ?, ?> key)
                       throws Exception {
                     Bound<?, ?> bound = (Bound<?, ?>) key.getTransform();
-                    return DoFnLifecycleManager.of(bound.getFn());
+                    return DoFnLifecycleManager.of(bound.getNewFn());
                   }
                 });
   }
@@ -92,7 +91,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
               stepContext,
               inputBundle,
               application,
-              (OldDoFn) fnLocal.get(),
+              fnLocal.get(),
               application.getTransform().getSideInputs(),
               mainOutputTag,
               Collections.<TupleTag<?>>emptyList(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
index 2e4fee2..9e2732e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
@@ -27,7 +27,7 @@ import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.hamcrest.Matchers;
 import org.junit.Before;
@@ -50,7 +50,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void delegatesToUnderlying() throws Exception {
     RecordingTransformEvaluator underlying = new RecordingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     TransformEvaluator<Object> evaluator =
         DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager);
     WindowedValue<Object> first = WindowedValue.valueInGlobalWindow(new Object());
@@ -67,7 +67,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void removesOnExceptionInProcessElement() throws Exception {
     ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     assertThat(original, not(nullValue()));
     TransformEvaluator<Object> evaluator =
         DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager);
@@ -75,7 +75,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
     try {
       evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object()));
     } catch (Exception e) {
-      assertThat(lifecycleManager.get(), not(Matchers.<OldDoFn<?, ?>>theInstance(original)));
+      assertThat(lifecycleManager.get(), not(Matchers.<DoFn<?, ?>>theInstance(original)));
       return;
     }
     fail("Expected ThrowingTransformEvaluator to throw on method call");
@@ -84,7 +84,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void removesOnExceptionInFinishBundle() throws Exception {
     ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     // the LifecycleManager is set when the evaluator starts
     assertThat(original, not(nullValue()));
     TransformEvaluator<Object> evaluator =
@@ -94,7 +94,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
       evaluator.finishBundle();
     } catch (Exception e) {
       assertThat(lifecycleManager.get(),
-          Matchers.not(Matchers.<OldDoFn<?, ?>>theInstance(original)));
+          Matchers.not(Matchers.<DoFn<?, ?>>theInstance(original)));
       return;
     }
     fail("Expected ThrowingTransformEvaluator to throw on method call");
@@ -134,8 +134,8 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   }
 
 
-  private static class TestFn extends OldDoFn<Object, Object> {
-    @Override
+  private static class TestFn extends DoFn<Object, Object> {
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
index 1f0af99..aef9d29 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.hamcrest.Matchers;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -101,7 +101,7 @@ public class DoFnLifecycleManagerTest {
     assertThat(obtained.setupCalled, is(true));
     assertThat(obtained.teardownCalled, is(true));
 
-    assertThat(mgr.get(), not(Matchers.<OldDoFn<?, ?>>theInstance(obtained)));
+    assertThat(mgr.get(), not(Matchers.<DoFn<?, ?>>theInstance(obtained)));
   }
 
   @Test
@@ -142,11 +142,11 @@ public class DoFnLifecycleManagerTest {
   }
 
 
-  private static class TestFn extends OldDoFn<Object, Object> {
+  private static class TestFn extends DoFn<Object, Object> {
     boolean setupCalled = false;
     boolean teardownCalled = false;
 
-    @Override
+    @Setup
     public void setup() {
       checkState(!setupCalled);
       checkState(!teardownCalled);
@@ -154,11 +154,11 @@ public class DoFnLifecycleManagerTest {
       setupCalled = true;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
 
-    @Override
+    @Teardown
     public void teardown() {
       checkState(setupCalled);
       checkState(!teardownCalled);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
index 39a4a9d..a19ff99 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
@@ -18,12 +18,15 @@
 
 package org.apache.beam.runners.direct;
 
+import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
 
 import com.google.common.collect.ImmutableList;
 import java.util.ArrayList;
 import java.util.Collection;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.UserCodeException;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
@@ -51,9 +54,15 @@ public class DoFnLifecycleManagersTest {
     third.get();
 
     final Collection<Matcher<? super Throwable>> suppressions = new ArrayList<>();
-    suppressions.add(new ThrowableMessageMatcher("foo"));
-    suppressions.add(new ThrowableMessageMatcher("bar"));
-    suppressions.add(new ThrowableMessageMatcher("baz"));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("foo"))));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("bar"))));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("baz"))));
 
     thrown.expect(
         new BaseMatcher<Exception>() {
@@ -90,18 +99,18 @@ public class DoFnLifecycleManagersTest {
     DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third));
   }
 
-  private static class ThrowsInCleanupFn extends OldDoFn<Object, Object> {
+  private static class ThrowsInCleanupFn extends DoFn<Object, Object> {
     private final String message;
 
     private ThrowsInCleanupFn(String message) {
       this.message = message;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
 
-    @Override
+    @Teardown
     public void teardown() throws Exception {
       throw new Exception(message);
     }
@@ -130,9 +139,32 @@ public class DoFnLifecycleManagersTest {
     }
   }
 
+  private static class CausedByMatcher extends BaseMatcher<Throwable> {
+    private final Matcher<Throwable> causeMatcher;
+
+    public CausedByMatcher(
+        Matcher<Throwable> causeMatcher) {
+      this.causeMatcher = causeMatcher;
+    }
 
-  private static class EmptyFn extends OldDoFn<Object, Object> {
     @Override
+    public boolean matches(Object item) {
+      if (!(item instanceof UserCodeException)) {
+        return false;
+      }
+      UserCodeException that = (UserCodeException) item;
+      return causeMatcher.matches(that.getCause());
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("a throwable with a cause ").appendDescriptionOf(causeMatcher);
+    }
+  }
+
+
+  private static class EmptyFn extends DoFn<Object, Object> {
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index 88e1484..8b0070b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -57,6 +57,7 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -236,6 +237,11 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
             WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStatePutsStateInResult() throws Exception {
     TestPipeline p = TestPipeline.create();
@@ -321,6 +327,11 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
         containsInAnyOrder("foo", "bara", "bazam"));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
     TestPipeline p = TestPipeline.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index 6a02e40..e562b28 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -54,6 +54,7 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -166,6 +167,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     assertThat(result.getAggregatorChanges(), equalTo(mutator));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStatePutsStateInResult() throws Exception {
     TestPipeline p = TestPipeline.create();
@@ -238,6 +244,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
         containsInAnyOrder("foo", "bara", "bazam"));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
     TestPipeline p = TestPipeline.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
index 84a0cd9..c164ce6 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
@@ -34,7 +34,6 @@ import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
@@ -46,6 +45,7 @@ import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.MutableDateTime;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -140,6 +140,9 @@ public class SplittableDoFnTest {
     }
   }
 
+  @Ignore(
+      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
+          + "It must be implemented as a primitive.")
   @Test
   public void testPairWithIndexBasic() throws ClassNotFoundException {
     Pipeline p = TestPipeline.create();
@@ -164,6 +167,9 @@ public class SplittableDoFnTest {
     p.run();
   }
 
+  @Ignore(
+      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
+          + "It must be implemented as a primitive.")
   @Test
   public void testPairWithIndexWindowedTimestamped() throws ClassNotFoundException {
     // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index a9f26a4..f16e0b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -73,6 +73,10 @@ import org.joda.time.Instant;
 public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDisplayData {
 
   public DoFn<InputT, OutputT> toDoFn() {
+    DoFn<InputT, OutputT> doFn = DoFnAdapters.getDoFn(this);
+    if (doFn != null) {
+      return doFn;
+    }
     if (this instanceof RequiresWindowAccess) {
       // No parameters as it just accesses `this`
       return new AdaptedRequiresWindowAccessDoFn();
@@ -553,8 +557,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
     private final DoFn<InputT, OutputT>.ProcessContext newContext;
 
-    public AdaptedProcessContext(
-        DoFn<InputT, OutputT>.ProcessContext newContext) {
+    public AdaptedProcessContext(DoFn<InputT, OutputT>.ProcessContext newContext) {
       this.newContext = newContext;
     }
 
@@ -632,21 +635,31 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
   private class AdaptedDoFn extends DoFn<InputT, OutputT> {
 
+    @Setup
+    public void setup() throws Exception {
+      OldDoFn.this.setup();
+    }
+
     @StartBundle
-    public void startBundle(DoFn.Context c) throws Exception {
+    public void startBundle(Context c) throws Exception {
       OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
     }
 
     @ProcessElement
-    public void processElement(DoFn.ProcessContext c) throws Exception {
+    public void processElement(ProcessContext c) throws Exception {
       OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c));
     }
 
     @FinishBundle
-    public void finishBundle(DoFn.Context c) throws Exception {
+    public void finishBundle(Context c) throws Exception {
       OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
     }
 
+    @Teardown
+    public void teardown() throws Exception {
+      OldDoFn.this.teardown();
+    }
+
     @Override
     public Duration getAllowedTimestampSkew() {
       return OldDoFn.this.getAllowedTimestampSkew();


[37/50] [abbrv] incubator-beam git commit: This closes #1158

Posted by dh...@apache.org.
This closes #1158


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

Branch: refs/heads/gearpump-runner
Commit: 0ec956d3f91656acbf17eb09eca9c4aa2b317b82
Parents: f603d43 5ce3aac
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 13:04:17 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/core/GroupAlsoByWindowsDoFn.java     | 19 -------------------
 1 file changed, 19 deletions(-)
----------------------------------------------------------------------



[29/50] [abbrv] incubator-beam git commit: Remove pieces of Trigger now owned by TriggerStateMachine

Posted by dh...@apache.org.
Remove pieces of Trigger now owned by TriggerStateMachine


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

Branch: refs/heads/gearpump-runner
Commit: 1eff320d5e7fb5510d13016e0826b14e5cf7f686
Parents: dfaf2a8
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 12:57:37 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 10:12:56 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/transforms/windowing/AfterAll.java |  49 --
 .../windowing/AfterDelayFromFirstElement.java   |  99 ----
 .../sdk/transforms/windowing/AfterEach.java     |  61 ---
 .../sdk/transforms/windowing/AfterFirst.java    |  50 --
 .../sdk/transforms/windowing/AfterPane.java     |  52 --
 .../windowing/AfterProcessingTime.java          |   7 -
 .../AfterSynchronizedProcessingTime.java        |   7 -
 .../transforms/windowing/AfterWatermark.java    | 158 ------
 .../transforms/windowing/DefaultTrigger.java    |  35 --
 .../beam/sdk/transforms/windowing/Never.java    |  17 -
 .../transforms/windowing/OrFinallyTrigger.java  |  46 --
 .../sdk/transforms/windowing/Repeatedly.java    |  30 --
 .../beam/sdk/transforms/windowing/Trigger.java  | 412 ++-------------
 .../apache/beam/sdk/util/ExecutableTrigger.java |  40 +-
 .../apache/beam/sdk/util/FinishedTriggers.java  |  44 --
 .../beam/sdk/util/FinishedTriggersBitSet.java   |  67 ---
 .../beam/sdk/util/FinishedTriggersSet.java      |  72 ---
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |  14 -
 .../beam/sdk/util/TriggerContextFactory.java    | 507 -------------------
 .../sdk/transforms/windowing/AfterAllTest.java  |  98 ----
 .../sdk/transforms/windowing/AfterEachTest.java |  64 ---
 .../transforms/windowing/AfterFirstTest.java    | 120 -----
 .../sdk/transforms/windowing/AfterPaneTest.java |  77 ---
 .../windowing/AfterProcessingTimeTest.java      |  94 ----
 .../AfterSynchronizedProcessingTimeTest.java    |  75 ---
 .../windowing/AfterWatermarkTest.java           | 308 -----------
 .../windowing/DefaultTriggerTest.java           | 130 -----
 .../sdk/transforms/windowing/NeverTest.java     |  34 +-
 .../windowing/OrFinallyTriggerTest.java         | 136 -----
 .../transforms/windowing/RepeatedlyTest.java    | 161 +-----
 .../sdk/transforms/windowing/StubTrigger.java   |  17 -
 .../sdk/transforms/windowing/TriggerTest.java   |  28 -
 .../beam/sdk/util/ExecutableTriggerTest.java    |  18 -
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 --
 .../sdk/util/FinishedTriggersProperties.java    | 110 ----
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ---
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  23 -
 .../org/apache/beam/sdk/util/TriggerTester.java | 410 ---------------
 38 files changed, 77 insertions(+), 3708 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
index 0e37d33..c3f0848 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
@@ -24,7 +24,6 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.joda.time.Instant;
 
 /**
@@ -46,27 +45,6 @@ public class AfterAll extends OnceTrigger {
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    for (ExecutableTrigger subTrigger : c.trigger().unfinishedSubTriggers()) {
-      // Since subTriggers are all OnceTriggers, they must either CONTINUE or FIRE_AND_FINISH.
-      // invokeElement will automatically mark the finish bit if they return FIRE_AND_FINISH.
-      subTrigger.invokeOnElement(c);
-    }
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      subTrigger.invokeOnMerge(c);
-    }
-    boolean allFinished = true;
-    for (ExecutableTrigger subTrigger1 : c.trigger().subTriggers()) {
-      allFinished &= c.forTrigger(subTrigger1).trigger().isFinished();
-    }
-    c.trigger().setFinished(allFinished);
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     // This trigger will fire after the latest of its sub-triggers.
     Instant deadline = BoundedWindow.TIMESTAMP_MIN_VALUE;
@@ -84,33 +62,6 @@ public class AfterAll extends OnceTrigger {
     return new AfterAll(continuationTriggers);
   }
 
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@code true} if all subtriggers return {@code true}.
-   */
-  @Override
-  public boolean shouldFire(TriggerContext context) throws Exception {
-    for (ExecutableTrigger subtrigger : context.trigger().subTriggers()) {
-      if (!context.forTrigger(subtrigger).trigger().isFinished()
-          && !subtrigger.invokeShouldFire(context)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Invokes {@link #onFire} for all subtriggers, eliding redundant calls to {@link #shouldFire}
-   * because they all must be ready to fire.
-   */
-  @Override
-  public void onOnlyFiring(TriggerContext context) throws Exception {
-    for (ExecutableTrigger subtrigger : context.trigger().subTriggers()) {
-      subtrigger.invokeOnFire(context);
-    }
-  }
-
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("AfterAll.of(");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
index 6078b34..9daecb2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
@@ -18,11 +18,9 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import com.google.common.collect.ImmutableList;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.util.List;
 import java.util.Locale;
 import java.util.Objects;
-import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.InstantCoder;
 import org.apache.beam.sdk.transforms.Combine;
@@ -31,10 +29,6 @@ import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.joda.time.Duration;
@@ -62,12 +56,6 @@ public abstract class AfterDelayFromFirstElement extends OnceTrigger {
   private static final PeriodFormatter PERIOD_FORMATTER = PeriodFormat.wordBased(Locale.ENGLISH);
 
   /**
-   * To complete an implementation, return the desired time from the TriggerContext.
-   */
-  @Nullable
-  public abstract Instant getCurrentTime(Trigger.TriggerContext context);
-
-  /**
    * To complete an implementation, return a new instance like this one, but incorporating
    * the provided timestamp mapping functions. Generally should be used by calling the
    * constructor of this class from the constructor of the subclass.
@@ -92,10 +80,6 @@ public abstract class AfterDelayFromFirstElement extends OnceTrigger {
     this.timeDomain = timeDomain;
   }
 
-  private Instant getTargetTimestamp(OnElementContext c) {
-    return computeTargetTimestamp(c.currentProcessingTime());
-  }
-
   /**
    * The time domain according for which this trigger sets timers.
    */
@@ -170,93 +154,10 @@ public abstract class AfterDelayFromFirstElement extends OnceTrigger {
   }
 
   @Override
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification =
-      "prefetch side effect")
-  public void prefetchOnElement(StateAccessor<?> state) {
-    state.access(DELAYED_UNTIL_TAG).readLater();
-  }
-
-  @Override
-  public void onElement(OnElementContext c) throws Exception {
-    CombiningState<Instant, Instant> delayUntilState = c.state().access(DELAYED_UNTIL_TAG);
-    Instant oldDelayUntil = delayUntilState.read();
-
-    // Since processing time can only advance, resulting in target wake-up times we would
-    // ignore anyhow, we don't bother with it if it is already set.
-    if (oldDelayUntil != null) {
-      return;
-    }
-
-    Instant targetTimestamp = getTargetTimestamp(c);
-    delayUntilState.add(targetTimestamp);
-    c.setTimer(targetTimestamp, timeDomain);
-  }
-
-  @Override
-  public void prefetchOnMerge(MergingStateAccessor<?, ?> state) {
-    super.prefetchOnMerge(state);
-    StateMerging.prefetchCombiningValues(state, DELAYED_UNTIL_TAG);
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    // NOTE: We could try to delete all timers which are still active, but we would
-    // need access to a timer context for each merging window.
-    // for (CombiningValueStateInternal<Instant, Combine.Holder<Instant>, Instant> state :
-    //    c.state().accessInEachMergingWindow(DELAYED_UNTIL_TAG).values()) {
-    //   Instant timestamp = state.get().read();
-    //   if (timestamp != null) {
-    //     <context for merging window>.deleteTimer(timestamp, timeDomain);
-    //   }
-    // }
-    // Instead let them fire and be ignored.
-
-    // If the trigger is already finished, there is no way it will become re-activated
-    if (c.trigger().isFinished()) {
-      StateMerging.clear(c.state(), DELAYED_UNTIL_TAG);
-      // NOTE: We do not attempt to delete  the timers.
-      return;
-    }
-
-    // Determine the earliest point across all the windows, and delay to that.
-    StateMerging.mergeCombiningValues(c.state(), DELAYED_UNTIL_TAG);
-
-    Instant earliestTargetTime = c.state().access(DELAYED_UNTIL_TAG).read();
-    if (earliestTargetTime != null) {
-      c.setTimer(earliestTargetTime, timeDomain);
-    }
-  }
-
-  @Override
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification =
-      "prefetch side effect")
-  public void prefetchShouldFire(StateAccessor<?> state) {
-    state.access(DELAYED_UNTIL_TAG).readLater();
-  }
-
-  @Override
-  public void clear(TriggerContext c) throws Exception {
-    c.state().access(DELAYED_UNTIL_TAG).clear();
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     return BoundedWindow.TIMESTAMP_MAX_VALUE;
   }
 
-  @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    Instant delayedUntil = context.state().access(DELAYED_UNTIL_TAG).read();
-    return delayedUntil != null
-        && getCurrentTime(context) != null
-        && getCurrentTime(context).isAfter(delayedUntil);
-  }
-
-  @Override
-  protected void onOnlyFiring(Trigger.TriggerContext context) throws Exception {
-    clear(context);
-  }
-
   protected Instant computeTargetTimestamp(Instant time) {
     Instant result = time;
     for (SerializableFunction<Instant, Instant> timestampMapper : timestampMappers) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
index 961d97f..872ad46 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
@@ -23,7 +23,6 @@ import com.google.common.base.Joiner;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.joda.time.Instant;
 
 /**
@@ -59,41 +58,6 @@ public class AfterEach extends Trigger {
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    if (!c.trigger().isMerging()) {
-      // If merges are not possible, we need only run the first unfinished subtrigger
-      c.trigger().firstUnfinishedSubTrigger().invokeOnElement(c);
-    } else {
-      // If merges are possible, we need to run all subtriggers in parallel
-      for (ExecutableTrigger subTrigger :  c.trigger().subTriggers()) {
-        // Even if the subTrigger is done, it may be revived via merging and must have
-        // adequate state.
-        subTrigger.invokeOnElement(c);
-      }
-    }
-  }
-
-  @Override
-  public void onMerge(OnMergeContext context) throws Exception {
-    // If merging makes a subtrigger no-longer-finished, it will automatically
-    // begin participating in shouldFire and onFire appropriately.
-
-    // All the following triggers are retroactively "not started" but that is
-    // also automatic because they are cleared whenever this trigger
-    // fires.
-    boolean priorTriggersAllFinished = true;
-    for (ExecutableTrigger subTrigger : context.trigger().subTriggers()) {
-      if (priorTriggersAllFinished) {
-        subTrigger.invokeOnMerge(context);
-        priorTriggersAllFinished &= context.forTrigger(subTrigger).trigger().isFinished();
-      } else {
-        subTrigger.invokeClear(context);
-      }
-    }
-    updateFinishedState(context);
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     // This trigger will fire at least once when the first trigger in the sequence
     // fires at least once.
@@ -106,27 +70,6 @@ public class AfterEach extends Trigger {
   }
 
   @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    ExecutableTrigger firstUnfinished = context.trigger().firstUnfinishedSubTrigger();
-    return firstUnfinished.invokeShouldFire(context);
-  }
-
-  @Override
-  public void onFire(Trigger.TriggerContext context) throws Exception {
-    context.trigger().firstUnfinishedSubTrigger().invokeOnFire(context);
-
-    // Reset all subtriggers if in a merging context; any may be revived by merging so they are
-    // all run in parallel for each pending pane.
-    if (context.trigger().isMerging()) {
-      for (ExecutableTrigger subTrigger : context.trigger().subTriggers()) {
-        subTrigger.invokeClear(context);
-      }
-    }
-
-    updateFinishedState(context);
-  }
-
-  @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("AfterEach.inOrder(");
     Joiner.on(", ").appendTo(builder, subTriggers);
@@ -134,8 +77,4 @@ public class AfterEach extends Trigger {
 
     return builder.toString();
   }
-
-  private void updateFinishedState(TriggerContext context) {
-    context.trigger().setFinished(context.trigger().firstUnfinishedSubTrigger() == null);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
index 7840fc4..a742b43 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
@@ -24,7 +24,6 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.joda.time.Instant;
 
 /**
@@ -47,21 +46,6 @@ public class AfterFirst extends OnceTrigger {
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      subTrigger.invokeOnElement(c);
-    }
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      subTrigger.invokeOnMerge(c);
-    }
-    updateFinishedStatus(c);
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     // This trigger will fire after the earliest of its sub-triggers.
     Instant deadline = BoundedWindow.TIMESTAMP_MAX_VALUE;
@@ -80,32 +64,6 @@ public class AfterFirst extends OnceTrigger {
   }
 
   @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    for (ExecutableTrigger subtrigger : context.trigger().subTriggers()) {
-      if (context.forTrigger(subtrigger).trigger().isFinished()
-          || subtrigger.invokeShouldFire(context)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  protected void onOnlyFiring(TriggerContext context) throws Exception {
-    for (ExecutableTrigger subtrigger : context.trigger().subTriggers()) {
-      TriggerContext subContext = context.forTrigger(subtrigger);
-      if (subtrigger.invokeShouldFire(subContext)) {
-        // If the trigger is ready to fire, then do whatever it needs to do.
-        subtrigger.invokeOnFire(subContext);
-      } else {
-        // If the trigger is not ready to fire, it is nonetheless true that whatever
-        // pending pane it was tracking is now gone.
-        subtrigger.invokeClear(subContext);
-      }
-    }
-  }
-
-  @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("AfterFirst.of(");
     Joiner.on(", ").appendTo(builder, subTriggers);
@@ -113,12 +71,4 @@ public class AfterFirst extends OnceTrigger {
 
     return builder.toString();
   }
-
-  private void updateFinishedStatus(TriggerContext c) {
-    boolean anyFinished = false;
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      anyFinished |= c.forTrigger(subTrigger).trigger().isFinished();
-    }
-    c.trigger().setFinished(anyFinished);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
index 4d59d58..4a706e6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.sdk.transforms.windowing;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.util.List;
 import java.util.Objects;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -25,9 +24,6 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.joda.time.Instant;
@@ -65,49 +61,6 @@ private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Lo
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    c.state().access(ELEMENTS_IN_PANE_TAG).add(1L);
-  }
-
-  @Override
-  public void prefetchOnMerge(MergingStateAccessor<?, ?> state) {
-    super.prefetchOnMerge(state);
-    StateMerging.prefetchCombiningValues(state, ELEMENTS_IN_PANE_TAG);
-  }
-
-  @Override
-  public void onMerge(OnMergeContext context) throws Exception {
-    // If we've already received enough elements and finished in some window,
-    // then this trigger is just finished.
-    if (context.trigger().finishedInAnyMergingWindow()) {
-      context.trigger().setFinished(true);
-      StateMerging.clear(context.state(), ELEMENTS_IN_PANE_TAG);
-      return;
-    }
-
-    // Otherwise, compute the sum of elements in all the active panes.
-    StateMerging.mergeCombiningValues(context.state(), ELEMENTS_IN_PANE_TAG);
-  }
-
-  @Override
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification =
-      "prefetch side effect")
-  public void prefetchShouldFire(StateAccessor<?> state) {
-    state.access(ELEMENTS_IN_PANE_TAG).readLater();
-  }
-
-  @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    long count = context.state().access(ELEMENTS_IN_PANE_TAG).read();
-    return count >= countElems;
-  }
-
-  @Override
-  public void clear(TriggerContext c) throws Exception {
-    c.state().access(ELEMENTS_IN_PANE_TAG).clear();
-  }
-
-  @Override
   public boolean isCompatible(Trigger other) {
     return this.equals(other);
   }
@@ -143,9 +96,4 @@ private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Lo
   public int hashCode() {
     return Objects.hash(countElems);
   }
-
-  @Override
-  protected void onOnlyFiring(Trigger.TriggerContext context) throws Exception {
-    clear(context);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
index f551118..09f288e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
@@ -19,7 +19,6 @@ package org.apache.beam.sdk.transforms.windowing;
 
 import java.util.List;
 import java.util.Objects;
-import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.TimeDomain;
@@ -36,12 +35,6 @@ import org.joda.time.Instant;
 @Experimental(Experimental.Kind.TRIGGER)
 public class AfterProcessingTime extends AfterDelayFromFirstElement {
 
-  @Override
-  @Nullable
-  public Instant getCurrentTime(Trigger.TriggerContext context) {
-    return context.currentProcessingTime();
-  }
-
   private AfterProcessingTime(List<SerializableFunction<Instant, Instant>> transforms) {
     super(TimeDomain.PROCESSING_TIME, transforms);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
index b96b293..b6258f8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms.windowing;
 import com.google.common.base.Objects;
 import java.util.Collections;
 import java.util.List;
-import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.joda.time.Instant;
@@ -31,12 +30,6 @@ import org.joda.time.Instant;
  */
 public class AfterSynchronizedProcessingTime extends AfterDelayFromFirstElement {
 
-  @Override
-  @Nullable
-  public Instant getCurrentTime(Trigger.TriggerContext context) {
-    return context.currentSynchronizedProcessingTime();
-  }
-
   public AfterSynchronizedProcessingTime() {
     super(TimeDomain.SYNCHRONIZED_PROCESSING_TIME,
         Collections.<SerializableFunction<Instant, Instant>>emptyList());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
index 89c1ba9..37b73a6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -25,7 +25,6 @@ import java.util.Objects;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.joda.time.Instant;
 
@@ -111,50 +110,6 @@ public class AfterWatermark {
     }
 
     @Override
-    public void onElement(OnElementContext c) throws Exception {
-      if (!c.trigger().isMerging()) {
-        // If merges can never happen, we just run the unfinished subtrigger
-        c.trigger().firstUnfinishedSubTrigger().invokeOnElement(c);
-      } else {
-        // If merges can happen, we run for all subtriggers because they might be
-        // de-activated or re-activated
-        for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-          subTrigger.invokeOnElement(c);
-        }
-      }
-    }
-
-    @Override
-    public void onMerge(OnMergeContext c) throws Exception {
-      // NOTE that the ReduceFnRunner will delete all end-of-window timers for the
-      // merged-away windows.
-
-      ExecutableTrigger earlySubtrigger = c.trigger().subTrigger(EARLY_INDEX);
-      // We check the early trigger to determine if we are still processing it or
-      // if the end of window has transitioned us to the late trigger
-      OnMergeContext earlyContext = c.forTrigger(earlySubtrigger);
-
-      // If the early trigger is still active in any merging window then it is still active in
-      // the new merged window, because even if the merged window is "done" some pending elements
-      // haven't had a chance to fire.
-      if (!earlyContext.trigger().finishedInAllMergingWindows() || !endOfWindowReached(c)) {
-        earlyContext.trigger().setFinished(false);
-        if (lateTrigger != null) {
-          ExecutableTrigger lateSubtrigger = c.trigger().subTrigger(LATE_INDEX);
-          OnMergeContext lateContext = c.forTrigger(lateSubtrigger);
-          lateContext.trigger().setFinished(false);
-          lateSubtrigger.invokeClear(lateContext);
-        }
-      } else {
-        // Otherwise the early trigger and end-of-window bit is done for good.
-        earlyContext.trigger().setFinished(true);
-        if (lateTrigger != null) {
-          c.trigger().subTrigger(LATE_INDEX).invokeOnMerge(c);
-        }
-      }
-    }
-
-    @Override
     public Trigger getContinuationTrigger() {
       return new AfterWatermarkEarlyAndLate(
           earlyTrigger.getContinuationTrigger(),
@@ -173,38 +128,6 @@ public class AfterWatermark {
       return window.maxTimestamp();
     }
 
-    private boolean endOfWindowReached(Trigger.TriggerContext context) {
-      return context.currentEventTime() != null
-          && context.currentEventTime().isAfter(context.window().maxTimestamp());
-    }
-
-    @Override
-    public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-      if (!context.trigger().isFinished(EARLY_INDEX)) {
-        // We have not yet transitioned to late firings.
-        // We should fire if either the trigger is ready or we reach the end of the window.
-        return context.trigger().subTrigger(EARLY_INDEX).invokeShouldFire(context)
-            || endOfWindowReached(context);
-      } else if (lateTrigger == null) {
-        return false;
-      } else {
-        // We are running the late trigger
-        return context.trigger().subTrigger(LATE_INDEX).invokeShouldFire(context);
-      }
-    }
-
-    @Override
-    public void onFire(Trigger.TriggerContext context) throws Exception {
-      if (!context.forTrigger(context.trigger().subTrigger(EARLY_INDEX)).trigger().isFinished()) {
-        onNonLateFiring(context);
-      } else if (lateTrigger != null) {
-        onLateFiring(context);
-      } else {
-        // all done
-        context.trigger().setFinished(true);
-      }
-    }
-
     @Override
     public String toString() {
       StringBuilder builder = new StringBuilder(TO_STRING);
@@ -225,47 +148,6 @@ public class AfterWatermark {
 
       return builder.toString();
     }
-
-    private void onNonLateFiring(Trigger.TriggerContext context) throws Exception {
-      // We have not yet transitioned to late firings.
-      ExecutableTrigger earlySubtrigger = context.trigger().subTrigger(EARLY_INDEX);
-      Trigger.TriggerContext earlyContext = context.forTrigger(earlySubtrigger);
-
-      if (!endOfWindowReached(context)) {
-        // This is an early firing, since we have not arrived at the end of the window
-        // Implicitly repeats
-        earlySubtrigger.invokeOnFire(context);
-        earlySubtrigger.invokeClear(context);
-        earlyContext.trigger().setFinished(false);
-      } else {
-        // We have arrived at the end of the window; terminate the early trigger
-        // and clear out the late trigger's state
-        if (earlySubtrigger.invokeShouldFire(context)) {
-          earlySubtrigger.invokeOnFire(context);
-        }
-        earlyContext.trigger().setFinished(true);
-        earlySubtrigger.invokeClear(context);
-
-        if (lateTrigger == null) {
-          // Done if there is no late trigger.
-          context.trigger().setFinished(true);
-        } else {
-          // If there is a late trigger, we transition to it, and need to clear its state
-          // because it was run in parallel.
-          context.trigger().subTrigger(LATE_INDEX).invokeClear(context);
-        }
-      }
-
-    }
-
-    private void onLateFiring(Trigger.TriggerContext context) throws Exception {
-      // We are firing the late trigger, with implicit repeat
-      ExecutableTrigger lateSubtrigger = context.trigger().subTrigger(LATE_INDEX);
-      lateSubtrigger.invokeOnFire(context);
-      // It is a OnceTrigger, so it must have finished; unfinished it and clear it
-      lateSubtrigger.invokeClear(context);
-      context.forTrigger(lateSubtrigger).trigger().setFinished(false);
-    }
   }
 
   /**
@@ -296,33 +178,6 @@ public class AfterWatermark {
     }
 
     @Override
-    public void onElement(OnElementContext c) throws Exception {
-      // We're interested in knowing when the input watermark passes the end of the window.
-      // (It is possible this has already happened, in which case the timer will be fired
-      // almost immediately).
-      c.setTimer(c.window().maxTimestamp(), TimeDomain.EVENT_TIME);
-    }
-
-    @Override
-    public void onMerge(OnMergeContext c) throws Exception {
-      // NOTE that the ReduceFnRunner will delete all end-of-window timers for the
-      // merged-away windows.
-
-      if (!c.trigger().finishedInAllMergingWindows()) {
-        // If the trigger is still active in any merging window then it is still active in the new
-        // merged window, because even if the merged window is "done" some pending elements haven't
-        // had a chance to fire
-        c.trigger().setFinished(false);
-      } else if (!endOfWindowReached(c)) {
-        // If the end of the new window has not been reached, then the trigger is active again.
-        c.trigger().setFinished(false);
-      } else {
-        // Otherwise it is done for good
-        c.trigger().setFinished(true);
-      }
-    }
-
-    @Override
     public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
       return window.maxTimestamp();
     }
@@ -346,18 +201,5 @@ public class AfterWatermark {
     public int hashCode() {
       return Objects.hash(getClass());
     }
-
-    @Override
-    public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-      return endOfWindowReached(context);
-    }
-
-    private boolean endOfWindowReached(Trigger.TriggerContext context) {
-      return context.currentEventTime() != null
-          && context.currentEventTime().isAfter(context.window().maxTimestamp());
-    }
-
-    @Override
-    protected void onOnlyFiring(Trigger.TriggerContext context) throws Exception { }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
index fee7cdf..a649b4f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
@@ -19,7 +19,6 @@ package org.apache.beam.sdk.transforms.windowing;
 
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.TimeDomain;
 import org.joda.time.Instant;
 
 /**
@@ -41,27 +40,6 @@ public class DefaultTrigger extends Trigger{
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    // If the end of the window has already been reached, then we are already ready to fire
-    // and do not need to set a wake-up timer.
-    if (!endOfWindowReached(c)) {
-      c.setTimer(c.window().maxTimestamp(), TimeDomain.EVENT_TIME);
-    }
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    // If the end of the window has already been reached, then we are already ready to fire
-    // and do not need to set a wake-up timer.
-    if (!endOfWindowReached(c)) {
-      c.setTimer(c.window().maxTimestamp(), TimeDomain.EVENT_TIME);
-    }
-  }
-
-  @Override
-  public void clear(TriggerContext c) throws Exception { }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     return window.maxTimestamp();
   }
@@ -76,17 +54,4 @@ public class DefaultTrigger extends Trigger{
   public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
     return this;
   }
-
-  @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    return endOfWindowReached(context);
-  }
-
-  private boolean endOfWindowReached(Trigger.TriggerContext context) {
-    return context.currentEventTime() != null
-        && context.currentEventTime().isAfter(context.window().maxTimestamp());
-  }
-
-  @Override
-  public void onFire(Trigger.TriggerContext context) throws Exception { }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
index 07b70f4..664ae83 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
@@ -48,12 +48,6 @@ public final class Never {
     }
 
     @Override
-    public void onElement(OnElementContext c) {}
-
-    @Override
-    public void onMerge(OnMergeContext c) {}
-
-    @Override
     protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
       return this;
     }
@@ -62,16 +56,5 @@ public final class Never {
     public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
       return BoundedWindow.TIMESTAMP_MAX_VALUE;
     }
-
-    @Override
-    public boolean shouldFire(Trigger.TriggerContext context) {
-      return false;
-    }
-
-    @Override
-    protected void onOnlyFiring(Trigger.TriggerContext context) {
-      throw new UnsupportedOperationException(
-          String.format("%s should never fire", getClass().getSimpleName()));
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
index 9bef45a..1ed9b55 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms.windowing;
 import com.google.common.annotations.VisibleForTesting;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.joda.time.Instant;
 
 /**
@@ -51,20 +50,6 @@ public class OrFinallyTrigger extends Trigger {
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    c.trigger().subTrigger(ACTUAL).invokeOnElement(c);
-    c.trigger().subTrigger(UNTIL).invokeOnElement(c);
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      subTrigger.invokeOnMerge(c);
-    }
-    updateFinishedState(c);
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     // This trigger fires once either the trigger or the until trigger fires.
     Instant actualDeadline = subTriggers.get(ACTUAL).getWatermarkThatGuaranteesFiring(window);
@@ -83,38 +68,7 @@ public class OrFinallyTrigger extends Trigger {
   }
 
   @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    return context.trigger().subTrigger(ACTUAL).invokeShouldFire(context)
-        || context.trigger().subTrigger(UNTIL).invokeShouldFire(context);
-  }
-
-  @Override
-  public void onFire(Trigger.TriggerContext context) throws Exception {
-    ExecutableTrigger actualSubtrigger = context.trigger().subTrigger(ACTUAL);
-    ExecutableTrigger untilSubtrigger = context.trigger().subTrigger(UNTIL);
-
-    if (untilSubtrigger.invokeShouldFire(context)) {
-      untilSubtrigger.invokeOnFire(context);
-      actualSubtrigger.invokeClear(context);
-    } else {
-      // If until didn't fire, then the actual must have (or it is forbidden to call
-      // onFire) so we are done only if actual is done.
-      actualSubtrigger.invokeOnFire(context);
-      // Do not clear the until trigger, because it tracks data cross firings.
-    }
-    updateFinishedState(context);
-  }
-
-  @Override
   public String toString() {
     return String.format("%s.orFinally(%s)", subTriggers.get(ACTUAL), subTriggers.get(UNTIL));
   }
-
-  private void updateFinishedState(TriggerContext c) throws Exception {
-    boolean anyStillFinished = false;
-    for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) {
-      anyStillFinished |= c.forTrigger(subTrigger).trigger().isFinished();
-    }
-    c.trigger().setFinished(anyStillFinished);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
index 45bc6c1..4d79a2c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
@@ -19,7 +19,6 @@ package org.apache.beam.sdk.transforms.windowing;
 
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.util.ExecutableTrigger;
 import org.joda.time.Instant;
 
 /**
@@ -61,16 +60,6 @@ public class Repeatedly extends Trigger {
   }
 
   @Override
-  public void onElement(OnElementContext c) throws Exception {
-    getRepeated(c).invokeOnElement(c);
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-    getRepeated(c).invokeOnMerge(c);
-  }
-
-  @Override
   public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
     // This trigger fires once the repeated trigger fires.
     return subTriggers.get(REPEATED).getWatermarkThatGuaranteesFiring(window);
@@ -82,26 +71,7 @@ public class Repeatedly extends Trigger {
   }
 
   @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    return getRepeated(context).invokeShouldFire(context);
-  }
-
-  @Override
-  public void onFire(TriggerContext context) throws Exception {
-    getRepeated(context).invokeOnFire(context);
-
-    if (context.trigger().isFinished(REPEATED)) {
-      // Reset tree will recursively clear the finished bits, and invoke clear.
-      context.forTrigger(getRepeated(context)).trigger().resetTree();
-    }
-  }
-
-  @Override
   public String toString() {
     return String.format("Repeatedly.forever(%s)", subTriggers.get(REPEATED));
   }
-
-  private ExecutableTrigger getRepeated(TriggerContext context) {
-    return context.trigger().subTrigger(REPEATED);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
index 18b7a62..1cc807e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
@@ -23,22 +23,18 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
-import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.transforms.GroupByKey;
 import org.joda.time.Instant;
 
 /**
- * {@code Trigger}s control when the elements for a specific key and window are output. As elements
- * arrive, they are put into one or more windows by a {@link Window} transform and its associated
- * {@link WindowFn}, and then passed to the associated {@code Trigger} to determine if the
- * {@code Window}s contents should be output.
+ * {@link Trigger Triggers} control when the elements for a specific key and window are output. As
+ * elements arrive, they are put into one or more windows by a {@link Window} transform and its
+ * associated {@link WindowFn}, and then passed to the associated {@link Trigger} to determine if
+ * the {@link BoundedWindow Window's} contents should be output.
  *
- * <p>See {@link org.apache.beam.sdk.transforms.GroupByKey} and {@link Window}
- * for more information about how grouping with windows works.
+ * <p>See {@link GroupByKey} and {@link Window} for more information about how grouping with windows
+ * works.
  *
  * <p>The elements that are assigned to a window since the last time it was fired (or since the
  * window was created) are placed into the current window pane. Triggers are evaluated against the
@@ -46,224 +42,34 @@ import org.joda.time.Instant;
  * output. When the root trigger finishes (indicating it will never fire again), the window is
  * closed and any new elements assigned to that window are discarded.
  *
- * <p>Several predefined {@code Trigger}s are provided:
+ * <p>Several predefined {@link Trigger Triggers} are provided:
+ *
  * <ul>
- *   <li> {@link AfterWatermark} for firing when the watermark passes a timestamp determined from
- *   either the end of the window or the arrival of the first element in a pane.
- *   <li> {@link AfterProcessingTime} for firing after some amount of processing time has elapsed
- *   (typically since the first element in a pane).
- *   <li> {@link AfterPane} for firing off a property of the elements in the current pane, such as
- *   the number of elements that have been assigned to the current pane.
+ * <li> {@link AfterWatermark} for firing when the watermark passes a timestamp determined from
+ *     either the end of the window or the arrival of the first element in a pane.
+ * <li> {@link AfterProcessingTime} for firing after some amount of processing time has elapsed
+ *     (typically since the first element in a pane).
+ * <li> {@link AfterPane} for firing off a property of the elements in the current pane, such as the
+ *     number of elements that have been assigned to the current pane.
  * </ul>
  *
  * <p>In addition, {@code Trigger}s can be combined in a variety of ways:
- * <ul>
- *   <li> {@link Repeatedly#forever} to create a trigger that executes forever. Any time its
- *   argument finishes it gets reset and starts over. Can be combined with
- *   {@link Trigger#orFinally} to specify a condition that causes the repetition to stop.
- *   <li> {@link AfterEach#inOrder} to execute each trigger in sequence, firing each (and every)
- *   time that a trigger fires, and advancing to the next trigger in the sequence when it finishes.
- *   <li> {@link AfterFirst#of} to create a trigger that fires after at least one of its arguments
- *   fires. An {@link AfterFirst} trigger finishes after it fires once.
- *   <li> {@link AfterAll#of} to create a trigger that fires after all least one of its arguments
- *   have fired at least once. An {@link AfterAll} trigger finishes after it fires once.
- * </ul>
  *
- * <p>Each trigger tree is instantiated per-key and per-window. Every trigger in the tree is in one
- * of the following states:
  * <ul>
- *   <li> Never Existed - before the trigger has started executing, there is no state associated
- *   with it anywhere in the system. A trigger moves to the executing state as soon as it
- *   processes in the current pane.
- *   <li> Executing - while the trigger is receiving items and may fire. While it is in this state,
- *   it may persist book-keeping information to persisted state, set timers, etc.
- *   <li> Finished - after a trigger finishes, all of its book-keeping data is cleaned up, and the
- *   system remembers only that it is finished. Entering this state causes us to discard any
- *   elements in the buffer for that window, as well.
+ * <li> {@link Repeatedly#forever} to create a trigger that executes forever. Any time its argument
+ *     finishes it gets reset and starts over. Can be combined with {@link Trigger#orFinally} to
+ *     specify a condition that causes the repetition to stop.
+ * <li> {@link AfterEach#inOrder} to execute each trigger in sequence, firing each (and every) time
+ *     that a trigger fires, and advancing to the next trigger in the sequence when it finishes.
+ * <li> {@link AfterFirst#of} to create a trigger that fires after at least one of its arguments
+ *     fires. An {@link AfterFirst} trigger finishes after it fires once.
+ * <li> {@link AfterAll#of} to create a trigger that fires after all least one of its arguments have
+ *     fired at least once. An {@link AfterAll} trigger finishes after it fires once.
  * </ul>
- *
- * <p>Once finished, a trigger cannot return itself back to an earlier state, however a composite
- * trigger could reset its sub-triggers.
- *
- * <p>Triggers should not build up any state internally since they may be recreated
- * between invocations of the callbacks. All important values should be persisted using
- * state before the callback returns.
  */
 @Experimental(Experimental.Kind.TRIGGER)
 public abstract class Trigger implements Serializable {
 
-  /**
-   * Interface for accessing information about the trigger being executed and other triggers in the
-   * same tree.
-   */
-  public interface TriggerInfo {
-
-    /**
-     * Returns true if the windowing strategy of the current {@code PCollection} is a merging
-     * WindowFn. If true, the trigger execution needs to keep enough information to support the
-     * possibility of {@link Trigger#onMerge} being called. If false, {@link Trigger#onMerge} will
-     * never be called.
-     */
-    boolean isMerging();
-
-    /**
-     * Access the executable versions of the sub-triggers of the current trigger.
-     */
-    Iterable<ExecutableTrigger> subTriggers();
-
-    /**
-     * Access the executable version of the specified sub-trigger.
-     */
-    ExecutableTrigger subTrigger(int subtriggerIndex);
-
-    /**
-     * Returns true if the current trigger is marked finished.
-     */
-    boolean isFinished();
-
-    /**
-     * Return true if the given subtrigger is marked finished.
-     */
-    boolean isFinished(int subtriggerIndex);
-
-    /**
-     * Returns true if all the sub-triggers of the current trigger are marked finished.
-     */
-    boolean areAllSubtriggersFinished();
-
-    /**
-     * Returns an iterable over the unfinished sub-triggers of the current trigger.
-     */
-    Iterable<ExecutableTrigger> unfinishedSubTriggers();
-
-    /**
-     * Returns the first unfinished sub-trigger.
-     */
-    ExecutableTrigger firstUnfinishedSubTrigger();
-
-    /**
-     * Clears all keyed state for triggers in the current sub-tree and unsets all the associated
-     * finished bits.
-     */
-    void resetTree() throws Exception;
-
-    /**
-     * Sets the finished bit for the current trigger.
-     */
-    void setFinished(boolean finished);
-
-    /**
-     * Sets the finished bit for the given sub-trigger.
-     */
-    void setFinished(boolean finished, int subTriggerIndex);
-  }
-
-  /**
-   * Interact with properties of the trigger being executed, with extensions to deal with the
-   * merging windows.
-   */
-  public interface MergingTriggerInfo extends TriggerInfo {
-
-    /** Return true if the trigger is finished in any window being merged. */
-    boolean finishedInAnyMergingWindow();
-
-    /** Return true if the trigger is finished in all windows being merged. */
-    boolean finishedInAllMergingWindows();
-  }
-
-  /**
-   * Information accessible to all operational hooks in this {@code Trigger}.
-   *
-   * <p>Used directly in {@link Trigger#shouldFire} and {@link Trigger#clear}, and
-   * extended with additional information in other methods.
-   */
-  public abstract class TriggerContext {
-
-    /** Returns the interface for accessing trigger info. */
-    public abstract TriggerInfo trigger();
-
-    /** Returns the interface for accessing persistent state. */
-    public abstract StateAccessor<?> state();
-
-    /** The window that the current context is executing in. */
-    public abstract BoundedWindow window();
-
-    /** Create a sub-context for the given sub-trigger. */
-    public abstract TriggerContext forTrigger(ExecutableTrigger trigger);
-
-    /**
-     * Removes the timer set in this trigger context for the given {@link Instant}
-     * and {@link TimeDomain}.
-     */
-    public abstract void deleteTimer(Instant timestamp, TimeDomain domain);
-
-    /** The current processing time. */
-    public abstract Instant currentProcessingTime();
-
-    /** The current synchronized upstream processing time or {@code null} if unknown. */
-    @Nullable
-    public abstract Instant currentSynchronizedProcessingTime();
-
-    /** The current event time for the input or {@code null} if unknown. */
-    @Nullable
-    public abstract Instant currentEventTime();
-  }
-
-  /**
-   * Extended {@link TriggerContext} containing information accessible to the {@link #onElement}
-   * operational hook.
-   */
-  public abstract class OnElementContext extends TriggerContext {
-    /** The event timestamp of the element currently being processed. */
-    public abstract Instant eventTimestamp();
-
-    /**
-     * Sets a timer to fire when the watermark or processing time is beyond the given timestamp.
-     * Timers are not guaranteed to fire immediately, but will be delivered at some time afterwards.
-     *
-     * <p>As with {@link #state}, timers are implicitly scoped to the current window. All
-     * timer firings for a window will be received, but the implementation should choose to ignore
-     * those that are not applicable.
-     *
-     * @param timestamp the time at which the trigger should be re-evaluated
-     * @param domain the domain that the {@code timestamp} applies to
-     */
-    public abstract void setTimer(Instant timestamp, TimeDomain domain);
-
-    /** Create an {@code OnElementContext} for executing the given trigger. */
-    @Override
-    public abstract OnElementContext forTrigger(ExecutableTrigger trigger);
-  }
-
-  /**
-   * Extended {@link TriggerContext} containing information accessible to the {@link #onMerge}
-   * operational hook.
-   */
-  public abstract class OnMergeContext extends TriggerContext {
-    /**
-     * Sets a timer to fire when the watermark or processing time is beyond the given timestamp.
-     * Timers are not guaranteed to fire immediately, but will be delivered at some time afterwards.
-     *
-     * <p>As with {@link #state}, timers are implicitly scoped to the current window. All
-     * timer firings for a window will be received, but the implementation should choose to ignore
-     * those that are not applicable.
-     *
-     * @param timestamp the time at which the trigger should be re-evaluated
-     * @param domain the domain that the {@code timestamp} applies to
-     */
-    public abstract void setTimer(Instant timestamp, TimeDomain domain);
-
-    /** Create an {@code OnMergeContext} for executing the given trigger. */
-    @Override
-    public abstract OnMergeContext forTrigger(ExecutableTrigger trigger);
-
-    @Override
-    public abstract MergingStateAccessor<?, ?> state();
-
-    @Override
-    public abstract MergingTriggerInfo trigger();
-  }
-
   protected final List<Trigger> subTriggers;
 
   protected Trigger(List<Trigger> subTriggers) {
@@ -274,114 +80,14 @@ public abstract class Trigger implements Serializable {
     this(Collections.EMPTY_LIST);
   }
 
-  /**
-   * Called every time an element is incorporated into a window.
-   */
-  public abstract void onElement(OnElementContext c) throws Exception;
-
-  /**
-   * Called immediately after windows have been merged.
-   *
-   * <p>Leaf triggers should update their state by inspecting their status and any state
-   * in the merging windows. Composite triggers should update their state by calling
-   * {@link ExecutableTrigger#invokeOnMerge} on their sub-triggers, and applying appropriate logic.
-   *
-   * <p>A trigger such as {@link AfterWatermark#pastEndOfWindow} may no longer be finished;
-   * it is the responsibility of the trigger itself to record this fact. It is forbidden for
-   * a trigger to become finished due to {@link #onMerge}, as it has not yet fired the pending
-   * elements that led to it being ready to fire.
-   *
-   * <p>The implementation does not need to clear out any state associated with the old windows.
-   */
-  public abstract void onMerge(OnMergeContext c) throws Exception;
-
-  /**
-   * Returns {@code true} if the current state of the trigger indicates that its condition
-   * is satisfied and it is ready to fire.
-   */
-  public abstract boolean shouldFire(TriggerContext context) throws Exception;
-
-  /**
-   * Adjusts the state of the trigger to be ready for the next pane. For example, a
-   * {@link Repeatedly} trigger will reset its inner trigger, since it has fired.
-   *
-   * <p>If the trigger is finished, it is the responsibility of the trigger itself to
-   * record that fact via the {@code context}.
-   */
-  public abstract void onFire(TriggerContext context) throws Exception;
-
-  /**
-   * Called to allow the trigger to prefetch any state it will likely need to read from during
-   * an {@link #onElement} call.
-   */
-  public void prefetchOnElement(StateAccessor<?> state) {
-    if (subTriggers != null) {
-      for (Trigger trigger : subTriggers) {
-        trigger.prefetchOnElement(state);
-      }
-    }
-  }
-
-  /**
-   * Called to allow the trigger to prefetch any state it will likely need to read from during
-   * an {@link #onMerge} call.
-   */
-  public void prefetchOnMerge(MergingStateAccessor<?, ?> state) {
-    if (subTriggers != null) {
-      for (Trigger trigger : subTriggers) {
-        trigger.prefetchOnMerge(state);
-      }
-    }
-  }
-
-  /**
-   * Called to allow the trigger to prefetch any state it will likely need to read from during
-   * an {@link #shouldFire} call.
-   */
-  public void prefetchShouldFire(StateAccessor<?> state) {
-    if (subTriggers != null) {
-      for (Trigger trigger : subTriggers) {
-        trigger.prefetchShouldFire(state);
-      }
-    }
-  }
-
-  /**
-   * Called to allow the trigger to prefetch any state it will likely need to read from during
-   * an {@link #onFire} call.
-   */
-  public void prefetchOnFire(StateAccessor<?> state) {
-    if (subTriggers != null) {
-      for (Trigger trigger : subTriggers) {
-        trigger.prefetchOnFire(state);
-      }
-    }
-  }
-
-  /**
-   * Clear any state associated with this trigger in the given window.
-   *
-   * <p>This is called after a trigger has indicated it will never fire again. The trigger system
-   * keeps enough information to know that the trigger is finished, so this trigger should clear all
-   * of its state.
-   */
-  public void clear(TriggerContext c) throws Exception {
-    if (subTriggers != null) {
-      for (ExecutableTrigger trigger : c.trigger().subTriggers()) {
-        trigger.invokeClear(c);
-      }
-    }
-  }
-
   public List<Trigger> subTriggers() {
     return subTriggers;
   }
 
   /**
-   * Return a trigger to use after a {@code GroupByKey} to preserve the
-   * intention of this trigger. Specifically, triggers that are time based
-   * and intended to provide speculative results should continue providing
-   * speculative results. Triggers that fire once (or multiple times) should
+   * Return a trigger to use after a {@link GroupByKey} to preserve the intention of this trigger.
+   * Specifically, triggers that are time based and intended to provide speculative results should
+   * continue providing speculative results. Triggers that fire once (or multiple times) should
    * continue firing once (or multiple times).
    */
   public Trigger getContinuationTrigger() {
@@ -397,27 +103,24 @@ public abstract class Trigger implements Serializable {
   }
 
   /**
-   * Return the {@link #getContinuationTrigger} of this {@code Trigger}. For convenience, this
-   * is provided the continuation trigger of each of the sub-triggers.
+   * Return the {@link #getContinuationTrigger} of this {@link Trigger}. For convenience, this is
+   * provided the continuation trigger of each of the sub-triggers.
    */
   protected abstract Trigger getContinuationTrigger(List<Trigger> continuationTriggers);
 
   /**
-   * Returns a bound in watermark time by which this trigger would have fired at least once
-   * for a given window had there been input data.  This is a static property of a trigger
-   * that does not depend on its state.
+   * Returns a bound in event time by which this trigger would have fired at least once for a given
+   * window had there been input data.
    *
-   * <p>For triggers that do not fire based on the watermark advancing, returns
-   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
+   * <p>For triggers that do not fire based on the watermark advancing, returns {@link
+   * BoundedWindow#TIMESTAMP_MAX_VALUE}.
    *
-   * <p>This estimate is used to determine that there are no elements in a side-input window, which
-   * causes the default value to be used instead.
+   * <p>This estimate may be used, for example, to determine that there are no elements in a
+   * side-input window, which causes the default value to be used instead.
    */
   public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window);
 
-  /**
-   * Returns whether this performs the same triggering as the given {@code Trigger}.
-   */
+  /** Returns whether this performs the same triggering as the given {@link Trigger}. */
   public boolean isCompatible(Trigger other) {
     if (!getClass().equals(other.getClass())) {
       return false;
@@ -472,31 +175,33 @@ public abstract class Trigger implements Serializable {
   }
 
   /**
-   * Specify an ending condition for this trigger. If the {@code until} fires then the combination
-   * fires.
+   * Specify an ending condition for this trigger. If the {@code until} {@link Trigger} fires then
+   * the combination fires.
    *
-   * <p>The expression {@code t1.orFinally(t2)} fires every time {@code t1} fires, and finishes
-   * as soon as either {@code t1} finishes or {@code t2} fires, in which case it fires one last time
-   * for {@code t2}. Both {@code t1} and {@code t2} are executed in parallel. This means that
-   * {@code t1} may have fired since {@code t2} started, so not all of the elements that {@code t2}
-   * has seen are necessarily in the current pane.
+   * <p>The expression {@code t1.orFinally(t2)} fires every time {@code t1} fires, and finishes as
+   * soon as either {@code t1} finishes or {@code t2} fires, in which case it fires one last time
+   * for {@code t2}. Both {@code t1} and {@code t2} are executed in parallel. This means that {@code
+   * t1} may have fired since {@code t2} started, so not all of the elements that {@code t2} has
+   * seen are necessarily in the current pane.
    *
    * <p>For example the final firing of the following trigger may only have 1 element:
-   * <pre> {@code
+   *
+   * <pre>{@code
    * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
    *     .orFinally(AfterPane.elementCountAtLeast(5))
-   * } </pre>
+   * }
+   * </pre>
    *
-   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same
-   * as {@code AfterFirst.of(t1, t2)}.
+   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same as
+   * {@code AfterFirst.of(t1, t2)}.
    */
   public OrFinallyTrigger orFinally(OnceTrigger until) {
     return new OrFinallyTrigger(this, until);
   }
 
   /**
-   * {@link Trigger}s that are guaranteed to fire at most once should extend from this, rather
-   * than the general {@link Trigger} class to indicate that behavior.
+   * {@link Trigger Triggers} that are guaranteed to fire at most once should extend {@link
+   * OnceTrigger} rather than the general {@link Trigger} class to indicate that behavior.
    */
   public abstract static class OnceTrigger extends Trigger {
     protected OnceTrigger(List<Trigger> subTriggers) {
@@ -511,20 +216,5 @@ public abstract class Trigger implements Serializable {
       }
       return (OnceTrigger) continuation;
     }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public final void onFire(TriggerContext context) throws Exception {
-      onOnlyFiring(context);
-      context.trigger().setFinished(true);
-    }
-
-    /**
-     * Called exactly once by {@link #onFire} when the trigger is fired. By default,
-     * invokes {@link #onFire} on all subtriggers for which {@link #shouldFire} is {@code true}.
-     */
-    protected abstract void onOnlyFiring(TriggerContext context) throws Exception;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
index 088c499..48a49aa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
@@ -29,9 +29,13 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 
 /**
  * A wrapper around a trigger used during execution. While an actual trigger may appear multiple
- * times (both in the same trigger expression and in other trigger expressions), the
- * {@code ExecutableTrigger} wrapped around them forms a tree (only one occurrence).
+ * times (both in the same trigger expression and in other trigger expressions), the {@code
+ * ExecutableTrigger} wrapped around them forms a tree (only one occurrence).
+ *
+ * @deprecated uses of {@link ExecutableTrigger} should be ported to
+ *     org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine.
  */
+@Deprecated
 public class ExecutableTrigger implements Serializable {
 
   /** Store the index assigned to this trigger. */
@@ -115,38 +119,6 @@ public class ExecutableTrigger implements Serializable {
   }
 
   /**
-   * Invoke the {@link Trigger#onElement} method for this trigger, ensuring that the bits are
-   * properly updated if the trigger finishes.
-   */
-  public void invokeOnElement(Trigger.OnElementContext c) throws Exception {
-    trigger.onElement(c.forTrigger(this));
-  }
-
-  /**
-   * Invoke the {@link Trigger#onMerge} method for this trigger, ensuring that the bits are properly
-   * updated.
-   */
-  public void invokeOnMerge(Trigger.OnMergeContext c) throws Exception {
-    Trigger.OnMergeContext subContext = c.forTrigger(this);
-    trigger.onMerge(subContext);
-  }
-
-  public boolean invokeShouldFire(Trigger.TriggerContext c) throws Exception {
-    return trigger.shouldFire(c.forTrigger(this));
-  }
-
-  public void invokeOnFire(Trigger.TriggerContext c) throws Exception {
-    trigger.onFire(c.forTrigger(this));
-  }
-
-  /**
-   * Invoke clear for the current this trigger.
-   */
-  public void invokeClear(Trigger.TriggerContext c) throws Exception {
-    trigger.clear(c.forTrigger(this));
-  }
-
-  /**
    * {@link ExecutableTrigger} that enforces the fact that the trigger should always FIRE_AND_FINISH
    * and never just FIRE.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
deleted file mode 100644
index ea14c40..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.util;
-
-/**
- * A mutable set which tracks whether any particular {@link ExecutableTrigger} is
- * finished.
- */
-public interface FinishedTriggers {
-  /**
-   * Returns {@code true} if the trigger is finished.
-   */
-  boolean isFinished(ExecutableTrigger trigger);
-
-  /**
-   * Sets the fact that the trigger is finished.
-   */
-  void setFinished(ExecutableTrigger trigger, boolean value);
-
-  /**
-   * Sets the trigger and all of its subtriggers to unfinished.
-   */
-  void clearRecursively(ExecutableTrigger trigger);
-
-  /**
-   * Create an independent copy of this mutable {@link FinishedTriggers}.
-   */
-  FinishedTriggers copy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
deleted file mode 100644
index 4cd617f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.util;
-
-import java.util.BitSet;
-
-/**
- * A {@link FinishedTriggers} implementation based on an underlying {@link BitSet}.
- */
-public class FinishedTriggersBitSet implements FinishedTriggers {
-
-  private final BitSet bitSet;
-
-  private FinishedTriggersBitSet(BitSet bitSet) {
-    this.bitSet = bitSet;
-  }
-
-  public static FinishedTriggersBitSet emptyWithCapacity(int capacity) {
-    return new FinishedTriggersBitSet(new BitSet(capacity));
-  }
-
-  public static FinishedTriggersBitSet fromBitSet(BitSet bitSet) {
-    return new FinishedTriggersBitSet(bitSet);
-  }
-
-  /**
-   * Returns the underlying {@link BitSet} for this {@link FinishedTriggersBitSet}.
-   */
-  public BitSet getBitSet() {
-    return bitSet;
-  }
-
-  @Override
-  public boolean isFinished(ExecutableTrigger trigger) {
-    return bitSet.get(trigger.getTriggerIndex());
-  }
-
-  @Override
-  public void setFinished(ExecutableTrigger trigger, boolean value) {
-    bitSet.set(trigger.getTriggerIndex(), value);
-  }
-
-  @Override
-  public void clearRecursively(ExecutableTrigger trigger) {
-    bitSet.clear(trigger.getTriggerIndex(), trigger.getFirstIndexAfterSubtree());
-  }
-
-  @Override
-  public FinishedTriggersBitSet copy() {
-    return new FinishedTriggersBitSet((BitSet) bitSet.clone());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
deleted file mode 100644
index a9feb73..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.util;
-
-import com.google.common.collect.Sets;
-import java.util.Set;
-
-/**
- * An implementation of {@link FinishedTriggers} atop a user-provided mutable {@link Set}.
- */
-public class FinishedTriggersSet implements FinishedTriggers {
-
-  private final Set<ExecutableTrigger> finishedTriggers;
-
-  private FinishedTriggersSet(Set<ExecutableTrigger> finishedTriggers) {
-    this.finishedTriggers = finishedTriggers;
-  }
-
-  public static FinishedTriggersSet fromSet(Set<ExecutableTrigger> finishedTriggers) {
-    return new FinishedTriggersSet(finishedTriggers);
-  }
-
-  /**
-   * Returns a mutable {@link Set} of the underlying triggers that are finished.
-   */
-  public Set<ExecutableTrigger> getFinishedTriggers() {
-    return finishedTriggers;
-  }
-
-  @Override
-  public boolean isFinished(ExecutableTrigger trigger) {
-    return finishedTriggers.contains(trigger);
-  }
-
-  @Override
-  public void setFinished(ExecutableTrigger trigger, boolean value) {
-    if (value) {
-      finishedTriggers.add(trigger);
-    } else {
-      finishedTriggers.remove(trigger);
-    }
-  }
-
-  @Override
-  public void clearRecursively(ExecutableTrigger trigger) {
-    finishedTriggers.remove(trigger);
-    for (ExecutableTrigger subTrigger : trigger.subTriggers()) {
-      clearRecursively(subTrigger);
-    }
-  }
-
-  @Override
-  public FinishedTriggersSet copy() {
-    return fromSet(Sets.newHashSet(finishedTriggers));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
index 437f14a..8dd648a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
@@ -35,12 +35,6 @@ public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
   }
 
   @Override
-  public void onElement(Trigger.OnElementContext c) { }
-
-  @Override
-  public void onMerge(Trigger.OnMergeContext c) { }
-
-  @Override
   protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
     return this;
   }
@@ -52,14 +46,6 @@ public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
   }
 
   @Override
-  public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-    return true;
-  }
-
-  @Override
-  public void onFire(Trigger.TriggerContext context) throws Exception { }
-
-  @Override
   public String toString() {
     return "ReshuffleTrigger()";
   }


[09/50] [abbrv] incubator-beam git commit: [BEAM-769] Spark: Make graceful stop the default.

Posted by dh...@apache.org.
[BEAM-769] Spark: Make graceful stop the default.

streaming tests fail on "nothing processed" if runtime env. is slow because timeout
is hit before processing is done.

Keep "pumping-in" the last batch in a mocked stream to handle overflowing batches in case of a
graceful stop.

Change tests accordingly.


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

Branch: refs/heads/gearpump-runner
Commit: e43228c92cd9dd8a81a28940d419b721a2aeb2d8
Parents: a9a41eb
Author: Sela <an...@paypal.com>
Authored: Fri Oct 21 01:20:33 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:07:27 2016 -0700

----------------------------------------------------------------------
 .../streaming/StreamingEvaluationContext.java            | 11 ++++++++---
 .../translation/streaming/EmptyStreamAssertionTest.java  | 10 +++++++---
 .../streaming/ResumeFromCheckpointStreamingTest.java     |  2 +-
 .../streaming/SimpleStreamingWordCountTest.java          |  1 -
 .../translation/streaming/utils/PAssertStreaming.java    |  6 +++---
 5 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
index 2652f2b..49afa26 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
@@ -92,6 +92,7 @@ public class StreamingEvaluationContext extends EvaluationContext {
             WindowedValue.getValueOnlyCoder(coder);
         // create the DStream from queue
         Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
+        JavaRDD<WindowedValue<T>> lastRDD = null;
         for (Iterable<T> v : values) {
           Iterable<WindowedValue<T>> windowedValues =
               Iterables.transform(v, WindowingHelpers.<T>windowValueFunction());
@@ -99,10 +100,14 @@ public class StreamingEvaluationContext extends EvaluationContext {
               CoderHelpers.toByteArrays(windowedValues, windowCoder)).map(
                   CoderHelpers.fromByteFunction(windowCoder));
           rddQueue.offer(rdd);
+          lastRDD = rdd;
         }
-        // create dstream from queue, one at a time, no defaults
-        // mainly for unit test so no reason to have this configurable
-        dStream = jssc.queueStream(rddQueue, true);
+        // create dstream from queue, one at a time,
+        // with last as default in case batches repeat (graceful stops for example).
+        // if the stream is empty, avoid creating a default empty RDD.
+        // mainly for unit test so no reason to have this configurable.
+        dStream = lastRDD != null ? jssc.queueStream(rddQueue, true, lastRDD)
+            : jssc.queueStream(rddQueue, true);
       }
       return dStream;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
index 1560c66..4f2a7c6 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
@@ -34,7 +34,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
-
+import org.junit.rules.TemporaryFolder;
 
 
 /**
@@ -48,11 +48,15 @@ public class EmptyStreamAssertionTest implements Serializable {
           + "     but: was <0>";
 
   @Rule
+  public TemporaryFolder checkpointParentDir = new TemporaryFolder();
+
+  @Rule
   public TestOptionsForStreaming commonOptions = new TestOptionsForStreaming();
 
   @Test
-  public void testFixedWindows() throws Exception {
-    SparkPipelineOptions options = commonOptions.getOptions();
+  public void testAssertion() throws Exception {
+    SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
+        checkpointParentDir.newFolder(getClass().getSimpleName()));
     Duration windowDuration = new Duration(options.getBatchIntervalMillis());
 
     Pipeline pipeline = Pipeline.create(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
index fc7fa34..fd1d11a 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
@@ -157,7 +157,7 @@ public class ResumeFromCheckpointStreamingTest {
 
     // requires a graceful stop so that checkpointing of the first run would finish successfully
     // before stopping and attempting to resume.
-    return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED, true);
+    return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
index 8f2dde3..4bc9a3d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
@@ -62,7 +62,6 @@ public class SimpleStreamingWordCountTest implements Serializable {
 
   @Test
   public void testFixedWindows() throws Exception {
-
     SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
         checkpointParentDir.newFolder(getClass().getSimpleName()));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
index 3bf1ef6..496735d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
@@ -81,13 +81,13 @@ public final class PAssertStreaming implements Serializable {
   }
 
   /**
-   * Default to stop immediately, useful for most tests except for the once that may require
-   * to finish writing checkpoints for example.
+   * Default to stop gracefully so that tests will finish processing even if slower for reasons
+   * such as a slow runtime environment.
    */
   public static <T> EvaluationResult runAndAssertContents(Pipeline p,
                                                           PCollection<T> actual,
                                                           T[] expected) {
-    return runAndAssertContents(p, actual, expected, false);
+    return runAndAssertContents(p, actual, expected, true);
   }
 
   private static class AssertDoFn<T> extends OldDoFn<Iterable<T>, Void> {


[40/50] [abbrv] incubator-beam git commit: This closes #1178

Posted by dh...@apache.org.
This closes #1178


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

Branch: refs/heads/gearpump-runner
Commit: 13af296dec5fd5b9b4f922dac6ce1dc9b026cf60
Parents: 7abdcbf 2872019
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 13:12:17 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../direct/WriteWithShardingFactory.java        |  6 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 32 +++----
 .../beam/runners/direct/DirectRunnerTest.java   | 40 +++++----
 .../ImmutabilityCheckingBundleFactoryTest.java  |  8 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  8 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |  8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  8 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 87 +++++++++---------
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 94 +++++++++-----------
 .../runners/direct/WatermarkManagerTest.java    |  8 +-
 .../apache/beam/sdk/util/ValueWithRecordId.java |  8 +-
 11 files changed, 146 insertions(+), 161 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] incubator-beam git commit: This closes #1174

Posted by dh...@apache.org.
This closes #1174


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

Branch: refs/heads/gearpump-runner
Commit: c03e3e926c38846ad02cf6d7bf857499927d54bd
Parents: 79bb2c2 7239ebb
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 10:46:43 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 10:46:43 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/DoFnLifecycleManager.java    | 56 +++++++++------
 .../direct/DoFnLifecycleManagerTest.java        | 74 ++++++++++++++++++--
 2 files changed, 104 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[15/50] [abbrv] incubator-beam git commit: [maven-release-plugin] prepare branch release-0.3.0-incubating

Posted by dh...@apache.org.
[maven-release-plugin] prepare branch release-0.3.0-incubating


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

Branch: refs/heads/gearpump-runner
Commit: c26b63baecde9105d32e3ba03874eb782162ecda
Parents: 69ae8bd
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 17:36:16 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 17:36:16 2016 +0200

----------------------------------------------------------------------
 pom.xml | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c26b63ba/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index de4bd63..6a24d98 100644
--- a/pom.xml
+++ b/pom.xml
@@ -48,7 +48,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary</url>
-    <tag>release-0.2.0-incubating</tag>
+    <tag>release-0.3.0-incubating</tag>
   </scm>
 
   <issueManagement>
@@ -92,13 +92,13 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <beam.javadoc_opts/>
+    <beam.javadoc_opts />
 
     <!-- Disable integration tests by default -->
     <skipITs>true</skipITs>
 
     <!-- Do not add additional surefire arguments by default -->
-    <beamSurefireArgline/>
+    <beamSurefireArgline />
 
     <!-- If updating dependencies, please update any relevant javadoc offlineLinks -->
     <avro.version>1.8.1</avro.version>
@@ -1023,7 +1023,7 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <ignore/>
+                    <ignore />
                   </action>
                 </pluginExecution>
                 <pluginExecution>
@@ -1037,7 +1037,7 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <ignore/>
+                    <ignore />
                   </action>
                 </pluginExecution>
               </pluginExecutions>


[03/50] [abbrv] incubator-beam git commit: Fix flipped conditional in SimpleDoFnRunner

Posted by dh...@apache.org.
Fix flipped conditional in SimpleDoFnRunner


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

Branch: refs/heads/gearpump-runner
Commit: f0c8d30d61c631fea642becde38b9cc52e873b5e
Parents: 043ebec
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 13:53:53 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0c8d30d/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index 1cf56a6..dec9905 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -125,11 +125,11 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
   @Override
   public void processElement(WindowedValue<InputT> compressedElem) {
     if (observesWindow) {
-      invokeProcessElement(compressedElem);
-    } else {
       for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {
         invokeProcessElement(elem);
       }
+    } else {
+      invokeProcessElement(compressedElem);
     }
   }
 


[19/50] [abbrv] incubator-beam git commit: [BEAM-806] update archetype poms in master

Posted by dh...@apache.org.
[BEAM-806] update archetype poms in master

Per the linked JIRA, maven release:branch did not do the right thing.


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

Branch: refs/heads/gearpump-runner
Commit: 5d011847a658ff1cbd1e4eb0e0c581ceecbfc2c0
Parents: b437c6b
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 11:12:41 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 11:12:41 2016 -0700

----------------------------------------------------------------------
 .../examples/src/main/resources/archetype-resources/pom.xml  | 8 ++++----
 .../starter/src/main/resources/archetype-resources/pom.xml   | 2 +-
 .../src/test/resources/projects/basic/reference/pom.xml      | 2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
index 147c2dc..7926578 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
@@ -108,21 +108,21 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- Adds a dependency on a specific version of the Dataflow runnner. -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-direct-java</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
       <scope>runtime</scope>
     </dependency>
 
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
       <scope>runtime</scope>
     </dependency>
 
@@ -130,7 +130,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
index 0f7d3de..5287d1f 100644
--- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
@@ -55,7 +55,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- slf4j API frontend binding with JUL backend -->

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
index 66091fe..cce2f99 100644
--- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
+++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
@@ -55,7 +55,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- slf4j API frontend binding with JUL backend -->


[16/50] [abbrv] incubator-beam git commit: [maven-release-plugin] prepare for next development iteration

Posted by dh...@apache.org.
[maven-release-plugin] prepare for next development iteration


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

Branch: refs/heads/gearpump-runner
Commit: 8dfadbf01acc4fed6f604fe531e87730bab2b2bc
Parents: c26b63b
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 17:36:25 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 17:36:25 2016 +0200

----------------------------------------------------------------------
 examples/java/pom.xml                       | 2 +-
 examples/java8/pom.xml                      | 2 +-
 examples/pom.xml                            | 2 +-
 pom.xml                                     | 4 ++--
 runners/core-java/pom.xml                   | 2 +-
 runners/direct-java/pom.xml                 | 2 +-
 runners/flink/examples/pom.xml              | 2 +-
 runners/flink/pom.xml                       | 2 +-
 runners/flink/runner/pom.xml                | 2 +-
 runners/google-cloud-dataflow-java/pom.xml  | 2 +-
 runners/pom.xml                             | 2 +-
 runners/spark/pom.xml                       | 2 +-
 sdks/java/build-tools/pom.xml               | 2 +-
 sdks/java/core/pom.xml                      | 2 +-
 sdks/java/extensions/join-library/pom.xml   | 2 +-
 sdks/java/extensions/pom.xml                | 2 +-
 sdks/java/io/google-cloud-platform/pom.xml  | 2 +-
 sdks/java/io/hdfs/pom.xml                   | 2 +-
 sdks/java/io/jdbc/pom.xml                   | 2 +-
 sdks/java/io/jms/pom.xml                    | 2 +-
 sdks/java/io/kafka/pom.xml                  | 2 +-
 sdks/java/io/kinesis/pom.xml                | 2 +-
 sdks/java/io/mongodb/pom.xml                | 2 +-
 sdks/java/io/pom.xml                        | 2 +-
 sdks/java/java8tests/pom.xml                | 2 +-
 sdks/java/maven-archetypes/examples/pom.xml | 2 +-
 sdks/java/maven-archetypes/pom.xml          | 2 +-
 sdks/java/maven-archetypes/starter/pom.xml  | 2 +-
 sdks/java/microbenchmarks/pom.xml           | 2 +-
 sdks/java/pom.xml                           | 2 +-
 sdks/pom.xml                                | 2 +-
 31 files changed, 32 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 37cb15a..fc82ed4 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-examples-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index 44cf918..e6408dc 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-examples-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 2efb146..2820473 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6a24d98..2338f55 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,7 +34,7 @@
   <url>http://beam.incubator.apache.org</url>
   <inceptionYear>2016</inceptionYear>
 
-  <version>0.3.0-incubating-SNAPSHOT</version>
+  <version>0.4.0-incubating-SNAPSHOT</version>
 
   <licenses>
     <license>
@@ -48,7 +48,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary</url>
-    <tag>release-0.3.0-incubating</tag>
+    <tag>release-0.2.0-incubating</tag>
   </scm>
 
   <issueManagement>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index d84c420..5f7ec08 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index aec0e71..c182d15 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index b8489fc..a0cf676 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 68e82d2..e012c4b 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 8759591..283d060 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 0f2d3b2..c5dd274 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 605c3b2..68cf29c 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index a246c19..ccec3c6 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/pom.xml b/sdks/java/build-tools/pom.xml
index 279555e..cc27bea 100644
--- a/sdks/java/build-tools/pom.xml
+++ b/sdks/java/build-tools/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index d44a494..9937cb8 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/extensions/join-library/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml
index e4acf82..4687554 100644
--- a/sdks/java/extensions/join-library/pom.xml
+++ b/sdks/java/extensions/join-library/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-extensions-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/extensions/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index 419b0ce..4328d3d 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index 1a598bb..054ee21 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 0ec542c..22c3187 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml
index 75eb5ed..3f513ca 100644
--- a/sdks/java/io/jdbc/pom.xml
+++ b/sdks/java/io/jdbc/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/jms/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml
index 7113434..5a74b34 100644
--- a/sdks/java/io/jms/pom.xml
+++ b/sdks/java/io/jms/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml
index e5c3285..f2b8326 100644
--- a/sdks/java/io/kafka/pom.xml
+++ b/sdks/java/io/kafka/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml
index 06817da..0f65d67 100644
--- a/sdks/java/io/kinesis/pom.xml
+++ b/sdks/java/io/kinesis/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml
index 5555173..ad0609d 100644
--- a/sdks/java/io/mongodb/pom.xml
+++ b/sdks/java/io/mongodb/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml
index 82cf8d0..3750202 100644
--- a/sdks/java/io/pom.xml
+++ b/sdks/java/io/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/java8tests/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
index 1fe41d7..13f5fce 100644
--- a/sdks/java/java8tests/pom.xml
+++ b/sdks/java/java8tests/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/examples/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml
index dcdf94e..e36c4fc 100644
--- a/sdks/java/maven-archetypes/examples/pom.xml
+++ b/sdks/java/maven-archetypes/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-maven-archetypes-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml
index dd2a16a..02bb150 100644
--- a/sdks/java/maven-archetypes/pom.xml
+++ b/sdks/java/maven-archetypes/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/starter/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/pom.xml b/sdks/java/maven-archetypes/starter/pom.xml
index 45b60a6..c38f80c 100644
--- a/sdks/java/maven-archetypes/starter/pom.xml
+++ b/sdks/java/maven-archetypes/starter/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-maven-archetypes-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/microbenchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/pom.xml b/sdks/java/microbenchmarks/pom.xml
index 4d8d922..5c6cf4d 100644
--- a/sdks/java/microbenchmarks/pom.xml
+++ b/sdks/java/microbenchmarks/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 4576e4f..eae5a58 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/pom.xml b/sdks/pom.xml
index aa9cbed..235d102 100644
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[18/50] [abbrv] incubator-beam git commit: This closes #1166

Posted by dh...@apache.org.
This closes #1166


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

Branch: refs/heads/gearpump-runner
Commit: b437c6b9e19754dc7624d043d23a4704329b83f7
Parents: c390a2a 59f042c
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 10:23:12 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Oct 24 10:23:12 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/core/ReduceFnRunner.java |  2 +-
 .../org/apache/beam/sdk/util/state/ReadableState.java     | 10 +++-------
 2 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[45/50] [abbrv] incubator-beam git commit: Closes #1182

Posted by dh...@apache.org.
Closes #1182


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

Branch: refs/heads/gearpump-runner
Commit: f2fe1ae466284b58ebcbff98ea1103027fe6f33e
Parents: 13af296 d255350
Author: Dan Halperin <dh...@google.com>
Authored: Tue Oct 25 13:49:56 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Oct 25 13:49:56 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/display/DisplayData.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------



[48/50] [abbrv] incubator-beam git commit: remove "pipeline" in runner name

Posted by dh...@apache.org.
remove "pipeline" in runner name


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

Branch: refs/heads/gearpump-runner
Commit: 94bd47cdb7e4b8f1d874ace1c60e4251636a8110
Parents: 8f013cb
Author: manuzhang <ow...@gmail.com>
Authored: Wed Oct 26 16:18:39 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Wed Oct 26 16:19:13 2016 +0800

----------------------------------------------------------------------
 .../gearpump/GearpumpPipelineRunner.java        | 191 -------------------
 .../GearpumpPipelineRunnerRegistrar.java        |  62 ------
 .../beam/runners/gearpump/GearpumpRunner.java   | 191 +++++++++++++++++++
 .../gearpump/GearpumpRunnerRegistrar.java       |  62 ++++++
 .../runners/gearpump/TestGearpumpRunner.java    |   4 +-
 .../gearpump/examples/StreamingWordCount.java   |   4 +-
 6 files changed, 257 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
deleted file mode 100644
index 9e32227..0000000
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * 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.runners.gearpump;
-
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigValueFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.beam.runners.core.AssignWindows;
-import org.apache.beam.runners.gearpump.translators.TranslationContext;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import org.apache.gearpump.cluster.ClusterConfig;
-import org.apache.gearpump.cluster.UserConfig;
-import org.apache.gearpump.cluster.client.ClientContext;
-import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
-import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to Gearpump Stream DSL
- * and then executing them on a Gearpump cluster.
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class GearpumpPipelineRunner extends PipelineRunner<GearpumpPipelineResult> {
-
-  private final GearpumpPipelineOptions options;
-
-  private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers";
-  private static final String DEFAULT_APPNAME = "beam_gearpump_app";
-
-  public GearpumpPipelineRunner(GearpumpPipelineOptions options) {
-    this.options = options;
-  }
-
-  public static GearpumpPipelineRunner fromOptions(PipelineOptions options) {
-    GearpumpPipelineOptions pipelineOptions =
-        PipelineOptionsValidator.validate(GearpumpPipelineOptions.class, options);
-    return new GearpumpPipelineRunner(pipelineOptions);
-  }
-
-
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    if (Window.Bound.class.equals(transform.getClass())) {
-      return (OutputT) super.apply(
-              new AssignWindowsAndSetStrategy((Window.Bound) transform), input);
-    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
-            && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of());
-    } else if (Create.Values.class.equals(transform.getClass())) {
-      return (OutputT) PCollection
-              .<OutputT>createPrimitiveOutputInternal(
-                      input.getPipeline(),
-                      WindowingStrategy.globalDefault(),
-                      PCollection.IsBounded.BOUNDED);
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-  @Override
-  public GearpumpPipelineResult run(Pipeline pipeline) {
-    String appName = options.getApplicationName();
-    if (null == appName) {
-      appName = DEFAULT_APPNAME;
-    }
-    Config config = registerSerializers(ClusterConfig.defaultConfig(),
-        options.getSerializers());
-    ClientContext clientContext = getClientContext(options, config);
-    options.setClientContext(clientContext);
-    JavaStreamApp streamApp = new JavaStreamApp(
-        appName, clientContext, UserConfig.empty());
-    TranslationContext translationContext = new TranslationContext(streamApp, options);
-    GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext);
-    translator.translate(pipeline);
-    streamApp.run();
-
-    return null;
-  }
-
-  private ClientContext getClientContext(GearpumpPipelineOptions options, Config config) {
-    EmbeddedCluster cluster = options.getEmbeddedCluster();
-    if (cluster != null) {
-      return cluster.newClientContext();
-    } else {
-      return ClientContext.apply(config);
-    }
-  }
-
-  /**
-   * register class with default kryo serializers.
-   */
-  private Config registerSerializers(Config config, Map<String, String> userSerializers) {
-    Map<String, String> serializers = new HashMap<>();
-    serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInSingleWindow", "");
-    serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo", "");
-    serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo$Timing", "");
-    serializers.put("org.joda.time.Instant", "");
-    serializers.put("org.apache.beam.sdk.values.KV", "");
-    serializers.put("org.apache.beam.sdk.transforms.windowing.IntervalWindow", "");
-    serializers.put("org.apache.beam.sdk.values.TimestampedValue", "");
-    if (userSerializers != null && !userSerializers.isEmpty()) {
-      serializers.putAll(userSerializers);
-    }
-    return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers));
-  }
-
-
-  /**
-   * copied from DirectPipelineRunner.
-   * used to replace Window.Bound till window function is added to Gearpump Stream DSL
-   */
-  private static class AssignWindowsAndSetStrategy<T, W extends BoundedWindow>
-      extends PTransform<PCollection<T>, PCollection<T>> {
-
-    private final Window.Bound<T> wrapped;
-
-    AssignWindowsAndSetStrategy(Window.Bound<T> wrapped) {
-      this.wrapped = wrapped;
-    }
-
-    @Override
-    public PCollection<T> apply(PCollection<T> input) {
-      WindowingStrategy<?, ?> outputStrategy =
-          wrapped.getOutputStrategyInternal(input.getWindowingStrategy());
-
-      WindowFn<T, BoundedWindow> windowFn =
-          (WindowFn<T, BoundedWindow>) outputStrategy.getWindowFn();
-
-      if (!windowFn.isNonMerging()) {
-        throw new UnsupportedOperationException(
-            "merging window is not supported in Gearpump pipeline");
-      }
-
-      // If the Window.Bound transform only changed parts other than the WindowFn, then
-      // we skip AssignWindows even though it should be harmless in a perfect world.
-      // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly
-      // crash if another GBK is performed without explicitly setting the WindowFn. So we skip
-      // AssignWindows in this case.
-      if (wrapped.getWindowFn() == null) {
-        return input.apply("Identity", ParDo.of(new IdentityFn<T>()))
-            .setWindowingStrategyInternal(outputStrategy);
-      } else {
-        return input
-            .apply("AssignWindows", new AssignWindows<>(windowFn))
-            .setWindowingStrategyInternal(outputStrategy);
-      }
-    }
-  }
-
-  private static class IdentityFn<T> extends OldDoFn<T, T> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
deleted file mode 100644
index ca173d1..0000000
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.runners.gearpump;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
-
-/**
- * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
- * {@link GearpumpPipelineRunner}.
- *
- * {@link AutoService} will register Gearpump's implementations of the {@link PipelineRunner}
- * and {@link PipelineOptions} as available pipeline runner services.
- */
-public class GearpumpPipelineRunnerRegistrar {
-  private GearpumpPipelineRunnerRegistrar() { }
-
-  /**
-   * Registers the {@link GearpumpPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class Runner implements PipelineRunnerRegistrar {
-
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(TestGearpumpRunner.class);
-    }
-  }
-
-  /**
-   * Registers the {@link GearpumpPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(GearpumpPipelineOptions.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
new file mode 100644
index 0000000..ed0813d
--- /dev/null
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
@@ -0,0 +1,191 @@
+/*
+ * 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.runners.gearpump;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.beam.runners.core.AssignWindows;
+import org.apache.beam.runners.gearpump.translators.TranslationContext;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import org.apache.gearpump.cluster.ClusterConfig;
+import org.apache.gearpump.cluster.UserConfig;
+import org.apache.gearpump.cluster.client.ClientContext;
+import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
+import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to Gearpump Stream DSL
+ * and then executing them on a Gearpump cluster.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class GearpumpRunner extends PipelineRunner<GearpumpPipelineResult> {
+
+  private final GearpumpPipelineOptions options;
+
+  private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers";
+  private static final String DEFAULT_APPNAME = "beam_gearpump_app";
+
+  public GearpumpRunner(GearpumpPipelineOptions options) {
+    this.options = options;
+  }
+
+  public static GearpumpRunner fromOptions(PipelineOptions options) {
+    GearpumpPipelineOptions pipelineOptions =
+        PipelineOptionsValidator.validate(GearpumpPipelineOptions.class, options);
+    return new GearpumpRunner(pipelineOptions);
+  }
+
+
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    if (Window.Bound.class.equals(transform.getClass())) {
+      return (OutputT) super.apply(
+              new AssignWindowsAndSetStrategy((Window.Bound) transform), input);
+    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
+            && ((PCollectionList<?>) input).size() == 0) {
+      return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of());
+    } else if (Create.Values.class.equals(transform.getClass())) {
+      return (OutputT) PCollection
+              .<OutputT>createPrimitiveOutputInternal(
+                      input.getPipeline(),
+                      WindowingStrategy.globalDefault(),
+                      PCollection.IsBounded.BOUNDED);
+    } else {
+      return super.apply(transform, input);
+    }
+  }
+
+  @Override
+  public GearpumpPipelineResult run(Pipeline pipeline) {
+    String appName = options.getApplicationName();
+    if (null == appName) {
+      appName = DEFAULT_APPNAME;
+    }
+    Config config = registerSerializers(ClusterConfig.defaultConfig(),
+        options.getSerializers());
+    ClientContext clientContext = getClientContext(options, config);
+    options.setClientContext(clientContext);
+    JavaStreamApp streamApp = new JavaStreamApp(
+        appName, clientContext, UserConfig.empty());
+    TranslationContext translationContext = new TranslationContext(streamApp, options);
+    GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext);
+    translator.translate(pipeline);
+    streamApp.run();
+
+    return null;
+  }
+
+  private ClientContext getClientContext(GearpumpPipelineOptions options, Config config) {
+    EmbeddedCluster cluster = options.getEmbeddedCluster();
+    if (cluster != null) {
+      return cluster.newClientContext();
+    } else {
+      return ClientContext.apply(config);
+    }
+  }
+
+  /**
+   * register class with default kryo serializers.
+   */
+  private Config registerSerializers(Config config, Map<String, String> userSerializers) {
+    Map<String, String> serializers = new HashMap<>();
+    serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInSingleWindow", "");
+    serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo", "");
+    serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo$Timing", "");
+    serializers.put("org.joda.time.Instant", "");
+    serializers.put("org.apache.beam.sdk.values.KV", "");
+    serializers.put("org.apache.beam.sdk.transforms.windowing.IntervalWindow", "");
+    serializers.put("org.apache.beam.sdk.values.TimestampedValue", "");
+    if (userSerializers != null && !userSerializers.isEmpty()) {
+      serializers.putAll(userSerializers);
+    }
+    return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers));
+  }
+
+
+  /**
+   * copied from DirectPipelineRunner.
+   * used to replace Window.Bound till window function is added to Gearpump Stream DSL
+   */
+  private static class AssignWindowsAndSetStrategy<T, W extends BoundedWindow>
+      extends PTransform<PCollection<T>, PCollection<T>> {
+
+    private final Window.Bound<T> wrapped;
+
+    AssignWindowsAndSetStrategy(Window.Bound<T> wrapped) {
+      this.wrapped = wrapped;
+    }
+
+    @Override
+    public PCollection<T> apply(PCollection<T> input) {
+      WindowingStrategy<?, ?> outputStrategy =
+          wrapped.getOutputStrategyInternal(input.getWindowingStrategy());
+
+      WindowFn<T, BoundedWindow> windowFn =
+          (WindowFn<T, BoundedWindow>) outputStrategy.getWindowFn();
+
+      if (!windowFn.isNonMerging()) {
+        throw new UnsupportedOperationException(
+            "merging window is not supported in Gearpump pipeline");
+      }
+
+      // If the Window.Bound transform only changed parts other than the WindowFn, then
+      // we skip AssignWindows even though it should be harmless in a perfect world.
+      // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly
+      // crash if another GBK is performed without explicitly setting the WindowFn. So we skip
+      // AssignWindows in this case.
+      if (wrapped.getWindowFn() == null) {
+        return input.apply("Identity", ParDo.of(new IdentityFn<T>()))
+            .setWindowingStrategyInternal(outputStrategy);
+      } else {
+        return input
+            .apply("AssignWindows", new AssignWindows<>(windowFn))
+            .setWindowingStrategyInternal(outputStrategy);
+      }
+    }
+  }
+
+  private static class IdentityFn<T> extends OldDoFn<T, T> {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java
new file mode 100644
index 0000000..b77e1e3
--- /dev/null
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java
@@ -0,0 +1,62 @@
+/*
+ * 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.runners.gearpump;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+/**
+ * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
+ * {@link GearpumpRunner}.
+ *
+ * {@link AutoService} will register Gearpump's implementations of the {@link PipelineRunner}
+ * and {@link PipelineOptions} as available pipeline runner services.
+ */
+public class GearpumpRunnerRegistrar {
+  private GearpumpRunnerRegistrar() { }
+
+  /**
+   * Registers the {@link GearpumpRunner}.
+   */
+  @AutoService(PipelineRunnerRegistrar.class)
+  public static class Runner implements PipelineRunnerRegistrar {
+
+    @Override
+    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(TestGearpumpRunner.class);
+    }
+  }
+
+  /**
+   * Registers the {@link GearpumpPipelineOptions}.
+   */
+  @AutoService(PipelineOptionsRegistrar.class)
+  public static class Options implements PipelineOptionsRegistrar {
+
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+      return ImmutableList.<Class<? extends PipelineOptions>>of(GearpumpPipelineOptions.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java
index cedd31f..89d31a6 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java
@@ -33,14 +33,14 @@ import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
  */
 public class TestGearpumpRunner extends PipelineRunner<GearpumpPipelineResult> {
 
-  private final GearpumpPipelineRunner delegate;
+  private final GearpumpRunner delegate;
   private final EmbeddedCluster cluster;
 
   private TestGearpumpRunner(GearpumpPipelineOptions options) {
     cluster = EmbeddedCluster.apply();
     cluster.start();
     options.setEmbeddedCluster(cluster);
-    delegate = GearpumpPipelineRunner.fromOptions(options);
+    delegate = GearpumpRunner.fromOptions(options);
   }
 
   public static TestGearpumpRunner fromOptions(PipelineOptions options) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/94bd47cd/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
index ba50de7..1d85c25 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
@@ -19,7 +19,7 @@
 package org.apache.beam.runners.gearpump.examples;
 
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
-import org.apache.beam.runners.gearpump.GearpumpPipelineRunner;
+import org.apache.beam.runners.gearpump.GearpumpRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -75,7 +75,7 @@ public class StreamingWordCount {
   public static void main(String[] args) {
     GearpumpPipelineOptions options = PipelineOptionsFactory
             .fromArgs(args).as(GearpumpPipelineOptions.class);
-    options.setRunner(GearpumpPipelineRunner.class);
+    options.setRunner(GearpumpRunner.class);
     options.setApplicationName("StreamingWordCount");
     options.setParallelism(1);
 


[50/50] [abbrv] incubator-beam git commit: Closes #1193

Posted by dh...@apache.org.
Closes #1193


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

Branch: refs/heads/gearpump-runner
Commit: 3933b5577da1808bdb8535e406f26682f62f57fb
Parents: 8f4334c 3f06382
Author: Dan Halperin <dh...@google.com>
Authored: Wed Oct 26 09:42:56 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 09:42:56 2016 -0700

----------------------------------------------------------------------
 .gitignore                                      |   8 +
 .travis.yml                                     |  10 +-
 examples/java/pom.xml                           | 154 ++-
 .../beam/examples/DebuggingWordCount.java       |   2 +-
 .../apache/beam/examples/MinimalWordCount.java  |   3 +-
 .../apache/beam/examples/WindowedWordCount.java |   2 +-
 .../org/apache/beam/examples/WordCount.java     |   4 +-
 .../common/ExampleBigQueryTableOptions.java     |   2 +-
 ...xamplePubsubTopicAndSubscriptionOptions.java |   2 +-
 .../common/ExamplePubsubTopicOptions.java       |   2 +-
 .../beam/examples/common/ExampleUtils.java      |   2 +-
 .../examples/common/PubsubFileInjector.java     | 153 ---
 .../beam/examples/complete/AutoComplete.java    |   2 +-
 .../apache/beam/examples/complete/TfIdf.java    |   2 +-
 .../examples/complete/TopWikipediaSessions.java |   2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../examples/cookbook/DatastoreWordCount.java   | 261 -----
 .../beam/examples/cookbook/DeDupExample.java    |   4 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../beam/examples/cookbook/JoinExamples.java    |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../beam/examples/cookbook/TriggerExample.java  |  28 +-
 .../beam/examples/WindowedWordCountIT.java      |  75 ++
 .../org/apache/beam/examples/WordCountIT.java   |   6 -
 .../examples/cookbook/BigQueryTornadoesIT.java  |  14 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |  10 +-
 .../examples/complete/game/HourlyTeamScore.java |   8 +-
 .../examples/complete/game/LeaderBoard.java     |  12 +-
 .../beam/examples/complete/game/UserScore.java  |  10 +-
 .../complete/game/injector/Injector.java        |  10 +-
 .../examples/complete/game/LeaderBoardTest.java |   5 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         | 134 ++-
 runners/core-java/pom.xml                       |   8 +-
 .../beam/runners/core/AggregatorFactory.java    |  39 +
 .../beam/runners/core/BatchTimerInternals.java  | 140 ---
 .../apache/beam/runners/core/DoFnRunner.java    |   8 +-
 .../beam/runners/core/DoFnRunnerBase.java       | 559 -----------
 .../apache/beam/runners/core/DoFnRunners.java   | 191 +++-
 .../runners/core/ElementAndRestriction.java     |  42 +
 .../core/ElementAndRestrictionCoder.java        |  67 ++
 .../runners/core/ElementByteSizeObservable.java |   5 +-
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 +
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   5 +
 .../runners/core/GroupAlsoByWindowsDoFn.java    |  19 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  12 +-
 .../runners/core/ReduceFnContextFactory.java    |   2 +-
 .../beam/runners/core/ReduceFnRunner.java       |  42 +-
 .../beam/runners/core/SideInputHandler.java     |   2 +-
 .../beam/runners/core/SimpleDoFnRunner.java     | 554 ++++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  | 521 ++++++++++
 .../beam/runners/core/SplittableParDo.java      | 469 +++++++++
 .../apache/beam/runners/core/TriggerRunner.java | 247 -----
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../core/triggers/AfterAllStateMachine.java     | 109 ++
 .../AfterDelayFromFirstElementStateMachine.java | 337 +++++++
 .../core/triggers/AfterEachStateMachine.java    | 130 +++
 .../core/triggers/AfterFirstStateMachine.java   | 112 +++
 .../core/triggers/AfterPaneStateMachine.java    | 139 +++
 .../AfterProcessingTimeStateMachine.java        |  93 ++
 ...rSynchronizedProcessingTimeStateMachine.java |  63 ++
 .../triggers/AfterWatermarkStateMachine.java    | 325 ++++++
 .../triggers/DefaultTriggerStateMachine.java    |  81 ++
 .../triggers/ExecutableTriggerStateMachine.java | 160 +++
 .../runners/core/triggers/FinishedTriggers.java |  44 +
 .../core/triggers/FinishedTriggersBitSet.java   |  67 ++
 .../core/triggers/FinishedTriggersSet.java      |  72 ++
 .../core/triggers/NeverStateMachine.java        |  60 ++
 .../core/triggers/OrFinallyStateMachine.java    |  85 ++
 .../core/triggers/RepeatedlyStateMachine.java   |  88 ++
 .../triggers/ReshuffleTriggerStateMachine.java  |  50 +
 .../core/triggers/TriggerStateMachine.java      | 487 +++++++++
 .../TriggerStateMachineContextFactory.java      | 509 ++++++++++
 .../triggers/TriggerStateMachineRunner.java     | 234 +++++
 .../core/triggers/TriggerStateMachines.java     | 215 ++++
 .../runners/core/triggers/package-info.java     |  23 +
 .../runners/core/BatchTimerInternalsTest.java   | 118 ---
 .../core/ElementAndRestrictionCoderTest.java    | 127 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   | 281 +++---
 .../beam/runners/core/ReduceFnTester.java       | 405 +++-----
 .../beam/runners/core/SimpleDoFnRunnerTest.java |  88 --
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  88 ++
 .../beam/runners/core/SplittableParDoTest.java  | 467 +++++++++
 .../core/triggers/AfterAllStateMachineTest.java | 140 +++
 .../triggers/AfterEachStateMachineTest.java     | 108 ++
 .../triggers/AfterFirstStateMachineTest.java    | 159 +++
 .../triggers/AfterPaneStateMachineTest.java     | 117 +++
 .../AfterProcessingTimeStateMachineTest.java    | 172 ++++
 ...chronizedProcessingTimeStateMachineTest.java | 110 ++
 .../AfterWatermarkStateMachineTest.java         | 382 +++++++
 .../DefaultTriggerStateMachineTest.java         | 165 +++
 .../ExecutableTriggerStateMachineTest.java      | 108 ++
 .../triggers/FinishedTriggersBitSetTest.java    |  55 +
 .../triggers/FinishedTriggersProperties.java    | 115 +++
 .../core/triggers/FinishedTriggersSetTest.java  |  60 ++
 .../core/triggers/NeverStateMachineTest.java    |  59 ++
 .../triggers/OrFinallyStateMachineTest.java     | 177 ++++
 .../triggers/RepeatedlyStateMachineTest.java    | 200 ++++
 .../ReshuffleTriggerStateMachineTest.java       |  68 ++
 .../core/triggers/StubTriggerStateMachine.java  |  60 ++
 .../core/triggers/TriggerStateMachineTest.java  |  98 ++
 .../triggers/TriggerStateMachineTester.java     | 431 ++++++++
 .../core/triggers/TriggerStateMachinesTest.java | 199 ++++
 runners/direct-java/pom.xml                     |  26 +-
 .../runners/direct/AggregatorContainer.java     |   9 +-
 .../direct/BoundedReadEvaluatorFactory.java     | 155 +--
 .../beam/runners/direct/BundleFactory.java      |  15 +-
 .../runners/direct/CloningBundleFactory.java    |  98 ++
 .../beam/runners/direct/CompletionCallback.java |   4 +-
 .../runners/direct/DirectExecutionContext.java  |   2 +-
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  66 ++
 .../beam/runners/direct/DirectMetrics.java      | 338 +++++++
 .../beam/runners/direct/DirectOptions.java      |  40 +-
 .../beam/runners/direct/DirectRunner.java       | 103 +-
 .../runners/direct/DirectTimerInternals.java    |   2 +-
 .../runners/direct/DoFnLifecycleManager.java    |  86 +-
 .../beam/runners/direct/EmptyInputProvider.java |  45 +
 .../direct/EncodabilityEnforcementFactory.java  |  50 +-
 .../beam/runners/direct/EvaluationContext.java  |  27 +-
 .../direct/ExecutorServiceParallelExecutor.java | 124 ++-
 .../runners/direct/FlattenEvaluatorFactory.java |  26 +-
 .../runners/direct/ForwardingPTransform.java    |   2 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  14 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |  21 +-
 .../ImmutabilityCheckingBundleFactory.java      |  18 +-
 .../direct/ImmutableListBundleFactory.java      |  76 +-
 .../beam/runners/direct/KeyedResourcePool.java  |  47 -
 .../runners/direct/LockedKeyedResourcePool.java |  95 --
 .../beam/runners/direct/ParDoEvaluator.java     |  13 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |  29 +-
 .../runners/direct/ParDoOverrideFactory.java    |  55 +
 .../direct/ParDoSingleEvaluatorFactory.java     |  25 +-
 .../beam/runners/direct/PipelineExecutor.java   |   2 +-
 .../beam/runners/direct/RootInputProvider.java  |  46 +
 .../runners/direct/RootProviderRegistry.java    |  66 ++
 .../runners/direct/StepTransformResult.java     |  49 +-
 .../beam/runners/direct/StructuralKey.java      |  88 +-
 .../direct/TestStreamEvaluatorFactory.java      | 152 +--
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorFactory.java       |  22 +-
 .../direct/TransformEvaluatorRegistry.java      |  44 +-
 .../beam/runners/direct/TransformExecutor.java  |  53 +-
 .../beam/runners/direct/TransformResult.java    |  16 +-
 .../direct/UnboundedReadEvaluatorFactory.java   | 318 +++---
 .../direct/UncommittedBundleOutputManager.java  |   4 +-
 .../runners/direct/ViewEvaluatorFactory.java    |  16 +-
 .../beam/runners/direct/WatermarkManager.java   |  53 +-
 .../runners/direct/WindowEvaluatorFactory.java  |  18 +-
 .../direct/WriteWithShardingFactory.java        |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java | 181 ++--
 .../direct/CloningBundleFactoryTest.java        | 177 ++++
 .../runners/direct/CommittedResultTest.java     |  10 +-
 .../ConsumerTrackingPipelineVisitorTest.java    |  32 +-
 .../beam/runners/direct/DirectMetricsTest.java  | 133 +++
 .../beam/runners/direct/DirectRunnerTest.java   | 180 +++-
 ...leManagerRemovingTransformEvaluatorTest.java |  16 +-
 .../direct/DoFnLifecycleManagerTest.java        |  86 +-
 .../direct/DoFnLifecycleManagersTest.java       |  48 +-
 .../EncodabilityEnforcementFactoryTest.java     | 132 ++-
 .../runners/direct/EvaluationContextTest.java   |  31 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |  36 +-
 .../direct/ForwardingPTransformTest.java        |   7 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  27 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  35 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |  67 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  14 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  52 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   8 +-
 .../direct/LockedKeyedResourcePoolTest.java     | 163 ---
 .../beam/runners/direct/ParDoEvaluatorTest.java |  15 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 162 +--
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 145 ++-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 +++++
 .../runners/direct/StepTransformResultTest.java |   4 +-
 .../beam/runners/direct/StructuralKeyTest.java  |   9 +
 .../direct/TestStreamEvaluatorFactoryTest.java  | 223 ++---
 .../runners/direct/TransformExecutorTest.java   | 115 +--
 .../UnboundedReadEvaluatorFactoryTest.java      | 321 ++++--
 .../direct/ViewEvaluatorFactoryTest.java        |   6 +-
 .../runners/direct/WatermarkManagerTest.java    |  84 +-
 .../direct/WindowEvaluatorFactoryTest.java      |  12 +-
 runners/flink/README.md                         |  10 +-
 runners/flink/examples/pom.xml                  |   2 +-
 .../beam/runners/flink/examples/TFIDF.java      |  12 +-
 .../beam/runners/flink/examples/WordCount.java  |   4 +-
 .../flink/examples/streaming/AutoComplete.java  |   4 +-
 runners/flink/pom.xml                           |   4 +-
 runners/flink/runner/pom.xml                    |  10 +-
 .../flink/FlinkDetachedRunnerResult.java        |  76 ++
 .../FlinkPipelineExecutionEnvironment.java      |   7 +
 .../runners/flink/FlinkPipelineOptions.java     |  19 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  29 +-
 .../runners/flink/FlinkRunnerRegistrar.java     |   4 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |  17 +-
 .../beam/runners/flink/TestFlinkRunner.java     |   9 +-
 .../FlinkBatchPipelineTranslator.java           |   2 +-
 .../FlinkStreamingPipelineTranslator.java       |   2 +-
 .../functions/FlinkDoFnFunction.java            |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   2 +-
 .../functions/FlinkProcessContext.java          |  10 +-
 .../wrappers/streaming/DoFnOperator.java        |  31 +-
 .../wrappers/streaming/FlinkStateInternals.java |   8 +-
 .../wrappers/streaming/WindowDoFnOperator.java  | 177 +++-
 .../streaming/io/BoundedSourceWrapper.java      |   9 +-
 .../streaming/io/UnboundedSourceWrapper.java    | 187 +++-
 .../beam/runners/flink/FlinkTestPipeline.java   |   6 +-
 .../apache/beam/runners/flink/package-info.java |  22 -
 .../flink/streaming/DoFnOperatorTest.java       |   9 +-
 .../streaming/UnboundedSourceWrapperTest.java   | 123 +--
 runners/gearpump/pom.xml                        |   4 +-
 .../gearpump/GearpumpPipelineResult.java        |   6 +
 .../gearpump/GearpumpPipelineRunner.java        | 193 ----
 .../GearpumpPipelineRunnerRegistrar.java        |  62 --
 .../beam/runners/gearpump/GearpumpRunner.java   | 191 ++++
 .../gearpump/GearpumpRunnerRegistrar.java       |  62 ++
 .../runners/gearpump/TestGearpumpRunner.java    |   4 +-
 .../gearpump/examples/StreamingWordCount.java   |   4 +-
 .../translators/utils/GearpumpDoFnRunner.java   |  16 +-
 runners/google-cloud-dataflow-java/pom.xml      |  22 +-
 .../runners/dataflow/DataflowPipelineJob.java   |  52 +-
 .../dataflow/DataflowPipelineTranslator.java    | 137 ++-
 .../beam/runners/dataflow/DataflowRunner.java   |  57 +-
 .../dataflow/internal/AssignWindows.java        |   6 +-
 .../DataflowUnboundedReadFromBoundedSource.java |  18 +-
 .../runners/dataflow/internal/IsmFormat.java    |  13 +-
 .../dataflow/internal/ReadTranslator.java       |   2 +-
 .../options/DataflowPipelineDebugOptions.java   |   7 +-
 .../options/DataflowPipelineOptions.java        |  14 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  11 +-
 .../options/DataflowProfilingOptions.java       |   2 +-
 .../options/DataflowWorkerLoggingOptions.java   |   4 +-
 .../dataflow/testing/TestDataflowRunner.java    | 144 ++-
 .../beam/runners/dataflow/util/DoFnInfo.java    |  30 +-
 .../runners/dataflow/util/MonitoringUtil.java   |   2 +-
 .../runners/dataflow/util/RandomAccessData.java |   8 +-
 .../beam/runners/dataflow/util/Stager.java      |   2 +-
 .../dataflow/DataflowPipelineJobTest.java       | 120 +++
 .../DataflowPipelineTranslatorTest.java         |  40 +-
 .../runners/dataflow/DataflowRunnerTest.java    |  13 +-
 ...aflowUnboundedReadFromBoundedSourceTest.java |  83 ++
 .../testing/TestDataflowRunnerTest.java         | 287 +++++-
 .../dataflow/util/MonitoringUtilTest.java       |   4 +-
 runners/pom.xml                                 |   3 +-
 runners/spark/pom.xml                           |  94 +-
 .../beam/runners/spark/EvaluationResult.java    |   4 +-
 .../runners/spark/SparkPipelineOptions.java     |  48 +-
 .../apache/beam/runners/spark/SparkRunner.java  | 176 ++--
 .../beam/runners/spark/TestSparkRunner.java     |  17 +-
 .../spark/aggregators/AccumulatorSingleton.java |  53 +
 .../runners/spark/coders/WritableCoder.java     |   2 +-
 .../beam/runners/spark/examples/WordCount.java  |   2 +-
 .../apache/beam/runners/spark/io/SourceRDD.java | 200 ++++
 .../spark/io/hadoop/ShardNameTemplateAware.java |   2 +-
 .../runners/spark/translation/DoFnFunction.java |  75 +-
 .../spark/translation/EvaluationContext.java    |  32 +-
 .../translation/GroupCombineFunctions.java      | 313 ++++++
 .../spark/translation/MultiDoFnFunction.java    |  77 +-
 .../translation/SparkAbstractCombineFn.java     | 134 +++
 .../spark/translation/SparkContextFactory.java  |  50 +-
 .../spark/translation/SparkGlobalCombineFn.java | 260 +++++
 .../spark/translation/SparkKeyedCombineFn.java  | 273 +++++
 .../translation/SparkPipelineEvaluator.java     |  57 --
 .../translation/SparkPipelineTranslator.java    |   5 +-
 .../spark/translation/SparkProcessContext.java  | 168 +++-
 .../spark/translation/SparkRuntimeContext.java  |  44 +-
 .../spark/translation/TransformTranslator.java  | 592 +++--------
 .../spark/translation/TranslationUtils.java     | 197 ++++
 .../SparkRunnerStreamingContextFactory.java     | 106 ++
 .../streaming/StreamingEvaluationContext.java   |  81 +-
 .../streaming/StreamingTransformTranslator.java | 561 +++++++----
 .../spark/util/SparkSideInputReader.java        |  95 ++
 .../runners/spark/ClearAggregatorsRule.java     |  33 +
 .../apache/beam/runners/spark/DeDupTest.java    |  59 --
 .../beam/runners/spark/EmptyInputTest.java      |  75 --
 .../beam/runners/spark/SimpleWordCountTest.java | 105 --
 .../apache/beam/runners/spark/TfIdfTest.java    | 260 -----
 .../beam/runners/spark/io/AvroPipelineTest.java |   4 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   4 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   4 +-
 .../spark/translation/CombineGloballyTest.java  | 102 --
 .../spark/translation/CombinePerKeyTest.java    |  79 --
 .../spark/translation/DoFnOutputTest.java       |  67 --
 .../translation/MultiOutputWordCountTest.java   | 176 ----
 .../spark/translation/SerializationTest.java    | 201 ----
 .../spark/translation/SideEffectsTest.java      |  21 +-
 .../translation/SparkPipelineOptionsTest.java   |  42 -
 .../translation/TransformTranslatorTest.java    | 104 --
 .../translation/WindowedWordCountTest.java      | 120 ---
 .../streaming/EmptyStreamAssertionTest.java     |  80 ++
 .../streaming/FlattenStreamingTest.java         |  57 +-
 .../streaming/KafkaStreamingTest.java           |  32 +-
 .../ResumeFromCheckpointStreamingTest.java      | 182 ++++
 .../streaming/SimpleStreamingWordCountTest.java |  67 +-
 .../streaming/utils/PAssertStreaming.java       |  87 +-
 .../utils/TestOptionsForStreaming.java          |  55 +
 .../spark/src/test/resources/metrics.properties |  61 +-
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/checkstyle.xml      |  28 +-
 .../src/main/resources/beam/findbugs-filter.xml |   2 +-
 .../src/main/resources/beam/suppressions.xml    |  11 +-
 sdks/java/core/pom.xml                          |  43 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  14 +-
 .../org/apache/beam/sdk/PipelineResult.java     |  15 +-
 .../beam/sdk/annotations/Experimental.java      |  15 +-
 .../sdk/coders/CannotProvideCoderException.java |   2 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |  28 +-
 .../apache/beam/sdk/coders/CoderFactory.java    |   4 +-
 .../apache/beam/sdk/coders/CoderProvider.java   |   2 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   4 +-
 .../beam/sdk/coders/IterableLikeCoder.java      |  22 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |  10 +-
 .../apache/beam/sdk/coders/package-info.java    |   2 +-
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |   4 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     | 162 ++-
 .../java/org/apache/beam/sdk/io/AvroSource.java |  98 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  11 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |  19 +-
 .../apache/beam/sdk/io/CompressedSource.java    |  18 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   | 145 ++-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   6 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |  24 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |   2 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../sdk/io/SerializableAvroCodecFactory.java    | 112 +++
 .../main/java/org/apache/beam/sdk/io/Sink.java  |   5 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     | 236 +++--
 .../main/java/org/apache/beam/sdk/io/Write.java |   8 +-
 .../java/org/apache/beam/sdk/io/XmlSource.java  |  13 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   6 +-
 .../apache/beam/sdk/io/range/RangeTracker.java  |   1 +
 .../org/apache/beam/sdk/metrics/Counter.java    |  40 +
 .../apache/beam/sdk/metrics/CounterCell.java    |  76 ++
 .../org/apache/beam/sdk/metrics/DirtyState.java |  98 ++
 .../apache/beam/sdk/metrics/Distribution.java   |  30 +
 .../beam/sdk/metrics/DistributionCell.java      |  58 ++
 .../beam/sdk/metrics/DistributionData.java      |  59 ++
 .../beam/sdk/metrics/DistributionResult.java    |  42 +
 .../org/apache/beam/sdk/metrics/Metric.java     |  24 +
 .../org/apache/beam/sdk/metrics/MetricCell.java |  47 +
 .../org/apache/beam/sdk/metrics/MetricKey.java  |  40 +
 .../org/apache/beam/sdk/metrics/MetricName.java |  46 +
 .../beam/sdk/metrics/MetricNameFilter.java      |  60 ++
 .../beam/sdk/metrics/MetricQueryResults.java    |  33 +
 .../apache/beam/sdk/metrics/MetricResult.java   |  45 +
 .../apache/beam/sdk/metrics/MetricResults.java  |  34 +
 .../apache/beam/sdk/metrics/MetricUpdates.java  |  72 ++
 .../org/apache/beam/sdk/metrics/Metrics.java    | 110 ++
 .../beam/sdk/metrics/MetricsContainer.java      | 150 +++
 .../beam/sdk/metrics/MetricsEnvironment.java    |  85 ++
 .../apache/beam/sdk/metrics/MetricsFilter.java  |  86 ++
 .../org/apache/beam/sdk/metrics/MetricsMap.java |  86 ++
 .../apache/beam/sdk/metrics/package-info.java   |  28 +
 .../options/CloudResourceManagerOptions.java    |  40 +
 .../org/apache/beam/sdk/options/GcpOptions.java |   8 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |   4 +-
 .../beam/sdk/options/GoogleApiDebugOptions.java |   2 +-
 .../beam/sdk/options/PipelineOptions.java       |  52 +-
 .../sdk/options/PipelineOptionsFactory.java     | 269 +++--
 .../sdk/options/ProxyInvocationHandler.java     | 264 +++--
 .../apache/beam/sdk/options/ValueProvider.java  | 239 +++++
 .../java/org/apache/beam/sdk/package-info.java  |   2 +-
 .../apache/beam/sdk/runners/PipelineRunner.java |   1 -
 .../sdk/runners/PipelineRunnerRegistrar.java    |   2 +-
 .../apache/beam/sdk/runners/package-info.java   |  17 +-
 .../beam/sdk/testing/BigqueryMatcher.java       | 239 +++++
 .../org/apache/beam/sdk/testing/PAssert.java    |   4 +-
 .../beam/sdk/testing/SerializableMatchers.java  |   4 +-
 .../beam/sdk/testing/SourceTestUtils.java       |   3 +-
 .../apache/beam/sdk/testing/StreamingIT.java    |  35 +
 .../apache/beam/sdk/testing/TestPipeline.java   |   2 -
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/package-info.java   |   4 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |  11 +-
 .../sdk/transforms/ApproximateQuantiles.java    |   5 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  95 +-
 .../beam/sdk/transforms/CombineFnBase.java      |  17 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |  41 +-
 .../beam/sdk/transforms/CombineWithContext.java |   7 +-
 .../sdk/transforms/DelegatingAggregator.java    | 125 +++
 .../org/apache/beam/sdk/transforms/DoFn.java    | 431 +++++++-
 .../beam/sdk/transforms/DoFnAdapters.java       | 184 +++-
 .../apache/beam/sdk/transforms/DoFnTester.java  | 257 +++--
 .../beam/sdk/transforms/FlatMapElements.java    |   6 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   4 +-
 .../transforms/IntraBundleParallelization.java  | 361 -------
 .../org/apache/beam/sdk/transforms/Latest.java  |  12 +-
 .../apache/beam/sdk/transforms/MapElements.java |   8 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 318 ++++--
 .../apache/beam/sdk/transforms/PTransform.java  |  10 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 343 +++++--
 .../apache/beam/sdk/transforms/Partition.java   |   4 +-
 .../beam/sdk/transforms/RemoveDuplicates.java   |   5 +-
 .../sdk/transforms/SerializableFunction.java    |   2 +-
 .../org/apache/beam/sdk/transforms/ViewFn.java  |   2 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../sdk/transforms/display/DisplayData.java     | 530 ++++++----
 .../sdk/transforms/reflect/DoFnInvoker.java     |  48 +-
 .../sdk/transforms/reflect/DoFnInvokers.java    | 658 ++++++++----
 .../sdk/transforms/reflect/DoFnSignature.java   | 466 ++++++++-
 .../sdk/transforms/reflect/DoFnSignatures.java  | 946 +++++++++++++++---
 .../splittabledofn/RestrictionTracker.java      |  42 +
 .../transforms/splittabledofn/package-info.java |  22 +
 .../beam/sdk/transforms/windowing/AfterAll.java |  51 +-
 .../windowing/AfterDelayFromFirstElement.java   | 110 +-
 .../sdk/transforms/windowing/AfterEach.java     |  63 +-
 .../sdk/transforms/windowing/AfterFirst.java    |  52 +-
 .../sdk/transforms/windowing/AfterPane.java     |  59 +-
 .../windowing/AfterProcessingTime.java          |   7 -
 .../AfterSynchronizedProcessingTime.java        |  13 +-
 .../transforms/windowing/AfterWatermark.java    | 174 +---
 .../transforms/windowing/DefaultTrigger.java    |  37 +-
 .../beam/sdk/transforms/windowing/Never.java    |  28 +-
 .../transforms/windowing/OrFinallyTrigger.java  |  55 +-
 .../beam/sdk/transforms/windowing/PaneInfo.java |   6 +-
 .../sdk/transforms/windowing/Repeatedly.java    |  36 +-
 .../transforms/windowing/SlidingWindows.java    |   3 +-
 .../beam/sdk/transforms/windowing/Trigger.java  | 421 ++------
 .../beam/sdk/transforms/windowing/Window.java   |  20 +-
 .../beam/sdk/transforms/windowing/WindowFn.java |   2 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java | 173 ++++
 .../util/AttemptBoundedExponentialBackOff.java  |  86 ++
 .../beam/sdk/util/BaseExecutionContext.java     |   4 +-
 .../apache/beam/sdk/util/CredentialFactory.java |   2 +-
 .../apache/beam/sdk/util/ExecutableTrigger.java |  40 +-
 .../sdk/util/ExposedByteArrayOutputStream.java  |   1 +
 .../apache/beam/sdk/util/FinishedTriggers.java  |  44 -
 .../beam/sdk/util/FinishedTriggersBitSet.java   |  67 --
 .../beam/sdk/util/FinishedTriggersSet.java      |  72 --
 .../apache/beam/sdk/util/GatherAllPanes.java    |  10 +-
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 ++
 .../apache/beam/sdk/util/GcsPathValidator.java  |   2 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  94 +-
 .../apache/beam/sdk/util/IOChannelFactory.java  |   2 +-
 .../beam/sdk/util/MergingActiveWindowSet.java   |  12 +-
 .../apache/beam/sdk/util/PCollectionViews.java  |   9 +-
 .../org/apache/beam/sdk/util/PathValidator.java |   6 +-
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  26 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |   4 +-
 .../org/apache/beam/sdk/util/PropertyNames.java |   1 +
 .../org/apache/beam/sdk/util/PubsubClient.java  |   5 +-
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |  34 +-
 .../apache/beam/sdk/util/PubsubTestClient.java  |   2 +-
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |  16 +-
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   6 +-
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |  16 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   2 +-
 .../java/org/apache/beam/sdk/util/Timer.java    |  56 ++
 .../apache/beam/sdk/util/TimerInternals.java    |   6 +-
 .../org/apache/beam/sdk/util/TimerSpec.java     |  30 +
 .../org/apache/beam/sdk/util/TimerSpecs.java    |  41 +
 .../java/org/apache/beam/sdk/util/Timers.java   |  10 +-
 .../org/apache/beam/sdk/util/Transport.java     |  17 +
 .../beam/sdk/util/TriggerContextFactory.java    | 507 ----------
 .../apache/beam/sdk/util/ValueWithRecordId.java |   8 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |  22 -
 .../apache/beam/sdk/util/common/Reiterable.java |   2 +-
 .../apache/beam/sdk/util/common/Reiterator.java |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java     |   2 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 235 +++++
 .../beam/sdk/util/state/ReadableState.java      |  10 +-
 .../apache/beam/sdk/util/state/StateBinder.java |  67 ++
 .../beam/sdk/util/state/StateContext.java       |   6 +-
 .../apache/beam/sdk/util/state/StateSpec.java   |  39 +
 .../apache/beam/sdk/util/state/StateSpecs.java  | 452 +++++++++
 .../apache/beam/sdk/util/state/StateTag.java    |  82 +-
 .../apache/beam/sdk/util/state/StateTags.java   | 386 +-------
 .../util/state/TestInMemoryStateInternals.java  |  61 ++
 .../beam/sdk/util/state/TimerCallback.java      |  35 +
 .../apache/beam/sdk/values/PCollectionView.java |  15 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   8 +-
 .../org/apache/beam/sdk/values/POutput.java     |   8 +-
 .../java/org/apache/beam/sdk/values/PValue.java |   4 +-
 .../apache/beam/sdk/values/TypeDescriptors.java |  40 +-
 .../apache/beam/sdk/values/package-info.java    |   4 +-
 .../dataflow/util/GcsPathValidatorTest.java     | 103 --
 .../org/apache/beam/sdk/DataflowMatchers.java   |  64 --
 .../java/org/apache/beam/sdk/PipelineTest.java  |   2 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   4 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   1 +
 .../org/apache/beam/sdk/coders/KvCoderTest.java |  99 +-
 .../beam/sdk/coders/NullableCoderTest.java      |  60 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java | 107 +-
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  43 +
 .../io/BoundedReadFromUnboundedSourceTest.java  |  14 +-
 .../beam/sdk/io/CompressedSourceTest.java       |   4 +-
 .../sdk/io/DrunkWritableByteChannelFactory.java |  80 ++
 .../apache/beam/sdk/io/FileBasedSinkTest.java   | 108 ++
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  30 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |  62 +-
 .../io/SerializableAvroCodecFactoryTest.java    | 100 ++
 .../java/org/apache/beam/sdk/io/TextIOTest.java | 601 ++++++-----
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   6 +-
 .../beam/sdk/metrics/CounterCellTest.java       |  55 +
 .../apache/beam/sdk/metrics/DirtyStateTest.java |  56 ++
 .../beam/sdk/metrics/DistributionCellTest.java  |  53 +
 .../apache/beam/sdk/metrics/MetricMatchers.java |  99 ++
 .../beam/sdk/metrics/MetricsContainerTest.java  | 129 +++
 .../sdk/metrics/MetricsEnvironmentTest.java     |  63 ++
 .../apache/beam/sdk/metrics/MetricsMapTest.java | 103 ++
 .../apache/beam/sdk/metrics/MetricsTest.java    |  98 ++
 .../sdk/options/PipelineOptionsFactoryTest.java | 184 +++-
 .../beam/sdk/options/PipelineOptionsTest.java   |  49 +-
 .../options/PipelineOptionsValidatorTest.java   |  18 +-
 .../sdk/options/ProxyInvocationHandlerTest.java | 111 ++-
 .../beam/sdk/options/ValueProviderTest.java     | 220 ++++
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ++++
 .../sdk/testing/PCollectionViewTesting.java     |   7 -
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |   2 +-
 .../beam/sdk/testing/TestPipelineTest.java      |   4 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   7 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  81 +-
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     | 458 +++++----
 .../apache/beam/sdk/transforms/FlattenTest.java |  20 +
 .../IntraBundleParallelizationTest.java         | 280 ------
 .../beam/sdk/transforms/LatestFnTest.java       | 233 +++++
 .../beam/sdk/transforms/LatestFnTests.java      | 233 -----
 .../apache/beam/sdk/transforms/ParDoTest.java   | 101 +-
 .../display/DisplayDataEvaluator.java           |  13 +-
 .../transforms/display/DisplayDataMatchers.java | 141 ++-
 .../display/DisplayDataMatchersTest.java        |  67 +-
 .../sdk/transforms/display/DisplayDataTest.java | 367 ++++---
 .../transforms/reflect/DoFnInvokersTest.java    | 710 +++++++------
 .../reflect/DoFnInvokersTestHelper.java         | 116 ---
 .../DoFnSignaturesProcessElementTest.java       | 213 ++++
 .../DoFnSignaturesSplittableDoFnTest.java       | 543 ++++++++++
 .../transforms/reflect/DoFnSignaturesTest.java  | 991 ++++++++++++++-----
 .../reflect/DoFnSignaturesTestUtils.java        |  67 ++
 .../testhelper/DoFnInvokersTestHelper.java      | 124 +++
 .../sdk/transforms/windowing/AfterAllTest.java  |  98 --
 .../sdk/transforms/windowing/AfterEachTest.java |  64 --
 .../transforms/windowing/AfterFirstTest.java    | 120 ---
 .../sdk/transforms/windowing/AfterPaneTest.java |  77 --
 .../windowing/AfterProcessingTimeTest.java      |  94 --
 .../AfterSynchronizedProcessingTimeTest.java    |  75 --
 .../windowing/AfterWatermarkTest.java           | 308 ------
 .../windowing/DefaultTriggerTest.java           | 130 ---
 .../sdk/transforms/windowing/NeverTest.java     |  34 +-
 .../windowing/OrFinallyTriggerTest.java         | 136 ---
 .../transforms/windowing/RepeatedlyTest.java    | 161 +--
 .../sdk/transforms/windowing/StubTrigger.java   |  17 -
 .../sdk/transforms/windowing/TriggerTest.java   |  28 -
 .../sdk/transforms/windowing/WindowTest.java    |   4 +-
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  28 +-
 ...mptAndTimeBoundedExponentialBackOffTest.java | 213 ++++
 .../AttemptBoundedExponentialBackOffTest.java   |  85 ++
 .../beam/sdk/util/ExecutableTriggerTest.java    |  18 -
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 -
 .../sdk/util/FinishedTriggersProperties.java    | 110 --
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 --
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 ++
 .../beam/sdk/util/GcsPathValidatorTest.java     | 100 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 112 ++-
 .../beam/sdk/util/PubsubGrpcClientTest.java     | 108 +-
 .../beam/sdk/util/PubsubJsonClientTest.java     |  16 +-
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  23 -
 .../org/apache/beam/sdk/util/TriggerTester.java | 592 -----------
 .../util/state/InMemoryTimerInternalsTest.java  | 116 +++
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/io/google-cloud-platform/pom.xml      |  16 +-
 .../sdk/io/gcp/bigquery/BigQueryAvroUtils.java  |  79 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 239 +++--
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |  15 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 106 +-
 .../gcp/bigquery/BigQueryTableRowIterator.java  | 130 ++-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  56 +-
 .../beam/sdk/io/gcp/datastore/DatastoreIO.java  |   2 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  99 +-
 .../io/gcp/bigquery/BigQueryAvroUtilsTest.java  | 149 ++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 570 ++++++++++-
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 149 ++-
 .../bigquery/BigQueryTableRowIteratorTest.java  | 169 +++-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     | 104 +-
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    |   4 +-
 .../beam/sdk/io/gcp/datastore/V1ReadIT.java     |   2 +-
 .../beam/sdk/io/gcp/datastore/V1WriteIT.java    |   2 +-
 sdks/java/io/hdfs/pom.xml                       |   2 +-
 .../beam/sdk/io/hdfs/AvroHDFSFileSource.java    |   2 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |   2 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |  10 +-
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |   2 +-
 .../SimpleAuthAvroHDFSFileSource.java           |   2 +-
 .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java |   2 +-
 .../simpleauth/SimpleAuthHDFSFileSource.java    |   7 +-
 sdks/java/io/jdbc/pom.xml                       | 138 +++
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     | 443 +++++++++
 .../apache/beam/sdk/io/jdbc/package-info.java   |  22 +
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java | 271 +++++
 sdks/java/io/jms/pom.xml                        |   2 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  |  11 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../beam/sdk/io/kafka/KafkaCheckpointMark.java  |   8 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  90 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   | 166 +++-
 sdks/java/io/kinesis/pom.xml                    |   3 +-
 .../beam/sdk/io/kinesis/CustomOptional.java     |   7 +-
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |   2 +-
 .../sdk/io/kinesis/KinesisClientProvider.java   |   2 +-
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  18 +-
 .../beam/sdk/io/kinesis/KinesisReader.java      |  10 +-
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |   4 +-
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |   2 +-
 .../beam/sdk/io/kinesis/KinesisSource.java      |   6 +-
 .../beam/sdk/io/kinesis/RecordFilter.java       |   6 +-
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |   2 +-
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |   6 +-
 .../sdk/io/kinesis/ShardRecordsIterator.java    |   4 +-
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |   8 +-
 .../beam/sdk/io/kinesis/StartingPoint.java      |   2 +-
 .../beam/sdk/io/kinesis/KinesisTestOptions.java |   2 +-
 .../beam/sdk/io/kinesis/KinesisUploader.java    |   5 +-
 .../beam/sdk/io/kinesis/package-info.java       |  22 -
 sdks/java/io/mongodb/pom.xml                    |  14 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    | 449 +++++++++
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   | 310 +++---
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     | 276 ++++++
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  19 +-
 .../beam/sdk/io/mongodb/package-info.java       |  22 -
 sdks/java/io/pom.xml                            |   3 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../PipelineOptionsFactoryJava8Test.java        |   8 +-
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../main/resources/archetype-resources/pom.xml  |  26 +-
 .../src/main/java/DebuggingWordCount.java       |  34 +-
 .../src/main/java/MinimalWordCount.java         |  50 +-
 .../src/main/java/WindowedWordCount.java        | 139 +--
 .../src/main/java/WordCount.java                |  79 +-
 .../java/common/DataflowExampleOptions.java     |  32 -
 .../main/java/common/DataflowExampleUtils.java  | 391 --------
 .../common/ExampleBigQueryTableOptions.java     |  11 +-
 .../src/main/java/common/ExampleOptions.java    |  32 +
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 +
 .../java/common/ExamplePubsubTopicOptions.java  |  17 +-
 .../src/main/java/common/ExampleUtils.java      | 353 +++++++
 .../main/java/common/PubsubFileInjector.java    | 153 ---
 .../src/test/java/WordCountTest.java            |   9 +-
 sdks/java/maven-archetypes/pom.xml              |   2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |  10 +-
 .../main/resources/archetype-resources/pom.xml  |   8 +-
 .../resources/projects/basic/reference/pom.xml  |   8 +-
 sdks/java/microbenchmarks/pom.xml               |   2 +-
 .../transforms/DoFnInvokersBenchmark.java       |   7 +
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 647 files changed, 37183 insertions(+), 17519 deletions(-)
----------------------------------------------------------------------




[24/50] [abbrv] incubator-beam git commit: Closes #1135

Posted by dh...@apache.org.
Closes #1135


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

Branch: refs/heads/gearpump-runner
Commit: d285e85da7d24e7cfc3ede951f00248a11f21b76
Parents: 44b9f4e 7f82a57
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 15:04:15 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 15:04:15 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[17/50] [abbrv] incubator-beam git commit: This closes #1157

Posted by dh...@apache.org.
This closes #1157


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

Branch: refs/heads/gearpump-runner
Commit: c390a2a7ff1b3a58de213f85218eef689e712df4
Parents: 8dfadbf 2089c5c
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 09:10:04 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Oct 24 09:10:04 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/AggregatorFactory.java    |  39 ++++
 .../beam/runners/core/SimpleDoFnRunner.java     |   8 +-
 .../runners/direct/DoFnLifecycleManager.java    |  42 ++--
 .../beam/runners/direct/ParDoEvaluator.java     |   3 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |   6 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   5 +-
 ...leManagerRemovingTransformEvaluatorTest.java |  16 +-
 .../direct/DoFnLifecycleManagerTest.java        |  12 +-
 .../direct/DoFnLifecycleManagersTest.java       |  48 ++++-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  11 +
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  11 +
 .../beam/runners/direct/SplittableDoFnTest.java |   8 +-
 .../functions/FlinkProcessContext.java          |  10 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |  11 +-
 .../sdk/transforms/DelegatingAggregator.java    | 125 +++++++++++
 .../org/apache/beam/sdk/transforms/DoFn.java    |  55 +++--
 .../beam/sdk/transforms/DoFnAdapters.java       |   1 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  18 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 214 +++++++++----------
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |  16 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   2 +-
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  16 +-
 23 files changed, 463 insertions(+), 219 deletions(-)
----------------------------------------------------------------------



[10/50] [abbrv] incubator-beam git commit: Closes #1161

Posted by dh...@apache.org.
Closes #1161


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

Branch: refs/heads/gearpump-runner
Commit: 6d9d8bc6fc2795fa25a9120324e652b9bec20ca8
Parents: a9a41eb e43228c
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 07:07:44 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:07:44 2016 -0700

----------------------------------------------------------------------
 .../streaming/StreamingEvaluationContext.java            | 11 ++++++++---
 .../translation/streaming/EmptyStreamAssertionTest.java  | 10 +++++++---
 .../streaming/ResumeFromCheckpointStreamingTest.java     |  2 +-
 .../streaming/SimpleStreamingWordCountTest.java          |  1 -
 .../translation/streaming/utils/PAssertStreaming.java    |  6 +++---
 5 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[38/50] [abbrv] incubator-beam git commit: Add experimental ServiceAccount option

Posted by dh...@apache.org.
Add experimental ServiceAccount option


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

Branch: refs/heads/gearpump-runner
Commit: 60907b2128acdae28ecd5fe80d3b00a406b58892
Parents: 0ec956d
Author: sammcveety <sa...@gmail.com>
Authored: Sat Oct 22 10:27:53 2016 -0400
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Oct 25 13:05:16 2016 -0700

----------------------------------------------------------------------
 pom.xml                                                 |  2 +-
 .../runners/dataflow/DataflowPipelineTranslator.java    |  4 ++++
 .../dataflow/options/DataflowPipelineOptions.java       | 12 ++++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index df388d4..307e43e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,7 +106,7 @@
     <cloudresourcemanager.version>v1-rev6-1.22.0</cloudresourcemanager.version>
     <pubsubgrpc.version>0.1.0</pubsubgrpc.version>
     <clouddebugger.version>v2-rev8-1.22.0</clouddebugger.version>
-    <dataflow.version>v1b3-rev36-1.22.0</dataflow.version>
+    <dataflow.version>v1b3-rev42-1.22.0</dataflow.version>
     <dataflow.proto.version>0.5.160222</dataflow.proto.version>
     <datastore.client.version>1.2.0</datastore.client.version>
     <datastore.proto.version>1.2.0</datastore.proto.version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index ad49af7..12aa696 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -479,6 +479,10 @@ public class DataflowPipelineTranslator {
       workerPools.add(workerPool);
       environment.setWorkerPools(workerPools);
 
+      if (options.getServiceAccount() != null) {
+        environment.setServiceAccountEmail(options.getServiceAccount());
+      }
+
       pipeline.traverseTopologically(this);
       return job;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index ec4fec6..e853f22 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -22,6 +22,7 @@ import static com.google.common.base.Strings.isNullOrEmpty;
 
 import java.io.IOException;
 import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.BigQueryOptions;
 import org.apache.beam.sdk.options.Default;
@@ -29,6 +30,7 @@ import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.Hidden;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PubsubOptions;
 import org.apache.beam.sdk.options.StreamingOptions;
@@ -80,6 +82,16 @@ public interface DataflowPipelineOptions
   void setUpdate(boolean value);
 
   /**
+   * Run the job as a specific service account, instead of the default GCE robot.
+   */
+  @Hidden
+  @Experimental
+  @Description(
+      "Run the job as a specific service account, instead of the default GCE robot.")
+  String getServiceAccount();
+  void setServiceAccount(String value);
+
+  /**
    * Returns a default staging location under {@link GcpOptions#getGcpTempLocation}.
    */
   class StagingLocationFactory implements DefaultValueFactory<String> {


[13/50] [abbrv] incubator-beam git commit: Add property name to RuntimeValueProvider

Posted by dh...@apache.org.
Add property name to RuntimeValueProvider


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

Branch: refs/heads/gearpump-runner
Commit: 462a6417e925e2defb42d87191fb97353637ce7e
Parents: 43beb1e
Author: sammcveety <sa...@gmail.com>
Authored: Fri Oct 14 13:41:33 2016 -0400
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 08:15:05 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ProxyInvocationHandler.java   |  7 +++++--
 .../org/apache/beam/sdk/options/ValueProvider.java | 17 ++++++++++++++---
 .../apache/beam/sdk/options/ValueProviderTest.java |  7 +++++++
 3 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index 3e74916..f7c6da2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -493,12 +493,15 @@ class ProxyInvocationHandler implements InvocationHandler {
       }
     }
     if (method.getReturnType().equals(ValueProvider.class)) {
+      String propertyName = gettersToPropertyNames.get(method.getName());
       return defaultObject == null
         ? new RuntimeValueProvider(
-          method.getName(), (Class<? extends PipelineOptions>) method.getDeclaringClass(),
+          method.getName(), propertyName,
+          (Class<? extends PipelineOptions>) method.getDeclaringClass(),
           proxy.getOptionsId())
         : new RuntimeValueProvider(
-          method.getName(), (Class<? extends PipelineOptions>) method.getDeclaringClass(),
+          method.getName(), propertyName,
+          (Class<? extends PipelineOptions>) method.getDeclaringClass(),
           defaultObject, proxy.getOptionsId());
     } else if (defaultObject != null) {
       return defaultObject;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
index 04b6926..3f35a18 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
@@ -108,6 +108,7 @@ public interface ValueProvider<T> {
 
     private final Class<? extends PipelineOptions> klass;
     private final String methodName;
+    private final String propertyName;
     @Nullable
     private final T defaultValue;
     private final Long optionsId;
@@ -116,9 +117,10 @@ public interface ValueProvider<T> {
      * Creates a {@link RuntimeValueProvider} that will query the provided
      * {@code optionsId} for a value.
      */
-    RuntimeValueProvider(String methodName, Class<? extends PipelineOptions> klass,
-                         Long optionsId) {
+    RuntimeValueProvider(String methodName, String propertyName,
+                         Class<? extends PipelineOptions> klass, Long optionsId) {
       this.methodName = methodName;
+      this.propertyName = propertyName;
       this.klass = klass;
       this.defaultValue = null;
       this.optionsId = optionsId;
@@ -128,9 +130,11 @@ public interface ValueProvider<T> {
      * Creates a {@link RuntimeValueProvider} that will query the provided
      * {@code optionsId} for a value, or use the default if no value is available.
      */
-    RuntimeValueProvider(String methodName, Class<? extends PipelineOptions> klass,
+    RuntimeValueProvider(String methodName, String propertyName,
+                         Class<? extends PipelineOptions> klass,
       T defaultValue, Long optionsId) {
       this.methodName = methodName;
+      this.propertyName = propertyName;
       this.klass = klass;
       this.defaultValue = defaultValue;
       this.optionsId = optionsId;
@@ -167,6 +171,13 @@ public interface ValueProvider<T> {
       PipelineOptions options = optionsMap.get(optionsId);
       return options != null;
     }
+
+    /**
+     * Returns the property name that corresponds to this provider.
+     */
+    public String propertyName() {
+      return propertyName;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
index 0d93b31..c9eb479 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
@@ -98,6 +98,13 @@ public class ValueProviderTest {
   }
 
   @Test
+  public void testRuntimePropertyName() {
+    TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
+    ValueProvider<String> provider = options.getFoo();
+    assertEquals("foo", ((RuntimeValueProvider) provider).propertyName());
+  }
+
+  @Test
   public void testDefaultRuntimeProvider() {
     TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
     ValueProvider<String> provider = options.getBar();


[02/50] [abbrv] incubator-beam git commit: Fix binding of aggregator creating in OldDoFn

Posted by dh...@apache.org.
Fix binding of aggregator creating in OldDoFn


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

Branch: refs/heads/gearpump-runner
Commit: 4e185d0b0a7ec4c096380a25b9cbe4703621ec6b
Parents: 3094017
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 12:44:30 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4e185d0b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index b269f47..a9f26a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -511,6 +511,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     public AdaptedContext(
         DoFn<InputT, OutputT>.Context newContext) {
       this.newContext = newContext;
+      super.setupDelegateAggregators();
     }
 
     @Override
@@ -541,7 +542,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return null;
+      return newContext.createAggregator(name, combiner);
     }
   }
 
@@ -625,7 +626,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return null;
+      return newContext.createAggregator(name, combiner);
     }
   }
 


[14/50] [abbrv] incubator-beam git commit: Closes #1107

Posted by dh...@apache.org.
Closes #1107


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

Branch: refs/heads/gearpump-runner
Commit: 69ae8bd7656437497c2371deaaa26793f17989e2
Parents: 43beb1e 462a641
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 08:15:06 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 08:15:06 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ProxyInvocationHandler.java   |  7 +++++--
 .../org/apache/beam/sdk/options/ValueProvider.java | 17 ++++++++++++++---
 .../apache/beam/sdk/options/ValueProviderTest.java |  7 +++++++
 3 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[43/50] [abbrv] incubator-beam git commit: Remove OldDoFn to DoFn in DirectRunner

Posted by dh...@apache.org.
Remove OldDoFn to DoFn in DirectRunner


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

Branch: refs/heads/gearpump-runner
Commit: 287201916776cb51b98b9ddd27c169f87bb89e1a
Parents: 3d08685
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 11:18:10 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/direct/WriteWithShardingFactory.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28720191/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
index d74cd56..8727cb5 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
@@ -25,9 +25,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.apache.beam.sdk.io.Write;
 import org.apache.beam.sdk.io.Write.Bound;
 import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Values;
@@ -102,7 +102,7 @@ class WriteWithShardingFactory implements PTransformOverrideFactory {
   }
 
   @VisibleForTesting
-  static class KeyBasedOnCountFn<T> extends OldDoFn<T, KV<Integer, T>> {
+  static class KeyBasedOnCountFn<T> extends DoFn<T, KV<Integer, T>> {
     @VisibleForTesting
     static final int MIN_SHARDS_FOR_LOG = 3;
 
@@ -116,7 +116,7 @@ class WriteWithShardingFactory implements PTransformOverrideFactory {
       this.randomExtraShards = extraShards;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
       if (maxShards == 0) {
         maxShards = calculateShards(c.sideInput(numRecords));


[39/50] [abbrv] incubator-beam git commit: This closes #1163

Posted by dh...@apache.org.
This closes #1163


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

Branch: refs/heads/gearpump-runner
Commit: 7abdcbfb2fe1d1308f504dd17e46f1a4fc576407
Parents: 0ec956d 60907b2
Author: Davor Bonaci <da...@google.com>
Authored: Tue Oct 25 13:05:27 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Oct 25 13:05:27 2016 -0700

----------------------------------------------------------------------
 pom.xml                                                 |  2 +-
 .../runners/dataflow/DataflowPipelineTranslator.java    |  4 ++++
 .../dataflow/options/DataflowPipelineOptions.java       | 12 ++++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[44/50] [abbrv] incubator-beam git commit: Display data: minor cleanups

Posted by dh...@apache.org.
Display data: minor cleanups

* wrong @Nullable
* Remove unused function


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

Branch: refs/heads/gearpump-runner
Commit: d2553500b7ebff4025b2c1588d9b73bab8213490
Parents: 13af296
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 19:39:27 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Oct 25 13:49:55 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/display/DisplayData.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d2553500/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
index 8e4cf91..46cf94a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
@@ -36,7 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import org.apache.avro.reflect.Nullable;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -147,10 +147,6 @@ public class DisplayData implements Serializable {
     return builder.toString();
   }
 
-  private static String namespaceOf(Class<?> clazz) {
-    return clazz.getName();
-  }
-
   /**
    * Utility to build up display data from a component and its included
    * subcomponents.


[46/50] [abbrv] incubator-beam git commit: Merge branch 'master' into gearpump-runner

Posted by dh...@apache.org.
Merge branch 'master' into gearpump-runner


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

Branch: refs/heads/gearpump-runner
Commit: 9dc9be9e6d0aa202e170d64e1a1d7b5a57828c5a
Parents: 8f4334c f2fe1ae
Author: manuzhang <ow...@gmail.com>
Authored: Wed Oct 26 11:15:24 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Wed Oct 26 11:15:24 2016 +0800

----------------------------------------------------------------------
 .gitignore                                      |   8 +
 .travis.yml                                     |  10 +-
 examples/java/pom.xml                           | 154 ++-
 .../beam/examples/DebuggingWordCount.java       |   2 +-
 .../apache/beam/examples/MinimalWordCount.java  |   3 +-
 .../apache/beam/examples/WindowedWordCount.java |   2 +-
 .../org/apache/beam/examples/WordCount.java     |   4 +-
 .../common/ExampleBigQueryTableOptions.java     |   2 +-
 ...xamplePubsubTopicAndSubscriptionOptions.java |   2 +-
 .../common/ExamplePubsubTopicOptions.java       |   2 +-
 .../beam/examples/common/ExampleUtils.java      |   2 +-
 .../examples/common/PubsubFileInjector.java     | 153 ---
 .../beam/examples/complete/AutoComplete.java    |   2 +-
 .../apache/beam/examples/complete/TfIdf.java    |   2 +-
 .../examples/complete/TopWikipediaSessions.java |   2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../examples/cookbook/DatastoreWordCount.java   | 261 -----
 .../beam/examples/cookbook/DeDupExample.java    |   4 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../beam/examples/cookbook/JoinExamples.java    |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../beam/examples/cookbook/TriggerExample.java  |  28 +-
 .../beam/examples/WindowedWordCountIT.java      |  75 ++
 .../org/apache/beam/examples/WordCountIT.java   |   6 -
 .../examples/cookbook/BigQueryTornadoesIT.java  |  14 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |  10 +-
 .../examples/complete/game/HourlyTeamScore.java |   8 +-
 .../examples/complete/game/LeaderBoard.java     |  12 +-
 .../beam/examples/complete/game/UserScore.java  |  10 +-
 .../complete/game/injector/Injector.java        |  10 +-
 .../examples/complete/game/LeaderBoardTest.java |   5 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         | 134 ++-
 runners/core-java/pom.xml                       |   8 +-
 .../beam/runners/core/AggregatorFactory.java    |  39 +
 .../beam/runners/core/BatchTimerInternals.java  | 140 ---
 .../apache/beam/runners/core/DoFnRunner.java    |   8 +-
 .../beam/runners/core/DoFnRunnerBase.java       | 559 -----------
 .../apache/beam/runners/core/DoFnRunners.java   | 191 +++-
 .../runners/core/ElementAndRestriction.java     |  42 +
 .../core/ElementAndRestrictionCoder.java        |  67 ++
 .../runners/core/ElementByteSizeObservable.java |   5 +-
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 +
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   5 +
 .../runners/core/GroupAlsoByWindowsDoFn.java    |  19 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  12 +-
 .../runners/core/ReduceFnContextFactory.java    |   2 +-
 .../beam/runners/core/ReduceFnRunner.java       |  42 +-
 .../beam/runners/core/SideInputHandler.java     |   2 +-
 .../beam/runners/core/SimpleDoFnRunner.java     |  58 --
 .../beam/runners/core/SimpleOldDoFnRunner.java  | 521 ++++++++++
 .../beam/runners/core/SplittableParDo.java      | 469 +++++++++
 .../apache/beam/runners/core/TriggerRunner.java | 247 -----
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../core/triggers/AfterAllStateMachine.java     | 109 ++
 .../AfterDelayFromFirstElementStateMachine.java | 337 +++++++
 .../core/triggers/AfterEachStateMachine.java    | 130 +++
 .../core/triggers/AfterFirstStateMachine.java   | 112 +++
 .../core/triggers/AfterPaneStateMachine.java    | 139 +++
 .../AfterProcessingTimeStateMachine.java        |  93 ++
 ...rSynchronizedProcessingTimeStateMachine.java |  63 ++
 .../triggers/AfterWatermarkStateMachine.java    | 325 ++++++
 .../triggers/DefaultTriggerStateMachine.java    |  81 ++
 .../triggers/ExecutableTriggerStateMachine.java | 160 +++
 .../runners/core/triggers/FinishedTriggers.java |  44 +
 .../core/triggers/FinishedTriggersBitSet.java   |  67 ++
 .../core/triggers/FinishedTriggersSet.java      |  72 ++
 .../core/triggers/NeverStateMachine.java        |  60 ++
 .../core/triggers/OrFinallyStateMachine.java    |  85 ++
 .../core/triggers/RepeatedlyStateMachine.java   |  88 ++
 .../triggers/ReshuffleTriggerStateMachine.java  |  50 +
 .../core/triggers/TriggerStateMachine.java      | 487 +++++++++
 .../TriggerStateMachineContextFactory.java      | 509 ++++++++++
 .../triggers/TriggerStateMachineRunner.java     | 234 +++++
 .../core/triggers/TriggerStateMachines.java     | 215 ++++
 .../runners/core/triggers/package-info.java     |  23 +
 .../runners/core/BatchTimerInternalsTest.java   | 118 ---
 .../core/ElementAndRestrictionCoderTest.java    | 127 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   | 281 +++---
 .../beam/runners/core/ReduceFnTester.java       | 405 +++-----
 .../beam/runners/core/SimpleDoFnRunnerTest.java |  88 --
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  88 ++
 .../beam/runners/core/SplittableParDoTest.java  | 467 +++++++++
 .../core/triggers/AfterAllStateMachineTest.java | 140 +++
 .../triggers/AfterEachStateMachineTest.java     | 108 ++
 .../triggers/AfterFirstStateMachineTest.java    | 159 +++
 .../triggers/AfterPaneStateMachineTest.java     | 117 +++
 .../AfterProcessingTimeStateMachineTest.java    | 172 ++++
 ...chronizedProcessingTimeStateMachineTest.java | 110 ++
 .../AfterWatermarkStateMachineTest.java         | 382 +++++++
 .../DefaultTriggerStateMachineTest.java         | 165 +++
 .../ExecutableTriggerStateMachineTest.java      | 108 ++
 .../triggers/FinishedTriggersBitSetTest.java    |  55 +
 .../triggers/FinishedTriggersProperties.java    | 115 +++
 .../core/triggers/FinishedTriggersSetTest.java  |  60 ++
 .../core/triggers/NeverStateMachineTest.java    |  59 ++
 .../triggers/OrFinallyStateMachineTest.java     | 177 ++++
 .../triggers/RepeatedlyStateMachineTest.java    | 200 ++++
 .../ReshuffleTriggerStateMachineTest.java       |  68 ++
 .../core/triggers/StubTriggerStateMachine.java  |  60 ++
 .../core/triggers/TriggerStateMachineTest.java  |  98 ++
 .../triggers/TriggerStateMachineTester.java     | 431 ++++++++
 .../core/triggers/TriggerStateMachinesTest.java | 199 ++++
 runners/direct-java/pom.xml                     |  26 +-
 .../runners/direct/AggregatorContainer.java     |   9 +-
 .../direct/BoundedReadEvaluatorFactory.java     | 155 +--
 .../beam/runners/direct/BundleFactory.java      |  15 +-
 .../runners/direct/CloningBundleFactory.java    |  98 ++
 .../beam/runners/direct/CompletionCallback.java |   4 +-
 .../runners/direct/DirectExecutionContext.java  |   2 +-
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  66 ++
 .../beam/runners/direct/DirectMetrics.java      | 338 +++++++
 .../beam/runners/direct/DirectOptions.java      |  40 +-
 .../beam/runners/direct/DirectRunner.java       | 121 ++-
 .../runners/direct/DirectTimerInternals.java    |   2 +-
 .../runners/direct/DoFnLifecycleManager.java    |  86 +-
 .../beam/runners/direct/EmptyInputProvider.java |  45 +
 .../direct/EncodabilityEnforcementFactory.java  |  50 +-
 .../beam/runners/direct/EvaluationContext.java  |  27 +-
 .../direct/ExecutorServiceParallelExecutor.java | 124 ++-
 .../runners/direct/FlattenEvaluatorFactory.java |  26 +-
 .../runners/direct/ForwardingPTransform.java    |   2 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  14 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |  21 +-
 .../ImmutabilityCheckingBundleFactory.java      |  18 +-
 .../direct/ImmutableListBundleFactory.java      |  76 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  13 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |  29 +-
 .../runners/direct/ParDoOverrideFactory.java    |  55 +
 .../direct/ParDoSingleEvaluatorFactory.java     |  23 +-
 .../beam/runners/direct/PipelineExecutor.java   |   2 +-
 .../beam/runners/direct/RootInputProvider.java  |  46 +
 .../runners/direct/RootProviderRegistry.java    |  66 ++
 .../runners/direct/StepTransformResult.java     |  49 +-
 .../beam/runners/direct/StructuralKey.java      |  88 +-
 .../direct/TestStreamEvaluatorFactory.java      | 152 +--
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorFactory.java       |  22 +-
 .../direct/TransformEvaluatorRegistry.java      |  63 +-
 .../beam/runners/direct/TransformExecutor.java  |  53 +-
 .../beam/runners/direct/TransformResult.java    |  16 +-
 .../direct/UnboundedReadEvaluatorFactory.java   | 318 +++---
 .../direct/UncommittedBundleOutputManager.java  |   4 +-
 .../runners/direct/ViewEvaluatorFactory.java    |  16 +-
 .../beam/runners/direct/WatermarkManager.java   |  53 +-
 .../runners/direct/WindowEvaluatorFactory.java  |  18 +-
 .../direct/WriteWithShardingFactory.java        |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java | 181 ++--
 .../direct/CloningBundleFactoryTest.java        | 177 ++++
 .../runners/direct/CommittedResultTest.java     |  10 +-
 .../ConsumerTrackingPipelineVisitorTest.java    |  32 +-
 .../beam/runners/direct/DirectMetricsTest.java  | 133 +++
 .../beam/runners/direct/DirectRunnerTest.java   | 180 +++-
 ...leManagerRemovingTransformEvaluatorTest.java |  16 +-
 .../direct/DoFnLifecycleManagerTest.java        |  86 +-
 .../direct/DoFnLifecycleManagersTest.java       |  48 +-
 .../EncodabilityEnforcementFactoryTest.java     | 132 ++-
 .../runners/direct/EvaluationContextTest.java   |  31 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |  36 +-
 .../direct/ForwardingPTransformTest.java        |   7 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  27 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  35 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |  67 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  14 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  52 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  15 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 162 +--
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 145 ++-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 +++++
 .../runners/direct/StepTransformResultTest.java |   4 +-
 .../beam/runners/direct/StructuralKeyTest.java  |   9 +
 .../direct/TestStreamEvaluatorFactoryTest.java  | 223 ++---
 .../runners/direct/TransformExecutorTest.java   | 115 +--
 .../UnboundedReadEvaluatorFactoryTest.java      | 321 ++++--
 .../direct/ViewEvaluatorFactoryTest.java        |   6 +-
 .../runners/direct/WatermarkManagerTest.java    |  84 +-
 .../direct/WindowEvaluatorFactoryTest.java      |  12 +-
 runners/flink/README.md                         |  10 +-
 runners/flink/examples/pom.xml                  |   2 +-
 .../beam/runners/flink/examples/TFIDF.java      |  12 +-
 .../beam/runners/flink/examples/WordCount.java  |   4 +-
 .../flink/examples/streaming/AutoComplete.java  |   4 +-
 .../flink/examples/streaming/package-info.java  |  22 -
 runners/flink/pom.xml                           |   4 +-
 runners/flink/runner/pom.xml                    |  10 +-
 .../flink/FlinkDetachedRunnerResult.java        |  76 ++
 .../FlinkPipelineExecutionEnvironment.java      |   7 +
 .../runners/flink/FlinkPipelineOptions.java     |  19 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  29 +-
 .../runners/flink/FlinkRunnerRegistrar.java     |   4 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |  17 +-
 .../beam/runners/flink/TestFlinkRunner.java     |   9 +-
 .../FlinkBatchPipelineTranslator.java           |   2 +-
 .../FlinkStreamingPipelineTranslator.java       |   2 +-
 .../functions/FlinkDoFnFunction.java            |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   2 +-
 .../functions/FlinkProcessContext.java          |  10 +-
 .../wrappers/streaming/DoFnOperator.java        |  31 +-
 .../wrappers/streaming/FlinkStateInternals.java |   8 +-
 .../wrappers/streaming/WindowDoFnOperator.java  | 177 +++-
 .../streaming/io/BoundedSourceWrapper.java      |   9 +-
 .../streaming/io/UnboundedSourceWrapper.java    | 187 +++-
 .../beam/runners/flink/FlinkTestPipeline.java   |   6 +-
 .../flink/streaming/DoFnOperatorTest.java       |   9 +-
 .../streaming/UnboundedSourceWrapperTest.java   | 123 +--
 .../translators/TransformTranslator.java        |  30 -
 .../translators/io/UnboundedSourceWrapper.java  |  45 -
 runners/google-cloud-dataflow-java/pom.xml      |  22 +-
 .../runners/dataflow/DataflowPipelineJob.java   |  52 +-
 .../dataflow/DataflowPipelineTranslator.java    | 137 ++-
 .../beam/runners/dataflow/DataflowRunner.java   |  57 +-
 .../dataflow/internal/AssignWindows.java        |   6 +-
 .../DataflowUnboundedReadFromBoundedSource.java |  18 +-
 .../runners/dataflow/internal/IsmFormat.java    |  13 +-
 .../dataflow/internal/ReadTranslator.java       |   2 +-
 .../options/DataflowPipelineDebugOptions.java   |   7 +-
 .../options/DataflowPipelineOptions.java        |  14 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  11 +-
 .../options/DataflowProfilingOptions.java       |   2 +-
 .../options/DataflowWorkerLoggingOptions.java   |   4 +-
 .../dataflow/testing/TestDataflowRunner.java    | 144 ++-
 .../beam/runners/dataflow/util/DoFnInfo.java    |  29 +-
 .../runners/dataflow/util/MonitoringUtil.java   |   2 +-
 .../runners/dataflow/util/RandomAccessData.java |   8 +-
 .../beam/runners/dataflow/util/Stager.java      |   2 +-
 .../dataflow/DataflowPipelineJobTest.java       | 120 +++
 .../DataflowPipelineTranslatorTest.java         |  40 +-
 .../runners/dataflow/DataflowRunnerTest.java    |  13 +-
 ...aflowUnboundedReadFromBoundedSourceTest.java |  83 ++
 .../testing/TestDataflowRunnerTest.java         | 287 +++++-
 .../dataflow/util/MonitoringUtilTest.java       |   4 +-
 runners/pom.xml                                 |   3 +-
 runners/spark/pom.xml                           |  94 +-
 .../beam/runners/spark/EvaluationResult.java    |   4 +-
 .../runners/spark/SparkPipelineOptions.java     |  48 +-
 .../apache/beam/runners/spark/SparkRunner.java  | 176 ++--
 .../beam/runners/spark/TestSparkRunner.java     |  17 +-
 .../spark/aggregators/AccumulatorSingleton.java |  53 +
 .../runners/spark/coders/WritableCoder.java     |   2 +-
 .../beam/runners/spark/examples/WordCount.java  |   2 +-
 .../apache/beam/runners/spark/io/SourceRDD.java | 200 ++++
 .../spark/io/hadoop/ShardNameTemplateAware.java |   2 +-
 .../runners/spark/translation/DoFnFunction.java |  73 +-
 .../spark/translation/EvaluationContext.java    |  32 +-
 .../translation/GroupCombineFunctions.java      | 313 ++++++
 .../spark/translation/MultiDoFnFunction.java    |  77 +-
 .../translation/SparkAbstractCombineFn.java     | 134 +++
 .../spark/translation/SparkContextFactory.java  |  50 +-
 .../spark/translation/SparkGlobalCombineFn.java | 260 +++++
 .../spark/translation/SparkKeyedCombineFn.java  | 273 +++++
 .../translation/SparkPipelineEvaluator.java     |  57 --
 .../translation/SparkPipelineTranslator.java    |   5 +-
 .../spark/translation/SparkProcessContext.java  | 168 +++-
 .../spark/translation/SparkRuntimeContext.java  |  44 +-
 .../spark/translation/TransformTranslator.java  | 592 +++--------
 .../spark/translation/TranslationUtils.java     | 197 ++++
 .../SparkRunnerStreamingContextFactory.java     | 106 ++
 .../streaming/StreamingEvaluationContext.java   |  81 +-
 .../streaming/StreamingTransformTranslator.java | 561 +++++++----
 .../spark/util/SparkSideInputReader.java        |  95 ++
 .../runners/spark/ClearAggregatorsRule.java     |  33 +
 .../apache/beam/runners/spark/DeDupTest.java    |  59 --
 .../beam/runners/spark/EmptyInputTest.java      |  75 --
 .../beam/runners/spark/SimpleWordCountTest.java | 105 --
 .../apache/beam/runners/spark/TfIdfTest.java    | 260 -----
 .../beam/runners/spark/io/AvroPipelineTest.java |   4 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   4 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   4 +-
 .../spark/translation/CombineGloballyTest.java  | 102 --
 .../spark/translation/CombinePerKeyTest.java    |  79 --
 .../spark/translation/DoFnOutputTest.java       |  67 --
 .../translation/MultiOutputWordCountTest.java   | 176 ----
 .../spark/translation/SerializationTest.java    | 201 ----
 .../spark/translation/SideEffectsTest.java      |  21 +-
 .../translation/SparkPipelineOptionsTest.java   |  42 -
 .../translation/TransformTranslatorTest.java    | 104 --
 .../translation/WindowedWordCountTest.java      | 120 ---
 .../streaming/EmptyStreamAssertionTest.java     |  80 ++
 .../streaming/FlattenStreamingTest.java         |  57 +-
 .../streaming/KafkaStreamingTest.java           |  32 +-
 .../ResumeFromCheckpointStreamingTest.java      | 182 ++++
 .../streaming/SimpleStreamingWordCountTest.java |  67 +-
 .../streaming/utils/PAssertStreaming.java       |  87 +-
 .../utils/TestOptionsForStreaming.java          |  55 +
 .../spark/src/test/resources/metrics.properties |  61 +-
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/checkstyle.xml      |  28 +-
 .../src/main/resources/beam/findbugs-filter.xml |   2 +-
 .../src/main/resources/beam/suppressions.xml    |  11 +-
 sdks/java/core/pom.xml                          |  43 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  12 +-
 .../org/apache/beam/sdk/PipelineResult.java     |  15 +-
 .../beam/sdk/annotations/Experimental.java      |  15 +-
 .../sdk/coders/CannotProvideCoderException.java |   2 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |  28 +-
 .../apache/beam/sdk/coders/CoderFactory.java    |   4 +-
 .../apache/beam/sdk/coders/CoderProvider.java   |   2 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   4 +-
 .../beam/sdk/coders/IterableLikeCoder.java      |  22 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |  10 +-
 .../apache/beam/sdk/coders/package-info.java    |   2 +-
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |   4 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     | 162 ++-
 .../java/org/apache/beam/sdk/io/AvroSource.java |  98 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  11 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |  19 +-
 .../apache/beam/sdk/io/CompressedSource.java    |  18 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   | 145 ++-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   6 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |  24 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |   2 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../sdk/io/SerializableAvroCodecFactory.java    | 112 +++
 .../main/java/org/apache/beam/sdk/io/Sink.java  |   5 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     | 247 +++--
 .../main/java/org/apache/beam/sdk/io/Write.java |   8 +-
 .../java/org/apache/beam/sdk/io/XmlSource.java  |  13 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   6 +-
 .../apache/beam/sdk/io/range/RangeTracker.java  |   1 +
 .../org/apache/beam/sdk/metrics/Counter.java    |  40 +
 .../apache/beam/sdk/metrics/CounterCell.java    |  76 ++
 .../org/apache/beam/sdk/metrics/DirtyState.java |  98 ++
 .../apache/beam/sdk/metrics/Distribution.java   |  30 +
 .../beam/sdk/metrics/DistributionCell.java      |  58 ++
 .../beam/sdk/metrics/DistributionData.java      |  59 ++
 .../beam/sdk/metrics/DistributionResult.java    |  42 +
 .../org/apache/beam/sdk/metrics/Metric.java     |  24 +
 .../org/apache/beam/sdk/metrics/MetricCell.java |  47 +
 .../org/apache/beam/sdk/metrics/MetricKey.java  |  40 +
 .../org/apache/beam/sdk/metrics/MetricName.java |  46 +
 .../beam/sdk/metrics/MetricNameFilter.java      |  60 ++
 .../beam/sdk/metrics/MetricQueryResults.java    |  33 +
 .../apache/beam/sdk/metrics/MetricResult.java   |  45 +
 .../apache/beam/sdk/metrics/MetricResults.java  |  34 +
 .../apache/beam/sdk/metrics/MetricUpdates.java  |  72 ++
 .../org/apache/beam/sdk/metrics/Metrics.java    | 110 ++
 .../beam/sdk/metrics/MetricsContainer.java      | 150 +++
 .../beam/sdk/metrics/MetricsEnvironment.java    |  85 ++
 .../apache/beam/sdk/metrics/MetricsFilter.java  |  86 ++
 .../org/apache/beam/sdk/metrics/MetricsMap.java |  86 ++
 .../apache/beam/sdk/metrics/package-info.java   |  28 +
 .../options/CloudResourceManagerOptions.java    |  40 +
 .../org/apache/beam/sdk/options/GcpOptions.java |   8 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |   4 +-
 .../beam/sdk/options/GoogleApiDebugOptions.java |   2 +-
 .../beam/sdk/options/PipelineOptions.java       |  52 +-
 .../sdk/options/PipelineOptionsFactory.java     | 269 +++--
 .../sdk/options/ProxyInvocationHandler.java     | 264 +++--
 .../apache/beam/sdk/options/ValueProvider.java  | 239 +++++
 .../java/org/apache/beam/sdk/package-info.java  |   2 +-
 .../sdk/runners/PipelineRunnerRegistrar.java    |   2 +-
 .../apache/beam/sdk/runners/package-info.java   |  17 +-
 .../beam/sdk/testing/BigqueryMatcher.java       | 239 +++++
 .../org/apache/beam/sdk/testing/PAssert.java    |   4 +-
 .../beam/sdk/testing/SerializableMatchers.java  |   4 +-
 .../beam/sdk/testing/SourceTestUtils.java       |   3 +-
 .../apache/beam/sdk/testing/StreamingIT.java    |  35 +
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/package-info.java   |   4 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |  11 +-
 .../sdk/transforms/ApproximateQuantiles.java    |   5 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  95 +-
 .../beam/sdk/transforms/CombineFnBase.java      |  17 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |  41 +-
 .../beam/sdk/transforms/CombineWithContext.java |   7 +-
 .../sdk/transforms/DelegatingAggregator.java    | 125 +++
 .../org/apache/beam/sdk/transforms/DoFn.java    | 445 ++++++++-
 .../beam/sdk/transforms/DoFnAdapters.java       | 184 +++-
 .../apache/beam/sdk/transforms/DoFnTester.java  | 257 +++--
 .../beam/sdk/transforms/FlatMapElements.java    |   6 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   4 +-
 .../transforms/IntraBundleParallelization.java  | 361 -------
 .../org/apache/beam/sdk/transforms/Latest.java  |  12 +-
 .../apache/beam/sdk/transforms/MapElements.java |   8 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 318 ++++--
 .../apache/beam/sdk/transforms/PTransform.java  |  10 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 341 +++++--
 .../apache/beam/sdk/transforms/Partition.java   |   4 +-
 .../beam/sdk/transforms/RemoveDuplicates.java   |   5 +-
 .../sdk/transforms/SerializableFunction.java    |   2 +-
 .../org/apache/beam/sdk/transforms/ViewFn.java  |   2 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../sdk/transforms/display/DisplayData.java     | 530 ++++++----
 .../sdk/transforms/reflect/DoFnInvoker.java     |  48 +-
 .../sdk/transforms/reflect/DoFnInvokers.java    | 658 ++++++++----
 .../sdk/transforms/reflect/DoFnSignature.java   | 466 ++++++++-
 .../sdk/transforms/reflect/DoFnSignatures.java  | 946 +++++++++++++++---
 .../splittabledofn/RestrictionTracker.java      |  42 +
 .../transforms/splittabledofn/package-info.java |  22 +
 .../beam/sdk/transforms/windowing/AfterAll.java |  51 +-
 .../windowing/AfterDelayFromFirstElement.java   | 110 +-
 .../sdk/transforms/windowing/AfterEach.java     |  63 +-
 .../sdk/transforms/windowing/AfterFirst.java    |  52 +-
 .../sdk/transforms/windowing/AfterPane.java     |  59 +-
 .../windowing/AfterProcessingTime.java          |   7 -
 .../AfterSynchronizedProcessingTime.java        |  13 +-
 .../transforms/windowing/AfterWatermark.java    | 174 +---
 .../transforms/windowing/DefaultTrigger.java    |  37 +-
 .../beam/sdk/transforms/windowing/Never.java    |  28 +-
 .../transforms/windowing/OrFinallyTrigger.java  |  55 +-
 .../beam/sdk/transforms/windowing/PaneInfo.java |   6 +-
 .../sdk/transforms/windowing/Repeatedly.java    |  36 +-
 .../transforms/windowing/SlidingWindows.java    |   3 +-
 .../beam/sdk/transforms/windowing/Trigger.java  | 421 ++------
 .../beam/sdk/transforms/windowing/Window.java   |  20 +-
 .../beam/sdk/transforms/windowing/WindowFn.java |   2 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java | 173 ++++
 .../util/AttemptBoundedExponentialBackOff.java  |  86 ++
 .../beam/sdk/util/BaseExecutionContext.java     |   4 +-
 .../apache/beam/sdk/util/CredentialFactory.java |   2 +-
 .../apache/beam/sdk/util/ExecutableTrigger.java |  40 +-
 .../sdk/util/ExposedByteArrayOutputStream.java  |   1 +
 .../apache/beam/sdk/util/FinishedTriggers.java  |  44 -
 .../beam/sdk/util/FinishedTriggersBitSet.java   |  67 --
 .../beam/sdk/util/FinishedTriggersSet.java      |  72 --
 .../apache/beam/sdk/util/GatherAllPanes.java    |  10 +-
 .../apache/beam/sdk/util/GcpProjectUtil.java    | 106 ++
 .../apache/beam/sdk/util/GcsPathValidator.java  |   2 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  94 +-
 .../apache/beam/sdk/util/IOChannelFactory.java  |   2 +-
 .../beam/sdk/util/MergingActiveWindowSet.java   |  12 +-
 .../apache/beam/sdk/util/PCollectionViews.java  |   9 +-
 .../org/apache/beam/sdk/util/PathValidator.java |   6 +-
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  26 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |   4 +-
 .../org/apache/beam/sdk/util/PropertyNames.java |   1 +
 .../org/apache/beam/sdk/util/PubsubClient.java  |   5 +-
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |  34 +-
 .../apache/beam/sdk/util/PubsubTestClient.java  |   2 +-
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |  16 +-
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   6 +-
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |  16 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   2 +-
 .../java/org/apache/beam/sdk/util/Timer.java    |  56 ++
 .../apache/beam/sdk/util/TimerInternals.java    |   6 +-
 .../org/apache/beam/sdk/util/TimerSpec.java     |  30 +
 .../org/apache/beam/sdk/util/TimerSpecs.java    |  41 +
 .../java/org/apache/beam/sdk/util/Timers.java   |  10 +-
 .../org/apache/beam/sdk/util/Transport.java     |  17 +
 .../beam/sdk/util/TriggerContextFactory.java    | 507 ----------
 .../apache/beam/sdk/util/ValueWithRecordId.java |   8 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |  22 -
 .../apache/beam/sdk/util/common/Reiterable.java |   2 +-
 .../apache/beam/sdk/util/common/Reiterator.java |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java     |   2 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 235 +++++
 .../beam/sdk/util/state/ReadableState.java      |  10 +-
 .../apache/beam/sdk/util/state/StateBinder.java |  67 ++
 .../beam/sdk/util/state/StateContext.java       |   6 +-
 .../apache/beam/sdk/util/state/StateSpec.java   |  39 +
 .../apache/beam/sdk/util/state/StateSpecs.java  | 452 +++++++++
 .../apache/beam/sdk/util/state/StateTag.java    |  82 +-
 .../apache/beam/sdk/util/state/StateTags.java   | 386 +-------
 .../util/state/TestInMemoryStateInternals.java  |  61 ++
 .../beam/sdk/util/state/TimerCallback.java      |  35 +
 .../apache/beam/sdk/values/PCollectionView.java |  15 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   8 +-
 .../org/apache/beam/sdk/values/POutput.java     |   8 +-
 .../java/org/apache/beam/sdk/values/PValue.java |   4 +-
 .../apache/beam/sdk/values/TypeDescriptors.java |  40 +-
 .../apache/beam/sdk/values/package-info.java    |   4 +-
 .../dataflow/util/GcsPathValidatorTest.java     | 103 --
 .../org/apache/beam/sdk/DataflowMatchers.java   |  64 --
 .../java/org/apache/beam/sdk/PipelineTest.java  |   2 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   4 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   1 +
 .../org/apache/beam/sdk/coders/KvCoderTest.java |  99 +-
 .../beam/sdk/coders/NullableCoderTest.java      |  60 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java | 107 +-
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  43 +
 .../io/BoundedReadFromUnboundedSourceTest.java  |  14 +-
 .../beam/sdk/io/CompressedSourceTest.java       |   4 +-
 .../sdk/io/DrunkWritableByteChannelFactory.java |  80 ++
 .../apache/beam/sdk/io/FileBasedSinkTest.java   | 108 ++
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  30 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |  62 +-
 .../io/SerializableAvroCodecFactoryTest.java    | 100 ++
 .../java/org/apache/beam/sdk/io/TextIOTest.java | 601 ++++++-----
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   6 +-
 .../beam/sdk/metrics/CounterCellTest.java       |  55 +
 .../apache/beam/sdk/metrics/DirtyStateTest.java |  56 ++
 .../beam/sdk/metrics/DistributionCellTest.java  |  53 +
 .../apache/beam/sdk/metrics/MetricMatchers.java |  99 ++
 .../beam/sdk/metrics/MetricsContainerTest.java  | 129 +++
 .../sdk/metrics/MetricsEnvironmentTest.java     |  63 ++
 .../apache/beam/sdk/metrics/MetricsMapTest.java | 103 ++
 .../apache/beam/sdk/metrics/MetricsTest.java    |  98 ++
 .../sdk/options/PipelineOptionsFactoryTest.java | 184 +++-
 .../beam/sdk/options/PipelineOptionsTest.java   |  49 +-
 .../options/PipelineOptionsValidatorTest.java   |  18 +-
 .../sdk/options/ProxyInvocationHandlerTest.java | 111 ++-
 .../beam/sdk/options/ValueProviderTest.java     | 220 ++++
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 ++++
 .../sdk/testing/PCollectionViewTesting.java     |   7 -
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |   2 +-
 .../beam/sdk/testing/TestPipelineTest.java      |   4 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   7 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  81 +-
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     | 458 +++++----
 .../apache/beam/sdk/transforms/FlattenTest.java |  20 +
 .../IntraBundleParallelizationTest.java         | 280 ------
 .../beam/sdk/transforms/LatestFnTest.java       | 233 +++++
 .../apache/beam/sdk/transforms/ParDoTest.java   | 101 +-
 .../display/DisplayDataEvaluator.java           |  13 +-
 .../transforms/display/DisplayDataMatchers.java | 141 ++-
 .../display/DisplayDataMatchersTest.java        |  67 +-
 .../sdk/transforms/display/DisplayDataTest.java | 367 ++++---
 .../transforms/reflect/DoFnInvokersTest.java    | 710 +++++++------
 .../reflect/DoFnInvokersTestHelper.java         | 116 ---
 .../DoFnSignaturesProcessElementTest.java       | 213 ++++
 .../DoFnSignaturesSplittableDoFnTest.java       | 543 ++++++++++
 .../transforms/reflect/DoFnSignaturesTest.java  | 991 ++++++++++++++-----
 .../reflect/DoFnSignaturesTestUtils.java        |  67 ++
 .../testhelper/DoFnInvokersTestHelper.java      | 124 +++
 .../sdk/transforms/windowing/AfterAllTest.java  |  98 --
 .../sdk/transforms/windowing/AfterEachTest.java |  64 --
 .../transforms/windowing/AfterFirstTest.java    | 120 ---
 .../sdk/transforms/windowing/AfterPaneTest.java |  77 --
 .../windowing/AfterProcessingTimeTest.java      |  94 --
 .../AfterSynchronizedProcessingTimeTest.java    |  75 --
 .../windowing/AfterWatermarkTest.java           | 308 ------
 .../windowing/DefaultTriggerTest.java           | 130 ---
 .../sdk/transforms/windowing/NeverTest.java     |  34 +-
 .../windowing/OrFinallyTriggerTest.java         | 136 ---
 .../transforms/windowing/RepeatedlyTest.java    | 161 +--
 .../sdk/transforms/windowing/StubTrigger.java   |  17 -
 .../sdk/transforms/windowing/TriggerTest.java   |  28 -
 .../sdk/transforms/windowing/WindowTest.java    |   4 +-
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  28 +-
 ...mptAndTimeBoundedExponentialBackOffTest.java | 213 ++++
 .../AttemptBoundedExponentialBackOffTest.java   |  85 ++
 .../beam/sdk/util/ExecutableTriggerTest.java    |  18 -
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 -
 .../sdk/util/FinishedTriggersProperties.java    | 110 --
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 --
 .../beam/sdk/util/GcpProjectUtilTest.java       |  76 ++
 .../beam/sdk/util/GcsPathValidatorTest.java     | 100 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   | 112 ++-
 .../beam/sdk/util/PubsubGrpcClientTest.java     | 108 +-
 .../beam/sdk/util/PubsubJsonClientTest.java     |  16 +-
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  23 -
 .../org/apache/beam/sdk/util/TriggerTester.java | 592 -----------
 .../util/state/InMemoryTimerInternalsTest.java  | 116 +++
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/io/google-cloud-platform/pom.xml      |  16 +-
 .../sdk/io/gcp/bigquery/BigQueryAvroUtils.java  |  79 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 239 +++--
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |  15 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 106 +-
 .../gcp/bigquery/BigQueryTableRowIterator.java  | 130 ++-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  56 +-
 .../beam/sdk/io/gcp/datastore/DatastoreIO.java  |   2 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  99 +-
 .../io/gcp/bigquery/BigQueryAvroUtilsTest.java  | 149 ++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 570 ++++++++++-
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 149 ++-
 .../bigquery/BigQueryTableRowIteratorTest.java  | 169 +++-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     | 104 +-
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    |   4 +-
 .../beam/sdk/io/gcp/datastore/V1ReadIT.java     |   2 +-
 .../beam/sdk/io/gcp/datastore/V1WriteIT.java    |   2 +-
 sdks/java/io/hdfs/pom.xml                       |   2 +-
 .../beam/sdk/io/hdfs/AvroHDFSFileSource.java    |   2 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |   2 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |  10 +-
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |   2 +-
 .../SimpleAuthAvroHDFSFileSource.java           |   2 +-
 .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java |   2 +-
 .../simpleauth/SimpleAuthHDFSFileSource.java    |   7 +-
 sdks/java/io/jdbc/pom.xml                       | 138 +++
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     | 443 +++++++++
 .../apache/beam/sdk/io/jdbc/package-info.java   |  22 +
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java | 271 +++++
 sdks/java/io/jms/pom.xml                        |   2 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  |  11 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../beam/sdk/io/kafka/KafkaCheckpointMark.java  |   8 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  90 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   | 166 +++-
 sdks/java/io/kinesis/pom.xml                    |   3 +-
 .../beam/sdk/io/kinesis/CustomOptional.java     |   7 +-
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |   2 +-
 .../sdk/io/kinesis/KinesisClientProvider.java   |   2 +-
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  18 +-
 .../beam/sdk/io/kinesis/KinesisReader.java      |  10 +-
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |   4 +-
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |   2 +-
 .../beam/sdk/io/kinesis/KinesisSource.java      |   6 +-
 .../beam/sdk/io/kinesis/RecordFilter.java       |   6 +-
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |   2 +-
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |   6 +-
 .../sdk/io/kinesis/ShardRecordsIterator.java    |   4 +-
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |   8 +-
 .../beam/sdk/io/kinesis/StartingPoint.java      |   2 +-
 .../beam/sdk/io/kinesis/KinesisTestOptions.java |   2 +-
 .../beam/sdk/io/kinesis/KinesisUploader.java    |   5 +-
 sdks/java/io/mongodb/pom.xml                    |  14 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    | 449 +++++++++
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   | 310 +++---
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     | 276 ++++++
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  19 +-
 sdks/java/io/pom.xml                            |   3 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../PipelineOptionsFactoryJava8Test.java        |   8 +-
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../main/resources/archetype-resources/pom.xml  |  26 +-
 .../src/main/java/DebuggingWordCount.java       |  34 +-
 .../src/main/java/MinimalWordCount.java         |  50 +-
 .../src/main/java/WindowedWordCount.java        | 139 +--
 .../src/main/java/WordCount.java                |  79 +-
 .../java/common/DataflowExampleOptions.java     |  32 -
 .../main/java/common/DataflowExampleUtils.java  | 391 --------
 .../common/ExampleBigQueryTableOptions.java     |  11 +-
 .../src/main/java/common/ExampleOptions.java    |  32 +
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 +
 .../java/common/ExamplePubsubTopicOptions.java  |  17 +-
 .../src/main/java/common/ExampleUtils.java      | 353 +++++++
 .../main/java/common/PubsubFileInjector.java    | 153 ---
 .../src/test/java/WordCountTest.java            |   9 +-
 sdks/java/maven-archetypes/pom.xml              |   2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |  10 +-
 .../main/resources/archetype-resources/pom.xml  |   8 +-
 .../resources/projects/basic/reference/pom.xml  |   8 +-
 sdks/java/microbenchmarks/pom.xml               |   2 +-
 .../transforms/DoFnInvokersBenchmark.java       |   7 +
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 632 files changed, 36444 insertions(+), 16763 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --cc runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index df74ed3,dec9905..0000000
deleted file mode 100644,100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ /dev/null
@@@ -1,58 -1,586 +1,0 @@@
--/*
-- * 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.runners.core;
 -
 -import static com.google.common.base.Preconditions.checkNotNull;
--
 -import com.google.common.collect.Iterables;
 -import com.google.common.collect.Sets;
 -import java.io.IOException;
 -import java.util.Collection;
 -import java.util.Iterator;
--import java.util.List;
 -import java.util.Set;
--import org.apache.beam.runners.core.DoFnRunners.OutputManager;
 -import org.apache.beam.sdk.coders.Coder;
 -import org.apache.beam.sdk.coders.IterableCoder;
--import org.apache.beam.sdk.options.PipelineOptions;
 -import org.apache.beam.sdk.transforms.Aggregator;
--import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
- import org.apache.beam.sdk.transforms.OldDoFn;
 -import org.apache.beam.sdk.transforms.Combine.CombineFn;
 -import org.apache.beam.sdk.transforms.DoFn;
 -import org.apache.beam.sdk.transforms.DoFn.InputProvider;
 -import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
 -import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
 -import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 -import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
 -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 -import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 -import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 -import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 -import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 -import org.apache.beam.sdk.transforms.windowing.WindowFn;
--import org.apache.beam.sdk.util.ExecutionContext.StepContext;
--import org.apache.beam.sdk.util.SideInputReader;
 -import org.apache.beam.sdk.util.SystemDoFnInternal;
 -import org.apache.beam.sdk.util.TimerInternals;
 -import org.apache.beam.sdk.util.UserCodeException;
--import org.apache.beam.sdk.util.WindowedValue;
 -import org.apache.beam.sdk.util.WindowingInternals;
--import org.apache.beam.sdk.util.WindowingStrategy;
 -import org.apache.beam.sdk.util.state.StateInternals;
 -import org.apache.beam.sdk.values.PCollectionView;
--import org.apache.beam.sdk.values.TupleTag;
 -import org.joda.time.Instant;
 -import org.joda.time.format.PeriodFormat;
--
--/**
-  * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in.
 - * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in.
-- *
-  * @param <InputT> the type of the {@link OldDoFn} (main) input elements
-  * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
 - * @param <InputT> the type of the {@link DoFn} (main) input elements
 - * @param <OutputT> the type of the {@link DoFn} (main) output elements
-- */
- public class SimpleDoFnRunner<InputT, OutputT> extends DoFnRunnerBase<InputT, OutputT> {
 -public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
--
-   protected SimpleDoFnRunner(PipelineOptions options, OldDoFn<InputT, OutputT> fn,
 -  /** The {@link DoFn} being run. */
 -  private final DoFn<InputT, OutputT> fn;
 -
 -  /** The {@link DoFnInvoker} being run. */
 -  private final DoFnInvoker<InputT, OutputT> invoker;
 -
 -  /** The context used for running the {@link DoFn}. */
 -  private final DoFnContext<InputT, OutputT> context;
 -
 -  private final OutputManager outputManager;
 -
 -  private final TupleTag<OutputT> mainOutputTag;
 -
 -  private final boolean observesWindow;
 -
 -  public SimpleDoFnRunner(
 -      PipelineOptions options,
 -      DoFn<InputT, OutputT> fn,
--      SideInputReader sideInputReader,
--      OutputManager outputManager,
-       TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
-       AggregatorFactory aggregatorFactory, WindowingStrategy<?, ?> windowingStrategy) {
-     super(options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext,
-         aggregatorFactory, windowingStrategy);
 -      TupleTag<OutputT> mainOutputTag,
 -      List<TupleTag<?>> sideOutputTags,
 -      StepContext stepContext,
 -      AggregatorFactory aggregatorFactory,
 -      WindowingStrategy<?, ?> windowingStrategy) {
 -    this.fn = fn;
 -    this.observesWindow =
 -        DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().observesWindow();
 -    this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn);
 -    this.outputManager = outputManager;
 -    this.mainOutputTag = mainOutputTag;
 -    this.context =
 -        new DoFnContext<>(
 -            options,
 -            fn,
 -            sideInputReader,
 -            outputManager,
 -            mainOutputTag,
 -            sideOutputTags,
 -            stepContext,
 -            aggregatorFactory,
 -            windowingStrategy == null ? null : windowingStrategy.getWindowFn());
--  }
--
--  @Override
-   protected void invokeProcessElement(WindowedValue<InputT> elem) {
-     final OldDoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
 -  public void startBundle() {
--    // This can contain user code. Wrap it in case it throws an exception.
--    try {
-       fn.processElement(processContext);
 -      invoker.invokeStartBundle(context);
 -    } catch (Throwable t) {
 -      // Exception in user code.
 -      throw wrapUserCodeException(t);
 -    }
 -  }
 -
 -  @Override
 -  public void processElement(WindowedValue<InputT> compressedElem) {
 -    if (observesWindow) {
 -      for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {
 -        invokeProcessElement(elem);
 -      }
 -    } else {
 -      invokeProcessElement(compressedElem);
 -    }
 -  }
 -
 -  private void invokeProcessElement(WindowedValue<InputT> elem) {
 -    final DoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
 -
 -    // Note that if the element must be exploded into all its windows, that has to be done outside
 -    // of this runner.
 -    final DoFn.ExtraContextFactory<InputT, OutputT> extraContextFactory =
 -        createExtraContextFactory(elem);
 -
 -    // This can contain user code. Wrap it in case it throws an exception.
 -    try {
 -      invoker.invokeProcessElement(processContext, extraContextFactory);
--    } catch (Exception ex) {
--      throw wrapUserCodeException(ex);
 -    }
 -  }
 -
 -  @Override
 -  public void finishBundle() {
 -    // This can contain user code. Wrap it in case it throws an exception.
 -    try {
 -      invoker.invokeFinishBundle(context);
 -    } catch (Throwable t) {
 -      // Exception in user code.
 -      throw wrapUserCodeException(t);
 -    }
 -  }
 -
 -  /** Returns a new {@link DoFn.ProcessContext} for the given element. */
 -  private DoFn<InputT, OutputT>.ProcessContext createProcessContext(WindowedValue<InputT> elem) {
 -    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
 -  }
 -
 -  private DoFn.ExtraContextFactory<InputT, OutputT> createExtraContextFactory(
 -      WindowedValue<InputT> elem) {
 -    return new DoFnExtraContextFactory<InputT, OutputT>(elem.getWindows(), elem.getPane());
 -  }
 -
 -  private RuntimeException wrapUserCodeException(Throwable t) {
 -    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
 -  }
 -
 -  private boolean isSystemDoFn() {
 -    return invoker.getClass().isAnnotationPresent(SystemDoFnInternal.class);
 -  }
 -
 -  /**
 -   * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}.
 -   *
 -   * @param <InputT> the type of the {@link DoFn} (main) input elements
 -   * @param <OutputT> the type of the {@link DoFn} (main) output elements
 -   */
 -  private static class DoFnContext<InputT, OutputT> extends DoFn<InputT, OutputT>.Context {
 -    private static final int MAX_SIDE_OUTPUTS = 1000;
 -
 -    final PipelineOptions options;
 -    final DoFn<InputT, OutputT> fn;
 -    final SideInputReader sideInputReader;
 -    final OutputManager outputManager;
 -    final TupleTag<OutputT> mainOutputTag;
 -    final StepContext stepContext;
 -    final AggregatorFactory aggregatorFactory;
 -    final WindowFn<?, ?> windowFn;
 -
 -    /**
 -     * The set of known output tags, some of which may be undeclared, so we can throw an exception
 -     * when it exceeds {@link #MAX_SIDE_OUTPUTS}.
 -     */
 -    private Set<TupleTag<?>> outputTags;
 -
 -    public DoFnContext(
 -        PipelineOptions options,
 -        DoFn<InputT, OutputT> fn,
 -        SideInputReader sideInputReader,
 -        OutputManager outputManager,
 -        TupleTag<OutputT> mainOutputTag,
 -        List<TupleTag<?>> sideOutputTags,
 -        StepContext stepContext,
 -        AggregatorFactory aggregatorFactory,
 -        WindowFn<?, ?> windowFn) {
 -      fn.super();
 -      this.options = options;
 -      this.fn = fn;
 -      this.sideInputReader = sideInputReader;
 -      this.outputManager = outputManager;
 -      this.mainOutputTag = mainOutputTag;
 -      this.outputTags = Sets.newHashSet();
 -
 -      outputTags.add(mainOutputTag);
 -      for (TupleTag<?> sideOutputTag : sideOutputTags) {
 -        outputTags.add(sideOutputTag);
 -      }
 -
 -      this.stepContext = stepContext;
 -      this.aggregatorFactory = aggregatorFactory;
 -      this.windowFn = windowFn;
 -      super.setupDelegateAggregators();
 -    }
 -
 -    //////////////////////////////////////////////////////////////////////////////
 -
 -    @Override
 -    public PipelineOptions getPipelineOptions() {
 -      return options;
 -    }
 -
 -    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
 -        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
 -      final Instant inputTimestamp = timestamp;
 -
 -      if (timestamp == null) {
 -        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
 -      }
 -
 -      if (windows == null) {
 -        try {
 -          // The windowFn can never succeed at accessing the element, so its type does not
 -          // matter here
 -          @SuppressWarnings("unchecked")
 -          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
 -          windows =
 -              objectWindowFn.assignWindows(
 -                  objectWindowFn.new AssignContext() {
 -                    @Override
 -                    public Object element() {
 -                      throw new UnsupportedOperationException(
 -                          "WindowFn attempted to access input element when none was available");
 -                    }
 -
 -                    @Override
 -                    public Instant timestamp() {
 -                      if (inputTimestamp == null) {
 -                        throw new UnsupportedOperationException(
 -                            "WindowFn attempted to access input timestamp when none was available");
 -                      }
 -                      return inputTimestamp;
 -                    }
 -
 -                    @Override
 -                    public W window() {
 -                      throw new UnsupportedOperationException(
 -                          "WindowFn attempted to access input windows when none were available");
 -                    }
 -                  });
 -        } catch (Exception e) {
 -          throw UserCodeException.wrap(e);
 -        }
 -      }
 -
 -      return WindowedValue.of(output, timestamp, windows, pane);
 -    }
 -
 -    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
 -      if (!sideInputReader.contains(view)) {
 -        throw new IllegalArgumentException("calling sideInput() with unknown view");
 -      }
 -      BoundedWindow sideInputWindow =
 -          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
 -      return sideInputReader.get(view, sideInputWindow);
 -    }
 -
 -    void outputWindowedValue(
 -        OutputT output,
 -        Instant timestamp,
 -        Collection<? extends BoundedWindow> windows,
 -        PaneInfo pane) {
 -      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
 -    }
 -
 -    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
 -      outputManager.output(mainOutputTag, windowedElem);
 -      if (stepContext != null) {
 -        stepContext.noteOutput(windowedElem);
 -      }
 -    }
 -
 -    private <T> void sideOutputWindowedValue(
 -        TupleTag<T> tag,
 -        T output,
 -        Instant timestamp,
 -        Collection<? extends BoundedWindow> windows,
 -        PaneInfo pane) {
 -      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
 -    }
 -
 -    private <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
 -      if (!outputTags.contains(tag)) {
 -        // This tag wasn't declared nor was it seen before during this execution.
 -        // Thus, this must be a new, undeclared and unconsumed output.
 -        // To prevent likely user errors, enforce the limit on the number of side
 -        // outputs.
 -        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
 -          throw new IllegalArgumentException(
 -              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
 -        }
 -        outputTags.add(tag);
 -      }
 -
 -      outputManager.output(tag, windowedElem);
 -      if (stepContext != null) {
 -        stepContext.noteSideOutput(tag, windowedElem);
 -      }
 -    }
 -
 -    // Following implementations of output, outputWithTimestamp, and sideOutput
 -    // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by
 -    // ProcessContext's versions in DoFn.processElement.
 -    @Override
 -    public void output(OutputT output) {
 -      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
 -    }
 -
 -    @Override
 -    public void outputWithTimestamp(OutputT output, Instant timestamp) {
 -      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
 -    }
 -
 -    @Override
 -    public <T> void sideOutput(TupleTag<T> tag, T output) {
 -      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
 -      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
 -    }
 -
 -    @Override
 -    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
 -      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
 -      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
 -    }
 -
 -    @Override
 -    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
 -        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
 -      checkNotNull(combiner, "Combiner passed to createAggregator cannot be null");
 -      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
 -    }
 -  }
 -
 -  /**
 -   * A concrete implementation of {@link DoFn.ProcessContext} used for running a {@link DoFn} over a
 -   * single element.
 -   *
 -   * @param <InputT> the type of the {@link DoFn} (main) input elements
 -   * @param <OutputT> the type of the {@link DoFn} (main) output elements
 -   */
 -  private static class DoFnProcessContext<InputT, OutputT>
 -      extends DoFn<InputT, OutputT>.ProcessContext {
 -
 -    final DoFn<InputT, OutputT> fn;
 -    final DoFnContext<InputT, OutputT> context;
 -    final WindowedValue<InputT> windowedValue;
 -
 -    public DoFnProcessContext(
 -        DoFn<InputT, OutputT> fn,
 -        DoFnContext<InputT, OutputT> context,
 -        WindowedValue<InputT> windowedValue) {
 -      fn.super();
 -      this.fn = fn;
 -      this.context = context;
 -      this.windowedValue = windowedValue;
 -    }
 -
 -    @Override
 -    public PipelineOptions getPipelineOptions() {
 -      return context.getPipelineOptions();
 -    }
 -
 -    @Override
 -    public InputT element() {
 -      return windowedValue.getValue();
 -    }
 -
 -    @Override
 -    public <T> T sideInput(PCollectionView<T> view) {
 -      checkNotNull(view, "View passed to sideInput cannot be null");
 -      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
 -      BoundedWindow window;
 -      if (!windowIter.hasNext()) {
 -        if (context.windowFn instanceof GlobalWindows) {
 -          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
 -          // without windows
 -          window = GlobalWindow.INSTANCE;
 -        } else {
 -          throw new IllegalStateException(
 -              "sideInput called when main input element is not in any windows");
 -        }
 -      } else {
 -        window = windowIter.next();
 -        if (windowIter.hasNext()) {
 -          throw new IllegalStateException(
 -              "sideInput called when main input element is in multiple windows");
 -        }
 -      }
 -      return context.sideInput(view, window);
 -    }
 -
 -    @Override
 -    public PaneInfo pane() {
 -      return windowedValue.getPane();
 -    }
 -
 -    @Override
 -    public void output(OutputT output) {
 -      context.outputWindowedValue(windowedValue.withValue(output));
 -    }
 -
 -    @Override
 -    public void outputWithTimestamp(OutputT output, Instant timestamp) {
 -      checkTimestamp(timestamp);
 -      context.outputWindowedValue(
 -          output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
 -    }
 -
 -    void outputWindowedValue(
 -        OutputT output,
 -        Instant timestamp,
 -        Collection<? extends BoundedWindow> windows,
 -        PaneInfo pane) {
 -      context.outputWindowedValue(output, timestamp, windows, pane);
 -    }
 -
 -    @Override
 -    public <T> void sideOutput(TupleTag<T> tag, T output) {
 -      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
 -      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
 -    }
 -
 -    @Override
 -    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
 -      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
 -      checkTimestamp(timestamp);
 -      context.sideOutputWindowedValue(
 -          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
 -    }
 -
 -    @Override
 -    public Instant timestamp() {
 -      return windowedValue.getTimestamp();
 -    }
 -
 -    public Collection<? extends BoundedWindow> windows() {
 -      return windowedValue.getWindows();
 -    }
 -
 -    private void checkTimestamp(Instant timestamp) {
 -      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
 -        throw new IllegalArgumentException(
 -            String.format(
 -                "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
 -                    + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
 -                    + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed "
 -                    + "skew.",
 -                timestamp,
 -                windowedValue.getTimestamp(),
 -                PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
 -      }
 -    }
 -
 -    @Override
 -    protected <AggregatorInputT, AggregatorOutputT>
 -        Aggregator<AggregatorInputT, AggregatorOutputT> createAggregator(
 -            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
 -      return context.createAggregator(name, combiner);
 -    }
 -  }
 -
 -  private class DoFnExtraContextFactory<InputT, OutputT>
 -      implements DoFn.ExtraContextFactory<InputT, OutputT> {
 -
 -    /** The windows of the current element. */
 -    private final Collection<? extends BoundedWindow> windows;
 -
 -    /** The pane of the current element. */
 -    private final PaneInfo pane;
 -
 -    public DoFnExtraContextFactory(Collection<? extends BoundedWindow> windows, PaneInfo pane) {
 -      this.windows = windows;
 -      this.pane = pane;
 -    }
 -
 -    @Override
 -    public BoundedWindow window() {
 -      return Iterables.getOnlyElement(windows);
 -    }
 -
 -    @Override
 -    public InputProvider<InputT> inputProvider() {
 -      throw new UnsupportedOperationException("InputProvider parameters are not supported.");
 -    }
 -
 -    @Override
 -    public OutputReceiver<OutputT> outputReceiver() {
 -      throw new UnsupportedOperationException("OutputReceiver parameters are not supported.");
 -    }
 -
 -    @Override
 -    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
 -      throw new UnsupportedOperationException("RestrictionTracker parameters are not supported.");
 -    }
 -
 -    @Override
 -    public WindowingInternals<InputT, OutputT> windowingInternals() {
 -      return new WindowingInternals<InputT, OutputT>() {
 -        @Override
 -        public Collection<? extends BoundedWindow> windows() {
 -          return windows;
 -        }
 -
 -        @Override
 -        public PaneInfo pane() {
 -          return pane;
 -        }
 -
 -        @Override
 -        public TimerInternals timerInternals() {
 -          return context.stepContext.timerInternals();
 -        }
 -
 -        @Override
 -        public <T> void writePCollectionViewData(
 -            TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder)
 -            throws IOException {
 -          @SuppressWarnings("unchecked")
 -          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
 -
 -          context.stepContext.writePCollectionViewData(
 -              tag,
 -              data,
 -              IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
 -              window(),
 -              windowCoder);
 -        }
 -
 -        @Override
 -        public StateInternals<?> stateInternals() {
 -          return context.stepContext.stateInternals();
 -        }
 -
 -        @Override
 -        public void outputWindowedValue(
 -            OutputT output,
 -            Instant timestamp,
 -            Collection<? extends BoundedWindow> windows,
 -            PaneInfo pane) {}
 -
 -        @Override
 -        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
 -          return context.sideInput(view, mainInputWindow);
 -        }
 -      };
--    }
--  }
--}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --cc runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index b2d61c3,e02c8a6..f87f1c1
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@@ -401,31 -450,19 +450,41 @@@ public class DirectRunne
      }
  
      @Override
-     public State waitUntilFinish(Duration duration) throws IOException {
-       throw new UnsupportedOperationException(
-           "DirectPipelineResult does not support waitUntilFinish.");
+     public ExecutorService get() {
+       return Executors.newFixedThreadPool(options.getTargetParallelism());
+     }
+   }
+ 
+ 
+   /**
+    * A {@link Supplier} that creates a {@link NanosOffsetClock}.
+    */
+   private static class NanosOffsetClockSupplier implements Supplier<Clock> {
+     @Override
+     public Clock get() {
+       return NanosOffsetClock.create();
      }
    }
 +
 +  /**
 +   * A {@link Supplier} that creates a {@link ExecutorService} based on
 +   * {@link Executors#newFixedThreadPool(int)}.
 +   */
 +  private static class FixedThreadPoolSupplier implements Supplier<ExecutorService> {
 +    @Override
 +    public ExecutorService get() {
 +      return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
 +    }
 +  }
 +
 +
 +  /**
 +   * A {@link Supplier} that creates a {@link NanosOffsetClock}.
 +   */
 +  private static class NanosOffsetClockSupplier implements Supplier<Clock> {
 +    @Override
 +    public Clock get() {
 +      return NanosOffsetClock.create();
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --cc runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index 9edc50f,3dd44a7..6485714
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@@ -84,44 -82,38 +82,75 @@@ class TransformEvaluatorRegistry implem
        throws Exception {
      checkState(
          !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry");
-     TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
-     return factory.forApplication(application, inputBundle, evaluationContext);
+     Class<? extends PTransform> transformClass = application.getTransform().getClass();
+     TransformEvaluatorFactory factory =
+         checkNotNull(
+             factories.get(transformClass), "No evaluator for PTransform type %s", transformClass);
+     return factory.forApplication(application, inputBundle);
+   }
+ 
+   @Override
+   public void cleanup() throws Exception {
+     Collection<Exception> thrownInCleanup = new ArrayList<>();
+     for (TransformEvaluatorFactory factory : factories.values()) {
+       try {
+         factory.cleanup();
+       } catch (Exception e) {
+         if (e instanceof InterruptedException) {
+           Thread.currentThread().interrupt();
+         }
+         thrownInCleanup.add(e);
+       }
+     }
+     finished.set(true);
+     if (!thrownInCleanup.isEmpty()) {
+       LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup);
+       Exception toThrow = null;
+       for (Exception e : thrownInCleanup) {
+         if (toThrow == null) {
+           toThrow = e;
+         } else {
+           toThrow.addSuppressed(e);
+         }
+       }
+       throw toThrow;
+     }
    }
 +
 +  @Override
 +  public void cleanup() throws Exception {
 +    Collection<Exception> thrownInCleanup = new ArrayList<>();
 +    for (TransformEvaluatorFactory factory : factories.values()) {
 +      try {
 +        factory.cleanup();
 +      } catch (Exception e) {
 +        if (e instanceof InterruptedException) {
 +          Thread.currentThread().interrupt();
 +        }
 +        thrownInCleanup.add(e);
 +      }
 +    }
 +    finished.set(true);
 +    if (!thrownInCleanup.isEmpty()) {
 +      LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup);
 +      Exception toThrow = null;
 +      for (Exception e : thrownInCleanup) {
 +        if (toThrow == null) {
 +          toThrow = e;
 +        } else {
 +          toThrow.addSuppressed(e);
 +        }
 +      }
 +      throw toThrow;
 +    }
 +  }
 +
 +  /**
 +   * A factory to create Transform Evaluator Registries.
 +   */
 +  public static class Factory {
 +    public TransformEvaluatorRegistry create() {
 +      return TransformEvaluatorRegistry.defaultRegistry();
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
----------------------------------------------------------------------
diff --cc runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
index 58f41b6,58f41b6..0000000
deleted file mode 100644,100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
+++ /dev/null
@@@ -1,22 -1,22 +1,0 @@@
--/*
-- * 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.
-- */
--
--/**
-- * Flink Beam runner exemple.
-- */
--package org.apache.beam.runners.flink.examples.streaming;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
----------------------------------------------------------------------
diff --cc runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
index 949c381,b211c04..b84a1a8
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
@@@ -65,5 -68,12 +68,13 @@@ public class DoFnInfo<InputT, OutputT> 
    public Coder<InputT> getInputCoder() {
      return inputCoder;
    }
+ 
+   public long getMainOutput() {
+     return mainOutput;
+   }
+ 
+   public Map<Long, TupleTag<?>> getOutputMap() {
+     return outputMap;
+   }
  }
 +

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
----------------------------------------------------------------------
diff --cc runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
index 454b760,4dfbee6..69c450e
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
@@@ -37,22 -42,26 +42,28 @@@ import org.apache.spark.api.java.functi
   * @param <OutputT> Output element type.
   */
  public class DoFnFunction<InputT, OutputT>
-     implements FlatMapFunction<Iterator<WindowedValue<InputT>>,
-     WindowedValue<OutputT>> {
+     implements FlatMapFunction<Iterator<WindowedValue<InputT>>, WindowedValue<OutputT>> {
+   private final Accumulator<NamedAggregators> accum;
    private final OldDoFn<InputT, OutputT> mFunction;
 +  private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class);
 +
    private final SparkRuntimeContext mRuntimeContext;
-   private final Map<TupleTag<?>, BroadcastHelper<?>> mSideInputs;
+   private final Map<TupleTag<?>, KV<WindowingStrategy<?, ?>, BroadcastHelper<?>>> mSideInputs;
+   private final WindowFn<Object, ?> windowFn;
  
    /**
-    * @param fn         DoFunction to be wrapped.
-    * @param runtime    Runtime to apply function in.
-    * @param sideInputs Side inputs used in DoFunction.
+    * @param accum             The Spark Accumulator that handles the Beam Aggregators.
+    * @param fn                DoFunction to be wrapped.
+    * @param runtime           Runtime to apply function in.
+    * @param sideInputs        Side inputs used in DoFunction.
+    * @param windowFn          Input {@link WindowFn}.
     */
-   public DoFnFunction(OldDoFn<InputT, OutputT> fn,
-                SparkRuntimeContext runtime,
-                Map<TupleTag<?>, BroadcastHelper<?>> sideInputs) {
+   public DoFnFunction(Accumulator<NamedAggregators> accum,
+                       OldDoFn<InputT, OutputT> fn,
+                       SparkRuntimeContext runtime,
+                       Map<TupleTag<?>, KV<WindowingStrategy<?, ?>, BroadcastHelper<?>>> sideInputs,
+                       WindowFn<Object, ?> windowFn) {
+     this.accum = accum;
      this.mFunction = fn;
      this.mRuntimeContext = runtime;
      this.mSideInputs = sideInputs;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --cc sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index c0761b1,2dbcda7..78ea988
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@@ -32,21 -34,24 +33,36 @@@ import java.nio.channels.WritableByteCh
  import java.nio.charset.StandardCharsets;
  import java.util.NoSuchElementException;
  import java.util.regex.Pattern;
 -
  import javax.annotation.Nullable;
 +import org.apache.beam.sdk.coders.Coder;
 +import org.apache.beam.sdk.coders.Coder.Context;
 +import org.apache.beam.sdk.coders.StringUtf8Coder;
 +import org.apache.beam.sdk.coders.VoidCoder;
 +import org.apache.beam.sdk.io.Read.Bounded;
 +import org.apache.beam.sdk.options.PipelineOptions;
 +import org.apache.beam.sdk.transforms.PTransform;
 +import org.apache.beam.sdk.transforms.display.DisplayData;
 +import org.apache.beam.sdk.util.IOChannelUtils;
 +import org.apache.beam.sdk.util.MimeTypes;
 +import org.apache.beam.sdk.values.PBegin;
 +import org.apache.beam.sdk.values.PCollection;
 +import org.apache.beam.sdk.values.PDone;
  
+ import org.apache.beam.sdk.coders.Coder;
+ import org.apache.beam.sdk.coders.Coder.Context;
+ import org.apache.beam.sdk.coders.StringUtf8Coder;
+ import org.apache.beam.sdk.coders.VoidCoder;
+ import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory;
+ import org.apache.beam.sdk.io.Read.Bounded;
+ import org.apache.beam.sdk.options.PipelineOptions;
+ import org.apache.beam.sdk.transforms.PTransform;
+ import org.apache.beam.sdk.transforms.display.DisplayData;
+ import org.apache.beam.sdk.util.IOChannelUtils;
+ import org.apache.beam.sdk.util.MimeTypes;
+ import org.apache.beam.sdk.values.PBegin;
+ import org.apache.beam.sdk.values.PCollection;
+ import org.apache.beam.sdk.values.PDone;
+ 
  /**
   * {@link PTransform}s for reading and writing text files.
   *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --cc sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 59c8323,018877f..f2fa87c
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@@ -339,21 -423,135 +423,149 @@@ public abstract class DoFn<InputT, Outp
  
    /////////////////////////////////////////////////////////////////////////////
  
 +
 +  /**
 +   * Annotation for the method to use to prepare an instance for processing bundles of elements. The
 +   * method annotated with this must satisfy the following constraints
 +   * <ul>
 +   *   <li>It must have zero arguments.
 +   * </ul>
 +   */
 +  @Documented
 +  @Retention(RetentionPolicy.RUNTIME)
 +  @Target(ElementType.METHOD)
 +  public @interface Setup {
 +  }
 +
    /**
+    * Annotation for declaring and dereferencing state cells.
+    *
+    * <p><i>Not currently supported by any runner. When ready, the feature will work as described
+    * here.</i>
+    *
+    * <p>To declare a state cell, create a field of type {@link StateSpec} annotated with a {@link
+    * StateId}. To use the cell during processing, add a parameter of the appropriate {@link State}
+    * subclass to your {@link ProcessElement @ProcessElement} or {@link OnTimer @OnTimer} method, and
+    * annotate it with {@link StateId}. See the following code for an example:
+    *
+    * <pre>{@code
+    * new DoFn<KV<Key, Foo>, Baz>() {
+    *   @StateId("my-state-id")
+    *   private final StateSpec<K, ValueState<MyState>> myStateSpec =
+    *       StateSpecs.value(new MyStateCoder());
+    *
+    *   @ProcessElement
+    *   public void processElement(
+    *       ProcessContext c,
+    *       @StateId("my-state-id") ValueState<MyState> myState) {
+    *     myState.read();
+    *     myState.write(...);
+    *   }
+    * }
+    * }</pre>
+    *
+    * <p>State is subject to the following validity conditions:
+    *
+    * <ul>
+    * <li>Each state ID must be declared at most once.
+    * <li>Any state referenced in a parameter must be declared with the same state type.
+    * <li>State declarations must be final.
+    * </ul>
+    */
+   @Documented
+   @Retention(RetentionPolicy.RUNTIME)
+   @Target({ElementType.FIELD, ElementType.PARAMETER})
+   @Experimental(Kind.STATE)
+   public @interface StateId {
+     /** The state ID. */
+     String value();
+   }
+ 
+   /**
+    * Annotation for declaring and dereferencing timers.
+    *
+    * <p><i>Not currently supported by any runner. When ready, the feature will work as described
+    * here.</i>
+    *
+    * <p>To declare a timer, create a field of type {@link TimerSpec} annotated with a {@link
+    * TimerId}. To use the cell during processing, add a parameter of the type {@link Timer} to your
+    * {@link ProcessElement @ProcessElement} or {@link OnTimer @OnTimer} method, and annotate it with
+    * {@link TimerId}. See the following code for an example:
+    *
+    * <pre>{@code
+    * new DoFn<KV<Key, Foo>, Baz>() {
+    *   @TimerId("my-timer-id")
+    *   private final TimerSpec myTimer = TimerSpecs.timerForDomain(TimeDomain.EVENT_TIME);
+    *
+    *   @ProcessElement
+    *   public void processElement(
+    *       ProcessContext c,
+    *       @TimerId("my-timer-id") Timer myTimer) {
+    *     myTimer.setForNowPlus(Duration.standardSeconds(...));
+    *   }
+    *
+    *   @OnTimer("my-timer-id")
+    *   public void onMyTimer() {
+    *     ...
+    *   }
+    * }
+    * }</pre>
+    *
+    * <p>Timers are subject to the following validity conditions:
+    *
+    * <ul>
+    * <li>Each timer must have a distinct id.
+    * <li>Any timer referenced in a parameter must be declared.
+    * <li>Timer declarations must be final.
+    * <li>All declared timers must have a corresponding callback annotated with {@link
+    *     OnTimer @OnTimer}.
+    * </ul>
+    */
+   @Documented
+   @Retention(RetentionPolicy.RUNTIME)
+   @Target({ElementType.FIELD, ElementType.PARAMETER})
+   @Experimental(Kind.TIMERS)
+   public @interface TimerId {
+     /** The timer ID. */
+     String value();
+   }
+ 
+   /**
+    * Annotation for registering a callback for a timer.
+    *
+    * <p><i>Not currently supported by any runner. When ready, the feature will work as described
+    * here.</i>
+    *
+    * <p>See the javadoc for {@link TimerId} for use in a full example.
+    *
+    * <p>The method annotated with {@code @OnTimer} may have parameters according to the same logic
+    * as {@link ProcessElement}, but limited to the {@link BoundedWindow}, {@link State} subclasses,
+    * and {@link Timer}. State and timer parameters must be annotated with their {@link StateId} and
+    * {@link TimerId} respectively.
+    */
+   @Documented
+   @Retention(RetentionPolicy.RUNTIME)
+   @Target(ElementType.METHOD)
+   @Experimental(Kind.TIMERS)
+   public @interface OnTimer {
+     /** The timer ID. */
+     String value();
+   }
+ 
+   /**
+    * Annotation for the method to use to prepare an instance for processing bundles of elements. The
+    * method annotated with this must satisfy the following constraints
+    * <ul>
+    *   <li>It must have zero arguments.
+    * </ul>
+    */
+   @Documented
+   @Retention(RetentionPolicy.RUNTIME)
+   @Target(ElementType.METHOD)
+   public @interface Setup {
+   }
+ 
+   /**
     * Annotation for the method to use to prepare an instance for processing a batch of elements.
     * The method annotated with this must satisfy the following constraints:
     * <ul>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9dc9be9e/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------


[11/50] [abbrv] incubator-beam git commit: Closes #1091

Posted by dh...@apache.org.
Closes #1091


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

Branch: refs/heads/gearpump-runner
Commit: 43beb1ea337630121bb5972e31f6cff9b04004c1
Parents: 6d9d8bc 75d137b
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 07:37:41 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:37:41 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  7 ++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 35 +++++++++++++++-----
 2 files changed, 31 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[36/50] [abbrv] incubator-beam git commit: Remove unused GroupAlsoByWindowsDoFn.createDefault

Posted by dh...@apache.org.
Remove unused GroupAlsoByWindowsDoFn.createDefault


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

Branch: refs/heads/gearpump-runner
Commit: 5ce3aac2d3b7203b0cbab52c394a99142a46a5cf
Parents: f603d43
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 15:51:08 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/core/GroupAlsoByWindowsDoFn.java     | 19 -------------------
 1 file changed, 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ce3aac2/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
index 9851449..1b32d84 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
@@ -17,15 +17,12 @@
  */
 package org.apache.beam.runners.core;
 
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.KV;
 
 /**
@@ -47,20 +44,4 @@ public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends Bound
       createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
   protected final Aggregator<Long, Long> droppedDueToLateness =
       createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
-
-  /**
-   * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
-   * grouping.
-   *
-   * @param windowingStrategy The window function and trigger to use for grouping
-   * @param inputCoder the input coder to use
-   */
-  public static <K, V, W extends BoundedWindow>
-      GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W> createDefault(
-          WindowingStrategy<?, W> windowingStrategy,
-          StateInternalsFactory<K> stateInternalsFactory,
-          Coder<V> inputCoder) {
-    return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
-        windowingStrategy, stateInternalsFactory, SystemReduceFn.<K, V, W>buffering(inputCoder));
-  }
 }


[33/50] [abbrv] incubator-beam git commit: Improve teardown behavior in DoFnLifecycleManager

Posted by dh...@apache.org.
Improve teardown behavior in DoFnLifecycleManager

Use Cache invalidation hooks to teardown DoFns that are no longer in the
cache. Ensure that remove() and removeAll() report thrown exceptions
even though the exceptions are not thrown by the LoadingCache.


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

Branch: refs/heads/gearpump-runner
Commit: 7239ebb0c76f539f476cea0b44b1070e765cca41
Parents: 79bb2c2
Author: Thomas Groh <tg...@google.com>
Authored: Mon Oct 24 13:43:43 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 10:46:43 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/DoFnLifecycleManager.java    | 56 +++++++++------
 .../direct/DoFnLifecycleManagerTest.java        | 74 ++++++++++++++++++--
 2 files changed, 104 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7239ebb0/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
index 23460b6..472b28b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
@@ -21,17 +21,17 @@ package org.apache.beam.runners.direct;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import java.util.ArrayList;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
 import java.util.Collection;
-import java.util.Iterator;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.Setup;
 import org.apache.beam.sdk.transforms.DoFn.Teardown;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.util.SerializableUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Manages {@link DoFn} setup, teardown, and serialization.
@@ -42,16 +42,18 @@ import org.slf4j.LoggerFactory;
  * clearing all cached {@link DoFn DoFns}.
  */
 class DoFnLifecycleManager {
-  private static final Logger LOG = LoggerFactory.getLogger(DoFnLifecycleManager.class);
-
   public static DoFnLifecycleManager of(DoFn<?, ?> original) {
     return new DoFnLifecycleManager(original);
   }
 
   private final LoadingCache<Thread, DoFn<?, ?>> outstanding;
+  private final ConcurrentMap<Thread, Exception> thrownOnTeardown;
 
   private DoFnLifecycleManager(DoFn<?, ?> original) {
-    this.outstanding = CacheBuilder.newBuilder().build(new DeserializingCacheLoader(original));
+    this.outstanding = CacheBuilder.newBuilder()
+        .removalListener(new TeardownRemovedFnListener())
+        .build(new DeserializingCacheLoader(original));
+    thrownOnTeardown = new ConcurrentHashMap<>();
   }
 
   public DoFn<?, ?> get() throws Exception {
@@ -61,8 +63,15 @@ class DoFnLifecycleManager {
 
   public void remove() throws Exception {
     Thread currentThread = Thread.currentThread();
-    DoFn<?, ?> fn = outstanding.asMap().remove(currentThread);
-    DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
+    outstanding.invalidate(currentThread);
+    // Block until the invalidate is fully completed
+    outstanding.cleanUp();
+    // Remove to try too avoid reporting the same teardown exception twice. May still double-report,
+    // but the second will be suppressed.
+    Exception thrown = thrownOnTeardown.remove(currentThread);
+    if (thrown != null) {
+      throw thrown;
+    }
   }
 
   /**
@@ -73,21 +82,13 @@ class DoFnLifecycleManager {
    * DoFn.Teardown @Teardown} method, and the {@link PipelineRunner} should throw an exception.
    */
   public Collection<Exception> removeAll() throws Exception {
-    Iterator<DoFn<?, ?>> fns = outstanding.asMap().values().iterator();
-    Collection<Exception> thrown = new ArrayList<>();
-    while (fns.hasNext()) {
-      DoFn<?, ?> fn = fns.next();
-      fns.remove();
-      try {
-        DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
-      } catch (Exception e) {
-        thrown.add(e);
-      }
-    }
-    return thrown;
+    outstanding.invalidateAll();
+    // Make sure all of the teardowns are run
+    outstanding.cleanUp();
+    return thrownOnTeardown.values();
   }
 
-  private class DeserializingCacheLoader extends CacheLoader<Thread, DoFn<?, ?>> {
+  private static class DeserializingCacheLoader extends CacheLoader<Thread, DoFn<?, ?>> {
     private final byte[] original;
 
     public DeserializingCacheLoader(DoFn<?, ?> original) {
@@ -102,4 +103,15 @@ class DoFnLifecycleManager {
       return fn;
     }
   }
+
+  private class TeardownRemovedFnListener implements RemovalListener<Thread, DoFn<?, ?>> {
+    @Override
+    public void onRemoval(RemovalNotification<Thread, DoFn<?, ?>> notification) {
+      try {
+        DoFnInvokers.INSTANCE.newByteBuddyInvoker(notification.getValue()).invokeTeardown();
+      } catch (Exception e) {
+        thrownOnTeardown.put(notification.getKey(), e);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7239ebb0/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
index aef9d29..59e1e16 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
@@ -21,6 +21,7 @@ package org.apache.beam.runners.direct;
 import static com.google.common.base.Preconditions.checkState;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
 import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.theInstance;
 import static org.junit.Assert.assertThat;
@@ -34,8 +35,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.UserCodeException;
 import org.hamcrest.Matchers;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -44,6 +48,8 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class DoFnLifecycleManagerTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
   private TestFn fn = new TestFn();
   private DoFnLifecycleManager mgr = DoFnLifecycleManager.of(fn);
 
@@ -105,6 +111,17 @@ public class DoFnLifecycleManagerTest {
   }
 
   @Test
+  public void teardownThrowsRemoveThrows() throws Exception {
+    TestFn obtained = (TestFn) mgr.get();
+    obtained.teardown();
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(IllegalStateException.class));
+    thrown.expectMessage("Cannot call teardown: already torn down");
+    mgr.remove();
+  }
+
+  @Test
   public void teardownAllOnRemoveAll() throws Exception {
     CountDownLatch startSignal = new CountDownLatch(1);
     ExecutorService executor = Executors.newCachedThreadPool();
@@ -125,6 +142,38 @@ public class DoFnLifecycleManagerTest {
     }
   }
 
+  @Test
+  public void removeAndRemoveAllConcurrent() throws Exception {
+    CountDownLatch startSignal = new CountDownLatch(1);
+    ExecutorService executor = Executors.newCachedThreadPool();
+    List<Future<TestFn>> futures = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      futures.add(executor.submit(new GetFnCallable(mgr, startSignal)));
+    }
+    startSignal.countDown();
+    List<TestFn> fns = new ArrayList<>();
+    for (Future<TestFn> future : futures) {
+      fns.add(future.get(1L, TimeUnit.SECONDS));
+    }
+    CountDownLatch removeSignal = new CountDownLatch(1);
+    List<Future<Void>> removeFutures = new ArrayList<>();
+    for (int i = 0; i < 5; i++) {
+      // These will reuse the threads used in the GetFns
+      removeFutures.add(executor.submit(new TeardownFnCallable(mgr, removeSignal)));
+    }
+    removeSignal.countDown();
+    assertThat(mgr.removeAll(), Matchers.<Exception>emptyIterable());
+    for (Future<Void> removed : removeFutures) {
+      // Should not have thrown an exception.
+      removed.get();
+    }
+
+    for (TestFn fn : fns) {
+      assertThat(fn.setupCalled, is(true));
+      assertThat(fn.teardownCalled, is(true));
+    }
+  }
+
   private static class GetFnCallable implements Callable<TestFn> {
     private final DoFnLifecycleManager mgr;
     private final CountDownLatch startSignal;
@@ -141,6 +190,23 @@ public class DoFnLifecycleManagerTest {
     }
   }
 
+  private static class TeardownFnCallable implements Callable<Void> {
+    private final DoFnLifecycleManager mgr;
+    private final CountDownLatch startSignal;
+
+    private TeardownFnCallable(DoFnLifecycleManager mgr, CountDownLatch startSignal) {
+      this.mgr = mgr;
+      this.startSignal = startSignal;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      startSignal.await();
+      // Will throw an exception if the TestFn has already been removed from this thread
+      mgr.remove();
+      return null;
+    }
+  }
 
   private static class TestFn extends DoFn<Object, Object> {
     boolean setupCalled = false;
@@ -148,8 +214,8 @@ public class DoFnLifecycleManagerTest {
 
     @Setup
     public void setup() {
-      checkState(!setupCalled);
-      checkState(!teardownCalled);
+      checkState(!setupCalled, "Cannot call setup: already set up");
+      checkState(!teardownCalled, "Cannot call setup: already torn down");
 
       setupCalled = true;
     }
@@ -160,8 +226,8 @@ public class DoFnLifecycleManagerTest {
 
     @Teardown
     public void teardown() {
-      checkState(setupCalled);
-      checkState(!teardownCalled);
+      checkState(setupCalled, "Cannot call teardown: not set up");
+      checkState(!teardownCalled, "Cannot call teardown: already torn down");
 
       teardownCalled = true;
     }


[26/50] [abbrv] incubator-beam git commit: Closes #1175

Posted by dh...@apache.org.
Closes #1175


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

Branch: refs/heads/gearpump-runner
Commit: 7de8d571d6a86de0dfdbb1756e101c72319f5db4
Parents: d285e85 6291a6f
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 18:03:29 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 18:03:29 2016 -0700

----------------------------------------------------------------------
 pom.xml         | 3 ---
 runners/pom.xml | 1 +
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------



[34/50] [abbrv] incubator-beam git commit: Encode bundle elements in the DirectRunner

Posted by dh...@apache.org.
Encode bundle elements in the DirectRunner

This ensures that any changes that are caused when an element is encoded
and decoded is caught within the pipeline.


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

Branch: refs/heads/gearpump-runner
Commit: 2ceaa3effa8a6d9de3753a05db9d1648e8eed576
Parents: c03e3e9
Author: Thomas Groh <tg...@google.com>
Authored: Tue Sep 20 11:43:40 2016 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Tue Oct 25 11:03:43 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/CloningBundleFactory.java    |  98 ++++++++++
 .../beam/runners/direct/DirectRunner.java       |   5 +-
 .../direct/ImmutableListBundleFactory.java      |   4 +-
 .../direct/CloningBundleFactoryTest.java        | 177 +++++++++++++++++++
 .../EncodabilityEnforcementFactoryTest.java     |   6 +-
 5 files changed, 285 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
new file mode 100644
index 0000000..33241e3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.direct;
+
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+
+/**
+ * A {@link BundleFactory} where a created {@link UncommittedBundle} clones all elements added to it
+ * using the coder of the {@link PCollection}.
+ */
+class CloningBundleFactory implements BundleFactory {
+  private static final CloningBundleFactory INSTANCE = new CloningBundleFactory();
+
+  public static CloningBundleFactory create() {
+    return INSTANCE;
+  }
+
+  private final ImmutableListBundleFactory underlying;
+  private CloningBundleFactory() {
+    this.underlying = ImmutableListBundleFactory.create();
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createRootBundle() {
+    // The DirectRunner is responsible for these elements, but they need not be encodable.
+    return underlying.createRootBundle();
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createBundle(
+      PCollection<T> output) {
+    return new CloningBundle<>(underlying.createBundle(output));
+  }
+
+  @Override
+  public <K, T> UncommittedBundle<T> createKeyedBundle(
+      StructuralKey<K> key, PCollection<T> output) {
+    return new CloningBundle<>(underlying.createKeyedBundle(key, output));
+  }
+
+  private static class CloningBundle<T> implements UncommittedBundle<T> {
+    private final UncommittedBundle<T> underlying;
+    private final Coder<T> coder;
+
+    private CloningBundle(UncommittedBundle<T> underlying) {
+      this.underlying = underlying;
+      this.coder = underlying.getPCollection().getCoder();
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return underlying.getPCollection();
+    }
+
+    @Override
+    public UncommittedBundle<T> add(WindowedValue<T> element) {
+      try {
+        // Use the cloned value to ensure that if the coder behaves poorly (e.g. a NoOpCoder that
+        // does not expect to be used) that is reflected in the values given to downstream
+        // transforms
+        WindowedValue<T> clone = element.withValue(CoderUtils.clone(coder, element.getValue()));
+        underlying.add(clone);
+      } catch (CoderException e) {
+        throw UserCodeException.wrap(e);
+      }
+      return this;
+    }
+
+    @Override
+    public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
+      return underlying.commit(synchronizedProcessingTime);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index b79a42f..e02c8a6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -319,7 +319,10 @@ public class DirectRunner
   }
 
   private BundleFactory createBundleFactory(DirectOptions pipelineOptions) {
-    BundleFactory bundleFactory = ImmutableListBundleFactory.create();
+    BundleFactory bundleFactory =
+        pipelineOptions.isEnforceEncodability()
+            ? CloningBundleFactory.create()
+            : ImmutableListBundleFactory.create();
     if (pipelineOptions.isEnforceImmutability()) {
       bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
index db92542..abc6dd8 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
@@ -32,8 +32,10 @@ import org.joda.time.Instant;
  * A factory that produces bundles that perform no additional validation.
  */
 class ImmutableListBundleFactory implements BundleFactory {
+  private static final ImmutableListBundleFactory FACTORY = new ImmutableListBundleFactory();
+
   public static ImmutableListBundleFactory create() {
-    return new ImmutableListBundleFactory();
+    return FACTORY;
   }
 
   private ImmutableListBundleFactory() {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
new file mode 100644
index 0000000..03846d9
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.runners.direct;
+
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.isA;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.Record;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.RecordNoDecodeCoder;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.RecordNoEncodeCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link CloningBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class CloningBundleFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private CloningBundleFactory factory = CloningBundleFactory.create();
+
+  @Test
+  public void rootBundleSucceedsIgnoresCoder() {
+    WindowedValue<Record> one = WindowedValue.valueInGlobalWindow(new Record());
+    WindowedValue<Record> two = WindowedValue.valueInGlobalWindow(new Record());
+    CommittedBundle<Record> root =
+        factory.<Record>createRootBundle().add(one).add(two).commit(Instant.now());
+
+    assertThat(root.getElements(), containsInAnyOrder(one, two));
+  }
+
+  @Test
+  public void bundleWorkingCoderSucceedsClonesOutput() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of()));
+    PCollection<KV<String, Integer>> kvs =
+        created
+            .apply(WithKeys.<String, Integer>of("foo"))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+    WindowedValue<KV<String, Integer>> fooOne = WindowedValue.valueInGlobalWindow(KV.of("foo", 1));
+    WindowedValue<KV<String, Integer>> fooThree =
+        WindowedValue.valueInGlobalWindow(KV.of("foo", 3));
+    CommittedBundle<KV<String, Integer>> bundle =
+        factory.createBundle(kvs).add(fooOne).add(fooThree).commit(Instant.now());
+
+    assertThat(bundle.getElements(), containsInAnyOrder(fooOne, fooThree));
+    assertThat(
+        bundle.getElements(), not(containsInAnyOrder(theInstance(fooOne), theInstance(fooThree))));
+    for (WindowedValue<KV<String, Integer>> foo : bundle.getElements()) {
+      assertThat(
+          foo.getValue(),
+          not(anyOf(theInstance(fooOne.getValue()), theInstance(fooThree.getValue()))));
+    }
+    assertThat(bundle.getPCollection(), equalTo(kvs));
+  }
+
+  @Test
+  public void keyedBundleWorkingCoderSucceedsClonesOutput() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of()));
+
+    PCollection<KV<String, Iterable<Integer>>> keyed =
+        created
+            .apply(WithKeys.<String, Integer>of("foo"))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+            .apply(GroupByKey.<String, Integer>create());
+    WindowedValue<KV<String, Iterable<Integer>>> foos =
+        WindowedValue.valueInGlobalWindow(
+            KV.<String, Iterable<Integer>>of("foo", ImmutableList.of(1, 3)));
+    CommittedBundle<KV<String, Iterable<Integer>>> keyedBundle =
+        factory
+            .createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), keyed)
+            .add(foos)
+            .commit(Instant.now());
+
+    assertThat(keyedBundle.getElements(), containsInAnyOrder(foos));
+    assertThat(
+        Iterables.getOnlyElement(keyedBundle.getElements()).getValue(),
+        not(theInstance(foos.getValue())));
+    assertThat(keyedBundle.getPCollection(), equalTo(keyed));
+    assertThat(
+        keyedBundle.getKey(),
+        Matchers.<StructuralKey<?>>equalTo(StructuralKey.of("foo", StringUtf8Coder.of())));
+  }
+
+  @Test
+  public void bundleEncodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoEncodeCoder()));
+    UncommittedBundle<Record> bundle = factory.createBundle(pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Encode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void bundleDecodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoDecodeCoder()));
+    UncommittedBundle<Record> bundle = factory.createBundle(pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Decode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void keyedBundleEncodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoEncodeCoder()));
+    UncommittedBundle<Record> bundle =
+        factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Encode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void keyedBundleDecodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoDecodeCoder()));
+    UncommittedBundle<Record> bundle =
+        factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Decode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
index e62bf01..e6bdbd0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -208,8 +208,8 @@ public class EncodabilityEnforcementFactoryTest {
         Collections.<CommittedBundle<?>>emptyList());
   }
 
-  private static class Record {}
-  private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
+  static class Record {}
+  static class RecordNoEncodeCoder extends AtomicCoder<Record> {
 
     @Override
     public void encode(
@@ -228,7 +228,7 @@ public class EncodabilityEnforcementFactoryTest {
     }
   }
 
-  private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
+  static class RecordNoDecodeCoder extends AtomicCoder<Record> {
     @Override
     public void encode(
         Record value,


[04/50] [abbrv] incubator-beam git commit: Make DoFnTester aggregator initialization idempotent

Posted by dh...@apache.org.
Make DoFnTester aggregator initialization idempotent


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

Branch: refs/heads/gearpump-runner
Commit: 043ebecacf7a8e96939b025afa8480c6df2f3b41
Parents: 2ab955d
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 13:35:29 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/transforms/DoFnTester.java    | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/043ebeca/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 302bb02..7995719 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -543,6 +543,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     private <AinT, AccT, AoutT> Aggregator<AinT, AoutT> aggregator(
         final String name,
         final CombineFn<AinT, AccT, AoutT> combiner) {
+
       Aggregator<AinT, AoutT> aggregator = new Aggregator<AinT, AoutT>() {
         @Override
         public void addValue(AinT value) {
@@ -561,7 +562,22 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
           return combiner;
         }
       };
-      accumulators.put(name, combiner.createAccumulator());
+
+      // Aggregator instantiation is idempotent
+      if (accumulators.containsKey(name)) {
+        Class<?> currentAccumClass = accumulators.get(name).getClass();
+        Class<?> createAccumClass = combiner.createAccumulator().getClass();
+        checkState(
+            currentAccumClass.isAssignableFrom(createAccumClass),
+            "Aggregator %s already initialized with accumulator type %s "
+                + "but was re-initialized with accumulator type %s",
+            name,
+            currentAccumClass,
+            createAccumClass);
+
+      } else {
+        accumulators.put(name, combiner.createAccumulator());
+      }
       return aggregator;
     }
 


[12/50] [abbrv] incubator-beam git commit: Choose BigQuery Write implementation based on Input PCollection

Posted by dh...@apache.org.
Choose BigQuery Write implementation based on Input PCollection

Stop using PipelineOptions, and instead use the boundedness of the input
to choose how to write to BigQuery. This means that runners that don't
use the streaming flag still function appropriately.

Fixes BEAM-746


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

Branch: refs/heads/gearpump-runner
Commit: 75d137b81ae240ad1a2e8942627738a6871581c1
Parents: 6d9d8bc
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 12 17:58:57 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:37:41 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  7 ++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 35 +++++++++++++++-----
 2 files changed, 31 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75d137b8/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 5626067..50c5ae9 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -119,6 +119,7 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
@@ -1748,9 +1749,9 @@ public class BigQueryIO {
         BigQueryOptions options = p.getOptions().as(BigQueryOptions.class);
         BigQueryServices bqServices = getBigQueryServices();
 
-        // In a streaming job, or when a tablespec function is defined, we use StreamWithDeDup
-        // and BigQuery's streaming import API.
-        if (options.isStreaming() || tableRefFunction != null) {
+        // When writing an Unbounded PCollection, or when a tablespec function is defined, we use
+        // StreamWithDeDup and BigQuery's streaming import API.
+        if (input.isBounded() == IsBounded.UNBOUNDED || tableRefFunction != null) {
           return input.apply(
               new StreamWithDeDup(getTable(), tableRefFunction, getSchema(), bqServices));
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75d137b8/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
index 74c35a6..51a69a2 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -91,6 +91,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.TableRowJsonCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.io.CountingSource;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryQuerySource;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryTableSource;
@@ -120,8 +121,10 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -1190,13 +1193,12 @@ public class BigQueryIOTest implements Serializable {
     assertThat(displayData, hasDisplayItem("validation", false));
   }
 
-  private void testWriteValidatesDataset(boolean streaming) throws Exception {
+  private void testWriteValidatesDataset(boolean unbounded) throws Exception {
     String projectId = "someproject";
     String datasetId = "somedataset";
 
     BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
     options.setProject(projectId);
-    options.setStreaming(streaming);
 
     FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
         .withJobService(mockJobService)
@@ -1210,17 +1212,34 @@ public class BigQueryIOTest implements Serializable {
     tableRef.setDatasetId(datasetId);
     tableRef.setTableId("sometable");
 
+    PCollection<TableRow> tableRows;
+    if (unbounded) {
+      tableRows =
+          p.apply(CountingInput.unbounded())
+              .apply(
+                  MapElements.via(
+                      new SimpleFunction<Long, TableRow>() {
+                        @Override
+                        public TableRow apply(Long input) {
+                          return null;
+                        }
+                      }))
+              .setCoder(TableRowJsonCoder.of());
+    } else {
+      tableRows = p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()));
+    }
+
     thrown.expect(RuntimeException.class);
     // Message will be one of following depending on the execution environment.
     thrown.expectMessage(
         Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
             .or(Matchers.containsString("BigQuery dataset not found for table")));
-    p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
-     .apply(BigQueryIO.Write
-         .to(tableRef)
-         .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
-         .withSchema(new TableSchema())
-         .withTestServices(fakeBqServices));
+    tableRows
+        .apply(
+            BigQueryIO.Write.to(tableRef)
+                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+                .withSchema(new TableSchema())
+                .withTestServices(fakeBqServices));
   }
 
   @Test


[23/50] [abbrv] incubator-beam git commit: Add timestamps to maven output on Travis CI

Posted by dh...@apache.org.
Add timestamps to maven output on Travis CI


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

Branch: refs/heads/gearpump-runner
Commit: 7f82a573d00a5a30331b7bbb8757e55f4a2d93ae
Parents: 44b9f4e
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 14:34:17 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 15:04:14 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f82a573/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index a9ba6d1..277f883 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -48,7 +48,9 @@ before_install:
   - cat ~/.m2/settings.xml
   - sed -i.bak -e 's|https://nexus.codehaus.org/snapshots/|https://oss.sonatype.org/content/repositories/codehaus-snapshots/|g' ~/.m2/settings.xml
   - cat ~/.m2/settings.xml
-  - echo "MAVEN_OPTS='-Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal'" > ~/.mavenrc
+  - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
+  - echo $'MAVEN_OPTS="$MAVEN_OPTS -Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS"' >> ~/.mavenrc
+  - cat ~/.mavenrc
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
   - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
   - export BEAM_SUREFIRE_ARGLINE="-Xmx512m"


[35/50] [abbrv] incubator-beam git commit: This closes #1095

Posted by dh...@apache.org.
This closes #1095


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

Branch: refs/heads/gearpump-runner
Commit: f603d43e043d383ceb00c4d786459f01d9983586
Parents: c03e3e9 2ceaa3e
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 11:04:13 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 11:04:13 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/CloningBundleFactory.java    |  98 ++++++++++
 .../beam/runners/direct/DirectRunner.java       |   5 +-
 .../direct/ImmutableListBundleFactory.java      |   4 +-
 .../direct/CloningBundleFactoryTest.java        | 177 +++++++++++++++++++
 .../EncodabilityEnforcementFactoryTest.java     |   6 +-
 5 files changed, 285 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[42/50] [abbrv] incubator-beam git commit: Remove use of OldDoFn from some DirectRunner tests

Posted by dh...@apache.org.
Remove use of OldDoFn from some DirectRunner tests


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

Branch: refs/heads/gearpump-runner
Commit: 3d086857de87734b087076dad3eca92f625bb417
Parents: 4051357
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 16:09:13 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../ConsumerTrackingPipelineVisitorTest.java    | 32 +++----
 .../beam/runners/direct/DirectRunnerTest.java   | 40 +++++----
 .../ImmutabilityCheckingBundleFactoryTest.java  |  8 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  8 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |  8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  8 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 87 +++++++++---------
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 94 +++++++++-----------
 .../runners/direct/WatermarkManagerTest.java    |  8 +-
 9 files changed, 139 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
index 1c9b5a6..e8f2a7e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
@@ -26,8 +26,8 @@ import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.View;
@@ -60,9 +60,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
         p.apply("listCreate", Create.of("foo", "bar"))
             .apply(
                 ParDo.of(
-                    new OldDoFn<String, String>() {
-                      @Override
-                      public void processElement(OldDoFn<String, String>.ProcessContext c)
+                    new DoFn<String, String>() {
+                      @ProcessElement
+                      public void processElement(DoFn<String, String>.ProcessContext c)
                           throws Exception {
                         c.output(Integer.toString(c.element().length()));
                       }
@@ -107,9 +107,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -138,9 +138,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -155,9 +155,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     p.apply(Create.of("1", "2", "3"))
         .apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -180,9 +180,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index 4027d25..34a5469 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -59,7 +59,6 @@ import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.SimpleFunction;
@@ -221,8 +220,8 @@ public class DirectRunnerTest implements Serializable {
 
   @Test
   public void transformDisplayDataExceptionShouldFail() {
-    OldDoFn<Integer, Integer> brokenDoFn = new OldDoFn<Integer, Integer>() {
-      @Override
+    DoFn<Integer, Integer> brokenDoFn = new DoFn<Integer, Integer>() {
+      @ProcessElement
       public void processElement(ProcessContext c) throws Exception {}
 
       @Override
@@ -242,7 +241,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -251,8 +250,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
             c.output(outputList);
             outputList.set(0, 37);
@@ -267,7 +267,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -276,8 +276,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
             c.output(outputList);
             outputList.set(0, 37);
@@ -291,7 +292,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a bad equals() still fails
+   * Tests that a {@link DoFn} that mutates an output with a bad equals() still fails
    * in the {@link DirectRunner}.
    */
   @Test
@@ -300,8 +301,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, byte[]>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, byte[]>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             byte[] outputArray = new byte[]{0x1, 0x2, 0x3};
             c.output(outputArray);
             outputArray[0] = 0xa;
@@ -316,7 +318,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates its input with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates its input with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -326,8 +328,9 @@ public class DirectRunnerTest implements Serializable {
     pipeline
         .apply(Create.of(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))
             .withCoder(ListCoder.of(VarIntCoder.of())))
-        .apply(ParDo.of(new OldDoFn<List<Integer>, Integer>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<List<Integer>, Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> inputList = c.element();
             inputList.set(0, 37);
             c.output(12);
@@ -341,7 +344,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an input with a bad equals() still fails
+   * Tests that a {@link DoFn} that mutates an input with a bad equals() still fails
    * in the {@link DirectRunner}.
    */
   @Test
@@ -350,8 +353,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(new byte[]{0x1, 0x2, 0x3}, new byte[]{0x4, 0x5, 0x6}))
-        .apply(ParDo.of(new OldDoFn<byte[], Integer>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<byte[], Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             byte[] inputArray = c.element();
             inputArray[0] = 0xa;
             c.output(13);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index d445944..ea44125 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -26,7 +26,7 @@ import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
@@ -179,9 +179,9 @@ public class ImmutabilityCheckingBundleFactoryTest {
     intermediate.commit(Instant.now());
   }
 
-  private static class IdentityDoFn<T> extends OldDoFn<T, T> {
-    @Override
-    public void processElement(OldDoFn<T, T>.ProcessContext c) throws Exception {
+  private static class IdentityDoFn<T> extends DoFn<T, T> {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       c.output(c.element());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
index 812d7d5..a7277fe 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -24,7 +24,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.util.IllegalMutationException;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -57,9 +57,9 @@ public class ImmutabilityEnforcementFactoryTest implements Serializable {
         p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
             .apply(
                 ParDo.of(
-                    new OldDoFn<byte[], byte[]>() {
-                      @Override
-                      public void processElement(OldDoFn<byte[], byte[]>.ProcessContext c)
+                    new DoFn<byte[], byte[]>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext c)
                           throws Exception {
                         c.element()[0] = 'b';
                       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
index ee6b2b4..cf65936 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
@@ -31,9 +31,9 @@ import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.Keys;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.KV;
@@ -177,9 +177,9 @@ public class KeyedPValueTrackingVisitorTest {
     }
   }
 
-  private static class IdentityFn<K> extends OldDoFn<K, K> {
-    @Override
-    public void processElement(OldDoFn<K, K>.ProcessContext c) throws Exception {
+  private static class IdentityFn<K> extends DoFn<K, K> {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       c.output(c.element());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
index 1a742f0..6d00aa1 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
@@ -37,7 +37,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -168,7 +168,7 @@ public class ParDoEvaluatorTest {
         ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, output));
   }
 
-  private static class RecorderFn extends OldDoFn<Integer, Integer> {
+  private static class RecorderFn extends DoFn<Integer, Integer> {
     private Collection<Integer> processed;
     private final PCollectionView<Integer> view;
 
@@ -177,8 +177,8 @@ public class ParDoEvaluatorTest {
       this.view = view;
     }
 
-    @Override
-    public void processElement(OldDoFn<Integer, Integer>.ProcessContext c) throws Exception {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       processed.add(c.element());
       c.output(c.element() + c.sideInput(view));
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index 8b0070b..cc83323 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -32,7 +32,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -41,11 +41,16 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
@@ -81,8 +86,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
+                new DoFn<String, KV<String, Integer>>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(KV.<String, Integer>of(c.element(), c.element().length()));
                     c.sideOutput(elementTag, c.element());
@@ -170,8 +175,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
+                new DoFn<String, KV<String, Integer>>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(KV.<String, Integer>of(c.element(), c.element().length()));
                     c.sideOutput(elementTag, c.element());
@@ -258,20 +263,17 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
         StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(StateNamespaces.global(), watermarkTag)
-                        .add(new Instant(20202L + c.element().length()));
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(
-                            StateNamespaces.window(
-                                GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                            bagTag)
-                        .add(c.element());
+                new DoFn<String, KV<String, Integer>>() {
+                  private static final String STATE_ID = "my-state-id";
+
+                  @StateId(STATE_ID)
+                  private final StateSpec<Object, BagState<String>> bagSpec =
+                      StateSpecs.bag(StringUtf8Coder.of());
+
+                  @ProcessElement
+                  public void processElement(
+                      ProcessContext c, @StateId(STATE_ID) BagState<String> bagState) {
+                    bagState.add(c.element());
                   }
                 })
             .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
@@ -362,34 +364,25 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals().stateInternals();
-                    c.windowingInternals()
-                        .timerInternals()
-                        .setTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0).plus(Duration.standardMinutes(5)),
-                                        new Instant(1)
-                                            .plus(Duration.standardMinutes(5))
-                                            .plus(Duration.standardHours(1)))),
-                                new Instant(54541L),
-                                TimeDomain.EVENT_TIME));
-                    c.windowingInternals()
-                        .timerInternals()
-                        .deleteTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0),
-                                        new Instant(0).plus(Duration.standardHours(1)))),
-                                new Instant(3400000),
-                                TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+                new DoFn<String, KV<String, Integer>>() {
+                  private static final String EVENT_TIME_TIMER = "event-time-timer";
+                  private static final String SYNC_PROC_TIME_TIMER = "sync-proc-time-timer";
+
+                  @TimerId(EVENT_TIME_TIMER)
+                  TimerSpec myTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+                  @TimerId(SYNC_PROC_TIME_TIMER)
+                  TimerSpec syncProcTimerSpec =
+                      TimerSpecs.timer(TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+                  @ProcessElement
+                  public void processElement(
+                      ProcessContext c,
+                      @TimerId(EVENT_TIME_TIMER) Timer eventTimeTimer,
+                      @TimerId(SYNC_PROC_TIME_TIMER) Timer syncProcTimeTimer) {
+
+                    eventTimeTimer.setForNowPlus(Duration.standardMinutes(5));
+                    syncProcTimeTimer.cancel();
                   }
                 })
             .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index e562b28..d22643a 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -32,22 +32,25 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
@@ -74,8 +77,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     PCollection<Integer> collection =
         input.apply(
             ParDo.of(
-                new OldDoFn<String, Integer>() {
-                  @Override
+                new DoFn<String, Integer>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(c.element().length());
                   }
@@ -128,8 +131,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     PCollection<Integer> collection =
         input.apply(
             ParDo.of(
-                new OldDoFn<String, Integer>() {
-                  @Override
+                new DoFn<String, Integer>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.sideOutput(sideOutputTag, c.element().length());
                   }
@@ -178,26 +181,22 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
 
-    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
-        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
     final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
     final StateNamespace windowNs =
         StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
     ParDo.Bound<String, KV<String, Integer>> pardo =
         ParDo.of(
-            new OldDoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(StateNamespaces.global(), watermarkTag)
-                    .add(new Instant(124443L - c.element().length()));
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(
-                        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                        bagTag)
-                    .add(c.element());
+            new DoFn<String, KV<String, Integer>>() {
+              private static final String STATE_ID = "my-state-id";
+
+              @StateId(STATE_ID)
+              private final StateSpec<Object, BagState<String>> bagSpec =
+                  StateSpecs.bag(StringUtf8Coder.of());
+
+              @ProcessElement
+              public void processElement(
+                  ProcessContext c, @StateId(STATE_ID) BagState<String> bagState) {
+                bagState.add(c.element());
               }
             });
     PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
@@ -237,9 +236,6 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
     assertThat(result.getState(), not(nullValue()));
     assertThat(
-        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
-        equalTo(new Instant(124438L)));
-    assertThat(
         result.getState().state(windowNs, bagTag).read(),
         containsInAnyOrder("foo", "bara", "bazam"));
   }
@@ -255,6 +251,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
 
+    // TODO: this timer data is absolute, but the new API only support relative settings.
+    // It will require adjustments when @Ignore is removed
     final TimerData addedTimer =
         TimerData.of(
             StateNamespaces.window(
@@ -276,34 +274,24 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     ParDo.Bound<String, KV<String, Integer>> pardo =
         ParDo.of(
-            new OldDoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals().stateInternals();
-                c.windowingInternals()
-                    .timerInternals()
-                    .setTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0).plus(Duration.standardMinutes(5)),
-                                    new Instant(1)
-                                        .plus(Duration.standardMinutes(5))
-                                        .plus(Duration.standardHours(1)))),
-                            new Instant(54541L),
-                            TimeDomain.EVENT_TIME));
-                c.windowingInternals()
-                    .timerInternals()
-                    .deleteTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0),
-                                    new Instant(0).plus(Duration.standardHours(1)))),
-                            new Instant(3400000),
-                            TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+            new DoFn<String, KV<String, Integer>>() {
+              private static final String EVENT_TIME_TIMER = "event-time-timer";
+              private static final String SYNC_PROC_TIME_TIMER = "sync-proc-time-timer";
+
+              @TimerId(EVENT_TIME_TIMER)
+              TimerSpec myTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+              @TimerId(SYNC_PROC_TIME_TIMER)
+              TimerSpec syncProcTimerSpec =
+                  TimerSpecs.timer(TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+              @ProcessElement
+              public void processElement(
+                  ProcessContext c,
+                  @TimerId(EVENT_TIME_TIMER) Timer eventTimeTimer,
+                  @TimerId(SYNC_PROC_TIME_TIMER) Timer syncProcTimeTimer) {
+                eventTimeTimer.setForNowPlus(Duration.standardMinutes(5));
+                syncProcTimeTimer.cancel();
               }
             });
     PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
index 042abab..1954005 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -47,9 +47,9 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -101,9 +101,9 @@ public class WatermarkManagerTest implements Serializable {
     createdInts = p.apply("createdInts", Create.of(1, 2, 3));
 
     filtered = createdInts.apply("filtered", Filter.greaterThan(1));
-    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new OldDoFn<Integer, Integer>() {
-      @Override
-      public void processElement(OldDoFn<Integer, Integer>.ProcessContext c) throws Exception {
+    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
+      @ProcessElement
+      public void processElement(ProcessContext c) throws Exception {
         c.output(c.element() * 2);
       }
     }));


[25/50] [abbrv] incubator-beam git commit: pom.xml: make Dataflow runner like every other runner

Posted by dh...@apache.org.
pom.xml: make Dataflow runner like every other runner

Pom dependencies no longer require it to be treated specially.


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

Branch: refs/heads/gearpump-runner
Commit: 6291a6f674a98b8898d807e72ebf2ad5cc5d3ce7
Parents: d285e85
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 13:16:01 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 18:03:28 2016 -0700

----------------------------------------------------------------------
 pom.xml         | 3 ---
 runners/pom.xml | 1 +
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6291a6f6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2338f55..df388d4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -139,9 +139,6 @@
 
   <modules>
     <module>sdks</module>
-    <!-- Expose Dataflow runner as top level module to satisfy dependencies in sdks/java/maven-archetypes 
-      and examples/java. Until these are refactored out, we need to modify the build order. -->
-    <module>runners/google-cloud-dataflow-java</module>
     <module>runners</module>
     <!-- sdks/java/maven-archetypes has several dependencies on the DataflowPipelineRunner. 
       Until these are refactored out or a released artifact exists, we need to modify the 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6291a6f6/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 68cf29c..b2b613f 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -36,6 +36,7 @@
     <module>core-java</module>
     <module>direct-java</module>
     <module>flink</module>
+    <module>google-cloud-dataflow-java</module>
     <module>spark</module>
   </modules>
 


[30/50] [abbrv] incubator-beam git commit: Update DataflowRunner worker image

Posted by dh...@apache.org.
Update DataflowRunner worker image


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

Branch: refs/heads/gearpump-runner
Commit: 90c30cba9d5beaf19de5f90419df353a0511f8b5
Parents: 1eff320
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 15:36:45 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 10:12:56 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/dataflow/DataflowRunner.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90c30cba/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 89c364e..2324196 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -208,9 +208,9 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   // Default Docker container images that execute Dataflow worker harness, residing in Google
   // Container Registry, separately for Batch and Streaming.
   public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE =
-      "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161017";
+      "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161024";
   public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE =
-      "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161017";
+      "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161024";
 
   // The limit of CreateJob request size.
   private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;


[49/50] [abbrv] incubator-beam git commit: upgrade gearpump-runner to 0.4.0-incubating-SNAPSHOT

Posted by dh...@apache.org.
upgrade gearpump-runner to 0.4.0-incubating-SNAPSHOT


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

Branch: refs/heads/gearpump-runner
Commit: 3f06382d20a7b3bc686d46d834e489bdc79625b7
Parents: 94bd47c
Author: manuzhang <ow...@gmail.com>
Authored: Wed Oct 26 20:15:35 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Wed Oct 26 20:15:35 2016 +0800

----------------------------------------------------------------------
 runners/gearpump/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f06382d/runners/gearpump/pom.xml
----------------------------------------------------------------------
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index 6c104eb..abd135f 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[47/50] [abbrv] incubator-beam git commit: post-merge fix

Posted by dh...@apache.org.
post-merge fix


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

Branch: refs/heads/gearpump-runner
Commit: 8f013cb76fb85421da00eb8df0074dac0a8233fa
Parents: 9dc9be9
Author: manuzhang <ow...@gmail.com>
Authored: Wed Oct 26 11:22:56 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Wed Oct 26 14:02:54 2016 +0800

----------------------------------------------------------------------
 .../beam/runners/core/SimpleDoFnRunner.java     | 586 +++++++++++++++++++
 .../beam/runners/direct/DirectRunner.java       |  22 -
 .../beam/runners/direct/KeyedResourcePool.java  |  47 --
 .../runners/direct/LockedKeyedResourcePool.java |  95 ---
 .../direct/ParDoSingleEvaluatorFactory.java     |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |  37 --
 .../direct/LockedKeyedResourcePoolTest.java     | 163 ------
 .../flink/examples/streaming/package-info.java  |  22 +
 .../apache/beam/runners/flink/package-info.java |  22 -
 runners/gearpump/pom.xml                        |   2 +-
 .../gearpump/GearpumpPipelineResult.java        |   6 +
 .../gearpump/GearpumpPipelineRunner.java        |   2 -
 .../translators/TransformTranslator.java        |  30 +
 .../translators/io/UnboundedSourceWrapper.java  |  45 ++
 .../translators/utils/GearpumpDoFnRunner.java   |  16 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    |   1 -
 .../runners/spark/translation/DoFnFunction.java |   2 -
 .../main/java/org/apache/beam/sdk/Pipeline.java |   2 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  15 +-
 .../apache/beam/sdk/runners/PipelineRunner.java |   1 -
 .../apache/beam/sdk/testing/TestPipeline.java   |   2 -
 .../org/apache/beam/sdk/transforms/DoFn.java    |  14 -
 .../org/apache/beam/sdk/transforms/ParDo.java   |  12 -
 .../beam/sdk/transforms/LatestFnTests.java      | 233 --------
 .../beam/sdk/io/kinesis/package-info.java       |  22 -
 .../beam/sdk/io/mongodb/package-info.java       |  22 -
 26 files changed, 703 insertions(+), 720 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
new file mode 100644
index 0000000..dec9905
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -0,0 +1,586 @@
+/*
+ * 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.runners.core;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.InputProvider;
+import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+/**
+ * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the {@link DoFn} (main) input elements
+ * @param <OutputT> the type of the {@link DoFn} (main) output elements
+ */
+public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+  /** The {@link DoFn} being run. */
+  private final DoFn<InputT, OutputT> fn;
+
+  /** The {@link DoFnInvoker} being run. */
+  private final DoFnInvoker<InputT, OutputT> invoker;
+
+  /** The context used for running the {@link DoFn}. */
+  private final DoFnContext<InputT, OutputT> context;
+
+  private final OutputManager outputManager;
+
+  private final TupleTag<OutputT> mainOutputTag;
+
+  private final boolean observesWindow;
+
+  public SimpleDoFnRunner(
+      PipelineOptions options,
+      DoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    this.fn = fn;
+    this.observesWindow =
+        DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().observesWindow();
+    this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn);
+    this.outputManager = outputManager;
+    this.mainOutputTag = mainOutputTag;
+    this.context =
+        new DoFnContext<>(
+            options,
+            fn,
+            sideInputReader,
+            outputManager,
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            aggregatorFactory,
+            windowingStrategy == null ? null : windowingStrategy.getWindowFn());
+  }
+
+  @Override
+  public void startBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeStartBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> compressedElem) {
+    if (observesWindow) {
+      for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {
+        invokeProcessElement(elem);
+      }
+    } else {
+      invokeProcessElement(compressedElem);
+    }
+  }
+
+  private void invokeProcessElement(WindowedValue<InputT> elem) {
+    final DoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
+
+    // Note that if the element must be exploded into all its windows, that has to be done outside
+    // of this runner.
+    final DoFn.ExtraContextFactory<InputT, OutputT> extraContextFactory =
+        createExtraContextFactory(elem);
+
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeProcessElement(processContext, extraContextFactory);
+    } catch (Exception ex) {
+      throw wrapUserCodeException(ex);
+    }
+  }
+
+  @Override
+  public void finishBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeFinishBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  /** Returns a new {@link DoFn.ProcessContext} for the given element. */
+  private DoFn<InputT, OutputT>.ProcessContext createProcessContext(WindowedValue<InputT> elem) {
+    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
+  }
+
+  private DoFn.ExtraContextFactory<InputT, OutputT> createExtraContextFactory(
+      WindowedValue<InputT> elem) {
+    return new DoFnExtraContextFactory<InputT, OutputT>(elem.getWindows(), elem.getPane());
+  }
+
+  private RuntimeException wrapUserCodeException(Throwable t) {
+    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
+  }
+
+  private boolean isSystemDoFn() {
+    return invoker.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+  }
+
+  /**
+   * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}.
+   *
+   * @param <InputT> the type of the {@link DoFn} (main) input elements
+   * @param <OutputT> the type of the {@link DoFn} (main) output elements
+   */
+  private static class DoFnContext<InputT, OutputT> extends DoFn<InputT, OutputT>.Context {
+    private static final int MAX_SIDE_OUTPUTS = 1000;
+
+    final PipelineOptions options;
+    final DoFn<InputT, OutputT> fn;
+    final SideInputReader sideInputReader;
+    final OutputManager outputManager;
+    final TupleTag<OutputT> mainOutputTag;
+    final StepContext stepContext;
+    final AggregatorFactory aggregatorFactory;
+    final WindowFn<?, ?> windowFn;
+
+    /**
+     * The set of known output tags, some of which may be undeclared, so we can throw an exception
+     * when it exceeds {@link #MAX_SIDE_OUTPUTS}.
+     */
+    private Set<TupleTag<?>> outputTags;
+
+    public DoFnContext(
+        PipelineOptions options,
+        DoFn<InputT, OutputT> fn,
+        SideInputReader sideInputReader,
+        OutputManager outputManager,
+        TupleTag<OutputT> mainOutputTag,
+        List<TupleTag<?>> sideOutputTags,
+        StepContext stepContext,
+        AggregatorFactory aggregatorFactory,
+        WindowFn<?, ?> windowFn) {
+      fn.super();
+      this.options = options;
+      this.fn = fn;
+      this.sideInputReader = sideInputReader;
+      this.outputManager = outputManager;
+      this.mainOutputTag = mainOutputTag;
+      this.outputTags = Sets.newHashSet();
+
+      outputTags.add(mainOutputTag);
+      for (TupleTag<?> sideOutputTag : sideOutputTags) {
+        outputTags.add(sideOutputTag);
+      }
+
+      this.stepContext = stepContext;
+      this.aggregatorFactory = aggregatorFactory;
+      this.windowFn = windowFn;
+      super.setupDelegateAggregators();
+    }
+
+    //////////////////////////////////////////////////////////////////////////////
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
+        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
+      final Instant inputTimestamp = timestamp;
+
+      if (timestamp == null) {
+        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      }
+
+      if (windows == null) {
+        try {
+          // The windowFn can never succeed at accessing the element, so its type does not
+          // matter here
+          @SuppressWarnings("unchecked")
+          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
+          windows =
+              objectWindowFn.assignWindows(
+                  objectWindowFn.new AssignContext() {
+                    @Override
+                    public Object element() {
+                      throw new UnsupportedOperationException(
+                          "WindowFn attempted to access input element when none was available");
+                    }
+
+                    @Override
+                    public Instant timestamp() {
+                      if (inputTimestamp == null) {
+                        throw new UnsupportedOperationException(
+                            "WindowFn attempted to access input timestamp when none was available");
+                      }
+                      return inputTimestamp;
+                    }
+
+                    @Override
+                    public W window() {
+                      throw new UnsupportedOperationException(
+                          "WindowFn attempted to access input windows when none were available");
+                    }
+                  });
+        } catch (Exception e) {
+          throw UserCodeException.wrap(e);
+        }
+      }
+
+      return WindowedValue.of(output, timestamp, windows, pane);
+    }
+
+    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+      if (!sideInputReader.contains(view)) {
+        throw new IllegalArgumentException("calling sideInput() with unknown view");
+      }
+      BoundedWindow sideInputWindow =
+          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+      return sideInputReader.get(view, sideInputWindow);
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
+      outputManager.output(mainOutputTag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteOutput(windowedElem);
+      }
+    }
+
+    private <T> void sideOutputWindowedValue(
+        TupleTag<T> tag,
+        T output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    private <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
+      if (!outputTags.contains(tag)) {
+        // This tag wasn't declared nor was it seen before during this execution.
+        // Thus, this must be a new, undeclared and unconsumed output.
+        // To prevent likely user errors, enforce the limit on the number of side
+        // outputs.
+        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
+          throw new IllegalArgumentException(
+              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
+        }
+        outputTags.add(tag);
+      }
+
+      outputManager.output(tag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteSideOutput(tag, windowedElem);
+      }
+    }
+
+    // Following implementations of output, outputWithTimestamp, and sideOutput
+    // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by
+    // ProcessContext's versions in DoFn.processElement.
+    @Override
+    public void output(OutputT output) {
+      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      checkNotNull(combiner, "Combiner passed to createAggregator cannot be null");
+      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
+    }
+  }
+
+  /**
+   * A concrete implementation of {@link DoFn.ProcessContext} used for running a {@link DoFn} over a
+   * single element.
+   *
+   * @param <InputT> the type of the {@link DoFn} (main) input elements
+   * @param <OutputT> the type of the {@link DoFn} (main) output elements
+   */
+  private static class DoFnProcessContext<InputT, OutputT>
+      extends DoFn<InputT, OutputT>.ProcessContext {
+
+    final DoFn<InputT, OutputT> fn;
+    final DoFnContext<InputT, OutputT> context;
+    final WindowedValue<InputT> windowedValue;
+
+    public DoFnProcessContext(
+        DoFn<InputT, OutputT> fn,
+        DoFnContext<InputT, OutputT> context,
+        WindowedValue<InputT> windowedValue) {
+      fn.super();
+      this.fn = fn;
+      this.context = context;
+      this.windowedValue = windowedValue;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public InputT element() {
+      return windowedValue.getValue();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      checkNotNull(view, "View passed to sideInput cannot be null");
+      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
+      BoundedWindow window;
+      if (!windowIter.hasNext()) {
+        if (context.windowFn instanceof GlobalWindows) {
+          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
+          // without windows
+          window = GlobalWindow.INSTANCE;
+        } else {
+          throw new IllegalStateException(
+              "sideInput called when main input element is not in any windows");
+        }
+      } else {
+        window = windowIter.next();
+        if (windowIter.hasNext()) {
+          throw new IllegalStateException(
+              "sideInput called when main input element is in multiple windows");
+        }
+      }
+      return context.sideInput(view, window);
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return windowedValue.getPane();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.outputWindowedValue(windowedValue.withValue(output));
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      checkTimestamp(timestamp);
+      context.outputWindowedValue(
+          output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      context.outputWindowedValue(output, timestamp, windows, pane);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkTimestamp(timestamp);
+      context.sideOutputWindowedValue(
+          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    @Override
+    public Instant timestamp() {
+      return windowedValue.getTimestamp();
+    }
+
+    public Collection<? extends BoundedWindow> windows() {
+      return windowedValue.getWindows();
+    }
+
+    private void checkTimestamp(Instant timestamp) {
+      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+                    + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+                    + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed "
+                    + "skew.",
+                timestamp,
+                windowedValue.getTimestamp(),
+                PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
+      }
+    }
+
+    @Override
+    protected <AggregatorInputT, AggregatorOutputT>
+        Aggregator<AggregatorInputT, AggregatorOutputT> createAggregator(
+            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
+      return context.createAggregator(name, combiner);
+    }
+  }
+
+  private class DoFnExtraContextFactory<InputT, OutputT>
+      implements DoFn.ExtraContextFactory<InputT, OutputT> {
+
+    /** The windows of the current element. */
+    private final Collection<? extends BoundedWindow> windows;
+
+    /** The pane of the current element. */
+    private final PaneInfo pane;
+
+    public DoFnExtraContextFactory(Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+      this.windows = windows;
+      this.pane = pane;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return Iterables.getOnlyElement(windows);
+    }
+
+    @Override
+    public InputProvider<InputT> inputProvider() {
+      throw new UnsupportedOperationException("InputProvider parameters are not supported.");
+    }
+
+    @Override
+    public OutputReceiver<OutputT> outputReceiver() {
+      throw new UnsupportedOperationException("OutputReceiver parameters are not supported.");
+    }
+
+    @Override
+    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+      throw new UnsupportedOperationException("RestrictionTracker parameters are not supported.");
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return new WindowingInternals<InputT, OutputT>() {
+        @Override
+        public Collection<? extends BoundedWindow> windows() {
+          return windows;
+        }
+
+        @Override
+        public PaneInfo pane() {
+          return pane;
+        }
+
+        @Override
+        public TimerInternals timerInternals() {
+          return context.stepContext.timerInternals();
+        }
+
+        @Override
+        public <T> void writePCollectionViewData(
+            TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder)
+            throws IOException {
+          @SuppressWarnings("unchecked")
+          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
+
+          context.stepContext.writePCollectionViewData(
+              tag,
+              data,
+              IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
+              window(),
+              windowCoder);
+        }
+
+        @Override
+        public StateInternals<?> stateInternals() {
+          return context.stepContext.stateInternals();
+        }
+
+        @Override
+        public void outputWindowedValue(
+            OutputT output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {}
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+          return context.sideInput(view, mainInputWindow);
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index f87f1c1..e02c8a6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -465,26 +465,4 @@ public class DirectRunner
       return NanosOffsetClock.create();
     }
   }
-
-  /**
-   * A {@link Supplier} that creates a {@link ExecutorService} based on
-   * {@link Executors#newFixedThreadPool(int)}.
-   */
-  private static class FixedThreadPoolSupplier implements Supplier<ExecutorService> {
-    @Override
-    public ExecutorService get() {
-      return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
-    }
-  }
-
-
-  /**
-   * A {@link Supplier} that creates a {@link NanosOffsetClock}.
-   */
-  private static class NanosOffsetClockSupplier implements Supplier<Clock> {
-    @Override
-    public Clock get() {
-      return NanosOffsetClock.create();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
deleted file mode 100644
index b976b69..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.runners.direct;
-
-import com.google.common.base.Optional;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-
-/**
- * A pool of resources associated with specific keys. Implementations enforce specific use patterns,
- * such as limiting the the number of outstanding elements available per key.
- */
-interface KeyedResourcePool<K, V> {
-  /**
-   * Tries to acquire a value for the provided key, loading it via the provided loader if necessary.
-   *
-   * <p>If the returned {@link Optional} contains a value, the caller obtains ownership of that
-   * value. The value should be released back to this {@link KeyedResourcePool} after the
-   * caller no longer has use of it using {@link #release(Object, Object)}.
-   *
-   * <p>The provided {@link Callable} <b>must not</b> return null; it may either return a non-null
-   * value or throw an exception.
-   */
-  Optional<V> tryAcquire(K key, Callable<V> loader) throws ExecutionException;
-
-  /**
-   * Release the provided value, relinquishing ownership of it. Future calls to
-   * {@link #tryAcquire(Object, Callable)} may return the released value.
-   */
-  void release(K key, V value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
deleted file mode 100644
index 8b1e0b1..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.runners.direct;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ExecutionError;
-import com.google.common.util.concurrent.UncheckedExecutionException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-
-/**
- * A {@link KeyedResourcePool} which is limited to at most one outstanding instance at a time for
- * each key.
- */
-class LockedKeyedResourcePool<K, V> implements KeyedResourcePool<K, V> {
-  /**
-   * A map from each key to an {@link Optional} of the associated value. At most one value is stored
-   * per key, and it is obtained by at most one thread at a time.
-   *
-   * <p>For each key in this map:
-   *
-   * <ul>
-   * <li>If there is no associated value, then no value has been stored yet.
-   * <li>If the value is {@code Optional.absent()} then the value is currently in use.
-   * <li>If the value is {@code Optional.present()} then the contained value is available for use.
-   * </ul>
-   */
-  public static <K, V> LockedKeyedResourcePool<K, V> create() {
-    return new LockedKeyedResourcePool<>();
-  }
-
-  private final ConcurrentMap<K, Optional<V>> cache;
-
-  private LockedKeyedResourcePool() {
-    cache = new ConcurrentHashMap<>();
-  }
-
-  @Override
-  public Optional<V> tryAcquire(K key, Callable<V> loader) throws ExecutionException {
-    Optional<V> value = cache.replace(key, Optional.<V>absent());
-    if (value == null) {
-      // No value already existed, so populate the cache with the value returned by the loader
-      cache.putIfAbsent(key, Optional.of(load(loader)));
-      // Some other thread may obtain the result after the putIfAbsent, so retry acquisition
-      value = cache.replace(key, Optional.<V>absent());
-    }
-    return value;
-  }
-
-  private V load(Callable<V> loader) throws ExecutionException {
-    try {
-      return loader.call();
-    } catch (Error t) {
-      throw new ExecutionError(t);
-    } catch (RuntimeException e) {
-      throw new UncheckedExecutionException(e);
-    } catch (Exception e) {
-      throw new ExecutionException(e);
-    }
-  }
-
-  @Override
-  public void release(K key, V value) {
-    Optional<V> replaced = cache.replace(key, Optional.of(value));
-    checkNotNull(replaced, "Tried to release before a value was acquired");
-    checkState(
-        !replaced.isPresent(),
-        "Released a value to a %s where there is already a value present for key %s (%s). "
-            + "At most one value may be present at a time.",
-        LockedKeyedResourcePool.class.getSimpleName(),
-        key,
-        replaced);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
index 9a08e8f..0584e41 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -91,7 +91,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
               stepContext,
               inputBundle,
               application,
-              (OldDoFn) fnLocal.get(),
+              fnLocal.get(),
               application.getTransform().getSideInputs(),
               mainOutputTag,
               Collections.<TupleTag<?>>emptyList(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index 6485714..3dd44a7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -116,41 +116,4 @@ class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
       throw toThrow;
     }
   }
-
-  @Override
-  public void cleanup() throws Exception {
-    Collection<Exception> thrownInCleanup = new ArrayList<>();
-    for (TransformEvaluatorFactory factory : factories.values()) {
-      try {
-        factory.cleanup();
-      } catch (Exception e) {
-        if (e instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        thrownInCleanup.add(e);
-      }
-    }
-    finished.set(true);
-    if (!thrownInCleanup.isEmpty()) {
-      LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup);
-      Exception toThrow = null;
-      for (Exception e : thrownInCleanup) {
-        if (toThrow == null) {
-          toThrow = e;
-        } else {
-          toThrow.addSuppressed(e);
-        }
-      }
-      throw toThrow;
-    }
-  }
-
-  /**
-   * A factory to create Transform Evaluator Registries.
-   */
-  public static class Factory {
-    public TransformEvaluatorRegistry create() {
-      return TransformEvaluatorRegistry.defaultRegistry();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java
deleted file mode 100644
index e1e24a3..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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.runners.direct;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ExecutionError;
-import com.google.common.util.concurrent.UncheckedExecutionException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link LockedKeyedResourcePool}.
- */
-@RunWith(JUnit4.class)
-public class LockedKeyedResourcePoolTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private LockedKeyedResourcePool<String, Integer> cache =
-      LockedKeyedResourcePool.create();
-
-  @Test
-  public void acquireReleaseAcquireLastLoadedOrReleased() throws ExecutionException {
-    Optional<Integer> returned = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 3;
-      }
-    });
-    assertThat(returned.get(), equalTo(3));
-
-    cache.release("foo", 4);
-    Optional<Integer> reacquired = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 5;
-      }
-    });
-    assertThat(reacquired.get(), equalTo(4));
-  }
-
-  @Test
-  public void acquireReleaseReleaseThrows() throws ExecutionException {
-    Optional<Integer> returned = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 3;
-      }
-    });
-    assertThat(returned.get(), equalTo(3));
-
-    cache.release("foo", 4);
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("already a value present");
-    thrown.expectMessage("At most one");
-    cache.release("foo", 4);
-  }
-
-  @Test
-  public void releaseBeforeAcquireThrows() {
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("before a value was acquired");
-    cache.release("bar", 3);
-  }
-
-  @Test
-  public void multipleAcquireWithoutReleaseAbsent() throws ExecutionException {
-    Optional<Integer> returned = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 3;
-      }
-    });
-    Optional<Integer> secondReturned = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 3;
-      }
-    });
-    assertThat(secondReturned.isPresent(), is(false));
-  }
-
-  @Test
-  public void acquireMultipleKeysSucceeds() throws ExecutionException {
-    Optional<Integer> returned = cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 3;
-      }
-    });
-    Optional<Integer> secondReturned = cache.tryAcquire("bar", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        return 4;
-      }
-    });
-
-    assertThat(returned.get(), equalTo(3));
-    assertThat(secondReturned.get(), equalTo(4));
-  }
-
-  @Test
-  public void acquireThrowsExceptionWrapped() throws ExecutionException {
-    final Exception cause = new Exception("checkedException");
-    thrown.expect(ExecutionException.class);
-    thrown.expectCause(equalTo(cause));
-    cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        throw cause;
-      }
-    });
-  }
-
-  @Test
-  public void acquireThrowsRuntimeExceptionWrapped() throws ExecutionException {
-    final RuntimeException cause = new RuntimeException("UncheckedException");
-    thrown.expect(UncheckedExecutionException.class);
-    thrown.expectCause(equalTo(cause));
-    cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        throw cause;
-      }
-    });
-  }
-
-  @Test
-  public void acquireThrowsErrorWrapped() throws ExecutionException {
-    final Error cause = new Error("Error");
-    thrown.expect(ExecutionError.class);
-    thrown.expectCause(equalTo(cause));
-    cache.tryAcquire("foo", new Callable<Integer>() {
-      @Override
-      public Integer call() throws Exception {
-        throw cause;
-      }
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
new file mode 100644
index 0000000..58f41b6
--- /dev/null
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Flink Beam runner exemple.
+ */
+package org.apache.beam.runners.flink.examples.streaming;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
deleted file mode 100644
index 57f1e59..0000000
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Internal implementation of the Beam runner for Apache Flink.
- */
-package org.apache.beam.runners.flink;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/pom.xml
----------------------------------------------------------------------
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index 6576ba6..6c104eb 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -225,7 +225,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <type>test-jar</type>
+      <classifier>tests</classifier>
       <scope>test</scope>
       <exclusions>
         <exclusion>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
index 2011a4b..e7c621e 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
@@ -23,6 +23,7 @@ import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.AggregatorValues;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.sdk.transforms.Aggregator;
 
 import org.joda.time.Duration;
@@ -60,4 +61,9 @@ public class GearpumpPipelineResult implements PipelineResult {
         new UnsupportedOperationException());
   }
 
+  @Override
+  public MetricResults metrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
index ad7bb3e..9e32227 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
@@ -53,8 +53,6 @@ import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
  * A {@link PipelineRunner} that executes the operations in the
  * pipeline by first translating them to Gearpump Stream DSL
  * and then executing them on a Gearpump cluster.
- * <p>>
- * This is based on DataflowPipelineRunner.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class GearpumpPipelineRunner extends PipelineRunner<GearpumpPipelineResult> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
new file mode 100644
index 0000000..c8587d3
--- /dev/null
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
@@ -0,0 +1,30 @@
+/*
+ * 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.runners.gearpump.translators;
+
+import java.io.Serializable;
+
+import org.apache.beam.sdk.transforms.PTransform;
+
+/**
+ * translates {@link PTransform} to Gearpump functions.
+ */
+public interface TransformTranslator<T extends PTransform> extends Serializable {
+  void translate(T transform, TranslationContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
new file mode 100644
index 0000000..dfdecb2
--- /dev/null
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
@@ -0,0 +1,45 @@
+/*
+ * 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.runners.gearpump.translators.io;
+
+import java.io.IOException;
+
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * wrapper over UnboundedSource for Gearpump DataSource API.
+ */
+public class UnboundedSourceWrapper<OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+    extends GearpumpSource<OutputT> {
+
+  private final UnboundedSource<OutputT, CheckpointMarkT> source;
+
+  public UnboundedSourceWrapper(UnboundedSource<OutputT, CheckpointMarkT> source,
+      PipelineOptions options) {
+    super(options);
+    this.source = source;
+  }
+
+  @Override
+  protected Source.Reader<OutputT> createReader(PipelineOptions options) throws IOException {
+    return source.createReader(options, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
index e205575..ec86a8d 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
@@ -18,7 +18,8 @@
 
 package org.apache.beam.runners.gearpump.translators.utils;
 
-import com.google.common.base.Preconditions;
+import static org.apache.beam.sdk.repackaged.com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
@@ -59,6 +60,7 @@ import org.apache.beam.sdk.values.TupleTag;
 
 import org.joda.time.Instant;
 
+
 /**
  * a serializable {@link SimpleDoFnRunner}.
  */
@@ -330,20 +332,20 @@ public class GearpumpDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, O
 
     @Override
     public <T> void sideOutput(TupleTag<T> tag, T output) {
-      Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
       sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
     }
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
       sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
     }
 
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      Preconditions.checkNotNull(combiner,
+      checkNotNull(combiner,
           "Combiner passed to createAggregator cannot be null");
       throw new UnsupportedOperationException("aggregator not supported in Gearpump runner");
     }
@@ -386,7 +388,7 @@ public class GearpumpDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, O
 
     @Override
     public <T> T sideInput(PCollectionView<T> view) {
-      Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
+      checkNotNull(view, "View passed to sideInput cannot be null");
       Iterator<? extends BoundedWindow> windowIter = windows().iterator();
       BoundedWindow window;
       if (!windowIter.hasNext()) {
@@ -435,13 +437,13 @@ public class GearpumpDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, O
 
     @Override
     public <T> void sideOutput(TupleTag<T> tag, T output) {
-      Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
       context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
     }
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
       context.sideOutputWindowedValue(
           tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
index b84a1a8..b211c04 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
@@ -77,4 +77,3 @@ public class DoFnInfo<InputT, OutputT> implements Serializable {
     return outputMap;
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
index 69c450e..4dfbee6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java
@@ -45,8 +45,6 @@ public class DoFnFunction<InputT, OutputT>
     implements FlatMapFunction<Iterator<WindowedValue<InputT>>, WindowedValue<OutputT>> {
   private final Accumulator<NamedAggregators> accum;
   private final OldDoFn<InputT, OutputT> mFunction;
-  private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class);
-
   private final SparkRuntimeContext mRuntimeContext;
   private final Map<TupleTag<?>, KV<WindowingStrategy<?, ?>, BroadcastHelper<?>>> mSideInputs;
   private final WindowFn<Object, ?> windowFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index 0776786..2286832 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -134,7 +134,7 @@ public class Pipeline {
    */
   public static Pipeline create(PipelineOptions options) {
     Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options);
-    LOG.info("Creating {}", pipeline);
+    LOG.debug("Creating {}", pipeline);
     return pipeline;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 78ea988..2dbcda7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -23,6 +23,7 @@ import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -33,20 +34,8 @@ import java.nio.channels.WritableByteChannel;
 import java.nio.charset.StandardCharsets;
 import java.util.NoSuchElementException;
 import java.util.regex.Pattern;
+
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.Coder.Context;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.io.Read.Bounded;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.MimeTypes;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
 
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.Context;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
index 1ec4103..ede1507 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
@@ -57,7 +57,6 @@ public abstract class PipelineRunner<ResultT extends PipelineResult> {
         .fromFactoryMethod("fromOptions")
         .withArg(PipelineOptions.class, options)
         .build();
-    System.out.println("runner: " + result.getClass().getName());
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 3202000..f1bf09d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -95,7 +95,6 @@ public class TestPipeline extends Pipeline {
   }
 
   public static TestPipeline fromOptions(PipelineOptions options) {
-    System.out.println(options);
     return new TestPipeline(PipelineRunner.fromOptions(options), options);
   }
 
@@ -134,7 +133,6 @@ public class TestPipeline extends Pipeline {
       @Nullable String beamTestPipelineOptions =
           System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS);
 
-      System.out.println("options " + beamTestPipelineOptions);
       PipelineOptions options =
           Strings.isNullOrEmpty(beamTestPipelineOptions)
               ? PipelineOptionsFactory.create()

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index f2fa87c..018877f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -423,20 +423,6 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
 
   /////////////////////////////////////////////////////////////////////////////
 
-
-  /**
-   * Annotation for the method to use to prepare an instance for processing bundles of elements. The
-   * method annotated with this must satisfy the following constraints
-   * <ul>
-   *   <li>It must have zero arguments.
-   * </ul>
-   */
-  @Documented
-  @Retention(RetentionPolicy.RUNTIME)
-  @Target(ElementType.METHOD)
-  public @interface Setup {
-  }
-
   /**
    * Annotation for declaring and dereferencing state cells.
    *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index a7dc136..a3a306a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -670,18 +670,6 @@ public class ParDo {
 
     /**
      * Returns a new {@link ParDo} {@link PTransform} that's like this
-     * transform but which will invoke the given {@link DoFn}
-     * function, and which has its input and output types bound. Does
-     * not modify this transform. The resulting {@link PTransform} is
-     * sufficiently specified to be applied, but more properties can
-     * still be specified.
-     */
-    public <InputT, OutputT> Bound<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
-      return of(adapt(fn), fn.getClass());
-    }
-
-    /**
-     * Returns a new {@link ParDo} {@link PTransform} that's like this
      * transform but that will invoke the given {@link OldDoFn}
      * function, and that has its input and output types bound. Does
      * not modify this transform. The resulting {@link PTransform} is

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
deleted file mode 100644
index 84b5b68..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * 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.transforms;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.isOneOf;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertEquals;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Objects;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.NullableCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Unit tests for {@link Latest.LatestFn}.
- * */
-@RunWith(JUnit4.class)
-public class LatestFnTests {
-  private static final Instant INSTANT = new Instant(100);
-  private static final long VALUE = 100 * INSTANT.getMillis();
-
-  private static final TimestampedValue<Long> TV = TimestampedValue.of(VALUE, INSTANT);
-  private static final TimestampedValue<Long> TV_MINUS_TEN =
-      TimestampedValue.of(VALUE - 10, INSTANT.minus(10));
-  private static final TimestampedValue<Long> TV_PLUS_TEN =
-      TimestampedValue.of(VALUE + 10, INSTANT.plus(10));
-
-  @Rule
-  public final ExpectedException thrown = ExpectedException.none();
-
-  private final Latest.LatestFn<Long> fn = new Latest.LatestFn<>();
-  private final Instant baseTimestamp = Instant.now();
-
-  @Test
-  public void testDefaultValue() {
-    assertThat(fn.defaultValue(), nullValue());
-  }
-
-  @Test
-  public void testCreateAccumulator() {
-    assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.createAccumulator());
-  }
-
-  @Test
-  public void testAddInputInitialAdd() {
-    TimestampedValue<Long> input = TV;
-    assertEquals(input, fn.addInput(fn.createAccumulator(), input));
-  }
-
-  @Test
-  public void testAddInputMinTimestamp() {
-    TimestampedValue<Long> input = TimestampedValue.atMinimumTimestamp(1234L);
-    assertEquals(input, fn.addInput(fn.createAccumulator(), input));
-  }
-
-  @Test
-  public void testAddInputEarlierValue() {
-    assertEquals(TV, fn.addInput(TV, TV_MINUS_TEN));
-  }
-
-  @Test
-  public void testAddInputLaterValue() {
-    assertEquals(TV_PLUS_TEN, fn.addInput(TV, TV_PLUS_TEN));
-  }
-
-  @Test
-  public void testAddInputSameTimestamp() {
-    TimestampedValue<Long> accum = TimestampedValue.of(100L, INSTANT);
-    TimestampedValue<Long> input = TimestampedValue.of(200L, INSTANT);
-
-    assertThat("Latest for values with the same timestamp is chosen arbitrarily",
-        fn.addInput(accum, input), isOneOf(accum, input));
-  }
-
-  @Test
-  public void testAddInputNullAccumulator() {
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("accumulators");
-    fn.addInput(null, TV);
-  }
-
-  @Test
-  public void testAddInputNullInput() {
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("input");
-    fn.addInput(TV, null);
-  }
-
-  @Test
-  public void testAddInputNullValue() {
-    TimestampedValue<Long> input = TimestampedValue.of(null, INSTANT.plus(10));
-    assertEquals("Null values are allowed", input, fn.addInput(TV, input));
-  }
-
-  @Test
-  public void testMergeAccumulatorsMultipleValues() {
-    Iterable<TimestampedValue<Long>> accums = Lists.newArrayList(
-        TV,
-        TV_PLUS_TEN,
-        TV_MINUS_TEN
-    );
-
-    assertEquals(TV_PLUS_TEN, fn.mergeAccumulators(accums));
-  }
-
-  @Test
-  public void testMergeAccumulatorsSingleValue() {
-    assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV)));
-  }
-
-  @Test
-  public void testMergeAccumulatorsEmptyIterable() {
-    ArrayList<TimestampedValue<Long>> emptyAccums = Lists.newArrayList();
-    assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.mergeAccumulators(emptyAccums));
-  }
-
-  @Test
-  public void testMergeAccumulatorsDefaultAccumulator() {
-    TimestampedValue<Long> defaultAccum = fn.createAccumulator();
-    assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV, defaultAccum)));
-  }
-
-  @Test
-  public void testMergeAccumulatorsAllDefaultAccumulators() {
-    TimestampedValue<Long> defaultAccum = fn.createAccumulator();
-    assertEquals(defaultAccum, fn.mergeAccumulators(
-        Lists.newArrayList(defaultAccum, defaultAccum)));
-  }
-
-  @Test
-  public void testMergeAccumulatorsNullIterable() {
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("accumulators");
-    fn.mergeAccumulators(null);
-  }
-
-  @Test
-  public void testExtractOutput() {
-    assertEquals(TV.getValue(), fn.extractOutput(TV));
-  }
-
-  @Test
-  public void testExtractOutputDefaultAggregator() {
-    TimestampedValue<Long> accum = fn.createAccumulator();
-    assertThat(fn.extractOutput(accum), nullValue());
-  }
-
-  @Test
-  public void testExtractOutputNullValue() {
-    TimestampedValue<Long> accum = TimestampedValue.of(null, baseTimestamp);
-    assertEquals(null, fn.extractOutput(accum));
-  }
-
-  @Test
-  public void testAggregator() throws Exception {
-    LatestAggregatorsFn<Long> doFn = new LatestAggregatorsFn<>(TV_MINUS_TEN.getValue());
-    DoFnTester<Long, Long> harness = DoFnTester.of(doFn);
-    for (TimestampedValue<Long> element : Arrays.asList(TV, TV_PLUS_TEN, TV_MINUS_TEN)) {
-      harness.processTimestampedElement(element);
-    }
-
-    assertEquals(TV_PLUS_TEN.getValue(), harness.getAggregatorValue(doFn.allValuesAgg));
-    assertEquals(TV_MINUS_TEN.getValue(), harness.getAggregatorValue(doFn.specialValueAgg));
-    assertThat(harness.getAggregatorValue(doFn.noValuesAgg), nullValue());
-  }
-
-  @Test
-  public void testDefaultCoderHandlesNull() throws CannotProvideCoderException {
-    Latest.LatestFn<Long> fn = new Latest.LatestFn<>();
-
-    CoderRegistry registry = new CoderRegistry();
-    TimestampedValue.TimestampedValueCoder<Long> inputCoder =
-        TimestampedValue.TimestampedValueCoder.of(VarLongCoder.of());
-
-    assertThat("Default output coder should handle null values",
-        fn.getDefaultOutputCoder(registry, inputCoder), instanceOf(NullableCoder.class));
-    assertThat("Default accumulator coder should handle null values",
-        fn.getAccumulatorCoder(registry, inputCoder), instanceOf(NullableCoder.class));
-  }
-
-  static class LatestAggregatorsFn<T> extends DoFn<T, T> {
-    private final T specialValue;
-    LatestAggregatorsFn(T specialValue) {
-      this.specialValue = specialValue;
-    }
-
-    Aggregator<TimestampedValue<T>, T> allValuesAgg =
-        createAggregator("allValues", new Latest.LatestFn<T>());
-
-    Aggregator<TimestampedValue<T>, T> specialValueAgg =
-        createAggregator("oneValue", new Latest.LatestFn<T>());
-
-    Aggregator<TimestampedValue<T>, T> noValuesAgg =
-        createAggregator("noValues", new Latest.LatestFn<T>());
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      TimestampedValue<T> val = TimestampedValue.of(c.element(), c.timestamp());
-      allValuesAgg.addValue(val);
-      if (Objects.equals(c.element(), specialValue)) {
-        specialValueAgg.addValue(val);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java
deleted file mode 100644
index 44dbf4a..0000000
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Transforms for reading and writing from Amazon Kinesis.
- */
-package org.apache.beam.sdk.io.kinesis;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f013cb7/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
deleted file mode 100644
index fd08b58..0000000
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Transforms for reading and writing from MongoDB.
- */
-package org.apache.beam.sdk.io.mongodb;


[28/50] [abbrv] incubator-beam git commit: Remove pieces of Trigger now owned by TriggerStateMachine

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
deleted file mode 100644
index e09aac2..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
+++ /dev/null
@@ -1,507 +0,0 @@
-/*
- * 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.util;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import java.util.Collection;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.Trigger.MergingTriggerInfo;
-import org.apache.beam.sdk.transforms.windowing.Trigger.TriggerInfo;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.joda.time.Instant;
-
-/**
- * Factory for creating instances of the various {@link Trigger} contexts.
- *
- * <p>These contexts are highly interdependent and share many fields; it is inadvisable
- * to create them via any means other than this factory class.
- */
-public class TriggerContextFactory<W extends BoundedWindow> {
-
-  private final WindowFn<?, W> windowFn;
-  private StateInternals<?> stateInternals;
-  private final Coder<W> windowCoder;
-
-  public TriggerContextFactory(WindowFn<?, W> windowFn,
-      StateInternals<?> stateInternals, ActiveWindowSet<W> activeWindows) {
-    // Future triggers may be able to exploit the active window to state address window mapping.
-    this.windowFn = windowFn;
-    this.stateInternals = stateInternals;
-    this.windowCoder = windowFn.windowCoder();
-  }
-
-  public Trigger.TriggerContext base(W window, Timers timers,
-      ExecutableTrigger rootTrigger, FinishedTriggers finishedSet) {
-    return new TriggerContextImpl(window, timers, rootTrigger, finishedSet);
-  }
-
-  public Trigger.OnElementContext createOnElementContext(
-      W window, Timers timers, Instant elementTimestamp,
-      ExecutableTrigger rootTrigger, FinishedTriggers finishedSet) {
-    return new OnElementContextImpl(window, timers, rootTrigger, finishedSet, elementTimestamp);
-  }
-
-  public Trigger.OnMergeContext createOnMergeContext(W window, Timers timers,
-      ExecutableTrigger rootTrigger, FinishedTriggers finishedSet,
-      Map<W, FinishedTriggers> finishedSets) {
-    return new OnMergeContextImpl(window, timers, rootTrigger, finishedSet, finishedSets);
-  }
-
-  public StateAccessor<?> createStateAccessor(W window, ExecutableTrigger trigger) {
-    return new StateAccessorImpl(window, trigger);
-  }
-
-  public MergingStateAccessor<?, W> createMergingStateAccessor(
-      W mergeResult, Collection<W> mergingWindows, ExecutableTrigger trigger) {
-    return new MergingStateAccessorImpl(trigger, mergingWindows, mergeResult);
-  }
-
-  private class TriggerInfoImpl implements Trigger.TriggerInfo {
-
-    protected final ExecutableTrigger trigger;
-    protected final FinishedTriggers finishedSet;
-    private final Trigger.TriggerContext context;
-
-    public TriggerInfoImpl(ExecutableTrigger trigger, FinishedTriggers finishedSet,
-        Trigger.TriggerContext context) {
-      this.trigger = trigger;
-      this.finishedSet = finishedSet;
-      this.context = context;
-    }
-
-    @Override
-    public boolean isMerging() {
-      return !windowFn.isNonMerging();
-    }
-
-    @Override
-    public Iterable<ExecutableTrigger> subTriggers() {
-      return trigger.subTriggers();
-    }
-
-    @Override
-    public ExecutableTrigger subTrigger(int subtriggerIndex) {
-      return trigger.subTriggers().get(subtriggerIndex);
-    }
-
-    @Override
-    public boolean isFinished() {
-      return finishedSet.isFinished(trigger);
-    }
-
-    @Override
-    public boolean isFinished(int subtriggerIndex) {
-      return finishedSet.isFinished(subTrigger(subtriggerIndex));
-    }
-
-    @Override
-    public boolean areAllSubtriggersFinished() {
-      return Iterables.isEmpty(unfinishedSubTriggers());
-    }
-
-    @Override
-    public Iterable<ExecutableTrigger> unfinishedSubTriggers() {
-      return FluentIterable
-          .from(trigger.subTriggers())
-          .filter(new Predicate<ExecutableTrigger>() {
-            @Override
-            public boolean apply(ExecutableTrigger trigger) {
-              return !finishedSet.isFinished(trigger);
-            }
-          });
-    }
-
-    @Override
-    public ExecutableTrigger firstUnfinishedSubTrigger() {
-      for (ExecutableTrigger subTrigger : trigger.subTriggers()) {
-        if (!finishedSet.isFinished(subTrigger)) {
-          return subTrigger;
-        }
-      }
-      return null;
-    }
-
-    @Override
-    public void resetTree() throws Exception {
-      finishedSet.clearRecursively(trigger);
-      trigger.invokeClear(context);
-    }
-
-    @Override
-    public void setFinished(boolean finished) {
-      finishedSet.setFinished(trigger, finished);
-    }
-
-    @Override
-    public void setFinished(boolean finished, int subTriggerIndex) {
-      finishedSet.setFinished(subTrigger(subTriggerIndex), finished);
-    }
-  }
-
-  private class TriggerTimers implements Timers {
-
-    private final Timers timers;
-    private final W window;
-
-    public TriggerTimers(W window, Timers timers) {
-      this.timers = timers;
-      this.window = window;
-    }
-
-    @Override
-    public void setTimer(Instant timestamp, TimeDomain timeDomain) {
-      timers.setTimer(timestamp, timeDomain);
-    }
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
-      if (timeDomain == TimeDomain.EVENT_TIME
-          && timestamp.equals(window.maxTimestamp())) {
-        // Don't allow triggers to unset the at-max-timestamp timer. This is necessary for on-time
-        // state transitions.
-        return;
-      }
-      timers.deleteTimer(timestamp, timeDomain);
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timers.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timers.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    public Instant currentEventTime() {
-      return timers.currentEventTime();
-    }
-  }
-
-  private class MergingTriggerInfoImpl
-      extends TriggerInfoImpl implements Trigger.MergingTriggerInfo {
-
-    private final Map<W, FinishedTriggers> finishedSets;
-
-    public MergingTriggerInfoImpl(
-        ExecutableTrigger trigger,
-        FinishedTriggers finishedSet,
-        Trigger.TriggerContext context,
-        Map<W, FinishedTriggers> finishedSets) {
-      super(trigger, finishedSet, context);
-      this.finishedSets = finishedSets;
-    }
-
-    @Override
-    public boolean finishedInAnyMergingWindow() {
-      for (FinishedTriggers finishedSet : finishedSets.values()) {
-        if (finishedSet.isFinished(trigger)) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public boolean finishedInAllMergingWindows() {
-      for (FinishedTriggers finishedSet : finishedSets.values()) {
-        if (!finishedSet.isFinished(trigger)) {
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-
-  private class StateAccessorImpl implements StateAccessor<Object> {
-    protected final int triggerIndex;
-    protected final StateNamespace windowNamespace;
-
-    public StateAccessorImpl(
-        W window,
-        ExecutableTrigger trigger) {
-      this.triggerIndex = trigger.getTriggerIndex();
-      this.windowNamespace = namespaceFor(window);
-    }
-
-    protected StateNamespace namespaceFor(W window) {
-      return StateNamespaces.windowAndTrigger(windowCoder, window, triggerIndex);
-    }
-
-    @Override
-    public <StateT extends State> StateT access(StateTag<? super Object, StateT> address) {
-      return stateInternals.state(windowNamespace, address);
-    }
-  }
-
-  private class MergingStateAccessorImpl extends StateAccessorImpl
-  implements MergingStateAccessor<Object, W> {
-    private final Collection<W> activeToBeMerged;
-
-    public MergingStateAccessorImpl(ExecutableTrigger trigger, Collection<W> activeToBeMerged,
-        W mergeResult) {
-      super(mergeResult, trigger);
-      this.activeToBeMerged = activeToBeMerged;
-    }
-
-    @Override
-    public <StateT extends State> StateT access(
-        StateTag<? super Object, StateT> address) {
-      return stateInternals.state(windowNamespace, address);
-    }
-
-    @Override
-    public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
-        StateTag<? super Object, StateT> address) {
-      ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
-      for (W mergingWindow : activeToBeMerged) {
-        StateT stateForWindow = stateInternals.state(namespaceFor(mergingWindow), address);
-        builder.put(mergingWindow, stateForWindow);
-      }
-      return builder.build();
-    }
-  }
-
-  private class TriggerContextImpl extends Trigger.TriggerContext {
-
-    private final W window;
-    private final StateAccessorImpl state;
-    private final Timers timers;
-    private final TriggerInfoImpl triggerInfo;
-
-    private TriggerContextImpl(
-        W window,
-        Timers timers,
-        ExecutableTrigger trigger,
-        FinishedTriggers finishedSet) {
-      trigger.getSpec().super();
-      this.window = window;
-      this.state = new StateAccessorImpl(window, trigger);
-      this.timers = new TriggerTimers(window, timers);
-      this.triggerInfo = new TriggerInfoImpl(trigger, finishedSet, this);
-    }
-
-    @Override
-    public Trigger.TriggerContext forTrigger(ExecutableTrigger trigger) {
-      return new TriggerContextImpl(window, timers, trigger, triggerInfo.finishedSet);
-    }
-
-    @Override
-    public TriggerInfo trigger() {
-      return triggerInfo;
-    }
-
-    @Override
-    public StateAccessor<?> state() {
-      return state;
-    }
-
-    @Override
-    public W window() {
-      return window;
-    }
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain domain) {
-      timers.deleteTimer(timestamp, domain);
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timers.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timers.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentEventTime() {
-      return timers.currentEventTime();
-    }
-  }
-
-  private class OnElementContextImpl extends Trigger.OnElementContext {
-
-    private final W window;
-    private final StateAccessorImpl state;
-    private final Timers timers;
-    private final TriggerInfoImpl triggerInfo;
-    private final Instant eventTimestamp;
-
-    private OnElementContextImpl(
-        W window,
-        Timers timers,
-        ExecutableTrigger trigger,
-        FinishedTriggers finishedSet,
-        Instant eventTimestamp) {
-      trigger.getSpec().super();
-      this.window = window;
-      this.state = new StateAccessorImpl(window, trigger);
-      this.timers = new TriggerTimers(window, timers);
-      this.triggerInfo = new TriggerInfoImpl(trigger, finishedSet, this);
-      this.eventTimestamp = eventTimestamp;
-    }
-
-
-    @Override
-    public Instant eventTimestamp() {
-      return eventTimestamp;
-    }
-
-    @Override
-    public Trigger.OnElementContext forTrigger(ExecutableTrigger trigger) {
-      return new OnElementContextImpl(
-          window, timers, trigger, triggerInfo.finishedSet, eventTimestamp);
-    }
-
-    @Override
-    public TriggerInfo trigger() {
-      return triggerInfo;
-    }
-
-    @Override
-    public StateAccessor<?> state() {
-      return state;
-    }
-
-    @Override
-    public W window() {
-      return window;
-    }
-
-    @Override
-    public void setTimer(Instant timestamp, TimeDomain domain) {
-      timers.setTimer(timestamp, domain);
-    }
-
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain domain) {
-      timers.deleteTimer(timestamp, domain);
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timers.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timers.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentEventTime() {
-      return timers.currentEventTime();
-    }
-  }
-
-  private class OnMergeContextImpl extends Trigger.OnMergeContext {
-    private final MergingStateAccessor<?, W> state;
-    private final W window;
-    private final Collection<W> mergingWindows;
-    private final Timers timers;
-    private final MergingTriggerInfoImpl triggerInfo;
-
-    private OnMergeContextImpl(
-        W window,
-        Timers timers,
-        ExecutableTrigger trigger,
-        FinishedTriggers finishedSet,
-        Map<W, FinishedTriggers> finishedSets) {
-      trigger.getSpec().super();
-      this.mergingWindows = finishedSets.keySet();
-      this.window = window;
-      this.state = new MergingStateAccessorImpl(trigger, mergingWindows, window);
-      this.timers = new TriggerTimers(window, timers);
-      this.triggerInfo = new MergingTriggerInfoImpl(trigger, finishedSet, this, finishedSets);
-    }
-
-    @Override
-    public Trigger.OnMergeContext forTrigger(ExecutableTrigger trigger) {
-      return new OnMergeContextImpl(
-          window, timers, trigger, triggerInfo.finishedSet, triggerInfo.finishedSets);
-    }
-
-    @Override
-    public MergingStateAccessor<?, W> state() {
-      return state;
-    }
-
-    @Override
-    public MergingTriggerInfo trigger() {
-      return triggerInfo;
-    }
-
-    @Override
-    public W window() {
-      return window;
-    }
-
-    @Override
-    public void setTimer(Instant timestamp, TimeDomain domain) {
-      timers.setTimer(timestamp, domain);
-    }
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain domain) {
-      timers.setTimer(timestamp, domain);
-
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timers.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timers.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentEventTime() {
-      return timers.currentEventTime();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
index b591229..5b213c7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
@@ -18,13 +18,8 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -36,99 +31,6 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class AfterAllTest {
 
-  private SimpleTriggerTester<IntervalWindow> tester;
-
-  @Test
-  public void testT1FiresFirst() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterAll.of(
-            AfterPane.elementCountAtLeast(1),
-            AfterPane.elementCountAtLeast(2)),
-        FixedWindows.of(Duration.millis(100)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  @Test
-  public void testT2FiresFirst() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterAll.of(
-            AfterPane.elementCountAtLeast(2),
-            AfterPane.elementCountAtLeast(1)),
-        FixedWindows.of(Duration.millis(100)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  /**
-   * Tests that the AfterAll properly unsets finished bits when a merge causing it to become
-   * unfinished.
-   */
-  @Test
-  public void testOnMergeRewinds() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            AfterAll.of(
-                AfterWatermark.pastEndOfWindow(),
-                AfterPane.elementCountAtLeast(1)),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(1))),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-
-    tester.injectElements(5);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-
-    // Finish the AfterAll in the first window
-    tester.advanceInputWatermark(new Instant(11));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Merge them; the AfterAll should not be finished
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-    assertFalse(tester.isMarkedFinished(mergedWindow));
-
-    // Confirm that we are back on the first trigger by probing that it is not ready to fire
-    // after an element (with merging)
-    tester.injectElements(3);
-    tester.mergeWindows();
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // Fire the AfterAll in the merged window
-    tester.advanceInputWatermark(new Instant(15));
-    assertTrue(tester.shouldFire(mergedWindow));
-    tester.fireIfShouldFire(mergedWindow);
-
-    // Confirm that we are on the second trigger by probing
-    tester.injectElements(2);
-    tester.mergeWindows();
-    assertTrue(tester.shouldFire(mergedWindow));
-    tester.fireIfShouldFire(mergedWindow);
-    tester.injectElements(2);
-    tester.mergeWindows();
-    assertTrue(tester.shouldFire(mergedWindow));
-    tester.fireIfShouldFire(mergedWindow);
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
index c413c6e..00d25e0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
@@ -18,19 +18,12 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.MockitoAnnotations;
 
 /**
  * Tests for {@link AfterEach}.
@@ -38,63 +31,6 @@ import org.mockito.MockitoAnnotations;
 @RunWith(JUnit4.class)
 public class AfterEachTest {
 
-  private SimpleTriggerTester<IntervalWindow> tester;
-
-  @Before
-  public void initMocks() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  /**
-   * Tests that the {@link AfterEach} trigger fires and finishes the first trigger then the second.
-   */
-  @Test
-  public void testAfterEachInSequence() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            Repeatedly.forever(AfterPane.elementCountAtLeast(2))
-                .orFinally(AfterPane.elementCountAtLeast(3)),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(5))
-                .orFinally(AfterWatermark.pastEndOfWindow())),
-            FixedWindows.of(Duration.millis(10)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-
-    // AfterCount(2) not ready
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    // AfterCount(2) ready, not finished
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // orFinally(AfterCount(3)) ready and will finish the first
-    tester.injectElements(1, 2, 3);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // Now running as the second trigger
-    assertFalse(tester.shouldFire(window));
-    // This quantity of elements would fire and finish if it were erroneously still the first
-    tester.injectElements(1, 2, 3, 4);
-    assertFalse(tester.shouldFire(window));
-
-    // Now fire once
-    tester.injectElements(5);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // This time advance the watermark to finish the whole mess.
-    tester.advanceInputWatermark(new Instant(10));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
index 415060b..2887edb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
@@ -18,22 +18,12 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.when;
 
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
 
 /**
  * Tests for {@link AfterFirst}.
@@ -41,116 +31,6 @@ import org.mockito.MockitoAnnotations;
 @RunWith(JUnit4.class)
 public class AfterFirstTest {
 
-  @Mock private OnceTrigger mockTrigger1;
-  @Mock private OnceTrigger mockTrigger2;
-  private SimpleTriggerTester<IntervalWindow> tester;
-  private static Trigger.TriggerContext anyTriggerContext() {
-    return Mockito.<Trigger.TriggerContext>any();
-  }
-
-  @Before
-  public void initMocks() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  @Test
-  public void testNeitherShouldFireFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterFirst.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10)));
-
-    tester.injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-
-    when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(false);
-    when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(false);
-
-    assertFalse(tester.shouldFire(window)); // should not fire
-    assertFalse(tester.isMarkedFinished(window)); // not finished
-  }
-
-  @Test
-  public void testOnlyT1ShouldFireFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterFirst.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10)));
-    tester.injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11));
-
-    when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(true);
-    when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(false);
-
-    assertTrue(tester.shouldFire(window)); // should fire
-
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  @Test
-  public void testOnlyT2ShouldFireFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-    AfterFirst.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10)));
-    tester.injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11));
-
-    when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(false);
-    when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(true);
-    assertTrue(tester.shouldFire(window)); // should fire
-
-    tester.fireIfShouldFire(window); // now finished
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  @Test
-  public void testBothShouldFireFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-    AfterFirst.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10)));
-    tester.injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11));
-
-    when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(true);
-    when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(true);
-    assertTrue(tester.shouldFire(window)); // should fire
-
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  /**
-   * Tests that if the first trigger rewinds to be non-finished in the merged window,
-   * then it becomes the currently active trigger again, with real triggers.
-   */
-  @Test
-  public void testShouldFireAfterMerge() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            AfterFirst.of(AfterPane.elementCountAtLeast(5),
-                AfterWatermark.pastEndOfWindow()),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(1))),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    // Finished the AfterFirst in the first window
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    assertFalse(tester.shouldFire(firstWindow));
-    tester.advanceInputWatermark(new Instant(11));
-    assertTrue(tester.shouldFire(firstWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Set up second window where it is not done
-    tester.injectElements(5);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Merge them, if the merged window were on the second trigger, it would be ready
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // Now adding 3 more makes the AfterFirst ready to fire
-    tester.injectElements(1, 2, 3, 4, 5);
-    tester.mergeWindows();
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
index 38d030e..1bff80a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
@@ -18,12 +18,7 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -35,78 +30,6 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class AfterPaneTest {
 
-  SimpleTriggerTester<IntervalWindow> tester;
-  /**
-   * Tests that the trigger does fire when enough elements are in a window, and that it only
-   * fires that window (no leakage).
-   */
-  @Test
-  public void testAfterPaneElementCountFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterPane.elementCountAtLeast(2),
-        FixedWindows.of(Duration.millis(10)));
-
-    tester.injectElements(1); // [0, 10)
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-    assertFalse(tester.shouldFire(window));
-
-    tester.injectElements(2); // [0, 10)
-    tester.injectElements(11); // [10, 20)
-
-    assertTrue(tester.shouldFire(window)); // ready to fire
-    tester.fireIfShouldFire(window); // and finished
-    assertTrue(tester.isMarkedFinished(window));
-
-    // But don't finish the other window
-    assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20))));
-  }
-
-  @Test
-  public void testClear() throws Exception {
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterPane.elementCountAtLeast(2),
-        FixedWindows.of(Duration.millis(10)));
-
-    tester.injectElements(1, 2, 3);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-    tester.clearState(window);
-    tester.assertCleared(window);
-  }
-
-  @Test
-  public void testAfterPaneElementCountSessions() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterPane.elementCountAtLeast(2),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(
-        1, // in [1, 11)
-        2); // in [2, 12)
-
-    assertFalse(tester.shouldFire(new IntervalWindow(new Instant(1), new Instant(11))));
-    assertFalse(tester.shouldFire(new IntervalWindow(new Instant(2), new Instant(12))));
-
-    tester.mergeWindows();
-
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(12));
-    assertTrue(tester.shouldFire(mergedWindow));
-    tester.fireIfShouldFire(mergedWindow);
-    assertTrue(tester.isMarkedFinished(mergedWindow));
-
-    // Because we closed the previous window, we don't have it around to merge with. So there
-    // will be a new FIRE_AND_FINISH result.
-    tester.injectElements(
-        7,  // in [7, 17)
-        9); // in [9, 19)
-
-    tester.mergeWindows();
-
-    IntervalWindow newMergedWindow = new IntervalWindow(new Instant(7), new Instant(19));
-    assertTrue(tester.shouldFire(newMergedWindow));
-    tester.fireIfShouldFire(newMergedWindow);
-    assertTrue(tester.isMarkedFinished(newMergedWindow));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
index 13a7acf..4984d7c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
@@ -18,12 +18,9 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
@@ -36,97 +33,6 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class AfterProcessingTimeTest {
 
-  /**
-   * Tests the basic property that the trigger does wait for processing time to be
-   * far enough advanced.
-   */
-  @Test
-  public void testAfterProcessingTimeFixedWindows() throws Exception {
-    Duration windowDuration = Duration.millis(10);
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.millis(5)),
-        FixedWindows.of(windowDuration));
-
-    tester.advanceProcessingTime(new Instant(10));
-
-    // Timer at 15
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
-    tester.advanceProcessingTime(new Instant(12));
-    assertFalse(tester.shouldFire(firstWindow));
-
-    // Load up elements in the next window, timer at 17 for them
-    tester.injectElements(11, 12, 13);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Not quite time to fire
-    tester.advanceProcessingTime(new Instant(14));
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Timer at 19 for these in the first window; it should be ignored since the 15 will fire first
-    tester.injectElements(2, 3);
-
-    // Advance past the first timer and fire, finishing the first window
-    tester.advanceProcessingTime(new Instant(16));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    assertTrue(tester.isMarkedFinished(firstWindow));
-
-    // The next window fires and finishes now
-    tester.advanceProcessingTime(new Instant(18));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(secondWindow);
-    assertTrue(tester.isMarkedFinished(secondWindow));
-  }
-
-  /**
-   * Tests that when windows merge, if the trigger is waiting for "N millis after the first
-   * element" that it is relative to the earlier of the two merged windows.
-   */
-  @Test
-  public void testClear() throws Exception {
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.millis(5)),
-        FixedWindows.of(Duration.millis(10)));
-
-    tester.injectElements(1, 2, 3);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-    tester.clearState(window);
-    tester.assertCleared(window);
-  }
-
-  @Test
-  public void testAfterProcessingTimeWithMergingWindow() throws Exception {
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.millis(5)),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.advanceProcessingTime(new Instant(10));
-    tester.injectElements(1); // in [1, 11), timer for 15
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    assertFalse(tester.shouldFire(firstWindow));
-
-    tester.advanceProcessingTime(new Instant(12));
-    tester.injectElements(3); // in [3, 13), timer for 17
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(3), new Instant(13));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(13));
-
-    tester.advanceProcessingTime(new Instant(16));
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
index 7e6e938..49d44c5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
@@ -18,12 +18,7 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -38,76 +33,6 @@ public class AfterSynchronizedProcessingTimeTest {
   private Trigger underTest = new AfterSynchronizedProcessingTime();
 
   @Test
-  public void testAfterProcessingTimeWithFixedWindows() throws Exception {
-    Duration windowDuration = Duration.millis(10);
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.millis(5)),
-        FixedWindows.of(windowDuration));
-
-    tester.advanceProcessingTime(new Instant(10));
-
-    // Timer at 15
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
-    tester.advanceProcessingTime(new Instant(12));
-    assertFalse(tester.shouldFire(firstWindow));
-
-    // Load up elements in the next window, timer at 17 for them
-    tester.injectElements(11, 12, 13);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Not quite time to fire
-    tester.advanceProcessingTime(new Instant(14));
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Timer at 19 for these in the first window; it should be ignored since the 15 will fire first
-    tester.injectElements(2, 3);
-
-    // Advance past the first timer and fire, finishing the first window
-    tester.advanceProcessingTime(new Instant(16));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    assertTrue(tester.isMarkedFinished(firstWindow));
-
-    // The next window fires and finishes now
-    tester.advanceProcessingTime(new Instant(18));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(secondWindow);
-    assertTrue(tester.isMarkedFinished(secondWindow));
-  }
-
-  @Test
-  public void testAfterProcessingTimeWithMergingWindow() throws Exception {
-    Duration windowDuration = Duration.millis(10);
-    SimpleTriggerTester<IntervalWindow> tester = TriggerTester.forTrigger(
-        AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.millis(5)),
-        Sessions.withGapDuration(windowDuration));
-
-    tester.advanceProcessingTime(new Instant(10));
-    tester.injectElements(1); // in [1, 11), timer for 15
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    assertFalse(tester.shouldFire(firstWindow));
-
-    tester.advanceProcessingTime(new Instant(12));
-    tester.injectElements(3); // in [3, 13), timer for 17
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(3), new Instant(13));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(13));
-
-    tester.advanceProcessingTime(new Instant(16));
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  @Test
   public void testFireDeadline() throws Exception {
     assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
         underTest.getWatermarkThatGuaranteesFiring(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
index 084027b..a418d63 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
@@ -18,23 +18,10 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.when;
 
-import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
 
 /**
  * Tests the {@link AfterWatermark} triggers.
@@ -42,301 +29,6 @@ import org.mockito.MockitoAnnotations;
 @RunWith(JUnit4.class)
 public class AfterWatermarkTest {
 
-  @Mock private OnceTrigger mockEarly;
-  @Mock private OnceTrigger mockLate;
-
-  private SimpleTriggerTester<IntervalWindow> tester;
-  private static Trigger.TriggerContext anyTriggerContext() {
-    return Mockito.<Trigger.TriggerContext>any();
-  }
-  private static Trigger.OnElementContext anyElementContext() {
-    return Mockito.<Trigger.OnElementContext>any();
-  }
-
-  private void injectElements(int... elements) throws Exception {
-    for (int element : elements) {
-      doNothing().when(mockEarly).onElement(anyElementContext());
-      doNothing().when(mockLate).onElement(anyElementContext());
-      tester.injectElements(element);
-    }
-  }
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  public void testRunningAsTrigger(OnceTrigger mockTrigger, IntervalWindow window)
-      throws Exception {
-
-    // Don't fire due to mock saying no
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    assertFalse(tester.shouldFire(window)); // not ready
-
-    // Fire due to mock trigger; early trigger is required to be a OnceTrigger
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    assertTrue(tester.shouldFire(window)); // ready
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-  }
-
-  @Test
-  public void testEarlyAndAtWatermark() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(mockEarly),
-        FixedWindows.of(Duration.millis(100)));
-
-    injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    testRunningAsTrigger(mockEarly, window);
-
-    // Fire due to watermark
-    when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(100));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  @Test
-  public void testAtWatermarkAndLate() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterWatermark.pastEndOfWindow()
-            .withLateFirings(mockLate),
-        FixedWindows.of(Duration.millis(100)));
-
-    injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    // No early firing, just double checking
-    when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(true);
-    assertFalse(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // Fire due to watermark
-    when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(100));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    testRunningAsTrigger(mockLate, window);
-  }
-
-  @Test
-  public void testEarlyAndAtWatermarkAndLate() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(mockEarly)
-            .withLateFirings(mockLate),
-        FixedWindows.of(Duration.millis(100)));
-
-    injectElements(1);
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    testRunningAsTrigger(mockEarly, window);
-
-    // Fire due to watermark
-    when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(100));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    testRunningAsTrigger(mockLate, window);
-  }
-
-  /**
-   * Tests that if the EOW is finished in both as well as the merged window, then
-   * it is finished in the merged result.
-   *
-   * <p>Because windows are discarded when a trigger finishes, we need to embed this
-   * in a sequence in order to check that it is re-activated. So this test is potentially
-   * sensitive to other triggers' correctness.
-   */
-  @Test
-  public void testOnMergeAlreadyFinished() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            AfterWatermark.pastEndOfWindow(),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(1))),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    tester.injectElements(5);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-
-    // Finish the AfterWatermark.pastEndOfWindow() trigger in both windows
-    tester.advanceInputWatermark(new Instant(15));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    tester.fireIfShouldFire(secondWindow);
-
-    // Confirm that we are on the second trigger by probing
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.injectElements(1);
-    tester.injectElements(5);
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    tester.fireIfShouldFire(secondWindow);
-
-    // Merging should leave it finished
-    tester.mergeWindows();
-
-    // Confirm that we are on the second trigger by probing
-    assertFalse(tester.shouldFire(mergedWindow));
-    tester.injectElements(1);
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  /**
-   * Tests that the trigger rewinds to be non-finished in the merged window.
-   *
-   * <p>Because windows are discarded when a trigger finishes, we need to embed this
-   * in a sequence in order to check that it is re-activated. So this test is potentially
-   * sensitive to other triggers' correctness.
-   */
-  @Test
-  public void testOnMergeRewinds() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            AfterWatermark.pastEndOfWindow(),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(1))),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    tester.injectElements(5);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-
-    // Finish the AfterWatermark.pastEndOfWindow() trigger in only the first window
-    tester.advanceInputWatermark(new Instant(11));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Confirm that we are on the second trigger by probing
-    assertFalse(tester.shouldFire(firstWindow));
-    tester.injectElements(1);
-    assertTrue(tester.shouldFire(firstWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Merging should re-activate the watermark trigger in the merged window
-    tester.mergeWindows();
-
-    // Confirm that we are not on the second trigger by probing
-    assertFalse(tester.shouldFire(mergedWindow));
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // And confirm that advancing the watermark fires again
-    tester.advanceInputWatermark(new Instant(15));
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  /**
-   * Tests that if the EOW is finished in both as well as the merged window, then
-   * it is finished in the merged result.
-   *
-   * <p>Because windows are discarded when a trigger finishes, we need to embed this
-   * in a sequence in order to check that it is re-activated. So this test is potentially
-   * sensitive to other triggers' correctness.
-   */
-  @Test
-  public void testEarlyAndLateOnMergeAlreadyFinished() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(AfterPane.elementCountAtLeast(100))
-            .withLateFirings(AfterPane.elementCountAtLeast(1)),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    tester.injectElements(5);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-
-    // Finish the AfterWatermark.pastEndOfWindow() bit of the trigger in both windows
-    tester.advanceInputWatermark(new Instant(15));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    tester.fireIfShouldFire(secondWindow);
-
-    // Confirm that we are on the late trigger by probing
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.injectElements(1);
-    tester.injectElements(5);
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-    tester.fireIfShouldFire(secondWindow);
-
-    // Merging should leave it on the late trigger
-    tester.mergeWindows();
-
-    // Confirm that we are on the late trigger by probing
-    assertFalse(tester.shouldFire(mergedWindow));
-    tester.injectElements(1);
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  /**
-   * Tests that the trigger rewinds to be non-finished in the merged window.
-   *
-   * <p>Because windows are discarded when a trigger finishes, we need to embed this
-   * in a sequence in order to check that it is re-activated. So this test is potentially
-   * sensitive to other triggers' correctness.
-   */
-  @Test
-  public void testEarlyAndLateOnMergeRewinds() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(AfterPane.elementCountAtLeast(100))
-            .withLateFirings(AfterPane.elementCountAtLeast(1)),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    tester.injectElements(5);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-
-    // Finish the AfterWatermark.pastEndOfWindow() bit of the trigger in only the first window
-    tester.advanceInputWatermark(new Instant(11));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Confirm that we are on the late trigger by probing
-    assertFalse(tester.shouldFire(firstWindow));
-    tester.injectElements(1);
-    assertTrue(tester.shouldFire(firstWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Merging should re-activate the early trigger in the merged window
-    tester.mergeWindows();
-
-    // Confirm that we are not on the second trigger by probing
-    assertFalse(tester.shouldFire(mergedWindow));
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // And confirm that advancing the watermark fires again
-    tester.advanceInputWatermark(new Instant(15));
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
   @Test
   public void testFromEndOfWindowToString() {
     Trigger trigger = AfterWatermark.pastEndOfWindow();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
index 673e555..ee1c44a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
@@ -18,12 +18,7 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -36,131 +31,6 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class DefaultTriggerTest {
 
-  SimpleTriggerTester<IntervalWindow> tester;
-
-  @Test
-  public void testDefaultTriggerFixedWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-        DefaultTrigger.of(),
-        FixedWindows.of(Duration.millis(100)));
-
-    tester.injectElements(
-        1, // [0, 100)
-        101); // [100, 200)
-
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(100), new Instant(200));
-
-    // Advance the watermark almost to the end of the first window.
-    tester.advanceInputWatermark(new Instant(99));
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Advance watermark past end of the first window, which is then ready
-    tester.advanceInputWatermark(new Instant(100));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Fire, but the first window is still allowed to fire
-    tester.fireIfShouldFire(firstWindow);
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Advance watermark to 200, then both are ready
-    tester.advanceInputWatermark(new Instant(200));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-
-    assertFalse(tester.isMarkedFinished(firstWindow));
-    assertFalse(tester.isMarkedFinished(secondWindow));
-  }
-
-  @Test
-  public void testDefaultTriggerSlidingWindows() throws Exception {
-    tester = TriggerTester.forTrigger(
-        DefaultTrigger.of(),
-        SlidingWindows.of(Duration.millis(100)).every(Duration.millis(50)));
-
-    tester.injectElements(
-        1, // [-50, 50), [0, 100)
-        50); // [0, 100), [50, 150)
-
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(-50), new Instant(50));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(100));
-    IntervalWindow thirdWindow = new IntervalWindow(new Instant(50), new Instant(150));
-
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(thirdWindow));
-
-    // At 50, the first becomes ready; it stays ready after firing
-    tester.advanceInputWatermark(new Instant(50));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(thirdWindow));
-    tester.fireIfShouldFire(firstWindow);
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(thirdWindow));
-
-    // At 99, the first is still the only one ready
-    tester.advanceInputWatermark(new Instant(99));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(thirdWindow));
-
-    // At 100, the first and second are ready
-    tester.advanceInputWatermark(new Instant(100));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(thirdWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    assertFalse(tester.isMarkedFinished(firstWindow));
-    assertFalse(tester.isMarkedFinished(secondWindow));
-    assertFalse(tester.isMarkedFinished(thirdWindow));
-  }
-
-  @Test
-  public void testDefaultTriggerSessions() throws Exception {
-    tester = TriggerTester.forTrigger(
-        DefaultTrigger.of(),
-        Sessions.withGapDuration(Duration.millis(100)));
-
-    tester.injectElements(
-        1, // [1, 101)
-        50); // [50, 150)
-    tester.mergeWindows();
-
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(101));
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(50), new Instant(150));
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(150));
-
-    // Not ready in any window yet
-    tester.advanceInputWatermark(new Instant(100));
-    assertFalse(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // The first window is "ready": the caller owns knowledge of which windows are merged away
-    tester.advanceInputWatermark(new Instant(149));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertFalse(tester.shouldFire(secondWindow));
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // Now ready on all windows
-    tester.advanceInputWatermark(new Instant(150));
-    assertTrue(tester.shouldFire(firstWindow));
-    assertTrue(tester.shouldFire(secondWindow));
-    assertTrue(tester.shouldFire(mergedWindow));
-
-    // Ensure it repeats
-    tester.fireIfShouldFire(mergedWindow);
-    assertTrue(tester.shouldFire(mergedWindow));
-
-    assertFalse(tester.isMarkedFinished(mergedWindow));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     assertEquals(new Instant(9), DefaultTrigger.of().getWatermarkThatGuaranteesFiring(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
index fb2b4d5..1052873 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
@@ -17,40 +17,26 @@
  */
 package org.apache.beam.sdk.transforms.windowing;
 
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertEquals;
 
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * Tests for {@link Never}.
- */
+/** Tests for {@link Never}. */
 @RunWith(JUnit4.class)
 public class NeverTest {
-  private SimpleTriggerTester<IntervalWindow> triggerTester;
-
-  @Before
-  public void setup() throws Exception {
-    triggerTester =
-        TriggerTester.forTrigger(
-            Never.ever(), FixedWindows.of(Duration.standardMinutes(5)));
+  @Test
+  public void testFireDeadline() throws Exception {
+    assertEquals(
+        BoundedWindow.TIMESTAMP_MAX_VALUE,
+        Never.ever()
+            .getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10))));
   }
 
   @Test
-  public void falseAfterEndOfWindow() throws Exception {
-    triggerTester.injectElements(TimestampedValue.of(1, new Instant(1)));
-    IntervalWindow window =
-        new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardMinutes(5)));
-    assertThat(triggerTester.shouldFire(window), is(false));
-    triggerTester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(triggerTester.shouldFire(window), is(false));
+  public void testContinuation() throws Exception {
+    assertEquals(Never.ever(), Never.ever().getContinuationTrigger());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
index 7289d97..6e61e10 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
@@ -18,13 +18,8 @@
 package org.apache.beam.sdk.transforms.windowing;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
-import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -36,137 +31,6 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class OrFinallyTriggerTest {
 
-  private SimpleTriggerTester<IntervalWindow> tester;
-
-  /**
-   * Tests that for {@code OrFinally(actual, ...)} when {@code actual}
-   * fires and finishes, the {@code OrFinally} also fires and finishes.
-   */
-  @Test
-  public void testActualFiresAndFinishes() throws Exception {
-    tester = TriggerTester.forTrigger(
-        new OrFinallyTrigger(
-            AfterPane.elementCountAtLeast(2),
-            AfterPane.elementCountAtLeast(100)),
-        FixedWindows.of(Duration.millis(100)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    // Not yet firing
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-    assertFalse(tester.isMarkedFinished(window));
-
-    // The actual fires and finishes
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
-  /**
-   * Tests that for {@code OrFinally(actual, ...)} when {@code actual}
-   * fires but does not finish, the {@code OrFinally} also fires and also does not
-   * finish.
-   */
-  @Test
-  public void testActualFiresOnly() throws Exception {
-    tester = TriggerTester.forTrigger(
-        new OrFinallyTrigger(
-            Repeatedly.forever(AfterPane.elementCountAtLeast(2)),
-            AfterPane.elementCountAtLeast(100)),
-        FixedWindows.of(Duration.millis(100)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100));
-
-    // Not yet firing
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-    assertFalse(tester.isMarkedFinished(window));
-
-    // The actual fires but does not finish
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // And again
-    tester.injectElements(3, 4);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-  }
-
-  /**
-   * Tests that if the first trigger rewinds to be non-finished in the merged window,
-   * then it becomes the currently active trigger again, with real triggers.
-   */
-  @Test
-  public void testShouldFireAfterMerge() throws Exception {
-    tester = TriggerTester.forTrigger(
-        AfterEach.inOrder(
-            AfterPane.elementCountAtLeast(5)
-                .orFinally(AfterWatermark.pastEndOfWindow()),
-            Repeatedly.forever(AfterPane.elementCountAtLeast(1))),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    // Finished the orFinally in the first window
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    assertFalse(tester.shouldFire(firstWindow));
-    tester.advanceInputWatermark(new Instant(11));
-    assertTrue(tester.shouldFire(firstWindow));
-    tester.fireIfShouldFire(firstWindow);
-
-    // Set up second window where it is not done
-    tester.injectElements(5);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Merge them, if the merged window were on the second trigger, it would be ready
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-    assertFalse(tester.shouldFire(mergedWindow));
-
-    // Now adding 3 more makes the main trigger ready to fire
-    tester.injectElements(1, 2, 3, 4, 5);
-    tester.mergeWindows();
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  /**
-   * Tests that for {@code OrFinally(actual, until)} when {@code actual}
-   * fires but does not finish, then {@code until} fires and finishes, the
-   * whole thing fires and finished.
-   */
-  @Test
-  public void testActualFiresButUntilFinishes() throws Exception {
-    tester = TriggerTester.forTrigger(
-        new OrFinallyTrigger(
-            Repeatedly.forever(AfterPane.elementCountAtLeast(2)),
-                AfterPane.elementCountAtLeast(3)),
-        FixedWindows.of(Duration.millis(10)));
-
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-
-    // Before any firing
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-    assertFalse(tester.isMarkedFinished(window));
-
-    // The actual fires but doesn't finish
-    tester.injectElements(2);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.isMarkedFinished(window));
-
-    // The until fires and finishes; the trigger is finished
-    tester.injectElements(3);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertTrue(tester.isMarkedFinished(window));
-  }
-
   @Test
   public void testFireDeadline() throws Exception {
     BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
index 6e8930d..55cb77e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
@@ -19,14 +19,9 @@ package org.apache.beam.sdk.transforms.windowing;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import org.apache.beam.sdk.util.TriggerTester;
-import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
@@ -36,51 +31,17 @@ import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
 
-/**
- * Tests for {@link Repeatedly}.
- */
+/** Tests for {@link Repeatedly}. */
 @RunWith(JUnit4.class)
 public class RepeatedlyTest {
 
   @Mock private Trigger mockTrigger;
-  private SimpleTriggerTester<IntervalWindow> tester;
-  private static Trigger.TriggerContext anyTriggerContext() {
-    return Mockito.<Trigger.TriggerContext>any();
-  }
 
   public void setUp(WindowFn<Object, IntervalWindow> windowFn) throws Exception {
     MockitoAnnotations.initMocks(this);
-    tester = TriggerTester.forTrigger(Repeatedly.forever(mockTrigger), windowFn);
-  }
-
-  /**
-   * Tests that onElement correctly passes the data on to the subtrigger.
-   */
-  @Test
-  public void testOnElement() throws Exception {
-    setUp(FixedWindows.of(Duration.millis(10)));
-    tester.injectElements(37);
-    verify(mockTrigger).onElement(Mockito.<Trigger.OnElementContext>any());
-  }
-
-  /**
-   * Tests that the repeatedly is ready to fire whenever the subtrigger is ready.
-   */
-  @Test
-  public void testShouldFire() throws Exception {
-    setUp(FixedWindows.of(Duration.millis(10)));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    assertTrue(tester.shouldFire(new IntervalWindow(new Instant(0), new Instant(10))));
-
-    when(mockTrigger.shouldFire(Mockito.<Trigger.TriggerContext>any()))
-        .thenReturn(false);
-    assertFalse(tester.shouldFire(new IntervalWindow(new Instant(0), new Instant(10))));
   }
 
-  /**
-   * Tests that the watermark that guarantees firing is that of the subtrigger.
-   */
+  /** Tests that the watermark that guarantees firing is that of the subtrigger. */
   @Test
   public void testFireDeadline() throws Exception {
     setUp(FixedWindows.of(Duration.millis(10)));
@@ -107,118 +68,16 @@ public class RepeatedlyTest {
   }
 
   @Test
-  public void testShouldFireAfterMerge() throws Exception {
-    tester = TriggerTester.forTrigger(
-        Repeatedly.forever(AfterPane.elementCountAtLeast(2)),
-        Sessions.withGapDuration(Duration.millis(10)));
-
-    tester.injectElements(1);
-    IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11));
-    assertFalse(tester.shouldFire(firstWindow));
-
-    tester.injectElements(5);
-    IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15));
-    assertFalse(tester.shouldFire(secondWindow));
-
-    // Merge them, if the merged window were on the second trigger, it would be ready
-    tester.mergeWindows();
-    IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15));
-    assertTrue(tester.shouldFire(mergedWindow));
-  }
-
-  @Test
-  public void testRepeatedlyAfterFirstElementCount() throws Exception {
-    SimpleTriggerTester<GlobalWindow> tester =
-        TriggerTester.forTrigger(
-            Repeatedly.forever(
-                AfterFirst.of(
-                    AfterProcessingTime.pastFirstElementInPane()
-                        .plusDelayOf(Duration.standardMinutes(15)),
-                    AfterPane.elementCountAtLeast(5))),
-            new GlobalWindows());
-
-    GlobalWindow window = GlobalWindow.INSTANCE;
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.injectElements(2, 3, 4, 5);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.shouldFire(window));
-  }
-
-  @Test
-  public void testRepeatedlyAfterFirstProcessingTime() throws Exception {
-    SimpleTriggerTester<GlobalWindow> tester =
-        TriggerTester.forTrigger(
-            Repeatedly.forever(
-                AfterFirst.of(
-                    AfterProcessingTime.pastFirstElementInPane()
-                        .plusDelayOf(Duration.standardMinutes(15)),
-                    AfterPane.elementCountAtLeast(5))),
-            new GlobalWindows());
-
-    GlobalWindow window = GlobalWindow.INSTANCE;
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.advanceProcessingTime(new Instant(0).plus(Duration.standardMinutes(15)));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.shouldFire(window));
-  }
-
-  @Test
-  public void testRepeatedlyElementCount() throws Exception {
-    SimpleTriggerTester<GlobalWindow> tester =
-        TriggerTester.forTrigger(
-            Repeatedly.forever(AfterPane.elementCountAtLeast(5)),
-            new GlobalWindows());
-
-    GlobalWindow window = GlobalWindow.INSTANCE;
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.injectElements(2, 3, 4, 5);
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.shouldFire(window));
-  }
-
-  @Test
-  public void testRepeatedlyProcessingTime() throws Exception {
-    SimpleTriggerTester<GlobalWindow> tester =
-        TriggerTester.forTrigger(
-            Repeatedly.forever(
-                    AfterProcessingTime.pastFirstElementInPane()
-                        .plusDelayOf(Duration.standardMinutes(15))),
-            new GlobalWindows());
-
-    GlobalWindow window = GlobalWindow.INSTANCE;
-
-    tester.injectElements(1);
-    assertFalse(tester.shouldFire(window));
-
-    tester.advanceProcessingTime(new Instant(0).plus(Duration.standardMinutes(15)));
-    assertTrue(tester.shouldFire(window));
-    tester.fireIfShouldFire(window);
-    assertFalse(tester.shouldFire(window));
-  }
-
-
-  @Test
   public void testToString() {
-    Trigger trigger = Repeatedly.forever(new StubTrigger() {
-        @Override
-        public String toString() {
-          return "innerTrigger";
-        }
-      });
+    Trigger trigger =
+        Repeatedly.forever(
+            new StubTrigger() {
+              @Override
+              public String toString() {
+                return "innerTrigger";
+              }
+            });
 
     assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
index b258a79..0fc74e7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
@@ -42,23 +42,6 @@ abstract class StubTrigger extends Trigger.OnceTrigger {
   }
 
   @Override
-  protected void onOnlyFiring(TriggerContext context) throws Exception {
-  }
-
-  @Override
-  public void onElement(OnElementContext c) throws Exception {
-  }
-
-  @Override
-  public void onMerge(OnMergeContext c) throws Exception {
-  }
-
-  @Override
-  public boolean shouldFire(TriggerContext context) throws Exception {
-    return false;
-  }
-
-  @Override
   protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
index cfc03b2..2602f79 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
@@ -59,12 +59,6 @@ public class TriggerTest {
     }
 
     @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
     protected Trigger getContinuationTrigger(
         List<Trigger> continuationTriggers) {
       return null;
@@ -74,14 +68,6 @@ public class TriggerTest {
     public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
       return null;
     }
-
-    @Override
-    public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-      return false;
-    }
-
-    @Override
-    public void onFire(Trigger.TriggerContext context) throws Exception { }
   }
 
   private static class Trigger2 extends Trigger {
@@ -91,12 +77,6 @@ public class TriggerTest {
     }
 
     @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
     protected Trigger getContinuationTrigger(
         List<Trigger> continuationTriggers) {
       return null;
@@ -106,13 +86,5 @@ public class TriggerTest {
     public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
       return null;
     }
-
-    @Override
-    public boolean shouldFire(Trigger.TriggerContext context) throws Exception {
-      return false;
-    }
-
-    @Override
-    public void onFire(Trigger.TriggerContext context) throws Exception { }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
index 1e3a1ff..befc07e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
@@ -92,16 +92,6 @@ public class ExecutableTriggerTest {
     }
 
     @Override
-    public void onElement(OnElementContext c) throws Exception { }
-
-    @Override
-    public void onMerge(OnMergeContext c) throws Exception { }
-
-    @Override
-    public void clear(TriggerContext c) throws Exception {
-    }
-
-    @Override
     public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
       return BoundedWindow.TIMESTAMP_MAX_VALUE;
     }
@@ -115,13 +105,5 @@ public class ExecutableTriggerTest {
     public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
       return this;
     }
-
-    @Override
-    public boolean shouldFire(TriggerContext c) {
-      return false;
-    }
-
-    @Override
-    public void onFire(TriggerContext c) { }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
deleted file mode 100644
index 7f74620..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersBitSetTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.util;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FinishedTriggersBitSet}.
- */
-@RunWith(JUnit4.class)
-public class FinishedTriggersBitSetTest {
-  /**
-   * Tests that after a trigger is set to finished, it reads back as finished.
-   */
-  @Test
-  public void testSetGet() {
-    FinishedTriggersProperties.verifyGetAfterSet(FinishedTriggersBitSet.emptyWithCapacity(1));
-  }
-
-  /**
-   * Tests that clearing a trigger recursively clears all of that triggers subTriggers, but no
-   * others.
-   */
-  @Test
-  public void testClearRecursively() {
-    FinishedTriggersProperties.verifyClearRecursively(FinishedTriggersBitSet.emptyWithCapacity(1));
-  }
-
-  @Test
-  public void testCopy() throws Exception {
-    FinishedTriggersBitSet finishedSet = FinishedTriggersBitSet.emptyWithCapacity(10);
-    assertThat(finishedSet.copy().getBitSet(), not(theInstance(finishedSet.getBitSet())));
-  }
-}


[08/50] [abbrv] incubator-beam git commit: Fix Javadoc of ReadableState

Posted by dh...@apache.org.
Fix Javadoc of ReadableState


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

Branch: refs/heads/gearpump-runner
Commit: caf1fb705e785cb312cedc334efa73e081469da0
Parents: a9a41eb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 10:35:41 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 10:39:07 2016 +0200

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/state/ReadableState.java     | 10 +++-------
 1 file changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/caf1fb70/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
index ea8a231..3b4cb7b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
@@ -21,14 +21,10 @@ import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 
 /**
- * A {@code StateContents} is produced by the read methods on all {@link State} objects.
- * Calling {@link #read} returns the associated value.
+ * {@link State} that can be read via {@link #read()}.
  *
- * <p>This class is similar to {@link java.util.concurrent.Future}, but each invocation of
- * {@link #read} need not return the same value.
- *
- * <p>Getting the {@code StateContents} from a read method indicates the desire to eventually
- * read a value. Depending on the runner this may or may not immediately start the read.
+ * <p>Use {@link #readLater()} for marking several states for prefetching. Runners
+ * can potentially batch these into one read.
  *
  * @param <T> The type of value returned by {@link #read}.
  */


[06/50] [abbrv] incubator-beam git commit: Make aggregator registration idempotent in FlinkRunner

Posted by dh...@apache.org.
Make aggregator registration idempotent in FlinkRunner


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

Branch: refs/heads/gearpump-runner
Commit: 2089c5cd2662a2eeea39ac7ebd1bfd8bcdc1aa16
Parents: 1919d8b
Author: Kenneth Knowles <kl...@google.com>
Authored: Sun Oct 23 21:26:48 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:26:48 2016 -0700

----------------------------------------------------------------------
 .../flink/translation/functions/FlinkProcessContext.java  | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2089c5cd/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
index fa5eb1a..baf97cb 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.common.collect.Iterables;
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
@@ -40,6 +41,7 @@ import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
@@ -316,7 +318,13 @@ class FlinkProcessContext<InputT, OutputT>
   createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
     SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper =
         new SerializableFnAggregatorWrapper<>(combiner);
-    runtimeContext.addAccumulator(name, wrapper);
+    Accumulator<?, ?> existingAccum =
+        (Accumulator<AggInputT, Serializable>) runtimeContext.getAccumulator(name);
+    if (existingAccum != null) {
+      return wrapper;
+    } else {
+      runtimeContext.addAccumulator(name, wrapper);
+    }
     return wrapper;
   }
 }


[27/50] [abbrv] incubator-beam git commit: Remove pieces of Trigger now owned by TriggerStateMachine

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
deleted file mode 100644
index a66f74f..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersProperties.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.util;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.beam.sdk.transforms.windowing.AfterAll;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-
-/**
- * Generalized tests for {@link FinishedTriggers} implementations.
- */
-public class FinishedTriggersProperties {
-  /**
-   * Tests that for the provided trigger and {@link FinishedTriggers}, when the trigger is set
-   * finished, it is correctly reported as finished.
-   */
-  public static void verifyGetAfterSet(FinishedTriggers finishedSet, ExecutableTrigger trigger) {
-    assertFalse(finishedSet.isFinished(trigger));
-    finishedSet.setFinished(trigger, true);
-    assertTrue(finishedSet.isFinished(trigger));
-  }
-
-  /**
-   * For a few arbitrary triggers, tests that when the trigger is set finished it is correctly
-   * reported as finished.
-   */
-  public static void verifyGetAfterSet(FinishedTriggers finishedSet) {
-    ExecutableTrigger trigger = ExecutableTrigger.create(AfterAll.of(
-        AfterFirst.of(AfterPane.elementCountAtLeast(3), AfterWatermark.pastEndOfWindow()),
-        AfterAll.of(
-            AfterPane.elementCountAtLeast(10), AfterProcessingTime.pastFirstElementInPane())));
-
-    verifyGetAfterSet(finishedSet, trigger);
-    verifyGetAfterSet(finishedSet, trigger.subTriggers().get(0).subTriggers().get(1));
-    verifyGetAfterSet(finishedSet, trigger.subTriggers().get(0));
-    verifyGetAfterSet(finishedSet, trigger.subTriggers().get(1));
-    verifyGetAfterSet(finishedSet, trigger.subTriggers().get(1).subTriggers().get(1));
-    verifyGetAfterSet(finishedSet, trigger.subTriggers().get(1).subTriggers().get(0));
-  }
-
-  /**
-   * Tests that clearing a trigger recursively clears all of that triggers subTriggers, but no
-   * others.
-   */
-  public static void verifyClearRecursively(FinishedTriggers finishedSet) {
-    ExecutableTrigger trigger = ExecutableTrigger.create(AfterAll.of(
-        AfterFirst.of(AfterPane.elementCountAtLeast(3), AfterWatermark.pastEndOfWindow()),
-        AfterAll.of(
-            AfterPane.elementCountAtLeast(10), AfterProcessingTime.pastFirstElementInPane())));
-
-    // Set them all finished. This method is not on a trigger as it makes no sense outside tests.
-    setFinishedRecursively(finishedSet, trigger);
-    assertTrue(finishedSet.isFinished(trigger));
-    assertTrue(finishedSet.isFinished(trigger.subTriggers().get(0)));
-    assertTrue(finishedSet.isFinished(trigger.subTriggers().get(0).subTriggers().get(0)));
-    assertTrue(finishedSet.isFinished(trigger.subTriggers().get(0).subTriggers().get(1)));
-
-    // Clear just the second AfterAll
-    finishedSet.clearRecursively(trigger.subTriggers().get(1));
-
-    // Check that the first and all that are still finished
-    assertTrue(finishedSet.isFinished(trigger));
-    verifyFinishedRecursively(finishedSet, trigger.subTriggers().get(0));
-    verifyUnfinishedRecursively(finishedSet, trigger.subTriggers().get(1));
-  }
-
-  private static void setFinishedRecursively(
-      FinishedTriggers finishedSet, ExecutableTrigger trigger) {
-    finishedSet.setFinished(trigger, true);
-    for (ExecutableTrigger subTrigger : trigger.subTriggers()) {
-      setFinishedRecursively(finishedSet, subTrigger);
-    }
-  }
-
-  private static void verifyFinishedRecursively(
-      FinishedTriggers finishedSet, ExecutableTrigger trigger) {
-    assertTrue(finishedSet.isFinished(trigger));
-    for (ExecutableTrigger subTrigger : trigger.subTriggers()) {
-      verifyFinishedRecursively(finishedSet, subTrigger);
-    }
-  }
-
-  private static void verifyUnfinishedRecursively(
-      FinishedTriggers finishedSet, ExecutableTrigger trigger) {
-    assertFalse(finishedSet.isFinished(trigger));
-    for (ExecutableTrigger subTrigger : trigger.subTriggers()) {
-      verifyUnfinishedRecursively(finishedSet, subTrigger);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
deleted file mode 100644
index 072d264..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.util;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import java.util.HashSet;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FinishedTriggersSet}.
- */
-@RunWith(JUnit4.class)
-public class FinishedTriggersSetTest {
-  /**
-   * Tests that after a trigger is set to finished, it reads back as finished.
-   */
-  @Test
-  public void testSetGet() {
-    FinishedTriggersProperties.verifyGetAfterSet(
-        FinishedTriggersSet.fromSet(new HashSet<ExecutableTrigger>()));
-  }
-
-  /**
-   * Tests that clearing a trigger recursively clears all of that triggers subTriggers, but no
-   * others.
-   */
-  @Test
-  public void testClearRecursively() {
-    FinishedTriggersProperties.verifyClearRecursively(
-        FinishedTriggersSet.fromSet(new HashSet<ExecutableTrigger>()));
-  }
-
-  @Test
-  public void testCopy() throws Exception {
-    FinishedTriggersSet finishedSet =
-        FinishedTriggersSet.fromSet(new HashSet<ExecutableTrigger>());
-    assertThat(finishedSet.copy().getFinishedTriggers(),
-        not(theInstance(finishedSet.getFinishedTriggers())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
index 83077f4..63c71ed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java
@@ -18,15 +18,9 @@
 package org.apache.beam.sdk.util;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -43,23 +37,6 @@ public class ReshuffleTriggerTest {
   }
 
   @Test
-  public void testShouldFire() throws Exception {
-    TriggerTester<Integer, IntervalWindow> tester = TriggerTester.forTrigger(
-        new ReshuffleTrigger<IntervalWindow>(), FixedWindows.of(Duration.millis(100)));
-    IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(300), new Instant(400));
-    assertTrue(tester.shouldFire(arbitraryWindow));
-  }
-
-  @Test
-  public void testOnTimer() throws Exception {
-    TriggerTester<Integer, IntervalWindow> tester = TriggerTester.forTrigger(
-        new ReshuffleTrigger<IntervalWindow>(), FixedWindows.of(Duration.millis(100)));
-    IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(100), new Instant(200));
-    tester.fireIfShouldFire(arbitraryWindow);
-    assertFalse(tester.isMarkedFinished(arbitraryWindow));
-  }
-
-  @Test
   public void testToString() {
     Trigger trigger = new ReshuffleTrigger<>();
     assertEquals("ReshuffleTrigger()", trigger.toString());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1eff320d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
deleted file mode 100644
index 5fe17ad..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
+++ /dev/null
@@ -1,410 +0,0 @@
-/*
- * 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.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.ActiveWindowSet.MergeCallback;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowAndTriggerNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
-import org.apache.beam.sdk.util.state.TestInMemoryStateInternals;
-import org.apache.beam.sdk.util.state.TimerCallback;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Test utility that runs a {@link Trigger}, using in-memory stub implementation to provide
- * the {@link StateInternals}.
- *
- * @param <W> The type of windows being used.
- */
-public class TriggerTester<InputT, W extends BoundedWindow> {
-
-  /**
-   * A {@link TriggerTester} specialized to {@link Integer} values, so elements and timestamps
-   * can be conflated. Today, triggers should not observed the element type, so this is the
-   * only trigger tester that needs to be used.
-   */
-  public static class SimpleTriggerTester<W extends BoundedWindow>
-      extends TriggerTester<Integer, W> {
-
-    private SimpleTriggerTester(WindowingStrategy<Object, W> windowingStrategy) throws Exception {
-      super(windowingStrategy);
-    }
-
-    public void injectElements(int... values) throws Exception {
-      List<TimestampedValue<Integer>> timestampedValues =
-          Lists.newArrayListWithCapacity(values.length);
-      for (int value : values) {
-        timestampedValues.add(TimestampedValue.of(value, new Instant(value)));
-      }
-      injectElements(timestampedValues);
-    }
-
-    public SimpleTriggerTester<W> withAllowedLateness(Duration allowedLateness) throws Exception {
-      return new SimpleTriggerTester<>(
-          windowingStrategy.withAllowedLateness(allowedLateness));
-    }
-  }
-
-  protected final WindowingStrategy<Object, W> windowingStrategy;
-
-  private final TestInMemoryStateInternals<?> stateInternals =
-      new TestInMemoryStateInternals<Object>(null /* key */);
-  private final InMemoryTimerInternals timerInternals = new InMemoryTimerInternals();
-  private final TriggerContextFactory<W> contextFactory;
-  private final WindowFn<Object, W> windowFn;
-  private final ActiveWindowSet<W> activeWindows;
-  private final Map<W, W> windowToMergeResult;
-
-  /**
-   * An {@link ExecutableTrigger} built from the {@link Trigger} or {@link Trigger}
-   * under test.
-   */
-  private final ExecutableTrigger executableTrigger;
-
-  /**
-   * A map from a window and trigger to whether that trigger is finished for the window.
-   */
-  private final Map<W, FinishedTriggers> finishedSets;
-
-  public static <W extends BoundedWindow> SimpleTriggerTester<W> forTrigger(
-      Trigger trigger, WindowFn<Object, W> windowFn)
-          throws Exception {
-    WindowingStrategy<Object, W> windowingStrategy =
-        WindowingStrategy.of(windowFn).withTrigger(trigger)
-        // Merging requires accumulation mode or early firings can break up a session.
-        // Not currently an issue with the tester (because we never GC) but we don't want
-        // mystery failures due to violating this need.
-        .withMode(windowFn.isNonMerging()
-            ? AccumulationMode.DISCARDING_FIRED_PANES
-            : AccumulationMode.ACCUMULATING_FIRED_PANES);
-
-    return new SimpleTriggerTester<>(windowingStrategy);
-  }
-
-  public static <InputT, W extends BoundedWindow> TriggerTester<InputT, W> forAdvancedTrigger(
-      Trigger trigger, WindowFn<Object, W> windowFn) throws Exception {
-    WindowingStrategy<Object, W> strategy =
-        WindowingStrategy.of(windowFn).withTrigger(trigger)
-        // Merging requires accumulation mode or early firings can break up a session.
-        // Not currently an issue with the tester (because we never GC) but we don't want
-        // mystery failures due to violating this need.
-        .withMode(windowFn.isNonMerging()
-            ? AccumulationMode.DISCARDING_FIRED_PANES
-            : AccumulationMode.ACCUMULATING_FIRED_PANES);
-
-    return new TriggerTester<>(strategy);
-  }
-
-  protected TriggerTester(WindowingStrategy<Object, W> windowingStrategy) throws Exception {
-    this.windowingStrategy = windowingStrategy;
-    this.windowFn = windowingStrategy.getWindowFn();
-    this.executableTrigger = windowingStrategy.getTrigger();
-    this.finishedSets = new HashMap<>();
-
-    this.activeWindows =
-        windowFn.isNonMerging()
-            ? new NonMergingActiveWindowSet<W>()
-            : new MergingActiveWindowSet<W>(windowFn, stateInternals);
-    this.windowToMergeResult = new HashMap<>();
-
-    this.contextFactory =
-        new TriggerContextFactory<>(windowingStrategy.getWindowFn(), stateInternals, activeWindows);
-  }
-
-  /**
-   * Instructs the trigger to clear its state for the given window.
-   */
-  public void clearState(W window) throws Exception {
-    executableTrigger.invokeClear(contextFactory.base(window,
-        new TestTimers(windowNamespace(window)), executableTrigger, getFinishedSet(window)));
-  }
-
-  /**
-   * Asserts that the trigger has actually cleared all of its state for the given window. Since
-   * the trigger under test is the root, this makes the assert for all triggers regardless
-   * of their position in the trigger tree.
-   */
-  public void assertCleared(W window) {
-    for (StateNamespace untypedNamespace : stateInternals.getNamespacesInUse()) {
-      if (untypedNamespace instanceof WindowAndTriggerNamespace) {
-        @SuppressWarnings("unchecked")
-        WindowAndTriggerNamespace<W> namespace = (WindowAndTriggerNamespace<W>) untypedNamespace;
-        if (namespace.getWindow().equals(window)) {
-          Set<?> tagsInUse = stateInternals.getTagsInUse(namespace);
-          assertTrue("Trigger has not cleared tags: " + tagsInUse, tagsInUse.isEmpty());
-        }
-      }
-    }
-  }
-
-  /**
-   * Returns {@code true} if the {@link Trigger} under test is finished for the given window.
-   */
-  public boolean isMarkedFinished(W window) {
-    FinishedTriggers finishedSet = finishedSets.get(window);
-    if (finishedSet == null) {
-      return false;
-    }
-
-    return finishedSet.isFinished(executableTrigger);
-  }
-
-  private StateNamespace windowNamespace(W window) {
-    return StateNamespaces.window(windowFn.windowCoder(), checkNotNull(window));
-  }
-
-  /**
-   * Advance the input watermark to the specified time, then advance the output watermark as far as
-   * possible.
-   */
-  public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
-    // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694
-    timerInternals.advanceInputWatermark(TimerCallback.NO_OP, newInputWatermark);
-  }
-
-  /** Advance the processing time to the specified time. */
-  public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
-    // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694
-    timerInternals.advanceProcessingTime(TimerCallback.NO_OP, newProcessingTime);
-  }
-
-  /**
-   * Inject all the timestamped values (after passing through the window function) as if they
-   * arrived in a single chunk of a bundle (or work-unit).
-   */
-  @SafeVarargs
-  public final void injectElements(TimestampedValue<InputT>... values) throws Exception {
-    injectElements(Arrays.asList(values));
-  }
-
-  public final void injectElements(Collection<TimestampedValue<InputT>> values) throws Exception {
-    for (TimestampedValue<InputT> value : values) {
-      WindowTracing.trace("TriggerTester.injectElements: {}", value);
-    }
-
-    List<WindowedValue<InputT>> windowedValues = Lists.newArrayListWithCapacity(values.size());
-
-    for (TimestampedValue<InputT> input : values) {
-      try {
-        InputT value = input.getValue();
-        Instant timestamp = input.getTimestamp();
-        Collection<W> assignedWindows = windowFn.assignWindows(new TestAssignContext<W>(
-            windowFn, value, timestamp, GlobalWindow.INSTANCE));
-
-        for (W window : assignedWindows) {
-          activeWindows.addActiveForTesting(window);
-
-          // Today, triggers assume onTimer firing at the watermark time, whether or not they
-          // explicitly set the timer themselves. So this tester must set it.
-          timerInternals.setTimer(
-              TimerData.of(windowNamespace(window), window.maxTimestamp(), TimeDomain.EVENT_TIME));
-        }
-
-        windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING));
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    for (WindowedValue<InputT> windowedValue : windowedValues) {
-      for (BoundedWindow untypedWindow : windowedValue.getWindows()) {
-        // SDK is responsible for type safety
-        @SuppressWarnings("unchecked")
-        W window = mergeResult((W) untypedWindow);
-
-        Trigger.OnElementContext context = contextFactory.createOnElementContext(window,
-            new TestTimers(windowNamespace(window)), windowedValue.getTimestamp(),
-            executableTrigger, getFinishedSet(window));
-
-        if (!context.trigger().isFinished()) {
-          executableTrigger.invokeOnElement(context);
-        }
-      }
-    }
-  }
-
-  public boolean shouldFire(W window) throws Exception {
-    Trigger.TriggerContext context = contextFactory.base(
-        window,
-        new TestTimers(windowNamespace(window)),
-        executableTrigger, getFinishedSet(window));
-    executableTrigger.getSpec().prefetchShouldFire(context.state());
-    return executableTrigger.invokeShouldFire(context);
-  }
-
-  public void fireIfShouldFire(W window) throws Exception {
-    Trigger.TriggerContext context = contextFactory.base(
-        window,
-        new TestTimers(windowNamespace(window)),
-        executableTrigger, getFinishedSet(window));
-
-    executableTrigger.getSpec().prefetchShouldFire(context.state());
-    if (executableTrigger.invokeShouldFire(context)) {
-      executableTrigger.getSpec().prefetchOnFire(context.state());
-      executableTrigger.invokeOnFire(context);
-      if (context.trigger().isFinished()) {
-        activeWindows.remove(window);
-        executableTrigger.invokeClear(context);
-      }
-    }
-  }
-
-  public void setSubTriggerFinishedForWindow(int subTriggerIndex, W window, boolean value) {
-    getFinishedSet(window).setFinished(executableTrigger.subTriggers().get(subTriggerIndex), value);
-  }
-
-  /**
-   * Invokes merge from the {@link WindowFn} a single time and passes the resulting merge
-   * events on to the trigger under test. Does not persist the fact that merging happened,
-   * since it is just to test the trigger's {@code OnMerge} method.
-   */
-  public final void mergeWindows() throws Exception {
-    windowToMergeResult.clear();
-    activeWindows.merge(new MergeCallback<W>() {
-      @Override
-      public void prefetchOnMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {}
-
-      @Override
-      public void onMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {
-        List<W> activeToBeMerged = new ArrayList<W>();
-        for (W window : toBeMerged) {
-          windowToMergeResult.put(window, mergeResult);
-          if (activeWindows.isActive(window)) {
-            activeToBeMerged.add(window);
-          }
-        }
-        Map<W, FinishedTriggers> mergingFinishedSets =
-            Maps.newHashMapWithExpectedSize(activeToBeMerged.size());
-        for (W oldWindow : activeToBeMerged) {
-          mergingFinishedSets.put(oldWindow, getFinishedSet(oldWindow));
-        }
-        executableTrigger.invokeOnMerge(contextFactory.createOnMergeContext(mergeResult,
-            new TestTimers(windowNamespace(mergeResult)), executableTrigger,
-            getFinishedSet(mergeResult), mergingFinishedSets));
-        timerInternals.setTimer(TimerData.of(
-            windowNamespace(mergeResult), mergeResult.maxTimestamp(), TimeDomain.EVENT_TIME));
-      }
-    });
-  }
-
-  public  W mergeResult(W window) {
-    W result = windowToMergeResult.get(window);
-    return result == null ? window : result;
-  }
-
-  private FinishedTriggers getFinishedSet(W window) {
-    FinishedTriggers finishedSet = finishedSets.get(window);
-    if (finishedSet == null) {
-      finishedSet = FinishedTriggersSet.fromSet(new HashSet<ExecutableTrigger>());
-      finishedSets.put(window, finishedSet);
-    }
-    return finishedSet;
-  }
-
-  private static class TestAssignContext<W extends BoundedWindow>
-      extends WindowFn<Object, W>.AssignContext {
-    private Object element;
-    private Instant timestamp;
-    private BoundedWindow window;
-
-    public TestAssignContext(
-        WindowFn<Object, W> windowFn, Object element, Instant timestamp, BoundedWindow window) {
-      windowFn.super();
-      this.element = element;
-      this.timestamp = timestamp;
-      this.window = window;
-    }
-
-    @Override
-    public Object element() {
-      return element;
-    }
-
-    @Override
-    public Instant timestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public BoundedWindow window() {
-      return window;
-    }
-  }
-
-  private class TestTimers implements Timers {
-    private final StateNamespace namespace;
-
-    public TestTimers(StateNamespace namespace) {
-      checkArgument(namespace instanceof WindowNamespace);
-      this.namespace = namespace;
-    }
-
-    @Override
-    public void setTimer(Instant timestamp, TimeDomain timeDomain) {
-      timerInternals.setTimer(TimerData.of(namespace, timestamp, timeDomain));
-    }
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
-      timerInternals.deleteTimer(TimerData.of(namespace, timestamp, timeDomain));
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timerInternals.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timerInternals.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    public Instant currentEventTime() {
-      return timerInternals.currentInputWatermarkTime();
-    }
-  }
-}


[22/50] [abbrv] incubator-beam git commit: Closes #1168

Posted by dh...@apache.org.
Closes #1168


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

Branch: refs/heads/gearpump-runner
Commit: 44b9f4ebbc3229d8a9ec46b0c377e57ad0176dbe
Parents: dfaf2a8 7e9ad99
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 13:16:51 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 13:16:51 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  4 +--
 .../beam/sdk/io/OffsetBasedSourceTest.java      | 30 +++++++++++++++-----
 2 files changed, 25 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[31/50] [abbrv] incubator-beam git commit: This closes #1172

Posted by dh...@apache.org.
This closes #1172


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

Branch: refs/heads/gearpump-runner
Commit: 79bb2c282c6b716713ab6a1308755f9e06543358
Parents: 7de8d57 90c30cb
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 10:13:36 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 10:13:36 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/DataflowRunner.java   |   4 +-
 .../beam/sdk/transforms/windowing/AfterAll.java |  49 --
 .../windowing/AfterDelayFromFirstElement.java   |  99 ----
 .../sdk/transforms/windowing/AfterEach.java     |  61 ---
 .../sdk/transforms/windowing/AfterFirst.java    |  50 --
 .../sdk/transforms/windowing/AfterPane.java     |  52 --
 .../windowing/AfterProcessingTime.java          |   7 -
 .../AfterSynchronizedProcessingTime.java        |   7 -
 .../transforms/windowing/AfterWatermark.java    | 158 ------
 .../transforms/windowing/DefaultTrigger.java    |  35 --
 .../beam/sdk/transforms/windowing/Never.java    |  17 -
 .../transforms/windowing/OrFinallyTrigger.java  |  46 --
 .../sdk/transforms/windowing/Repeatedly.java    |  30 --
 .../beam/sdk/transforms/windowing/Trigger.java  | 412 ++-------------
 .../apache/beam/sdk/util/ExecutableTrigger.java |  40 +-
 .../apache/beam/sdk/util/FinishedTriggers.java  |  44 --
 .../beam/sdk/util/FinishedTriggersBitSet.java   |  67 ---
 .../beam/sdk/util/FinishedTriggersSet.java      |  72 ---
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |  14 -
 .../beam/sdk/util/TriggerContextFactory.java    | 507 -------------------
 .../sdk/transforms/windowing/AfterAllTest.java  |  98 ----
 .../sdk/transforms/windowing/AfterEachTest.java |  64 ---
 .../transforms/windowing/AfterFirstTest.java    | 120 -----
 .../sdk/transforms/windowing/AfterPaneTest.java |  77 ---
 .../windowing/AfterProcessingTimeTest.java      |  94 ----
 .../AfterSynchronizedProcessingTimeTest.java    |  75 ---
 .../windowing/AfterWatermarkTest.java           | 308 -----------
 .../windowing/DefaultTriggerTest.java           | 130 -----
 .../sdk/transforms/windowing/NeverTest.java     |  34 +-
 .../windowing/OrFinallyTriggerTest.java         | 136 -----
 .../transforms/windowing/RepeatedlyTest.java    | 161 +-----
 .../sdk/transforms/windowing/StubTrigger.java   |  17 -
 .../sdk/transforms/windowing/TriggerTest.java   |  28 -
 .../beam/sdk/util/ExecutableTriggerTest.java    |  18 -
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 --
 .../sdk/util/FinishedTriggersProperties.java    | 110 ----
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ---
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  23 -
 .../org/apache/beam/sdk/util/TriggerTester.java | 410 ---------------
 39 files changed, 79 insertions(+), 3710 deletions(-)
----------------------------------------------------------------------



[21/50] [abbrv] incubator-beam git commit: OffsetBasedSource: allow empty sources

Posted by dh...@apache.org.
OffsetBasedSource: allow empty sources

As one example, empty files exist.


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

Branch: refs/heads/gearpump-runner
Commit: 7e9ad9954e50bd01fba4cda84c182af895b2c23f
Parents: dfaf2a8
Author: Dan Halperin <dh...@google.com>
Authored: Fri Oct 21 12:21:45 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 13:16:50 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  4 +--
 .../beam/sdk/io/OffsetBasedSourceTest.java      | 30 +++++++++++++++-----
 2 files changed, 25 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7e9ad995/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
index 6e49cc3..e9a398d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
@@ -146,8 +146,8 @@ public abstract class OffsetBasedSource<T> extends BoundedSource<T> {
         this.endOffset >= 0,
         "End offset has value %s, must be non-negative", this.endOffset);
     checkArgument(
-        this.startOffset < this.endOffset,
-        "Start offset %s must be before end offset %s",
+        this.startOffset <= this.endOffset,
+        "Start offset %s may not be larger than end offset %s",
         this.startOffset, this.endOffset);
     checkArgument(
         this.minBundleSize >= 0,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7e9ad995/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
index 923b4b4..6584e5d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
@@ -62,7 +62,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+    public boolean producesSortedKeys(PipelineOptions options) {
       return false;
     }
 
@@ -85,7 +85,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public OffsetBasedReader<Integer> createReader(PipelineOptions options) throws IOException {
+    public CoarseRangeReader createReader(PipelineOptions options) {
       return new CoarseRangeReader(this);
     }
   }
@@ -105,7 +105,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean startImpl() throws IOException {
+    public boolean startImpl() {
       current = getCurrentSource().getStartOffset();
       while (current % granularity != 0) {
         ++current;
@@ -114,7 +114,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean advanceImpl() throws IOException {
+    public boolean advanceImpl() {
       ++current;
       return true;
     }
@@ -130,7 +130,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public void close() throws IOException { }
+    public void close() { }
   }
 
   public static void assertSplitsAre(List<? extends OffsetBasedSource<?>> splits,
@@ -211,7 +211,7 @@ public class OffsetBasedSourceTest {
     // in the face of that.
     PipelineOptions options = PipelineOptionsFactory.create();
     CoarseRangeSource source = new CoarseRangeSource(13, 35, 1, 10);
-    try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
+    try (CoarseRangeReader reader = source.createReader(options)) {
       List<Integer> items = new ArrayList<>();
 
       assertEquals(0.0, reader.getFractionConsumed(), 1e-6);
@@ -304,7 +304,7 @@ public class OffsetBasedSourceTest {
   public void testSplitAtFraction() throws IOException {
     PipelineOptions options = PipelineOptionsFactory.create();
     CoarseRangeSource source = new CoarseRangeSource(13, 35, 1, 10);
-    try (CoarseRangeReader reader = (CoarseRangeReader) source.createReader(options)) {
+    try (CoarseRangeReader reader = source.createReader(options)) {
       List<Integer> originalItems = new ArrayList<>();
       assertTrue(reader.start());
       originalItems.add(reader.getCurrent());
@@ -341,4 +341,20 @@ public class OffsetBasedSourceTest {
     CoarseRangeSource original = new CoarseRangeSource(13, 35, 1, 10);
     assertSplitAtFractionExhaustive(original, options);
   }
+
+  @Test
+  public void testEmptyOffsetRange() throws Exception {
+    CoarseRangeSource empty = new CoarseRangeSource(0, 0, 1, 1);
+    try (CoarseRangeReader reader = empty.createReader(PipelineOptionsFactory.create())) {
+      assertEquals(0, reader.getSplitPointsConsumed());
+      assertEquals(OffsetBasedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining());
+      assertEquals(0.0, reader.getFractionConsumed(), 0.0001);
+
+      assertFalse(reader.start());
+
+      assertEquals(0, reader.getSplitPointsConsumed());
+      assertEquals(0, reader.getSplitPointsRemaining());
+      assertEquals(1.0, reader.getFractionConsumed(), 0.0001);
+    }
+  }
 }


[20/50] [abbrv] incubator-beam git commit: Closes #1169

Posted by dh...@apache.org.
Closes #1169


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

Branch: refs/heads/gearpump-runner
Commit: dfaf2a896029aac62d9db1f0000141206d42209d
Parents: b437c6b 5d01184
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 12:22:29 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 12:22:29 2016 -0700

----------------------------------------------------------------------
 .../examples/src/main/resources/archetype-resources/pom.xml  | 8 ++++----
 .../starter/src/main/resources/archetype-resources/pom.xml   | 2 +-
 .../src/test/resources/projects/basic/reference/pom.xml      | 2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[07/50] [abbrv] incubator-beam git commit: Fix ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers()

Posted by dh...@apache.org.
Fix ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers()

Before, it was only removing the end-of-window timer. Now, it also
removed that GC timer.


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

Branch: refs/heads/gearpump-runner
Commit: 59f042c85a7265d95620afd5aea565416cd35ff9
Parents: caf1fb7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 10:36:20 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 10:39:07 2016 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59f042c8/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
index 78c4e0b..66fb27c 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
@@ -943,7 +943,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> impleme
     directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
     Instant gc = garbageCollectionTime(directContext.window());
     if (gc.isAfter(eow)) {
-      directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+      directContext.timers().deleteTimer(gc, TimeDomain.EVENT_TIME);
     }
   }
 


[41/50] [abbrv] incubator-beam git commit: Remove use of OldDoFn in ValueWithRecordId

Posted by dh...@apache.org.
Remove use of OldDoFn in ValueWithRecordId


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

Branch: refs/heads/gearpump-runner
Commit: 4051357870b88ed18d09ef502ee8b1cf5142979f
Parents: 7abdcbf
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 15:57:18 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/util/ValueWithRecordId.java     | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/40513578/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
index d6b5fe3..abca598 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
@@ -31,7 +31,7 @@ import java.util.Objects;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StandardCoder;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 
 /**
  * Immutable struct containing a value as well as a unique id identifying the value.
@@ -136,9 +136,9 @@ public class ValueWithRecordId<ValueT> {
     ByteArrayCoder idCoder;
   }
 
-  /** {@link OldDoFn} to turn a {@code ValueWithRecordId<T>} back to the value {@code T}. */
-  public static class StripIdsDoFn<T> extends OldDoFn<ValueWithRecordId<T>, T> {
-    @Override
+  /** {@link DoFn} to turn a {@code ValueWithRecordId<T>} back to the value {@code T}. */
+  public static class StripIdsDoFn<T> extends DoFn<ValueWithRecordId<T>, T> {
+    @ProcessElement
     public void processElement(ProcessContext c) {
       c.output(c.element().getValue());
     }


[05/50] [abbrv] incubator-beam git commit: Fix binding of aggregator delegation in DoFnAdapters

Posted by dh...@apache.org.
Fix binding of aggregator delegation in DoFnAdapters


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

Branch: refs/heads/gearpump-runner
Commit: 2ab955d5501c87adea3fd17ad5dd1ad73be13364
Parents: 4e185d0
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 12:44:47 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ab955d5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
index 0b0d207..ca724cd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
@@ -278,6 +278,7 @@ public class DoFnAdapters {
     private ContextAdapter(DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.Context context) {
       fn.super();
       this.context = context;
+      super.setupDelegateAggregators();
     }
 
     @Override