You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2016/10/13 22:22:35 UTC

[01/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Repository: incubator-beam
Updated Branches:
  refs/heads/master 73226168a -> e969f3d38


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
new file mode 100644
index 0000000..119c937
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
@@ -0,0 +1,382 @@
+/*
+ * 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.triggers;
+
+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.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+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 AfterWatermarkStateMachine} triggers.
+ */
+@RunWith(JUnit4.class)
+public class AfterWatermarkStateMachineTest {
+
+  @Mock private OnceTriggerStateMachine mockEarly;
+  @Mock private OnceTriggerStateMachine mockLate;
+
+  private SimpleTriggerStateMachineTester<IntervalWindow> tester;
+  private static TriggerStateMachine.TriggerContext anyTriggerContext() {
+    return Mockito.<TriggerStateMachine.TriggerContext>any();
+  }
+  private static TriggerStateMachine.OnElementContext anyElementContext() {
+    return Mockito.<TriggerStateMachine.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(OnceTriggerStateMachine 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 = TriggerStateMachineTester.forTrigger(
+        AfterWatermarkStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterWatermarkStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterWatermarkStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            AfterWatermarkStateMachine.pastEndOfWindow(),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            AfterWatermarkStateMachine.pastEndOfWindow(),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterWatermarkStateMachine.pastEndOfWindow()
+            .withEarlyFirings(AfterPaneStateMachine.elementCountAtLeast(100))
+            .withLateFirings(AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterWatermarkStateMachine.pastEndOfWindow()
+            .withEarlyFirings(AfterPaneStateMachine.elementCountAtLeast(100))
+            .withLateFirings(AfterPaneStateMachine.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() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow();
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyFiringsToString() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow()
+        .withEarlyFirings(StubTriggerStateMachine.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testLateFiringsToString() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow()
+        .withLateFirings(StubTriggerStateMachine.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyAndLateFiringsToString() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow()
+        .withEarlyFirings(StubTriggerStateMachine.named("t1"))
+        .withLateFirings(StubTriggerStateMachine.named("t2"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testToStringExcludesNeverTrigger() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow()
+        .withEarlyFirings(NeverStateMachine.ever())
+        .withLateFirings(NeverStateMachine.ever());
+
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
new file mode 100644
index 0000000..b11d319
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link DefaultTriggerStateMachine}, which should be equivalent to
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+ */
+@RunWith(JUnit4.class)
+public class DefaultTriggerStateMachineTest {
+
+  SimpleTriggerStateMachineTester<IntervalWindow> tester;
+
+  @Test
+  public void testDefaultTriggerFixedWindows() throws Exception {
+    tester = TriggerStateMachineTester.forTrigger(
+        DefaultTriggerStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        DefaultTriggerStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        DefaultTriggerStateMachine.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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachineTest.java
new file mode 100644
index 0000000..744c220
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachineTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+
+import java.util.Arrays;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ExecutableTriggerStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class ExecutableTriggerStateMachineTest {
+
+  @Test
+  public void testIndexAssignmentLeaf() throws Exception {
+    StubStateMachine t1 = new StubStateMachine();
+    ExecutableTriggerStateMachine executable = ExecutableTriggerStateMachine.create(t1);
+    assertEquals(0, executable.getTriggerIndex());
+  }
+
+  @Test
+  public void testIndexAssignmentOneLevel() throws Exception {
+    StubStateMachine t1 = new StubStateMachine();
+    StubStateMachine t2 = new StubStateMachine();
+    StubStateMachine t = new StubStateMachine(t1, t2);
+
+    ExecutableTriggerStateMachine executable = ExecutableTriggerStateMachine.create(t);
+
+    assertEquals(0, executable.getTriggerIndex());
+    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
+    assertSame(t1, executable.subTriggers().get(0).getSpec());
+    assertEquals(2, executable.subTriggers().get(1).getTriggerIndex());
+    assertSame(t2, executable.subTriggers().get(1).getSpec());
+  }
+
+  @Test
+  public void testIndexAssignmentTwoLevel() throws Exception {
+    StubStateMachine t11 = new StubStateMachine();
+    StubStateMachine t12 = new StubStateMachine();
+    StubStateMachine t13 = new StubStateMachine();
+    StubStateMachine t14 = new StubStateMachine();
+    StubStateMachine t21 = new StubStateMachine();
+    StubStateMachine t22 = new StubStateMachine();
+    StubStateMachine t1 = new StubStateMachine(t11, t12, t13, t14);
+    StubStateMachine t2 = new StubStateMachine(t21, t22);
+    StubStateMachine t = new StubStateMachine(t1, t2);
+
+    ExecutableTriggerStateMachine executable = ExecutableTriggerStateMachine.create(t);
+
+    assertEquals(0, executable.getTriggerIndex());
+    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
+    assertEquals(6, executable.subTriggers().get(0).getFirstIndexAfterSubtree());
+    assertEquals(6, executable.subTriggers().get(1).getTriggerIndex());
+
+    assertSame(t1, executable.getSubTriggerContaining(1).getSpec());
+    assertSame(t2, executable.getSubTriggerContaining(6).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(2).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(3).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(5).getSpec());
+    assertSame(t2, executable.getSubTriggerContaining(7).getSpec());
+  }
+
+  private static class StubStateMachine extends TriggerStateMachine {
+
+    @SafeVarargs
+    protected StubStateMachine(TriggerStateMachine... subTriggers) {
+      super(Arrays.asList(subTriggers));
+    }
+
+    @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 boolean shouldFire(TriggerContext c) {
+      return false;
+    }
+
+    @Override
+    public void onFire(TriggerContext c) { }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSetTest.java
new file mode 100644
index 0000000..16bf49d
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSetTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.triggers;
+
+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())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersProperties.java
new file mode 100644
index 0000000..31d17c1
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersProperties.java
@@ -0,0 +1,115 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * 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, ExecutableTriggerStateMachine 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) {
+    ExecutableTriggerStateMachine trigger =
+        ExecutableTriggerStateMachine.create(
+            AfterAllStateMachine.of(
+                AfterFirstStateMachine.of(
+                    AfterPaneStateMachine.elementCountAtLeast(3),
+                    AfterWatermarkStateMachine.pastEndOfWindow()),
+                AfterAllStateMachine.of(
+                    AfterPaneStateMachine.elementCountAtLeast(10),
+                    AfterProcessingTimeStateMachine.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) {
+    ExecutableTriggerStateMachine trigger =
+        ExecutableTriggerStateMachine.create(
+            AfterAllStateMachine.of(
+                AfterFirstStateMachine.of(
+                    AfterPaneStateMachine.elementCountAtLeast(3),
+                    AfterWatermarkStateMachine.pastEndOfWindow()),
+                AfterAllStateMachine.of(
+                    AfterPaneStateMachine.elementCountAtLeast(10),
+                    AfterProcessingTimeStateMachine.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, ExecutableTriggerStateMachine trigger) {
+    finishedSet.setFinished(trigger, true);
+    for (ExecutableTriggerStateMachine subTrigger : trigger.subTriggers()) {
+      setFinishedRecursively(finishedSet, subTrigger);
+    }
+  }
+
+  private static void verifyFinishedRecursively(
+      FinishedTriggers finishedSet, ExecutableTriggerStateMachine trigger) {
+    assertTrue(finishedSet.isFinished(trigger));
+    for (ExecutableTriggerStateMachine subTrigger : trigger.subTriggers()) {
+      verifyFinishedRecursively(finishedSet, subTrigger);
+    }
+  }
+
+  private static void verifyUnfinishedRecursively(
+      FinishedTriggers finishedSet, ExecutableTriggerStateMachine trigger) {
+    assertFalse(finishedSet.isFinished(trigger));
+    for (ExecutableTriggerStateMachine subTrigger : trigger.subTriggers()) {
+      verifyUnfinishedRecursively(finishedSet, subTrigger);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersSetTest.java
new file mode 100644
index 0000000..fae73f3
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/FinishedTriggersSetTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.triggers;
+
+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<ExecutableTriggerStateMachine>()));
+  }
+
+  /**
+   * 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<ExecutableTriggerStateMachine>()));
+  }
+
+  @Test
+  public void testCopy() throws Exception {
+    FinishedTriggersSet finishedSet =
+        FinishedTriggersSet.fromSet(new HashSet<ExecutableTriggerStateMachine>());
+    assertThat(finishedSet.copy().getFinishedTriggers(),
+        not(theInstance(finishedSet.getFinishedTriggers())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
new file mode 100644
index 0000000..6d8a344
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.triggers;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+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.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 NeverStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class NeverStateMachineTest {
+  private SimpleTriggerStateMachineTester<IntervalWindow> triggerTester;
+
+  @Before
+  public void setup() throws Exception {
+    triggerTester =
+        TriggerStateMachineTester.forTrigger(
+            NeverStateMachine.ever(), FixedWindows.of(Duration.standardMinutes(5)));
+  }
+
+  @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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java
new file mode 100644
index 0000000..6e093c5
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.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.core.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link OrFinallyStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class OrFinallyStateMachineTest {
+
+  private SimpleTriggerStateMachineTester<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 = TriggerStateMachineTester.forTrigger(
+        new OrFinallyStateMachine(
+            AfterPaneStateMachine.elementCountAtLeast(2),
+            AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        new OrFinallyStateMachine(
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.elementCountAtLeast(2)),
+            AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            AfterPaneStateMachine.elementCountAtLeast(5)
+                .orFinally(AfterWatermarkStateMachine.pastEndOfWindow()),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        new OrFinallyStateMachine(
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.elementCountAtLeast(2)),
+                AfterPaneStateMachine.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 testToString() {
+    TriggerStateMachine trigger =
+        StubTriggerStateMachine.named("t1").orFinally(StubTriggerStateMachine.named("t2"));
+    assertEquals("t1.orFinally(t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
new file mode 100644
index 0000000..b52f41d
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+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 RepeatedlyStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class RepeatedlyStateMachineTest {
+
+  @Mock private TriggerStateMachine mockTrigger;
+  private SimpleTriggerStateMachineTester<IntervalWindow> tester;
+  private static TriggerStateMachine.TriggerContext anyTriggerContext() {
+    return Mockito.<TriggerStateMachine.TriggerContext>any();
+  }
+
+  public void setUp(WindowFn<Object, IntervalWindow> windowFn) throws Exception {
+    MockitoAnnotations.initMocks(this);
+    tester = TriggerStateMachineTester
+        .forTrigger(RepeatedlyStateMachine.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.<TriggerStateMachine.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.<TriggerStateMachine.TriggerContext>any()))
+        .thenReturn(false);
+    assertFalse(tester.shouldFire(new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testShouldFireAfterMerge() throws Exception {
+    tester = TriggerStateMachineTester.forTrigger(
+        RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 {
+    SimpleTriggerStateMachineTester<GlobalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            RepeatedlyStateMachine.forever(
+                AfterFirstStateMachine.of(
+                    AfterProcessingTimeStateMachine.pastFirstElementInPane()
+                        .plusDelayOf(Duration.standardMinutes(15)),
+                    AfterPaneStateMachine.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 {
+    SimpleTriggerStateMachineTester<GlobalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            RepeatedlyStateMachine.forever(
+                AfterFirstStateMachine.of(
+                    AfterProcessingTimeStateMachine.pastFirstElementInPane()
+                        .plusDelayOf(Duration.standardMinutes(15)),
+                    AfterPaneStateMachine.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 {
+    SimpleTriggerStateMachineTester<GlobalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 {
+    SimpleTriggerStateMachineTester<GlobalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            RepeatedlyStateMachine.forever(
+                    AfterProcessingTimeStateMachine.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() {
+    TriggerStateMachine trigger = RepeatedlyStateMachine.forever(new StubTriggerStateMachine() {
+        @Override
+        public String toString() {
+          return "innerTrigger";
+        }
+      });
+
+    assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
new file mode 100644
index 0000000..ef74bb5
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.triggers;
+
+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.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ReshuffleTriggerStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class ReshuffleTriggerStateMachineTest {
+
+  /** Public so that other tests can instantiate {@link ReshuffleTriggerStateMachine}. */
+  public static <W extends BoundedWindow> ReshuffleTriggerStateMachine forTest() {
+    return new ReshuffleTriggerStateMachine();
+  }
+
+  @Test
+  public void testShouldFire() throws Exception {
+    TriggerStateMachineTester<Integer, IntervalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            new ReshuffleTriggerStateMachine(), 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 {
+    TriggerStateMachineTester<Integer, IntervalWindow> tester =
+        TriggerStateMachineTester.forTrigger(
+            new ReshuffleTriggerStateMachine(), 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() {
+    TriggerStateMachine trigger = new ReshuffleTriggerStateMachine();
+    assertEquals("ReshuffleTriggerStateMachine()", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/StubTriggerStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/StubTriggerStateMachine.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/StubTriggerStateMachine.java
new file mode 100644
index 0000000..4512848
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/StubTriggerStateMachine.java
@@ -0,0 +1,60 @@
+/*
+ * 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.triggers;
+
+import com.google.common.collect.Lists;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+
+/**
+ * No-op {@link OnceTriggerStateMachine} implementation for testing.
+ */
+abstract class StubTriggerStateMachine extends OnceTriggerStateMachine {
+  /**
+   * Create a stub {@link TriggerStateMachine} instance which returns the specified name on {@link
+   * #toString()}.
+   */
+  static StubTriggerStateMachine named(final String name) {
+    return new StubTriggerStateMachine() {
+      @Override
+      public String toString() {
+        return name;
+      }
+    };
+  }
+
+  protected StubTriggerStateMachine() {
+    super(Lists.<TriggerStateMachine>newArrayList());
+  }
+
+  @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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java
new file mode 100644
index 0000000..e06eb85
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.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.core.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link TriggerStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class TriggerStateMachineTest {
+
+  @Test
+  public void testTriggerToString() throws Exception {
+    assertEquals(
+        "AfterWatermark.pastEndOfWindow()",
+        AfterWatermarkStateMachine.pastEndOfWindow().toString());
+    assertEquals(
+        "Repeatedly.forever(AfterWatermark.pastEndOfWindow())",
+        RepeatedlyStateMachine.forever(AfterWatermarkStateMachine.pastEndOfWindow()).toString());
+  }
+
+  @Test
+  public void testIsCompatible() throws Exception {
+    assertTrue(new Trigger1(null).isCompatible(new Trigger1(null)));
+    assertTrue(new Trigger1(Arrays.<TriggerStateMachine>asList(new Trigger2(null)))
+        .isCompatible(new Trigger1(Arrays.<TriggerStateMachine>asList(new Trigger2(null)))));
+
+    assertFalse(new Trigger1(null).isCompatible(new Trigger2(null)));
+    assertFalse(new Trigger1(Arrays.<TriggerStateMachine>asList(new Trigger1(null)))
+        .isCompatible(new Trigger1(Arrays.<TriggerStateMachine>asList(new Trigger2(null)))));
+  }
+
+  private static class Trigger1 extends TriggerStateMachine {
+
+    private Trigger1(List<TriggerStateMachine> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(TriggerStateMachine.OnElementContext c) { }
+
+    @Override
+    public void onMerge(TriggerStateMachine.OnMergeContext c) { }
+
+    @Override
+    public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+      return false;
+    }
+
+    @Override
+    public void onFire(TriggerStateMachine.TriggerContext context) throws Exception { }
+  }
+
+  private static class Trigger2 extends TriggerStateMachine {
+
+    private Trigger2(List<TriggerStateMachine> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(TriggerStateMachine.OnElementContext c) { }
+
+    @Override
+    public void onMerge(TriggerStateMachine.OnMergeContext c) { }
+
+    @Override
+    public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+      return false;
+    }
+
+    @Override
+    public void onFire(TriggerStateMachine.TriggerContext context) throws Exception { }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
new file mode 100644
index 0000000..1ccca17
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
@@ -0,0 +1,431 @@
+/*
+ * 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.triggers;
+
+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.WindowFn;
+import org.apache.beam.sdk.util.ActiveWindowSet;
+import org.apache.beam.sdk.util.ActiveWindowSet.MergeCallback;
+import org.apache.beam.sdk.util.MergingActiveWindowSet;
+import org.apache.beam.sdk.util.NonMergingActiveWindowSet;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.Timers;
+import org.apache.beam.sdk.util.WindowTracing;
+import org.apache.beam.sdk.util.WindowedValue;
+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 TriggerStateMachine}, using in-memory stub implementation to
+ * provide the {@link StateInternals}.
+ *
+ * @param <W> The type of windows being used.
+ */
+public class TriggerStateMachineTester<InputT, W extends BoundedWindow> {
+
+  /**
+   * A {@link TriggerStateMachineTester} 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 SimpleTriggerStateMachineTester<W extends BoundedWindow>
+      extends TriggerStateMachineTester<Integer, W> {
+
+    private SimpleTriggerStateMachineTester(
+        ExecutableTriggerStateMachine executableTriggerStateMachine,
+        WindowFn<Object, W> windowFn,
+        Duration allowedLateness)
+        throws Exception {
+      super(executableTriggerStateMachine, windowFn, allowedLateness);
+    }
+
+    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 SimpleTriggerStateMachineTester<W> withAllowedLateness(Duration allowedLateness)
+        throws Exception {
+      return new SimpleTriggerStateMachineTester<>(
+              executableTrigger,
+              windowFn,
+              allowedLateness);
+    }
+  }
+
+  private final TestInMemoryStateInternals<?> stateInternals =
+      new TestInMemoryStateInternals<Object>(null /* key */);
+  private final InMemoryTimerInternals timerInternals = new InMemoryTimerInternals();
+  private final TriggerStateMachineContextFactory<W> contextFactory;
+  protected final WindowFn<Object, W> windowFn;
+  private final ActiveWindowSet<W> activeWindows;
+  private final Map<W, W> windowToMergeResult;
+
+  /**
+   * An {@link ExecutableTriggerStateMachine} under test.
+   */
+  protected final ExecutableTriggerStateMachine 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> SimpleTriggerStateMachineTester<W> forTrigger(
+      TriggerStateMachine stateMachine, WindowFn<Object, W> windowFn)
+          throws Exception {
+
+    ExecutableTriggerStateMachine executableTriggerStateMachine =
+        ExecutableTriggerStateMachine.create(stateMachine);
+
+    // 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.
+    AccumulationMode mode =
+        windowFn.isNonMerging()
+            ? AccumulationMode.DISCARDING_FIRED_PANES
+            : AccumulationMode.ACCUMULATING_FIRED_PANES;
+
+    return new SimpleTriggerStateMachineTester<>(
+        executableTriggerStateMachine, windowFn, Duration.ZERO);
+  }
+
+  public static <InputT, W extends BoundedWindow>
+  TriggerStateMachineTester<InputT, W> forAdvancedTrigger(
+          TriggerStateMachine stateMachine, WindowFn<Object, W> windowFn) throws Exception {
+    ExecutableTriggerStateMachine executableTriggerStateMachine =
+        ExecutableTriggerStateMachine.create(stateMachine);
+
+    // 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.
+    AccumulationMode mode =
+            windowFn.isNonMerging()
+                    ? AccumulationMode.DISCARDING_FIRED_PANES
+                    : AccumulationMode.ACCUMULATING_FIRED_PANES;
+
+    return new TriggerStateMachineTester<>(executableTriggerStateMachine, windowFn, Duration.ZERO);
+  }
+
+  protected TriggerStateMachineTester(
+                                 ExecutableTriggerStateMachine executableTriggerStateMachine,
+          WindowFn<Object, W> windowFn,
+                                 Duration allowedLateness) throws Exception {
+    this.windowFn = windowFn;
+    this.executableTrigger = executableTriggerStateMachine;
+    this.finishedSets = new HashMap<>();
+
+    this.activeWindows =
+        windowFn.isNonMerging()
+            ? new NonMergingActiveWindowSet<W>()
+            : new MergingActiveWindowSet<W>(windowFn, stateInternals);
+    this.windowToMergeResult = new HashMap<>();
+
+    this.contextFactory =
+        new TriggerStateMachineContextFactory<>(
+            windowFn, 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 TriggerStateMachine} 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);
+
+        TriggerStateMachine.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 {
+    TriggerStateMachine.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 {
+    TriggerStateMachine.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<ExecutableTriggerStateMachine>());
+      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();
+    }
+  }
+}


[12/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java
new file mode 100644
index 0000000..a9feb73
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java
@@ -0,0 +1,72 @@
+/*
+ * 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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
new file mode 100644
index 0000000..5f20465
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
@@ -0,0 +1,75 @@
+/*
+ * 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.windowing;
+
+import java.util.List;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.joda.time.Instant;
+
+/**
+ * A trigger which never fires.
+ *
+ * <p>Using this trigger will only produce output when the watermark passes the end of the
+ * {@link BoundedWindow window} plus the {@link Window#withAllowedLateness allowed
+ * lateness}.
+ */
+public final class Never {
+  /**
+   * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
+   * when the {@link BoundedWindow} closes.
+   */
+  public static OnceTrigger ever() {
+    // NeverTrigger ignores all inputs and is Window-type independent.
+    return new NeverTrigger();
+  }
+
+  // package-private in order to check identity for string formatting.
+  static class NeverTrigger extends OnceTrigger {
+    protected NeverTrigger() {
+      super(null);
+    }
+
+    @Override
+    public void onElement(OnElementContext c) {}
+
+    @Override
+    public void onMerge(OnMergeContext c) {}
+
+    @Override
+    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+      return this;
+    }
+
+    @Override
+    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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
new file mode 100644
index 0000000..25b7b34
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
@@ -0,0 +1,105 @@
+/*
+ * 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.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;
+
+/**
+ * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
+ */
+class OrFinallyTrigger extends Trigger {
+
+  private static final int ACTUAL = 0;
+  private static final int UNTIL = 1;
+
+  @VisibleForTesting OrFinallyTrigger(Trigger actual, Trigger.OnceTrigger until) {
+    super(Arrays.asList(actual, until));
+  }
+
+  @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);
+    Instant untilDeadline = subTriggers.get(UNTIL).getWatermarkThatGuaranteesFiring(window);
+    return actualDeadline.isBefore(untilDeadline) ? actualDeadline : untilDeadline;
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    // Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL
+    // may not be a OnceTrigger.
+    return Repeatedly.forever(
+        new OrFinallyTrigger(
+            continuationTriggers.get(ACTUAL),
+            (Trigger.OnceTrigger) continuationTriggers.get(UNTIL)));
+  }
+
+  @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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
new file mode 100644
index 0000000..8858798
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
@@ -0,0 +1,101 @@
+/*
+ * 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.windowing;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Repeat a trigger, either until some condition is met or forever.
+ *
+ * <p>For example, to fire after the end of the window, and every time late data arrives:
+ * <pre> {@code
+ *     Repeatedly.forever(AfterWatermark.isPastEndOfWindow());
+ * } </pre>
+ *
+ * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite
+ * {@code AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
+ */
+public class Repeatedly extends Trigger {
+
+  private static final int REPEATED = 0;
+
+  /**
+   * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each
+   * time it fires and ignoring any indications to finish.
+   *
+   * <p>Unless used with {@link Trigger#orFinally} the composite trigger will never finish.
+   *
+   * @param repeated the trigger to execute repeatedly.
+   */
+  public static Repeatedly forever(Trigger repeated) {
+    return new Repeatedly(repeated);
+  }
+
+  private Repeatedly(Trigger repeated) {
+    super(Arrays.asList(repeated));
+  }
+
+
+  @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);
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new Repeatedly(continuationTriggers.get(REPEATED));
+  }
+
+  @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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
new file mode 100644
index 0000000..9e2c27d
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
@@ -0,0 +1,66 @@
+/*
+ * 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.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.joda.time.Instant;
+
+/**
+ * The trigger used with {@link Reshuffle} which triggers on every element
+ * and never buffers state.
+ *
+ * @param <W> The kind of window that is being reshuffled.
+ */
+public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
+
+  public ReshuffleTrigger() {
+    super(null);
+  }
+
+  @Override
+  public void onElement(Trigger.OnElementContext c) { }
+
+  @Override
+  public void onMerge(Trigger.OnMergeContext c) { }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return this;
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    throw new UnsupportedOperationException(
+        "ReshuffleTrigger should not be used outside of Reshuffle");
+  }
+
+  @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()";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
new file mode 100644
index 0000000..a960aa4
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
@@ -0,0 +1,527 @@
+/*
+ * 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.windowing;
+
+import com.google.common.base.Joiner;
+import java.io.Serializable;
+import java.util.ArrayList;
+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.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.
+ *
+ * <p>See {@link org.apache.beam.sdk.transforms.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
+ * elements as they are added. When the root trigger fires, the elements in the current pane will be
+ * 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:
+ * <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.
+ * </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.
+ * </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. */
+    public abstract boolean finishedInAnyMergingWindow();
+
+    /** Return true if the trigger is finished in all windows being merged. */
+    public abstract 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();
+  }
+
+  @Nullable
+  protected final List<Trigger> subTriggers;
+
+  protected Trigger(@Nullable List<Trigger> subTriggers) {
+    this.subTriggers = subTriggers;
+  }
+
+
+  /**
+   * 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 Iterable<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
+   * continue firing once (or multiple times).
+   */
+  public Trigger getContinuationTrigger() {
+    if (subTriggers == null) {
+      return getContinuationTrigger(null);
+    }
+
+    List<Trigger> subTriggerContinuations = new ArrayList<>();
+    for (Trigger subTrigger : subTriggers) {
+      subTriggerContinuations.add(subTrigger.getContinuationTrigger());
+    }
+    return getContinuationTrigger(subTriggerContinuations);
+  }
+
+  /**
+   * Return the {@link #getContinuationTrigger} of this {@code 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.
+   *
+   * <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.
+   */
+  public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window);
+
+  /**
+   * Returns whether this performs the same triggering as the given {@code Trigger}.
+   */
+  public boolean isCompatible(Trigger other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    if (subTriggers == null) {
+      return other.subTriggers == null;
+    } else if (other.subTriggers == null) {
+      return false;
+    } else if (subTriggers.size() != other.subTriggers.size()) {
+      return false;
+    }
+
+    for (int i = 0; i < subTriggers.size(); i++) {
+      if (!subTriggers.get(i).isCompatible(other.subTriggers.get(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    String simpleName = getClass().getSimpleName();
+    if (getClass().getEnclosingClass() != null) {
+      simpleName = getClass().getEnclosingClass().getSimpleName() + "." + simpleName;
+    }
+    if (subTriggers == null || subTriggers.size() == 0) {
+      return simpleName;
+    } else {
+      return simpleName + "(" + Joiner.on(", ").join(subTriggers) + ")";
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof Trigger)) {
+      return false;
+    }
+    Trigger that = (Trigger) obj;
+    return Objects.equals(getClass(), that.getClass())
+        && Objects.equals(subTriggers, that.subTriggers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), subTriggers);
+  }
+
+  /**
+   * Specify an ending condition for this trigger. If the {@code until} 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>For example the final firing of the following trigger may only have 1 element:
+   * <pre> {@code
+   * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
+   *     .orFinally(AfterPane.elementCountAtLeast(5))
+   * } </pre>
+   *
+   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same
+   * as {@code AfterFirst.of(t1, t2)}.
+   */
+  public Trigger 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.
+   */
+  public abstract static class OnceTrigger extends Trigger {
+    protected OnceTrigger(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public final OnceTrigger getContinuationTrigger() {
+      Trigger continuation = super.getContinuationTrigger();
+      if (!(continuation instanceof OnceTrigger)) {
+        throw new IllegalStateException("Continuation of a OnceTrigger must be a OnceTrigger");
+      }
+      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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java
new file mode 100644
index 0000000..e09aac2
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java
@@ -0,0 +1,507 @@
+/*
+ * 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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
new file mode 100644
index 0000000..8d0f322
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
@@ -0,0 +1,247 @@
+/*
+ * 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.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.BitSetCoder;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.FinishedTriggers;
+import org.apache.beam.sdk.util.FinishedTriggersBitSet;
+import org.apache.beam.sdk.util.Timers;
+import org.apache.beam.sdk.util.TriggerContextFactory;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.joda.time.Instant;
+
+/**
+ * Executes a trigger while managing persistence of information about which subtriggers are
+ * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
+ *
+ * <p>Specifically, the responsibilities are:
+ *
+ * <ul>
+ *   <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
+ *       constructing the appropriate trigger contexts.</li>
+ *   <li>Committing a record of which subtriggers are finished to persistent state.</li>
+ *   <li>Restoring the record of which subtriggers are finished from persistent state.</li>
+ *   <li>Clearing out the persisted finished set when a caller indicates
+ *       (via {#link #clearFinished}) that it is no longer needed.</li>
+ * </ul>
+ *
+ * <p>These responsibilities are intertwined: trigger contexts include mutable information about
+ * which subtriggers are finished. This class provides the information when building the contexts
+ * and commits the information when the method of the {@link ExecutableTrigger} returns.
+ *
+ * @param <W> The kind of windows being processed.
+ */
+public class TriggerRunner<W extends BoundedWindow> {
+  @VisibleForTesting
+  static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
+      StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
+
+  private final ExecutableTrigger rootTrigger;
+  private final TriggerContextFactory<W> contextFactory;
+
+  public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
+    checkState(rootTrigger.getTriggerIndex() == 0);
+    this.rootTrigger = rootTrigger;
+    this.contextFactory = contextFactory;
+  }
+
+  private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // If no trigger in the tree will ever have finished bits, then we don't need to read them.
+      // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
+      // finished) for each trigger in the tree.
+      return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
+    }
+
+    BitSet bitSet = state.read();
+    return bitSet == null
+        ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
+            : FinishedTriggersBitSet.fromBitSet(bitSet);
+  }
+
+
+  private void clearFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // Nothing to clear.
+      return;
+    }
+    state.clear();
+  }
+
+  /** Return true if the trigger is closed in the window corresponding to the specified state. */
+  public boolean isClosed(StateAccessor<?> state) {
+    return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchForValue(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnElement(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchOnFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchShouldFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchShouldFire(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  /**
+   * Run the trigger logic to deal with a new value.
+   */
+  public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
+      throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
+        window, timers, timestamp, rootTrigger, finishedSet);
+    rootTrigger.invokeOnElement(triggerContext);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchForMerge(
+      W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
+    if (isFinishedSetNeeded()) {
+      for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
+        value.readLater();
+      }
+    }
+    rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
+        window, mergingWindows, rootTrigger));
+  }
+
+  /**
+   * Run the trigger merging logic as part of executing the specified merge.
+   */
+  public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+
+    // And read the finished bits in each merging window.
+    ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
+    for (Map.Entry<W, ValueState<BitSet>> entry :
+        state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
+      // Don't need to clone these, since the trigger context doesn't allow modification
+      builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
+      // Clear the underlying finished bits.
+      clearFinishedBits(entry.getValue());
+    }
+    ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
+
+    Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
+        window, timers, rootTrigger, finishedSet, mergingFinishedSets);
+
+    // Run the merge from the trigger
+    rootTrigger.invokeOnMerge(mergeContext);
+
+    persistFinishedSet(state, finishedSet);
+  }
+
+  public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    return rootTrigger.invokeShouldFire(context);
+  }
+
+  public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // shouldFire should be false.
+    // However it is too expensive to assert.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    rootTrigger.invokeOnFire(context);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  private void persistFinishedSet(
+      StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
+    if (!isFinishedSetNeeded()) {
+      return;
+    }
+
+    ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
+    if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
+      if (modifiedFinishedSet.getBitSet().isEmpty()) {
+        finishedSetState.clear();
+      } else {
+        finishedSetState.write(modifiedFinishedSet.getBitSet());
+      }
+    }
+  }
+
+  /**
+   * Clear the finished bits.
+   */
+  public void clearFinished(StateAccessor<?> state) {
+    clearFinishedBits(state.access(FINISHED_BITS_TAG));
+  }
+
+  /**
+   * Clear the state used for executing triggers, but leave the finished set to indicate
+   * the window is closed.
+   */
+  public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // Don't need to clone, because we'll be clearing the finished bits anyways.
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
+    rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
+  }
+
+  private boolean isFinishedSetNeeded() {
+    // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
+    // lookup. Right now, we special case this for the DefaultTrigger.
+    return !(rootTrigger.getSpec() instanceof DefaultTrigger);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
new file mode 100644
index 0000000..b591229
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterAll}.
+ */
+@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));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterAll.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(1))
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterAll = AfterAll.of(trigger1, trigger2);
+    assertEquals(
+        AfterAll.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
+        afterAll.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterAll.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
+    assertEquals("AfterAll.of(t1, t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
new file mode 100644
index 0000000..c413c6e
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.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}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        AfterEach.inOrder(AfterWatermark.pastEndOfWindow(),
+                          AfterPane.elementCountAtLeast(4))
+            .getWatermarkThatGuaranteesFiring(window));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterEach.inOrder(AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow())
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterEach = AfterEach.inOrder(trigger1, trigger2);
+    assertEquals(
+        Repeatedly.forever(AfterFirst.of(
+            trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger())),
+        afterEach.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterEach.inOrder(
+        StubTrigger.named("t1"),
+        StubTrigger.named("t2"),
+        StubTrigger.named("t3"));
+
+    assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
new file mode 100644
index 0000000..415060b
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.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}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        AfterFirst.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4))
+            .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterFirst.of(AfterPane.elementCountAtLeast(2), AfterPane.elementCountAtLeast(1))
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterFirst = AfterFirst.of(trigger1, trigger2);
+    assertEquals(
+        AfterFirst.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
+        afterFirst.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterFirst.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
+    assertEquals("AfterFirst.of(t1, t2)", trigger.toString());
+  }
+}


[08/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 7e6e938..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
+++ /dev/null
@@ -1,121 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link AfterSynchronizedProcessingTime}.
- */
-@RunWith(JUnit4.class)
-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(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(underTest, underTest.getContinuationTrigger());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 084027b..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
+++ /dev/null
@@ -1,380 +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.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.
- */
-@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();
-    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
-  }
-
-  @Test
-  public void testEarlyFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow().withEarlyFirings(StubTrigger.named("t1"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
-  }
-
-  @Test
-  public void testLateFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow().withLateFirings(StubTrigger.named("t1"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
-  }
-
-  @Test
-  public void testEarlyAndLateFiringsToString() {
-    Trigger trigger =
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(StubTrigger.named("t1"))
-            .withLateFirings(StubTrigger.named("t2"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
-        trigger.toString());
-  }
-
-  @Test
-  public void testToStringExcludesNeverTrigger() {
-    Trigger trigger =
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(Never.ever())
-            .withLateFirings(Never.ever());
-
-    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 673e555..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
+++ /dev/null
@@ -1,176 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link DefaultTrigger}, which should be equivalent to
- * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
- */
-@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(
-        new IntervalWindow(new Instant(0), new Instant(10))));
-    assertEquals(GlobalWindow.INSTANCE.maxTimestamp(),
-        DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(DefaultTrigger.of(), DefaultTrigger.of().getContinuationTrigger());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index fb2b4d5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
+++ /dev/null
@@ -1,56 +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.windowing;
-
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-
-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}.
- */
-@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 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));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 7289d97..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
+++ /dev/null
@@ -1,215 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link OrFinallyTrigger}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        Repeatedly.forever(AfterWatermark.pastEndOfWindow())
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow())
-        .orFinally(AfterPane.elementCountAtLeast(1))
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9), Repeatedly.forever(AfterPane.elementCountAtLeast(1))
-        .orFinally(AfterWatermark.pastEndOfWindow())
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9),
-        AfterPane.elementCountAtLeast(100)
-            .orFinally(AfterWatermark.pastEndOfWindow())
-            .getWatermarkThatGuaranteesFiring(window));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        Repeatedly.forever(AfterPane.elementCountAtLeast(1))
-        .orFinally(AfterPane.elementCountAtLeast(10))
-        .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger triggerA = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger triggerB = AfterWatermark.pastEndOfWindow();
-    Trigger aOrFinallyB = triggerA.orFinally(triggerB);
-    Trigger bOrFinallyA = triggerB.orFinally(triggerA);
-    assertEquals(
-        Repeatedly.forever(
-            triggerA.getContinuationTrigger().orFinally(triggerB.getContinuationTrigger())),
-        aOrFinallyB.getContinuationTrigger());
-    assertEquals(
-        Repeatedly.forever(
-            triggerB.getContinuationTrigger().orFinally(triggerA.getContinuationTrigger())),
-        bOrFinallyA.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = StubTrigger.named("t1").orFinally(StubTrigger.named("t2"));
-    assertEquals("t1.orFinally(t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 6e8930d..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
+++ /dev/null
@@ -1,224 +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.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;
-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 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.
-   */
-  @Test
-  public void testFireDeadline() throws Exception {
-    setUp(FixedWindows.of(Duration.millis(10)));
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-    Instant arbitraryInstant = new Instant(34957849);
-
-    when(mockTrigger.getWatermarkThatGuaranteesFiring(Mockito.<IntervalWindow>any()))
-        .thenReturn(arbitraryInstant);
-
-    assertThat(
-        Repeatedly.forever(mockTrigger).getWatermarkThatGuaranteesFiring(window),
-        equalTo(arbitraryInstant));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
-    Trigger repeatedly = Repeatedly.forever(trigger);
-    assertEquals(
-        Repeatedly.forever(trigger.getContinuationTrigger()), repeatedly.getContinuationTrigger());
-    assertEquals(
-        Repeatedly.forever(trigger.getContinuationTrigger().getContinuationTrigger()),
-        repeatedly.getContinuationTrigger().getContinuationTrigger());
-  }
-
-  @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";
-        }
-      });
-
-    assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index b258a79..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
+++ /dev/null
@@ -1,70 +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.windowing;
-
-import com.google.common.collect.Lists;
-import java.util.List;
-import org.joda.time.Instant;
-
-/**
- * No-op {@link OnceTrigger} implementation for testing.
- */
-abstract class StubTrigger extends Trigger.OnceTrigger {
-  /**
-   * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}.
-   */
-  static StubTrigger named(final String name) {
-    return new StubTrigger() {
-      @Override
-      public String toString() {
-        return name;
-      }
-    };
-  }
-
-  protected StubTrigger() {
-    super(Lists.<Trigger>newArrayList());
-  }
-
-  @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;
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index cfc03b2..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
+++ /dev/null
@@ -1,118 +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.windowing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Arrays;
-import java.util.List;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link Trigger}.
- */
-@RunWith(JUnit4.class)
-public class TriggerTest {
-
-  @Test
-  public void testTriggerToString() throws Exception {
-    assertEquals("AfterWatermark.pastEndOfWindow()", AfterWatermark.pastEndOfWindow().toString());
-    assertEquals("Repeatedly.forever(AfterWatermark.pastEndOfWindow())",
-        Repeatedly.forever(AfterWatermark.pastEndOfWindow()).toString());
-  }
-
-  @Test
-  public void testIsCompatible() throws Exception {
-    assertTrue(new Trigger1(null).isCompatible(new Trigger1(null)));
-    assertTrue(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))
-        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
-
-    assertFalse(new Trigger1(null).isCompatible(new Trigger2(null)));
-    assertFalse(new Trigger1(Arrays.<Trigger>asList(new Trigger1(null)))
-        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
-  }
-
-  private static class Trigger1 extends Trigger {
-
-    private Trigger1(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
-    protected Trigger getContinuationTrigger(
-        List<Trigger> continuationTriggers) {
-      return null;
-    }
-
-    @Override
-    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 {
-
-    private Trigger2(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
-    protected Trigger getContinuationTrigger(
-        List<Trigger> continuationTriggers) {
-      return null;
-    }
-
-    @Override
-    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/e4398e1e/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
deleted file mode 100644
index 1e3a1ff..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java
+++ /dev/null
@@ -1,127 +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.assertEquals;
-import static org.junit.Assert.assertSame;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ExecutableTrigger}.
- */
-@RunWith(JUnit4.class)
-public class ExecutableTriggerTest {
-
-  @Test
-  public void testIndexAssignmentLeaf() throws Exception {
-    StubTrigger t1 = new StubTrigger();
-    ExecutableTrigger executable = ExecutableTrigger.create(t1);
-    assertEquals(0, executable.getTriggerIndex());
-  }
-
-  @Test
-  public void testIndexAssignmentOneLevel() throws Exception {
-    StubTrigger t1 = new StubTrigger();
-    StubTrigger t2 = new StubTrigger();
-    StubTrigger t = new StubTrigger(t1, t2);
-
-    ExecutableTrigger executable = ExecutableTrigger.create(t);
-
-    assertEquals(0, executable.getTriggerIndex());
-    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
-    assertSame(t1, executable.subTriggers().get(0).getSpec());
-    assertEquals(2, executable.subTriggers().get(1).getTriggerIndex());
-    assertSame(t2, executable.subTriggers().get(1).getSpec());
-  }
-
-  @Test
-  public void testIndexAssignmentTwoLevel() throws Exception {
-    StubTrigger t11 = new StubTrigger();
-    StubTrigger t12 = new StubTrigger();
-    StubTrigger t13 = new StubTrigger();
-    StubTrigger t14 = new StubTrigger();
-    StubTrigger t21 = new StubTrigger();
-    StubTrigger t22 = new StubTrigger();
-    StubTrigger t1 = new StubTrigger(t11, t12, t13, t14);
-    StubTrigger t2 = new StubTrigger(t21, t22);
-    StubTrigger t = new StubTrigger(t1, t2);
-
-    ExecutableTrigger executable = ExecutableTrigger.create(t);
-
-    assertEquals(0, executable.getTriggerIndex());
-    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
-    assertEquals(6, executable.subTriggers().get(0).getFirstIndexAfterSubtree());
-    assertEquals(6, executable.subTriggers().get(1).getTriggerIndex());
-
-    assertSame(t1, executable.getSubTriggerContaining(1).getSpec());
-    assertSame(t2, executable.getSubTriggerContaining(6).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(2).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(3).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(5).getSpec());
-    assertSame(t2, executable.getSubTriggerContaining(7).getSpec());
-  }
-
-  private static class StubTrigger extends Trigger {
-
-    @SafeVarargs
-    protected StubTrigger(Trigger... subTriggers) {
-      super(Arrays.asList(subTriggers));
-    }
-
-    @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;
-    }
-
-    @Override
-    public boolean isCompatible(Trigger other) {
-      return false;
-    }
-
-    @Override
-    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/e4398e1e/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())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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/e4398e1e/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/e4398e1e/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
deleted file mode 100644
index 83077f4..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.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 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;
-
-/**
- * Tests for {@link ReshuffleTrigger}.
- */
-@RunWith(JUnit4.class)
-public class ReshuffleTriggerTest {
-
-  /** Public so that other tests can instantiate {@link ReshuffleTrigger}. */
-  public static <W extends BoundedWindow> ReshuffleTrigger<W> forTest() {
-    return new ReshuffleTrigger<>();
-  }
-
-  @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());
-  }
-}


[07/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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();
-    }
-  }
-}


[16/17] incubator-beam git commit: Restore prior trigger files, for temporary compatibility

Posted by ke...@apache.org.
Restore prior trigger files, for temporary compatibility


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

Branch: refs/heads/master
Commit: a64acb2f84ac26bd1a3f297085477f13b0252570
Parents: 69b1efd
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 11 21:35:02 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 13 14:34:35 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/TriggerRunner.java | 247 +++++++++
 .../beam/sdk/transforms/windowing/AfterAll.java | 122 +++++
 .../windowing/AfterDelayFromFirstElement.java   | 335 ++++++++++++
 .../sdk/transforms/windowing/AfterEach.java     | 141 +++++
 .../sdk/transforms/windowing/AfterFirst.java    | 124 +++++
 .../sdk/transforms/windowing/AfterPane.java     | 144 +++++
 .../windowing/AfterProcessingTime.java          | 102 ++++
 .../AfterSynchronizedProcessingTime.java        |  73 +++
 .../transforms/windowing/AfterWatermark.java    | 355 +++++++++++++
 .../transforms/windowing/DefaultTrigger.java    |  92 ++++
 .../beam/sdk/transforms/windowing/Never.java    |  75 +++
 .../transforms/windowing/OrFinallyTrigger.java  | 105 ++++
 .../sdk/transforms/windowing/Repeatedly.java    | 101 ++++
 .../beam/sdk/transforms/windowing/Trigger.java  | 527 +++++++++++++++++++
 .../apache/beam/sdk/util/ExecutableTrigger.java | 159 ++++++
 .../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  |  66 +++
 .../beam/sdk/util/TriggerContextFactory.java    | 507 ++++++++++++++++++
 .../sdk/transforms/windowing/AfterAllTest.java  | 156 ++++++
 .../sdk/transforms/windowing/AfterEachTest.java | 132 +++++
 .../transforms/windowing/AfterFirstTest.java    | 181 +++++++
 .../sdk/transforms/windowing/AfterPaneTest.java | 132 +++++
 .../windowing/AfterProcessingTimeTest.java      | 187 +++++++
 .../AfterSynchronizedProcessingTimeTest.java    | 121 +++++
 .../windowing/AfterWatermarkTest.java           | 380 +++++++++++++
 .../windowing/DefaultTriggerTest.java           | 176 +++++++
 .../sdk/transforms/windowing/NeverTest.java     |  56 ++
 .../windowing/OrFinallyTriggerTest.java         | 215 ++++++++
 .../transforms/windowing/RepeatedlyTest.java    | 224 ++++++++
 .../sdk/transforms/windowing/StubTrigger.java   |  70 +++
 .../sdk/transforms/windowing/TriggerTest.java   | 118 +++++
 .../org/apache/beam/sdk/util/TriggerTester.java | 410 +++++++++++++++
 34 files changed, 6016 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
new file mode 100644
index 0000000..8d0f322
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
@@ -0,0 +1,247 @@
+/*
+ * 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.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.BitSetCoder;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.FinishedTriggers;
+import org.apache.beam.sdk.util.FinishedTriggersBitSet;
+import org.apache.beam.sdk.util.Timers;
+import org.apache.beam.sdk.util.TriggerContextFactory;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.joda.time.Instant;
+
+/**
+ * Executes a trigger while managing persistence of information about which subtriggers are
+ * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
+ *
+ * <p>Specifically, the responsibilities are:
+ *
+ * <ul>
+ *   <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
+ *       constructing the appropriate trigger contexts.</li>
+ *   <li>Committing a record of which subtriggers are finished to persistent state.</li>
+ *   <li>Restoring the record of which subtriggers are finished from persistent state.</li>
+ *   <li>Clearing out the persisted finished set when a caller indicates
+ *       (via {#link #clearFinished}) that it is no longer needed.</li>
+ * </ul>
+ *
+ * <p>These responsibilities are intertwined: trigger contexts include mutable information about
+ * which subtriggers are finished. This class provides the information when building the contexts
+ * and commits the information when the method of the {@link ExecutableTrigger} returns.
+ *
+ * @param <W> The kind of windows being processed.
+ */
+public class TriggerRunner<W extends BoundedWindow> {
+  @VisibleForTesting
+  static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
+      StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
+
+  private final ExecutableTrigger rootTrigger;
+  private final TriggerContextFactory<W> contextFactory;
+
+  public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
+    checkState(rootTrigger.getTriggerIndex() == 0);
+    this.rootTrigger = rootTrigger;
+    this.contextFactory = contextFactory;
+  }
+
+  private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // If no trigger in the tree will ever have finished bits, then we don't need to read them.
+      // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
+      // finished) for each trigger in the tree.
+      return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
+    }
+
+    BitSet bitSet = state.read();
+    return bitSet == null
+        ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
+            : FinishedTriggersBitSet.fromBitSet(bitSet);
+  }
+
+
+  private void clearFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // Nothing to clear.
+      return;
+    }
+    state.clear();
+  }
+
+  /** Return true if the trigger is closed in the window corresponding to the specified state. */
+  public boolean isClosed(StateAccessor<?> state) {
+    return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchForValue(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnElement(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchOnFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchShouldFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchShouldFire(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  /**
+   * Run the trigger logic to deal with a new value.
+   */
+  public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
+      throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
+        window, timers, timestamp, rootTrigger, finishedSet);
+    rootTrigger.invokeOnElement(triggerContext);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+      justification = "prefetch side effect")
+  public void prefetchForMerge(
+      W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
+    if (isFinishedSetNeeded()) {
+      for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
+        value.readLater();
+      }
+    }
+    rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
+        window, mergingWindows, rootTrigger));
+  }
+
+  /**
+   * Run the trigger merging logic as part of executing the specified merge.
+   */
+  public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+
+    // And read the finished bits in each merging window.
+    ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
+    for (Map.Entry<W, ValueState<BitSet>> entry :
+        state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
+      // Don't need to clone these, since the trigger context doesn't allow modification
+      builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
+      // Clear the underlying finished bits.
+      clearFinishedBits(entry.getValue());
+    }
+    ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
+
+    Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
+        window, timers, rootTrigger, finishedSet, mergingFinishedSets);
+
+    // Run the merge from the trigger
+    rootTrigger.invokeOnMerge(mergeContext);
+
+    persistFinishedSet(state, finishedSet);
+  }
+
+  public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    return rootTrigger.invokeShouldFire(context);
+  }
+
+  public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // shouldFire should be false.
+    // However it is too expensive to assert.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    Trigger.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    rootTrigger.invokeOnFire(context);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  private void persistFinishedSet(
+      StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
+    if (!isFinishedSetNeeded()) {
+      return;
+    }
+
+    ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
+    if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
+      if (modifiedFinishedSet.getBitSet().isEmpty()) {
+        finishedSetState.clear();
+      } else {
+        finishedSetState.write(modifiedFinishedSet.getBitSet());
+      }
+    }
+  }
+
+  /**
+   * Clear the finished bits.
+   */
+  public void clearFinished(StateAccessor<?> state) {
+    clearFinishedBits(state.access(FINISHED_BITS_TAG));
+  }
+
+  /**
+   * Clear the state used for executing triggers, but leave the finished set to indicate
+   * the window is closed.
+   */
+  public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // Don't need to clone, because we'll be clearing the finished bits anyways.
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
+    rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
+  }
+
+  private boolean isFinishedSetNeeded() {
+    // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
+    // lookup. Right now, we special case this for the DefaultTrigger.
+    return !(rootTrigger.getSpec() instanceof DefaultTrigger);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..cc8c97f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
@@ -0,0 +1,122 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterAll extends OnceTrigger {
+
+  private AfterAll(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
+   */
+  public static OnceTrigger of(OnceTrigger... triggers) {
+    return new AfterAll(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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;
+    for (Trigger subTrigger : subTriggers) {
+      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
+      if (deadline.isBefore(subDeadline)) {
+        deadline = subDeadline;
+      }
+    }
+    return deadline;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    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(");
+    Joiner.on(", ").appendTo(builder, subTriggers);
+    builder.append(")");
+
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..c4bc946
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
@@ -0,0 +1,335 @@
+/*
+ * 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.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;
+import org.apache.beam.sdk.transforms.Min;
+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;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+import org.joda.time.format.PeriodFormatter;
+
+/**
+ * A base class for triggers that happen after a processing time delay from the arrival
+ * of the first element in a pane.
+ *
+ * <p>This class is for internal use only and may change at any time.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public abstract class AfterDelayFromFirstElement extends OnceTrigger {
+
+  protected static final List<SerializableFunction<Instant, Instant>> IDENTITY =
+      ImmutableList.<SerializableFunction<Instant, Instant>>of();
+
+  protected static final StateTag<Object, AccumulatorCombiningState<Instant,
+                                              Combine.Holder<Instant>, Instant>> DELAYED_UNTIL_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "delayed", InstantCoder.of(), Min.MinFn.<Instant>naturalOrder()));
+
+  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.
+   */
+  protected abstract AfterDelayFromFirstElement newWith(
+      List<SerializableFunction<Instant, Instant>> transform);
+
+  /**
+   * A list of timestampMappers m1, m2, m3, ... m_n considered to be composed in sequence. The
+   * overall mapping for an instance `instance` is `m_n(... m3(m2(m1(instant))`,
+   * implemented via #computeTargetTimestamp
+   */
+  protected final List<SerializableFunction<Instant, Instant>> timestampMappers;
+
+  private final TimeDomain timeDomain;
+
+  public AfterDelayFromFirstElement(
+      TimeDomain timeDomain,
+      List<SerializableFunction<Instant, Instant>> timestampMappers) {
+    super(null);
+    this.timestampMappers = timestampMappers;
+    this.timeDomain = timeDomain;
+  }
+
+  private Instant getTargetTimestamp(OnElementContext c) {
+    return computeTargetTimestamp(c.currentProcessingTime());
+  }
+
+  /**
+   * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+   * than the timestamp.
+   *
+   * <p>TODO: Consider sharing this with FixedWindows, and bring over the equivalent of
+   * CalendarWindows.
+   */
+  public AfterDelayFromFirstElement alignedTo(final Duration size, final Instant offset) {
+    return newWith(new AlignFn(size, offset));
+  }
+
+  /**
+   * Aligns the time to be the smallest multiple of {@code size} greater than the timestamp
+   * since the epoch.
+   */
+  public AfterDelayFromFirstElement alignedTo(final Duration size) {
+    return alignedTo(size, new Instant(0));
+  }
+
+  /**
+   * Adds some delay to the original target time.
+   *
+   * @param delay the delay to add
+   * @return An updated time trigger that will wait the additional time before firing.
+   */
+  public AfterDelayFromFirstElement plusDelayOf(final Duration delay) {
+    return newWith(new DelayFn(delay));
+  }
+
+  /**
+   * @deprecated This will be removed in the next major version. Please use only
+   *             {@link #plusDelayOf} and {@link #alignedTo}.
+   */
+  @Deprecated
+  public OnceTrigger mappedTo(SerializableFunction<Instant, Instant> timestampMapper) {
+    return newWith(timestampMapper);
+  }
+
+  @Override
+  public boolean isCompatible(Trigger other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    AfterDelayFromFirstElement that = (AfterDelayFromFirstElement) other;
+    return this.timestampMappers.equals(that.timestampMappers);
+  }
+
+
+  private AfterDelayFromFirstElement newWith(
+      SerializableFunction<Instant, Instant> timestampMapper) {
+    return newWith(
+        ImmutableList.<SerializableFunction<Instant, Instant>>builder()
+            .addAll(timestampMappers)
+            .add(timestampMapper)
+            .build());
+  }
+
+  @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) {
+      result = timestampMapper.apply(result);
+    }
+    return result;
+  }
+
+  /**
+   * A {@link SerializableFunction} to delay the timestamp at which this triggers fires.
+   */
+  private static final class DelayFn implements SerializableFunction<Instant, Instant> {
+    private final Duration delay;
+
+    public DelayFn(Duration delay) {
+      this.delay = delay;
+    }
+
+    @Override
+    public Instant apply(Instant input) {
+      return input.plus(delay);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof DelayFn)) {
+        return false;
+      }
+
+      return this.delay.equals(((DelayFn) object).delay);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(delay);
+    }
+
+    @Override
+    public String toString() {
+      return PERIOD_FORMATTER.print(delay.toPeriod());
+    }
+  }
+
+  /**
+   * A {@link SerializableFunction} to align an instant to the nearest interval boundary.
+   */
+  static final class AlignFn implements SerializableFunction<Instant, Instant> {
+    private final Duration size;
+    private final Instant offset;
+
+
+    /**
+     * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+     * than the timestamp.
+     */
+    public AlignFn(Duration size, Instant offset) {
+      this.size = size;
+      this.offset = offset;
+    }
+
+    @Override
+    public Instant apply(Instant point) {
+      long millisSinceStart = new Duration(offset, point).getMillis() % size.getMillis();
+      return millisSinceStart == 0 ? point : point.plus(size).minus(millisSinceStart);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof AlignFn)) {
+        return false;
+      }
+
+      AlignFn other = (AlignFn) object;
+      return other.size.equals(this.size)
+          && other.offset.equals(this.offset);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(size, offset);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..629c640
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
@@ -0,0 +1,141 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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;
+
+/**
+ * A composite {@link Trigger} that executes its sub-triggers in order.
+ * Only one sub-trigger is executing at a time,
+ * and any time it fires the {@code AfterEach} fires. When the currently executing
+ * sub-trigger finishes, the {@code AfterEach} starts executing the next sub-trigger.
+ *
+ * <p>{@code AfterEach.inOrder(t1, t2, ...)} finishes when all of the sub-triggers have finished.
+ *
+ * <p>The following properties hold:
+ * <ul>
+ *   <li> {@code AfterEach.inOrder(AfterEach.inOrder(a, b), c)} behaves the same as
+ *   {@code AfterEach.inOrder(a, b, c)} and {@code AfterEach.inOrder(a, AfterEach.inOrder(b, c)}.
+ *   <li> {@code AfterEach.inOrder(Repeatedly.forever(a), b)} behaves the same as
+ *   {@code Repeatedly.forever(a)}, since the repeated trigger never finishes.
+ * </ul>
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterEach extends Trigger {
+
+  private AfterEach(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterEach} {@code Trigger} with the given subtriggers.
+   */
+  @SafeVarargs
+  public static Trigger inOrder(Trigger... triggers) {
+    return new AfterEach(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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.
+    return subTriggers.get(0).getWatermarkThatGuaranteesFiring(window);
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return Repeatedly.forever(new AfterFirst(continuationTriggers));
+  }
+
+  @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);
+    builder.append(")");
+
+    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/a64acb2f/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
new file mode 100644
index 0000000..6b06cfa
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
@@ -0,0 +1,124 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have
+ * fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterFirst extends OnceTrigger {
+
+  AfterFirst(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
+   */
+  public static OnceTrigger of(OnceTrigger... triggers) {
+    return new AfterFirst(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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;
+    for (Trigger subTrigger : subTriggers) {
+      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
+      if (deadline.isAfter(subDeadline)) {
+        deadline = subDeadline;
+      }
+    }
+    return deadline;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new AfterFirst(continuationTriggers);
+  }
+
+  @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);
+    builder.append(")");
+
+    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/a64acb2f/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
new file mode 100644
index 0000000..8c128dd
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
@@ -0,0 +1,144 @@
+/*
+ * 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.windowing;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.annotations.Experimental;
+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;
+
+/**
+ * {@link Trigger}s that fire based on properties of the elements in the current pane.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterPane extends OnceTrigger {
+
+private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
+      ELEMENTS_IN_PANE_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "count", VarLongCoder.of(), new Sum.SumLongFn()));
+
+  private final int countElems;
+
+  private AfterPane(int countElems) {
+    super(null);
+    this.countElems = countElems;
+  }
+
+  /**
+   * Creates a trigger that fires when the pane contains at least {@code countElems} elements.
+   */
+  public static AfterPane elementCountAtLeast(int countElems) {
+    return new AfterPane(countElems);
+  }
+
+  @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);
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return AfterPane.elementCountAtLeast(1);
+  }
+
+  @Override
+  public String toString() {
+    return "AfterPane.elementCountAtLeast(" + countElems + ")";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterPane)) {
+      return false;
+    }
+    AfterPane that = (AfterPane) obj;
+    return this.countElems == that.countElems;
+  }
+
+  @Override
+  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/a64acb2f/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
new file mode 100644
index 0000000..f551118
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
@@ -0,0 +1,102 @@
+/*
+ * 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.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;
+import org.joda.time.Instant;
+
+/**
+ * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in
+ * the real-time domain.
+ *
+ * <p>The time at which to fire the timer can be adjusted via the methods in
+ * {@link AfterDelayFromFirstElement}, such as {@link AfterDelayFromFirstElement#plusDelayOf} or
+ * {@link AfterDelayFromFirstElement#alignedTo}.
+ */
+@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);
+  }
+
+  /**
+   * Creates a trigger that fires when the current processing time passes the processing time
+   * at which this trigger saw the first element in a pane.
+   */
+  public static AfterProcessingTime pastFirstElementInPane() {
+    return new AfterProcessingTime(IDENTITY);
+  }
+
+  @Override
+  protected AfterProcessingTime newWith(
+      List<SerializableFunction<Instant, Instant>> transforms) {
+    return new AfterProcessingTime(transforms);
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new AfterSynchronizedProcessingTime();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()");
+    for (SerializableFunction<Instant, Instant> delayFn : timestampMappers) {
+      builder
+          .append(".plusDelayOf(")
+          .append(delayFn)
+          .append(")");
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterProcessingTime)) {
+      return false;
+    }
+    AfterProcessingTime that = (AfterProcessingTime) obj;
+    return Objects.equals(this.timestampMappers, that.timestampMappers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), this.timestampMappers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..59ece10
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
@@ -0,0 +1,73 @@
+/*
+ * 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.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;
+
+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());
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return this;
+  }
+
+  @Override
+  public String toString() {
+    return "AfterSynchronizedProcessingTime.pastFirstElementInPane()";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return this == obj || obj instanceof AfterSynchronizedProcessingTime;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(AfterSynchronizedProcessingTime.class);
+  }
+
+  @Override
+  protected AfterSynchronizedProcessingTime
+      newWith(List<SerializableFunction<Instant, Instant>> transforms) {
+    // ignore transforms
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..e2463d8
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -0,0 +1,355 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableList;
+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.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.joda.time.Instant;
+
+/**
+ * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
+ * lower-bound, sometimes heuristically established, on event times that have been fully processed
+ * by the pipeline.
+ *
+ * <p>For sources that provide non-heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
+ * watermark is a strict guarantee that no data with an event time earlier than
+ * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
+ * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
+ * of the window will be the last pane ever for that window.
+ *
+ * <p>For sources that provide heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
+ * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
+ * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
+ * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
+ * Thus, if absolute correctness over time is important to your use case, you may want to consider
+ * using a trigger that accounts for late data. The default trigger,
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires
+ * once when the watermark passes the end of the window and then immediately therafter when any
+ * late data arrives, is one such example.
+ *
+ * <p>The watermark is the clock that defines {@link TimeDomain#EVENT_TIME}.
+ *
+ * <p>Additionaly firings before or after the watermark can be requested by calling
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterWatermark {
+
+  private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
+
+  // Static factory class.
+  private AfterWatermark() {}
+
+  /**
+   * Creates a trigger that fires when the watermark passes the end of the window.
+   */
+  public static FromEndOfWindow pastEndOfWindow() {
+    return new FromEndOfWindow();
+  }
+
+  /**
+   * @see AfterWatermark
+   */
+  public static class AfterWatermarkEarlyAndLate extends Trigger {
+
+    private static final int EARLY_INDEX = 0;
+    private static final int LATE_INDEX = 1;
+
+    private final OnceTrigger earlyTrigger;
+    @Nullable
+    private final OnceTrigger lateTrigger;
+
+    @SuppressWarnings("unchecked")
+    public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
+      super(lateTrigger == null
+          ? ImmutableList.<Trigger>of(earlyTrigger)
+          : ImmutableList.<Trigger>of(earlyTrigger, lateTrigger));
+      this.earlyTrigger = checkNotNull(earlyTrigger, "earlyTrigger should not be null");
+      this.lateTrigger = lateTrigger;
+    }
+
+    public Trigger withEarlyFirings(OnceTrigger earlyTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    public Trigger withLateFirings(OnceTrigger lateTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    @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(),
+          lateTrigger == null ? null : lateTrigger.getContinuationTrigger());
+    }
+
+    @Override
+    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+      throw new UnsupportedOperationException(
+          "Should not call getContinuationTrigger(List<Trigger>)");
+    }
+
+    @Override
+    public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+      // Even without an early or late trigger, we'll still produce a firing at the watermark.
+      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);
+
+      if (!(earlyTrigger instanceof Never.NeverTrigger)) {
+        builder
+            .append(".withEarlyFirings(")
+            .append(earlyTrigger)
+            .append(")");
+      }
+
+      if (lateTrigger != null && !(lateTrigger instanceof Never.NeverTrigger)) {
+        builder
+            .append(".withLateFirings(")
+            .append(lateTrigger)
+            .append(")");
+      }
+
+      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);
+    }
+  }
+
+  /**
+   * A watermark trigger targeted relative to the end of the window.
+   */
+  public static class FromEndOfWindow extends OnceTrigger {
+
+    private FromEndOfWindow() {
+      super(null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires before the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTrigger earlyFirings) {
+      checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
+      return new AfterWatermarkEarlyAndLate(earlyFirings, null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires after the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withLateFirings(OnceTrigger lateFirings) {
+      checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
+      return new AfterWatermarkEarlyAndLate(Never.ever(), lateFirings);
+    }
+
+    @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();
+    }
+
+    @Override
+    public FromEndOfWindow getContinuationTrigger(List<Trigger> continuationTriggers) {
+      return this;
+    }
+
+    @Override
+    public String toString() {
+      return TO_STRING;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj instanceof FromEndOfWindow;
+    }
+
+    @Override
+    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/a64acb2f/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
new file mode 100644
index 0000000..d6b72ef
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
@@ -0,0 +1,92 @@
+/*
+ * 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.windowing;
+
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.joda.time.Instant;
+
+/**
+ * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+ * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class DefaultTrigger extends Trigger{
+
+  private DefaultTrigger() {
+    super(null);
+  }
+
+  /**
+   * Returns the default trigger.
+   */
+  public static DefaultTrigger of() {
+    return new DefaultTrigger();
+  }
+
+  @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();
+  }
+
+  @Override
+  public boolean isCompatible(Trigger other) {
+    // Semantically, all default triggers are identical
+    return other instanceof DefaultTrigger;
+  }
+
+  @Override
+  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/a64acb2f/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
new file mode 100644
index 0000000..5f20465
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
@@ -0,0 +1,75 @@
+/*
+ * 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.windowing;
+
+import java.util.List;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.joda.time.Instant;
+
+/**
+ * A trigger which never fires.
+ *
+ * <p>Using this trigger will only produce output when the watermark passes the end of the
+ * {@link BoundedWindow window} plus the {@link Window#withAllowedLateness allowed
+ * lateness}.
+ */
+public final class Never {
+  /**
+   * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
+   * when the {@link BoundedWindow} closes.
+   */
+  public static OnceTrigger ever() {
+    // NeverTrigger ignores all inputs and is Window-type independent.
+    return new NeverTrigger();
+  }
+
+  // package-private in order to check identity for string formatting.
+  static class NeverTrigger extends OnceTrigger {
+    protected NeverTrigger() {
+      super(null);
+    }
+
+    @Override
+    public void onElement(OnElementContext c) {}
+
+    @Override
+    public void onMerge(OnMergeContext c) {}
+
+    @Override
+    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+      return this;
+    }
+
+    @Override
+    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/a64acb2f/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
new file mode 100644
index 0000000..25b7b34
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
@@ -0,0 +1,105 @@
+/*
+ * 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.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;
+
+/**
+ * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
+ */
+class OrFinallyTrigger extends Trigger {
+
+  private static final int ACTUAL = 0;
+  private static final int UNTIL = 1;
+
+  @VisibleForTesting OrFinallyTrigger(Trigger actual, Trigger.OnceTrigger until) {
+    super(Arrays.asList(actual, until));
+  }
+
+  @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);
+    Instant untilDeadline = subTriggers.get(UNTIL).getWatermarkThatGuaranteesFiring(window);
+    return actualDeadline.isBefore(untilDeadline) ? actualDeadline : untilDeadline;
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    // Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL
+    // may not be a OnceTrigger.
+    return Repeatedly.forever(
+        new OrFinallyTrigger(
+            continuationTriggers.get(ACTUAL),
+            (Trigger.OnceTrigger) continuationTriggers.get(UNTIL)));
+  }
+
+  @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/a64acb2f/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
new file mode 100644
index 0000000..8858798
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
@@ -0,0 +1,101 @@
+/*
+ * 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.windowing;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Repeat a trigger, either until some condition is met or forever.
+ *
+ * <p>For example, to fire after the end of the window, and every time late data arrives:
+ * <pre> {@code
+ *     Repeatedly.forever(AfterWatermark.isPastEndOfWindow());
+ * } </pre>
+ *
+ * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite
+ * {@code AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
+ */
+public class Repeatedly extends Trigger {
+
+  private static final int REPEATED = 0;
+
+  /**
+   * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each
+   * time it fires and ignoring any indications to finish.
+   *
+   * <p>Unless used with {@link Trigger#orFinally} the composite trigger will never finish.
+   *
+   * @param repeated the trigger to execute repeatedly.
+   */
+  public static Repeatedly forever(Trigger repeated) {
+    return new Repeatedly(repeated);
+  }
+
+  private Repeatedly(Trigger repeated) {
+    super(Arrays.asList(repeated));
+  }
+
+
+  @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);
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new Repeatedly(continuationTriggers.get(REPEATED));
+  }
+
+  @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);
+  }
+}


[11/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
new file mode 100644
index 0000000..38d030e
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterPane}.
+ */
+@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,
+        AfterPane.elementCountAtLeast(1).getWatermarkThatGuaranteesFiring(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(
+        AfterPane.elementCountAtLeast(1),
+        AfterPane.elementCountAtLeast(100).getContinuationTrigger());
+    assertEquals(
+        AfterPane.elementCountAtLeast(1),
+        AfterPane.elementCountAtLeast(100).getContinuationTrigger().getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterPane.elementCountAtLeast(5);
+    assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
new file mode 100644
index 0000000..13a7acf
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterProcessingTime}.
+ */
+@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,
+        AfterProcessingTime.pastFirstElementInPane().getWatermarkThatGuaranteesFiring(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger firstElementPlus1 =
+        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.standardHours(1));
+    assertEquals(
+        new AfterSynchronizedProcessingTime(),
+        firstElementPlus1.getContinuationTrigger());
+  }
+
+  /**
+   * Basic test of compatibility check between identical triggers.
+   */
+  @Test
+  public void testCompatibilityIdentical() throws Exception {
+    Trigger t1 = AfterProcessingTime.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    Trigger t2 = AfterProcessingTime.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    assertTrue(t1.isCompatible(t2));
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
+    assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString());
+  }
+
+  @Test
+  public void testWithDelayToString() {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane()
+        .plusDelayOf(Duration.standardMinutes(5));
+
+    assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testBuiltUpToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow()
+        .withLateFirings(AfterProcessingTime
+            .pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(10)));
+
+    String expected = "AfterWatermark.pastEndOfWindow()"
+        + ".withLateFirings(AfterProcessingTime"
+        + ".pastFirstElementInPane()"
+        + ".plusDelayOf(10 minutes))";
+
+    assertEquals(expected, trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
new file mode 100644
index 0000000..7e6e938
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterSynchronizedProcessingTime}.
+ */
+@RunWith(JUnit4.class)
+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(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(underTest, underTest.getContinuationTrigger());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
new file mode 100644
index 0000000..084027b
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
@@ -0,0 +1,380 @@
+/*
+ * 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.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.
+ */
+@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();
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyFiringsToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withEarlyFirings(StubTrigger.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testLateFiringsToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withLateFirings(StubTrigger.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyAndLateFiringsToString() {
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(StubTrigger.named("t1"))
+            .withLateFirings(StubTrigger.named("t2"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testToStringExcludesNeverTrigger() {
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(Never.ever())
+            .withLateFirings(Never.ever());
+
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
new file mode 100644
index 0000000..673e555
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link DefaultTrigger}, which should be equivalent to
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+ */
+@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(
+        new IntervalWindow(new Instant(0), new Instant(10))));
+    assertEquals(GlobalWindow.INSTANCE.maxTimestamp(),
+        DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(DefaultTrigger.of(), DefaultTrigger.of().getContinuationTrigger());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
new file mode 100644
index 0000000..1e3a1ff
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.assertEquals;
+import static org.junit.Assert.assertSame;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ExecutableTrigger}.
+ */
+@RunWith(JUnit4.class)
+public class ExecutableTriggerTest {
+
+  @Test
+  public void testIndexAssignmentLeaf() throws Exception {
+    StubTrigger t1 = new StubTrigger();
+    ExecutableTrigger executable = ExecutableTrigger.create(t1);
+    assertEquals(0, executable.getTriggerIndex());
+  }
+
+  @Test
+  public void testIndexAssignmentOneLevel() throws Exception {
+    StubTrigger t1 = new StubTrigger();
+    StubTrigger t2 = new StubTrigger();
+    StubTrigger t = new StubTrigger(t1, t2);
+
+    ExecutableTrigger executable = ExecutableTrigger.create(t);
+
+    assertEquals(0, executable.getTriggerIndex());
+    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
+    assertSame(t1, executable.subTriggers().get(0).getSpec());
+    assertEquals(2, executable.subTriggers().get(1).getTriggerIndex());
+    assertSame(t2, executable.subTriggers().get(1).getSpec());
+  }
+
+  @Test
+  public void testIndexAssignmentTwoLevel() throws Exception {
+    StubTrigger t11 = new StubTrigger();
+    StubTrigger t12 = new StubTrigger();
+    StubTrigger t13 = new StubTrigger();
+    StubTrigger t14 = new StubTrigger();
+    StubTrigger t21 = new StubTrigger();
+    StubTrigger t22 = new StubTrigger();
+    StubTrigger t1 = new StubTrigger(t11, t12, t13, t14);
+    StubTrigger t2 = new StubTrigger(t21, t22);
+    StubTrigger t = new StubTrigger(t1, t2);
+
+    ExecutableTrigger executable = ExecutableTrigger.create(t);
+
+    assertEquals(0, executable.getTriggerIndex());
+    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
+    assertEquals(6, executable.subTriggers().get(0).getFirstIndexAfterSubtree());
+    assertEquals(6, executable.subTriggers().get(1).getTriggerIndex());
+
+    assertSame(t1, executable.getSubTriggerContaining(1).getSpec());
+    assertSame(t2, executable.getSubTriggerContaining(6).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(2).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(3).getSpec());
+    assertSame(t1, executable.getSubTriggerContaining(5).getSpec());
+    assertSame(t2, executable.getSubTriggerContaining(7).getSpec());
+  }
+
+  private static class StubTrigger extends Trigger {
+
+    @SafeVarargs
+    protected StubTrigger(Trigger... subTriggers) {
+      super(Arrays.asList(subTriggers));
+    }
+
+    @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;
+    }
+
+    @Override
+    public boolean isCompatible(Trigger other) {
+      return false;
+    }
+
+    @Override
+    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/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java
new file mode 100644
index 0000000..7f74620
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java
new file mode 100644
index 0000000..a66f74f
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java
@@ -0,0 +1,110 @@
+/*
+ * 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/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java
new file mode 100644
index 0000000..072d264
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
new file mode 100644
index 0000000..fb2b4d5
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.windowing;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+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}.
+ */
+@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 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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
new file mode 100644
index 0000000..7289d97
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
@@ -0,0 +1,215 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link OrFinallyTrigger}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        Repeatedly.forever(AfterWatermark.pastEndOfWindow())
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow())
+        .orFinally(AfterPane.elementCountAtLeast(1))
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9), Repeatedly.forever(AfterPane.elementCountAtLeast(1))
+        .orFinally(AfterWatermark.pastEndOfWindow())
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9),
+        AfterPane.elementCountAtLeast(100)
+            .orFinally(AfterWatermark.pastEndOfWindow())
+            .getWatermarkThatGuaranteesFiring(window));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        Repeatedly.forever(AfterPane.elementCountAtLeast(1))
+        .orFinally(AfterPane.elementCountAtLeast(10))
+        .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger triggerA = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger triggerB = AfterWatermark.pastEndOfWindow();
+    Trigger aOrFinallyB = triggerA.orFinally(triggerB);
+    Trigger bOrFinallyA = triggerB.orFinally(triggerA);
+    assertEquals(
+        Repeatedly.forever(
+            triggerA.getContinuationTrigger().orFinally(triggerB.getContinuationTrigger())),
+        aOrFinallyB.getContinuationTrigger());
+    assertEquals(
+        Repeatedly.forever(
+            triggerB.getContinuationTrigger().orFinally(triggerA.getContinuationTrigger())),
+        bOrFinallyA.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = StubTrigger.named("t1").orFinally(StubTrigger.named("t2"));
+    assertEquals("t1.orFinally(t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
new file mode 100644
index 0000000..6e8930d
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.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;
+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 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.
+   */
+  @Test
+  public void testFireDeadline() throws Exception {
+    setUp(FixedWindows.of(Duration.millis(10)));
+    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
+    Instant arbitraryInstant = new Instant(34957849);
+
+    when(mockTrigger.getWatermarkThatGuaranteesFiring(Mockito.<IntervalWindow>any()))
+        .thenReturn(arbitraryInstant);
+
+    assertThat(
+        Repeatedly.forever(mockTrigger).getWatermarkThatGuaranteesFiring(window),
+        equalTo(arbitraryInstant));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
+    Trigger repeatedly = Repeatedly.forever(trigger);
+    assertEquals(
+        Repeatedly.forever(trigger.getContinuationTrigger()), repeatedly.getContinuationTrigger());
+    assertEquals(
+        Repeatedly.forever(trigger.getContinuationTrigger().getContinuationTrigger()),
+        repeatedly.getContinuationTrigger().getContinuationTrigger());
+  }
+
+  @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";
+        }
+      });
+
+    assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java
new file mode 100644
index 0000000..83077f4
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.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;
+
+/**
+ * Tests for {@link ReshuffleTrigger}.
+ */
+@RunWith(JUnit4.class)
+public class ReshuffleTriggerTest {
+
+  /** Public so that other tests can instantiate {@link ReshuffleTrigger}. */
+  public static <W extends BoundedWindow> ReshuffleTrigger<W> forTest() {
+    return new ReshuffleTrigger<>();
+  }
+
+  @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/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
new file mode 100644
index 0000000..b258a79
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
@@ -0,0 +1,70 @@
+/*
+ * 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.windowing;
+
+import com.google.common.collect.Lists;
+import java.util.List;
+import org.joda.time.Instant;
+
+/**
+ * No-op {@link OnceTrigger} implementation for testing.
+ */
+abstract class StubTrigger extends Trigger.OnceTrigger {
+  /**
+   * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}.
+   */
+  static StubTrigger named(final String name) {
+    return new StubTrigger() {
+      @Override
+      public String toString() {
+        return name;
+      }
+    };
+  }
+
+  protected StubTrigger() {
+    super(Lists.<Trigger>newArrayList());
+  }
+
+  @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;
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
new file mode 100644
index 0000000..cfc03b2
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.windowing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link Trigger}.
+ */
+@RunWith(JUnit4.class)
+public class TriggerTest {
+
+  @Test
+  public void testTriggerToString() throws Exception {
+    assertEquals("AfterWatermark.pastEndOfWindow()", AfterWatermark.pastEndOfWindow().toString());
+    assertEquals("Repeatedly.forever(AfterWatermark.pastEndOfWindow())",
+        Repeatedly.forever(AfterWatermark.pastEndOfWindow()).toString());
+  }
+
+  @Test
+  public void testIsCompatible() throws Exception {
+    assertTrue(new Trigger1(null).isCompatible(new Trigger1(null)));
+    assertTrue(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))
+        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
+
+    assertFalse(new Trigger1(null).isCompatible(new Trigger2(null)));
+    assertFalse(new Trigger1(Arrays.<Trigger>asList(new Trigger1(null)))
+        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
+  }
+
+  private static class Trigger1 extends Trigger {
+
+    private Trigger1(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(Trigger.OnElementContext c) { }
+
+    @Override
+    public void onMerge(Trigger.OnMergeContext c) { }
+
+    @Override
+    protected Trigger getContinuationTrigger(
+        List<Trigger> continuationTriggers) {
+      return null;
+    }
+
+    @Override
+    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 {
+
+    private Trigger2(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(Trigger.OnElementContext c) { }
+
+    @Override
+    public void onMerge(Trigger.OnMergeContext c) { }
+
+    @Override
+    protected Trigger getContinuationTrigger(
+        List<Trigger> continuationTriggers) {
+      return null;
+    }
+
+    @Override
+    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 { }
+  }
+}



[10/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java
new file mode 100644
index 0000000..5fe17ad
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java
@@ -0,0 +1,410 @@
+/*
+ * 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();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index cc8c97f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
+++ /dev/null
@@ -1,122 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterAll extends OnceTrigger {
-
-  private AfterAll(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
-   */
-  public static OnceTrigger of(OnceTrigger... triggers) {
-    return new AfterAll(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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;
-    for (Trigger subTrigger : subTriggers) {
-      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
-      if (deadline.isBefore(subDeadline)) {
-        deadline = subDeadline;
-      }
-    }
-    return deadline;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    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(");
-    Joiner.on(", ").appendTo(builder, subTriggers);
-    builder.append(")");
-
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index c4bc946..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java
+++ /dev/null
@@ -1,335 +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.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;
-import org.apache.beam.sdk.transforms.Min;
-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;
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-import org.joda.time.format.PeriodFormatter;
-
-/**
- * A base class for triggers that happen after a processing time delay from the arrival
- * of the first element in a pane.
- *
- * <p>This class is for internal use only and may change at any time.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public abstract class AfterDelayFromFirstElement extends OnceTrigger {
-
-  protected static final List<SerializableFunction<Instant, Instant>> IDENTITY =
-      ImmutableList.<SerializableFunction<Instant, Instant>>of();
-
-  protected static final StateTag<Object, AccumulatorCombiningState<Instant,
-                                              Combine.Holder<Instant>, Instant>> DELAYED_UNTIL_TAG =
-      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
-          "delayed", InstantCoder.of(), Min.MinFn.<Instant>naturalOrder()));
-
-  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.
-   */
-  protected abstract AfterDelayFromFirstElement newWith(
-      List<SerializableFunction<Instant, Instant>> transform);
-
-  /**
-   * A list of timestampMappers m1, m2, m3, ... m_n considered to be composed in sequence. The
-   * overall mapping for an instance `instance` is `m_n(... m3(m2(m1(instant))`,
-   * implemented via #computeTargetTimestamp
-   */
-  protected final List<SerializableFunction<Instant, Instant>> timestampMappers;
-
-  private final TimeDomain timeDomain;
-
-  public AfterDelayFromFirstElement(
-      TimeDomain timeDomain,
-      List<SerializableFunction<Instant, Instant>> timestampMappers) {
-    super(null);
-    this.timestampMappers = timestampMappers;
-    this.timeDomain = timeDomain;
-  }
-
-  private Instant getTargetTimestamp(OnElementContext c) {
-    return computeTargetTimestamp(c.currentProcessingTime());
-  }
-
-  /**
-   * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
-   * than the timestamp.
-   *
-   * <p>TODO: Consider sharing this with FixedWindows, and bring over the equivalent of
-   * CalendarWindows.
-   */
-  public AfterDelayFromFirstElement alignedTo(final Duration size, final Instant offset) {
-    return newWith(new AlignFn(size, offset));
-  }
-
-  /**
-   * Aligns the time to be the smallest multiple of {@code size} greater than the timestamp
-   * since the epoch.
-   */
-  public AfterDelayFromFirstElement alignedTo(final Duration size) {
-    return alignedTo(size, new Instant(0));
-  }
-
-  /**
-   * Adds some delay to the original target time.
-   *
-   * @param delay the delay to add
-   * @return An updated time trigger that will wait the additional time before firing.
-   */
-  public AfterDelayFromFirstElement plusDelayOf(final Duration delay) {
-    return newWith(new DelayFn(delay));
-  }
-
-  /**
-   * @deprecated This will be removed in the next major version. Please use only
-   *             {@link #plusDelayOf} and {@link #alignedTo}.
-   */
-  @Deprecated
-  public OnceTrigger mappedTo(SerializableFunction<Instant, Instant> timestampMapper) {
-    return newWith(timestampMapper);
-  }
-
-  @Override
-  public boolean isCompatible(Trigger other) {
-    if (!getClass().equals(other.getClass())) {
-      return false;
-    }
-
-    AfterDelayFromFirstElement that = (AfterDelayFromFirstElement) other;
-    return this.timestampMappers.equals(that.timestampMappers);
-  }
-
-
-  private AfterDelayFromFirstElement newWith(
-      SerializableFunction<Instant, Instant> timestampMapper) {
-    return newWith(
-        ImmutableList.<SerializableFunction<Instant, Instant>>builder()
-            .addAll(timestampMappers)
-            .add(timestampMapper)
-            .build());
-  }
-
-  @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) {
-      result = timestampMapper.apply(result);
-    }
-    return result;
-  }
-
-  /**
-   * A {@link SerializableFunction} to delay the timestamp at which this triggers fires.
-   */
-  private static final class DelayFn implements SerializableFunction<Instant, Instant> {
-    private final Duration delay;
-
-    public DelayFn(Duration delay) {
-      this.delay = delay;
-    }
-
-    @Override
-    public Instant apply(Instant input) {
-      return input.plus(delay);
-    }
-
-    @Override
-    public boolean equals(Object object) {
-      if (object == this) {
-        return true;
-      }
-
-      if (!(object instanceof DelayFn)) {
-        return false;
-      }
-
-      return this.delay.equals(((DelayFn) object).delay);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(delay);
-    }
-
-    @Override
-    public String toString() {
-      return PERIOD_FORMATTER.print(delay.toPeriod());
-    }
-  }
-
-  /**
-   * A {@link SerializableFunction} to align an instant to the nearest interval boundary.
-   */
-  static final class AlignFn implements SerializableFunction<Instant, Instant> {
-    private final Duration size;
-    private final Instant offset;
-
-
-    /**
-     * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
-     * than the timestamp.
-     */
-    public AlignFn(Duration size, Instant offset) {
-      this.size = size;
-      this.offset = offset;
-    }
-
-    @Override
-    public Instant apply(Instant point) {
-      long millisSinceStart = new Duration(offset, point).getMillis() % size.getMillis();
-      return millisSinceStart == 0 ? point : point.plus(size).minus(millisSinceStart);
-    }
-
-    @Override
-    public boolean equals(Object object) {
-      if (object == this) {
-        return true;
-      }
-
-      if (!(object instanceof AlignFn)) {
-        return false;
-      }
-
-      AlignFn other = (AlignFn) object;
-      return other.size.equals(this.size)
-          && other.offset.equals(this.offset);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(size, offset);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 629c640..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java
+++ /dev/null
@@ -1,141 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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;
-
-/**
- * A composite {@link Trigger} that executes its sub-triggers in order.
- * Only one sub-trigger is executing at a time,
- * and any time it fires the {@code AfterEach} fires. When the currently executing
- * sub-trigger finishes, the {@code AfterEach} starts executing the next sub-trigger.
- *
- * <p>{@code AfterEach.inOrder(t1, t2, ...)} finishes when all of the sub-triggers have finished.
- *
- * <p>The following properties hold:
- * <ul>
- *   <li> {@code AfterEach.inOrder(AfterEach.inOrder(a, b), c)} behaves the same as
- *   {@code AfterEach.inOrder(a, b, c)} and {@code AfterEach.inOrder(a, AfterEach.inOrder(b, c)}.
- *   <li> {@code AfterEach.inOrder(Repeatedly.forever(a), b)} behaves the same as
- *   {@code Repeatedly.forever(a)}, since the repeated trigger never finishes.
- * </ul>
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterEach extends Trigger {
-
-  private AfterEach(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterEach} {@code Trigger} with the given subtriggers.
-   */
-  @SafeVarargs
-  public static Trigger inOrder(Trigger... triggers) {
-    return new AfterEach(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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.
-    return subTriggers.get(0).getWatermarkThatGuaranteesFiring(window);
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return Repeatedly.forever(new AfterFirst(continuationTriggers));
-  }
-
-  @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);
-    builder.append(")");
-
-    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/e4398e1e/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
deleted file mode 100644
index 6b06cfa..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
+++ /dev/null
@@ -1,124 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have
- * fired.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterFirst extends OnceTrigger {
-
-  AfterFirst(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
-   */
-  public static OnceTrigger of(OnceTrigger... triggers) {
-    return new AfterFirst(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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;
-    for (Trigger subTrigger : subTriggers) {
-      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
-      if (deadline.isAfter(subDeadline)) {
-        deadline = subDeadline;
-      }
-    }
-    return deadline;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new AfterFirst(continuationTriggers);
-  }
-
-  @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);
-    builder.append(")");
-
-    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/e4398e1e/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
deleted file mode 100644
index 8c128dd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
+++ /dev/null
@@ -1,144 +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.windowing;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.List;
-import java.util.Objects;
-import org.apache.beam.sdk.annotations.Experimental;
-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;
-
-/**
- * {@link Trigger}s that fire based on properties of the elements in the current pane.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterPane extends OnceTrigger {
-
-private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
-      ELEMENTS_IN_PANE_TAG =
-      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
-          "count", VarLongCoder.of(), new Sum.SumLongFn()));
-
-  private final int countElems;
-
-  private AfterPane(int countElems) {
-    super(null);
-    this.countElems = countElems;
-  }
-
-  /**
-   * Creates a trigger that fires when the pane contains at least {@code countElems} elements.
-   */
-  public static AfterPane elementCountAtLeast(int countElems) {
-    return new AfterPane(countElems);
-  }
-
-  @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);
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return AfterPane.elementCountAtLeast(1);
-  }
-
-  @Override
-  public String toString() {
-    return "AfterPane.elementCountAtLeast(" + countElems + ")";
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof AfterPane)) {
-      return false;
-    }
-    AfterPane that = (AfterPane) obj;
-    return this.countElems == that.countElems;
-  }
-
-  @Override
-  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/e4398e1e/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
deleted file mode 100644
index f551118..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
+++ /dev/null
@@ -1,102 +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.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;
-import org.joda.time.Instant;
-
-/**
- * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in
- * the real-time domain.
- *
- * <p>The time at which to fire the timer can be adjusted via the methods in
- * {@link AfterDelayFromFirstElement}, such as {@link AfterDelayFromFirstElement#plusDelayOf} or
- * {@link AfterDelayFromFirstElement#alignedTo}.
- */
-@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);
-  }
-
-  /**
-   * Creates a trigger that fires when the current processing time passes the processing time
-   * at which this trigger saw the first element in a pane.
-   */
-  public static AfterProcessingTime pastFirstElementInPane() {
-    return new AfterProcessingTime(IDENTITY);
-  }
-
-  @Override
-  protected AfterProcessingTime newWith(
-      List<SerializableFunction<Instant, Instant>> transforms) {
-    return new AfterProcessingTime(transforms);
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new AfterSynchronizedProcessingTime();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()");
-    for (SerializableFunction<Instant, Instant> delayFn : timestampMappers) {
-      builder
-          .append(".plusDelayOf(")
-          .append(delayFn)
-          .append(")");
-    }
-
-    return builder.toString();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof AfterProcessingTime)) {
-      return false;
-    }
-    AfterProcessingTime that = (AfterProcessingTime) obj;
-    return Objects.equals(this.timestampMappers, that.timestampMappers);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(getClass(), this.timestampMappers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 59ece10..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
+++ /dev/null
@@ -1,73 +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.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;
-
-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());
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return this;
-  }
-
-  @Override
-  public String toString() {
-    return "AfterSynchronizedProcessingTime.pastFirstElementInPane()";
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return this == obj || obj instanceof AfterSynchronizedProcessingTime;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(AfterSynchronizedProcessingTime.class);
-  }
-
-  @Override
-  protected AfterSynchronizedProcessingTime
-      newWith(List<SerializableFunction<Instant, Instant>> transforms) {
-    // ignore transforms
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index e2463d8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ /dev/null
@@ -1,355 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.ImmutableList;
-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.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.joda.time.Instant;
-
-/**
- * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
- * lower-bound, sometimes heuristically established, on event times that have been fully processed
- * by the pipeline.
- *
- * <p>For sources that provide non-heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
- * watermark is a strict guarantee that no data with an event time earlier than
- * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
- * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
- * of the window will be the last pane ever for that window.
- *
- * <p>For sources that provide heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
- * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
- * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
- * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
- * Thus, if absolute correctness over time is important to your use case, you may want to consider
- * using a trigger that accounts for late data. The default trigger,
- * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires
- * once when the watermark passes the end of the window and then immediately therafter when any
- * late data arrives, is one such example.
- *
- * <p>The watermark is the clock that defines {@link TimeDomain#EVENT_TIME}.
- *
- * <p>Additionaly firings before or after the watermark can be requested by calling
- * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or
- * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterWatermark {
-
-  private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
-
-  // Static factory class.
-  private AfterWatermark() {}
-
-  /**
-   * Creates a trigger that fires when the watermark passes the end of the window.
-   */
-  public static FromEndOfWindow pastEndOfWindow() {
-    return new FromEndOfWindow();
-  }
-
-  /**
-   * @see AfterWatermark
-   */
-  public static class AfterWatermarkEarlyAndLate extends Trigger {
-
-    private static final int EARLY_INDEX = 0;
-    private static final int LATE_INDEX = 1;
-
-    private final OnceTrigger earlyTrigger;
-    @Nullable
-    private final OnceTrigger lateTrigger;
-
-    @SuppressWarnings("unchecked")
-    public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
-      super(lateTrigger == null
-          ? ImmutableList.<Trigger>of(earlyTrigger)
-          : ImmutableList.<Trigger>of(earlyTrigger, lateTrigger));
-      this.earlyTrigger = checkNotNull(earlyTrigger, "earlyTrigger should not be null");
-      this.lateTrigger = lateTrigger;
-    }
-
-    public Trigger withEarlyFirings(OnceTrigger earlyTrigger) {
-      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
-    }
-
-    public Trigger withLateFirings(OnceTrigger lateTrigger) {
-      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
-    }
-
-    @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(),
-          lateTrigger == null ? null : lateTrigger.getContinuationTrigger());
-    }
-
-    @Override
-    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-      throw new UnsupportedOperationException(
-          "Should not call getContinuationTrigger(List<Trigger>)");
-    }
-
-    @Override
-    public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-      // Even without an early or late trigger, we'll still produce a firing at the watermark.
-      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);
-
-      if (!(earlyTrigger instanceof Never.NeverTrigger)) {
-        builder
-            .append(".withEarlyFirings(")
-            .append(earlyTrigger)
-            .append(")");
-      }
-
-      if (lateTrigger != null && !(lateTrigger instanceof Never.NeverTrigger)) {
-        builder
-            .append(".withLateFirings(")
-            .append(lateTrigger)
-            .append(")");
-      }
-
-      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);
-    }
-  }
-
-  /**
-   * A watermark trigger targeted relative to the end of the window.
-   */
-  public static class FromEndOfWindow extends OnceTrigger {
-
-    private FromEndOfWindow() {
-      super(null);
-    }
-
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires before the watermark has passed the end of the window.
-     */
-    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTrigger earlyFirings) {
-      checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
-      return new AfterWatermarkEarlyAndLate(earlyFirings, null);
-    }
-
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires after the watermark has passed the end of the window.
-     */
-    public AfterWatermarkEarlyAndLate withLateFirings(OnceTrigger lateFirings) {
-      checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
-      return new AfterWatermarkEarlyAndLate(Never.ever(), lateFirings);
-    }
-
-    @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();
-    }
-
-    @Override
-    public FromEndOfWindow getContinuationTrigger(List<Trigger> continuationTriggers) {
-      return this;
-    }
-
-    @Override
-    public String toString() {
-      return TO_STRING;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return obj instanceof FromEndOfWindow;
-    }
-
-    @Override
-    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/e4398e1e/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
deleted file mode 100644
index d6b72ef..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java
+++ /dev/null
@@ -1,92 +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.windowing;
-
-import java.util.List;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.joda.time.Instant;
-
-/**
- * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
- * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class DefaultTrigger extends Trigger{
-
-  private DefaultTrigger() {
-    super(null);
-  }
-
-  /**
-   * Returns the default trigger.
-   */
-  public static DefaultTrigger of() {
-    return new DefaultTrigger();
-  }
-
-  @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();
-  }
-
-  @Override
-  public boolean isCompatible(Trigger other) {
-    // Semantically, all default triggers are identical
-    return other instanceof DefaultTrigger;
-  }
-
-  @Override
-  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/e4398e1e/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
deleted file mode 100644
index 5f20465..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
+++ /dev/null
@@ -1,75 +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.windowing;
-
-import java.util.List;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.joda.time.Instant;
-
-/**
- * A trigger which never fires.
- *
- * <p>Using this trigger will only produce output when the watermark passes the end of the
- * {@link BoundedWindow window} plus the {@link Window#withAllowedLateness allowed
- * lateness}.
- */
-public final class Never {
-  /**
-   * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
-   * when the {@link BoundedWindow} closes.
-   */
-  public static OnceTrigger ever() {
-    // NeverTrigger ignores all inputs and is Window-type independent.
-    return new NeverTrigger();
-  }
-
-  // package-private in order to check identity for string formatting.
-  static class NeverTrigger extends OnceTrigger {
-    protected NeverTrigger() {
-      super(null);
-    }
-
-    @Override
-    public void onElement(OnElementContext c) {}
-
-    @Override
-    public void onMerge(OnMergeContext c) {}
-
-    @Override
-    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-      return this;
-    }
-
-    @Override
-    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/e4398e1e/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
deleted file mode 100644
index 25b7b34..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java
+++ /dev/null
@@ -1,105 +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.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;
-
-/**
- * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
- */
-class OrFinallyTrigger extends Trigger {
-
-  private static final int ACTUAL = 0;
-  private static final int UNTIL = 1;
-
-  @VisibleForTesting OrFinallyTrigger(Trigger actual, Trigger.OnceTrigger until) {
-    super(Arrays.asList(actual, until));
-  }
-
-  @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);
-    Instant untilDeadline = subTriggers.get(UNTIL).getWatermarkThatGuaranteesFiring(window);
-    return actualDeadline.isBefore(untilDeadline) ? actualDeadline : untilDeadline;
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    // Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL
-    // may not be a OnceTrigger.
-    return Repeatedly.forever(
-        new OrFinallyTrigger(
-            continuationTriggers.get(ACTUAL),
-            (Trigger.OnceTrigger) continuationTriggers.get(UNTIL)));
-  }
-
-  @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/e4398e1e/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
deleted file mode 100644
index 8858798..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java
+++ /dev/null
@@ -1,101 +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.windowing;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Repeat a trigger, either until some condition is met or forever.
- *
- * <p>For example, to fire after the end of the window, and every time late data arrives:
- * <pre> {@code
- *     Repeatedly.forever(AfterWatermark.isPastEndOfWindow());
- * } </pre>
- *
- * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite
- * {@code AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
- */
-public class Repeatedly extends Trigger {
-
-  private static final int REPEATED = 0;
-
-  /**
-   * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each
-   * time it fires and ignoring any indications to finish.
-   *
-   * <p>Unless used with {@link Trigger#orFinally} the composite trigger will never finish.
-   *
-   * @param repeated the trigger to execute repeatedly.
-   */
-  public static Repeatedly forever(Trigger repeated) {
-    return new Repeatedly(repeated);
-  }
-
-  private Repeatedly(Trigger repeated) {
-    super(Arrays.asList(repeated));
-  }
-
-
-  @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);
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new Repeatedly(continuationTriggers.get(REPEATED));
-  }
-
-  @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);
-  }
-}



[05/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
deleted file mode 100644
index 8858798..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Repeatedly.java
+++ /dev/null
@@ -1,101 +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.windowing;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Repeat a trigger, either until some condition is met or forever.
- *
- * <p>For example, to fire after the end of the window, and every time late data arrives:
- * <pre> {@code
- *     Repeatedly.forever(AfterWatermark.isPastEndOfWindow());
- * } </pre>
- *
- * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite
- * {@code AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
- */
-public class Repeatedly extends Trigger {
-
-  private static final int REPEATED = 0;
-
-  /**
-   * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each
-   * time it fires and ignoring any indications to finish.
-   *
-   * <p>Unless used with {@link Trigger#orFinally} the composite trigger will never finish.
-   *
-   * @param repeated the trigger to execute repeatedly.
-   */
-  public static Repeatedly forever(Trigger repeated) {
-    return new Repeatedly(repeated);
-  }
-
-  private Repeatedly(Trigger repeated) {
-    super(Arrays.asList(repeated));
-  }
-
-
-  @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);
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new Repeatedly(continuationTriggers.get(REPEATED));
-  }
-
-  @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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
deleted file mode 100644
index 9e2c27d..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ReshuffleTrigger.java
+++ /dev/null
@@ -1,66 +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.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.joda.time.Instant;
-
-/**
- * The trigger used with {@link Reshuffle} which triggers on every element
- * and never buffers state.
- *
- * @param <W> The kind of window that is being reshuffled.
- */
-public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
-
-  public ReshuffleTrigger() {
-    super(null);
-  }
-
-  @Override
-  public void onElement(Trigger.OnElementContext c) { }
-
-  @Override
-  public void onMerge(Trigger.OnMergeContext c) { }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return this;
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    throw new UnsupportedOperationException(
-        "ReshuffleTrigger should not be used outside of Reshuffle");
-  }
-
-  @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()";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
deleted file mode 100644
index a960aa4..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Trigger.java
+++ /dev/null
@@ -1,527 +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.windowing;
-
-import com.google.common.base.Joiner;
-import java.io.Serializable;
-import java.util.ArrayList;
-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.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.
- *
- * <p>See {@link org.apache.beam.sdk.transforms.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
- * elements as they are added. When the root trigger fires, the elements in the current pane will be
- * 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:
- * <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.
- * </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.
- * </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. */
-    public abstract boolean finishedInAnyMergingWindow();
-
-    /** Return true if the trigger is finished in all windows being merged. */
-    public abstract 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();
-  }
-
-  @Nullable
-  protected final List<Trigger> subTriggers;
-
-  protected Trigger(@Nullable List<Trigger> subTriggers) {
-    this.subTriggers = subTriggers;
-  }
-
-
-  /**
-   * 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 Iterable<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
-   * continue firing once (or multiple times).
-   */
-  public Trigger getContinuationTrigger() {
-    if (subTriggers == null) {
-      return getContinuationTrigger(null);
-    }
-
-    List<Trigger> subTriggerContinuations = new ArrayList<>();
-    for (Trigger subTrigger : subTriggers) {
-      subTriggerContinuations.add(subTrigger.getContinuationTrigger());
-    }
-    return getContinuationTrigger(subTriggerContinuations);
-  }
-
-  /**
-   * Return the {@link #getContinuationTrigger} of this {@code 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.
-   *
-   * <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.
-   */
-  public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window);
-
-  /**
-   * Returns whether this performs the same triggering as the given {@code Trigger}.
-   */
-  public boolean isCompatible(Trigger other) {
-    if (!getClass().equals(other.getClass())) {
-      return false;
-    }
-
-    if (subTriggers == null) {
-      return other.subTriggers == null;
-    } else if (other.subTriggers == null) {
-      return false;
-    } else if (subTriggers.size() != other.subTriggers.size()) {
-      return false;
-    }
-
-    for (int i = 0; i < subTriggers.size(); i++) {
-      if (!subTriggers.get(i).isCompatible(other.subTriggers.get(i))) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    String simpleName = getClass().getSimpleName();
-    if (getClass().getEnclosingClass() != null) {
-      simpleName = getClass().getEnclosingClass().getSimpleName() + "." + simpleName;
-    }
-    if (subTriggers == null || subTriggers.size() == 0) {
-      return simpleName;
-    } else {
-      return simpleName + "(" + Joiner.on(", ").join(subTriggers) + ")";
-    }
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof Trigger)) {
-      return false;
-    }
-    Trigger that = (Trigger) obj;
-    return Objects.equals(getClass(), that.getClass())
-        && Objects.equals(subTriggers, that.subTriggers);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(getClass(), subTriggers);
-  }
-
-  /**
-   * Specify an ending condition for this trigger. If the {@code until} 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>For example the final firing of the following trigger may only have 1 element:
-   * <pre> {@code
-   * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
-   *     .orFinally(AfterPane.elementCountAtLeast(5))
-   * } </pre>
-   *
-   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same
-   * as {@code AfterFirst.of(t1, t2)}.
-   */
-  public Trigger 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.
-   */
-  public abstract static class OnceTrigger extends Trigger {
-    protected OnceTrigger(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public final OnceTrigger getContinuationTrigger() {
-      Trigger continuation = super.getContinuationTrigger();
-      if (!(continuation instanceof OnceTrigger)) {
-        throw new IllegalStateException("Continuation of a OnceTrigger must be a OnceTrigger");
-      }
-      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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerContextFactory.java
deleted file mode 100644
index e09aac2..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
deleted file mode 100644
index 8d0f322..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/TriggerRunner.java
+++ /dev/null
@@ -1,247 +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.core;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.Map;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.BitSetCoder;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.apache.beam.sdk.util.FinishedTriggers;
-import org.apache.beam.sdk.util.FinishedTriggersBitSet;
-import org.apache.beam.sdk.util.Timers;
-import org.apache.beam.sdk.util.TriggerContextFactory;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.joda.time.Instant;
-
-/**
- * Executes a trigger while managing persistence of information about which subtriggers are
- * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
- *
- * <p>Specifically, the responsibilities are:
- *
- * <ul>
- *   <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
- *       constructing the appropriate trigger contexts.</li>
- *   <li>Committing a record of which subtriggers are finished to persistent state.</li>
- *   <li>Restoring the record of which subtriggers are finished from persistent state.</li>
- *   <li>Clearing out the persisted finished set when a caller indicates
- *       (via {#link #clearFinished}) that it is no longer needed.</li>
- * </ul>
- *
- * <p>These responsibilities are intertwined: trigger contexts include mutable information about
- * which subtriggers are finished. This class provides the information when building the contexts
- * and commits the information when the method of the {@link ExecutableTrigger} returns.
- *
- * @param <W> The kind of windows being processed.
- */
-public class TriggerRunner<W extends BoundedWindow> {
-  @VisibleForTesting
-  static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
-      StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
-
-  private final ExecutableTrigger rootTrigger;
-  private final TriggerContextFactory<W> contextFactory;
-
-  public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
-    checkState(rootTrigger.getTriggerIndex() == 0);
-    this.rootTrigger = rootTrigger;
-    this.contextFactory = contextFactory;
-  }
-
-  private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
-    if (!isFinishedSetNeeded()) {
-      // If no trigger in the tree will ever have finished bits, then we don't need to read them.
-      // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
-      // finished) for each trigger in the tree.
-      return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
-    }
-
-    BitSet bitSet = state.read();
-    return bitSet == null
-        ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
-            : FinishedTriggersBitSet.fromBitSet(bitSet);
-  }
-
-
-  private void clearFinishedBits(ValueState<BitSet> state) {
-    if (!isFinishedSetNeeded()) {
-      // Nothing to clear.
-      return;
-    }
-    state.clear();
-  }
-
-  /** Return true if the trigger is closed in the window corresponding to the specified state. */
-  public boolean isClosed(StateAccessor<?> state) {
-    return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchForValue(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchOnElement(
-        contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchOnFire(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchShouldFire(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchShouldFire(
-        contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  /**
-   * Run the trigger logic to deal with a new value.
-   */
-  public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
-      throws Exception {
-    // Clone so that we can detect changes and so that changes here don't pollute merging.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
-        window, timers, timestamp, rootTrigger, finishedSet);
-    rootTrigger.invokeOnElement(triggerContext);
-    persistFinishedSet(state, finishedSet);
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchForMerge(
-      W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
-    if (isFinishedSetNeeded()) {
-      for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
-        value.readLater();
-      }
-    }
-    rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
-        window, mergingWindows, rootTrigger));
-  }
-
-  /**
-   * Run the trigger merging logic as part of executing the specified merge.
-   */
-  public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
-    // Clone so that we can detect changes and so that changes here don't pollute merging.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-
-    // And read the finished bits in each merging window.
-    ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
-    for (Map.Entry<W, ValueState<BitSet>> entry :
-        state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
-      // Don't need to clone these, since the trigger context doesn't allow modification
-      builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
-      // Clear the underlying finished bits.
-      clearFinishedBits(entry.getValue());
-    }
-    ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
-
-    Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
-        window, timers, rootTrigger, finishedSet, mergingFinishedSets);
-
-    // Run the merge from the trigger
-    rootTrigger.invokeOnMerge(mergeContext);
-
-    persistFinishedSet(state, finishedSet);
-  }
-
-  public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.TriggerContext context = contextFactory.base(window, timers,
-        rootTrigger, finishedSet);
-    return rootTrigger.invokeShouldFire(context);
-  }
-
-  public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    // shouldFire should be false.
-    // However it is too expensive to assert.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.TriggerContext context = contextFactory.base(window, timers,
-        rootTrigger, finishedSet);
-    rootTrigger.invokeOnFire(context);
-    persistFinishedSet(state, finishedSet);
-  }
-
-  private void persistFinishedSet(
-      StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
-    if (!isFinishedSetNeeded()) {
-      return;
-    }
-
-    ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
-    if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
-      if (modifiedFinishedSet.getBitSet().isEmpty()) {
-        finishedSetState.clear();
-      } else {
-        finishedSetState.write(modifiedFinishedSet.getBitSet());
-      }
-    }
-  }
-
-  /**
-   * Clear the finished bits.
-   */
-  public void clearFinished(StateAccessor<?> state) {
-    clearFinishedBits(state.access(FINISHED_BITS_TAG));
-  }
-
-  /**
-   * Clear the state used for executing triggers, but leave the finished set to indicate
-   * the window is closed.
-   */
-  public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    // Don't need to clone, because we'll be clearing the finished bits anyways.
-    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
-    rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
-  }
-
-  private boolean isFinishedSetNeeded() {
-    // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
-    // lookup. Right now, we special case this for the DefaultTrigger.
-    return !(rootTrigger.getSpec() instanceof DefaultTrigger);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
new file mode 100644
index 0000000..2f4ad63
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
@@ -0,0 +1,104 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Joiner;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.annotations.Experimental;
+
+/**
+ * A {@link TriggerStateMachine} that fires and finishes once after all of its sub-triggers
+ * have fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterAllStateMachine extends OnceTriggerStateMachine {
+
+  private AfterAllStateMachine(List<TriggerStateMachine> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
+   */
+  @SafeVarargs
+  public static OnceTriggerStateMachine of(OnceTriggerStateMachine... triggers) {
+    return new AfterAllStateMachine(Arrays.<TriggerStateMachine>asList(triggers));
+  }
+
+  @Override
+  public void onElement(OnElementContext c) throws Exception {
+    for (ExecutableTriggerStateMachine 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 (ExecutableTriggerStateMachine subTrigger : c.trigger().subTriggers()) {
+      subTrigger.invokeOnMerge(c);
+    }
+    boolean allFinished = true;
+    for (ExecutableTriggerStateMachine subTrigger1 : c.trigger().subTriggers()) {
+      allFinished &= c.forTrigger(subTrigger1).trigger().isFinished();
+    }
+    c.trigger().setFinished(allFinished);
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code true} if all subtriggers return {@code true}.
+   */
+  @Override
+  public boolean shouldFire(TriggerContext context) throws Exception {
+    for (ExecutableTriggerStateMachine 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 (ExecutableTriggerStateMachine subtrigger : context.trigger().subTriggers()) {
+      subtrigger.invokeOnFire(context);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("AfterAll.of(");
+    Joiner.on(", ").appendTo(builder, subTriggers);
+    builder.append(")");
+
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
new file mode 100644
index 0000000..a6616fa
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
@@ -0,0 +1,322 @@
+/*
+ * 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.triggers;
+
+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.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Min;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+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;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+import org.joda.time.format.PeriodFormatter;
+
+/**
+ * A base class for triggers that happen after a processing time delay from the arrival
+ * of the first element in a pane.
+ *
+ * <p>This class is for internal use only and may change at any time.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public abstract class AfterDelayFromFirstElementStateMachine extends OnceTriggerStateMachine {
+
+  protected static final List<SerializableFunction<Instant, Instant>> IDENTITY =
+      ImmutableList.<SerializableFunction<Instant, Instant>>of();
+
+  protected static final StateTag<Object, AccumulatorCombiningState<Instant,
+                                              Combine.Holder<Instant>, Instant>> DELAYED_UNTIL_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "delayed", InstantCoder.of(), Min.MinFn.<Instant>naturalOrder()));
+
+  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(TriggerStateMachine.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.
+   */
+  protected abstract AfterDelayFromFirstElementStateMachine newWith(
+      List<SerializableFunction<Instant, Instant>> transform);
+
+  /**
+   * A list of timestampMappers m1, m2, m3, ... m_n considered to be composed in sequence. The
+   * overall mapping for an instance `instance` is `m_n(... m3(m2(m1(instant))`,
+   * implemented via #computeTargetTimestamp
+   */
+  protected final List<SerializableFunction<Instant, Instant>> timestampMappers;
+
+  private final TimeDomain timeDomain;
+
+  public AfterDelayFromFirstElementStateMachine(
+      TimeDomain timeDomain,
+      List<SerializableFunction<Instant, Instant>> timestampMappers) {
+    super(null);
+    this.timestampMappers = timestampMappers;
+    this.timeDomain = timeDomain;
+  }
+
+  private Instant getTargetTimestamp(OnElementContext c) {
+    return computeTargetTimestamp(c.currentProcessingTime());
+  }
+
+  /**
+   * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+   * than the timestamp.
+   *
+   * <p>TODO: Consider sharing this with FixedWindows, and bring over the equivalent of
+   * CalendarWindows.
+   */
+  public AfterDelayFromFirstElementStateMachine alignedTo(
+      final Duration size, final Instant offset) {
+    return newWith(new AlignFn(size, offset));
+  }
+
+  /**
+   * Aligns the time to be the smallest multiple of {@code size} greater than the timestamp
+   * since the epoch.
+   */
+  public AfterDelayFromFirstElementStateMachine alignedTo(final Duration size) {
+    return alignedTo(size, new Instant(0));
+  }
+
+  /**
+   * Adds some delay to the original target time.
+   *
+   * @param delay the delay to add
+   * @return An updated time trigger that will wait the additional time before firing.
+   */
+  public AfterDelayFromFirstElementStateMachine plusDelayOf(final Duration delay) {
+    return newWith(new DelayFn(delay));
+  }
+
+  @Override
+  public boolean isCompatible(TriggerStateMachine other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    AfterDelayFromFirstElementStateMachine that = (AfterDelayFromFirstElementStateMachine) other;
+    return this.timestampMappers.equals(that.timestampMappers);
+  }
+
+
+  private AfterDelayFromFirstElementStateMachine newWith(
+      SerializableFunction<Instant, Instant> timestampMapper) {
+    return newWith(
+        ImmutableList.<SerializableFunction<Instant, Instant>>builder()
+            .addAll(timestampMappers)
+            .add(timestampMapper)
+            .build());
+  }
+
+  @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 boolean shouldFire(TriggerStateMachine.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(TriggerStateMachine.TriggerContext context) throws Exception {
+    clear(context);
+  }
+
+  protected Instant computeTargetTimestamp(Instant time) {
+    Instant result = time;
+    for (SerializableFunction<Instant, Instant> timestampMapper : timestampMappers) {
+      result = timestampMapper.apply(result);
+    }
+    return result;
+  }
+
+  /**
+   * A {@link SerializableFunction} to delay the timestamp at which this triggers fires.
+   */
+  private static final class DelayFn implements SerializableFunction<Instant, Instant> {
+    private final Duration delay;
+
+    public DelayFn(Duration delay) {
+      this.delay = delay;
+    }
+
+    @Override
+    public Instant apply(Instant input) {
+      return input.plus(delay);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof DelayFn)) {
+        return false;
+      }
+
+      return this.delay.equals(((DelayFn) object).delay);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(delay);
+    }
+
+    @Override
+    public String toString() {
+      return PERIOD_FORMATTER.print(delay.toPeriod());
+    }
+  }
+
+  /**
+   * A {@link SerializableFunction} to align an instant to the nearest interval boundary.
+   */
+  static final class AlignFn implements SerializableFunction<Instant, Instant> {
+    private final Duration size;
+    private final Instant offset;
+
+
+    /**
+     * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+     * than the timestamp.
+     */
+    public AlignFn(Duration size, Instant offset) {
+      this.size = size;
+      this.offset = offset;
+    }
+
+    @Override
+    public Instant apply(Instant point) {
+      long millisSinceStart = new Duration(offset, point).getMillis() % size.getMillis();
+      return millisSinceStart == 0 ? point : point.plus(size).minus(millisSinceStart);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof AlignFn)) {
+        return false;
+      }
+
+      AlignFn other = (AlignFn) object;
+      return other.size.equals(this.size)
+          && other.offset.equals(this.offset);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(size, offset);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
new file mode 100644
index 0000000..140ac75
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
@@ -0,0 +1,125 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Joiner;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A composite {@link TriggerStateMachine} that executes its sub-triggers in order.
+ * Only one sub-trigger is executing at a time,
+ * and any time it fires the {@code AfterEach} fires. When the currently executing
+ * sub-trigger finishes, the {@code AfterEach} starts executing the next sub-trigger.
+ *
+ * <p>{@code AfterEach.inOrder(t1, t2, ...)} finishes when all of the sub-triggers have finished.
+ *
+ * <p>The following properties hold:
+ * <ul>
+ *   <li> {@code AfterEach.inOrder(AfterEach.inOrder(a, b), c)} behaves the same as
+ *   {@code AfterEach.inOrder(a, b, c)} and {@code AfterEach.inOrder(a, AfterEach.inOrder(b, c)}.
+ *   <li> {@code AfterEach.inOrder(Repeatedly.forever(a), b)} behaves the same as
+ *   {@code Repeatedly.forever(a)}, since the repeated trigger never finishes.
+ * </ul>
+ */
+public class AfterEachStateMachine extends TriggerStateMachine {
+
+  private AfterEachStateMachine(List<TriggerStateMachine> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterEach} {@code Trigger} with the given subtriggers.
+   */
+  @SafeVarargs
+  public static TriggerStateMachine inOrder(TriggerStateMachine... triggers) {
+    return new AfterEachStateMachine(Arrays.<TriggerStateMachine>asList(triggers));
+  }
+
+  @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 (ExecutableTriggerStateMachine 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 (ExecutableTriggerStateMachine subTrigger : context.trigger().subTriggers()) {
+      if (priorTriggersAllFinished) {
+        subTrigger.invokeOnMerge(context);
+        priorTriggersAllFinished &= context.forTrigger(subTrigger).trigger().isFinished();
+      } else {
+        subTrigger.invokeClear(context);
+      }
+    }
+    updateFinishedState(context);
+  }
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    ExecutableTriggerStateMachine firstUnfinished = context.trigger().firstUnfinishedSubTrigger();
+    return firstUnfinished.invokeShouldFire(context);
+  }
+
+  @Override
+  public void onFire(TriggerStateMachine.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 (ExecutableTriggerStateMachine subTrigger : context.trigger().subTriggers()) {
+        subTrigger.invokeClear(context);
+      }
+    }
+
+    updateFinishedState(context);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("AfterEach.inOrder(");
+    Joiner.on(", ").appendTo(builder, subTriggers);
+    builder.append(")");
+
+    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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
new file mode 100644
index 0000000..272c278
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
@@ -0,0 +1,106 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Joiner;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.annotations.Experimental;
+
+/**
+ * Create a composite {@link TriggerStateMachine} that fires once after at least one of its
+ * sub-triggers have fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterFirstStateMachine extends OnceTriggerStateMachine {
+
+  AfterFirstStateMachine(List<TriggerStateMachine> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
+   */
+  @SafeVarargs
+  public static OnceTriggerStateMachine of(
+      OnceTriggerStateMachine... triggers) {
+    return new AfterFirstStateMachine(Arrays.<TriggerStateMachine>asList(triggers));
+  }
+
+  @Override
+  public void onElement(OnElementContext c) throws Exception {
+    for (ExecutableTriggerStateMachine subTrigger : c.trigger().subTriggers()) {
+      subTrigger.invokeOnElement(c);
+    }
+  }
+
+  @Override
+  public void onMerge(OnMergeContext c) throws Exception {
+    for (ExecutableTriggerStateMachine subTrigger : c.trigger().subTriggers()) {
+      subTrigger.invokeOnMerge(c);
+    }
+    updateFinishedStatus(c);
+  }
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    for (ExecutableTriggerStateMachine 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 (ExecutableTriggerStateMachine 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);
+    builder.append(")");
+
+    return builder.toString();
+  }
+
+  private void updateFinishedStatus(TriggerContext c) {
+    boolean anyFinished = false;
+    for (ExecutableTriggerStateMachine 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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java
new file mode 100644
index 0000000..723aba6
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java
@@ -0,0 +1,132 @@
+/*
+ * 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.triggers;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.Objects;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Sum;
+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;
+
+/**
+ * {@link TriggerStateMachine}s that fire based on properties of the elements in the current pane.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterPaneStateMachine extends OnceTriggerStateMachine {
+
+private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
+      ELEMENTS_IN_PANE_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "count", VarLongCoder.of(), new Sum.SumLongFn()));
+
+  private final int countElems;
+
+  private AfterPaneStateMachine(int countElems) {
+    super(null);
+    this.countElems = countElems;
+  }
+
+  /**
+   * Creates a trigger that fires when the pane contains at least {@code countElems} elements.
+   */
+  public static AfterPaneStateMachine elementCountAtLeast(int countElems) {
+    return new AfterPaneStateMachine(countElems);
+  }
+
+  @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(TriggerStateMachine.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(TriggerStateMachine other) {
+    return this.equals(other);
+  }
+
+  @Override
+  public String toString() {
+    return "AfterPane.elementCountAtLeast(" + countElems + ")";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterPaneStateMachine)) {
+      return false;
+    }
+    AfterPaneStateMachine that = (AfterPaneStateMachine) obj;
+    return this.countElems == that.countElems;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(countElems);
+  }
+
+  @Override
+  protected void onOnlyFiring(TriggerStateMachine.TriggerContext context) throws Exception {
+    clear(context);
+  }
+}


[03/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java
new file mode 100644
index 0000000..0ffbbca
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java
@@ -0,0 +1,234 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.BitSetCoder;
+import org.apache.beam.sdk.util.Timers;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.joda.time.Instant;
+
+/**
+ * Executes a trigger while managing persistence of information about which subtriggers are
+ * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
+ *
+ * <p>Specifically, the responsibilities are:
+ *
+ * <ul>
+ *   <li>Invoking the trigger's methods via its {@link ExecutableTriggerStateMachine} wrapper by
+ *       constructing the appropriate trigger contexts.</li>
+ *   <li>Committing a record of which subtriggers are finished to persistent state.</li>
+ *   <li>Restoring the record of which subtriggers are finished from persistent state.</li>
+ *   <li>Clearing out the persisted finished set when a caller indicates
+ *       (via {#link #clearFinished}) that it is no longer needed.</li>
+ * </ul>
+ *
+ * <p>These responsibilities are intertwined: trigger contexts include mutable information about
+ * which subtriggers are finished. This class provides the information when building the contexts
+ * and commits the information when the method of the {@link ExecutableTriggerStateMachine} returns.
+ *
+ * @param <W> The kind of windows being processed.
+ */
+public class TriggerStateMachineRunner<W extends BoundedWindow> {
+  @VisibleForTesting
+  static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
+      StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
+
+  private final ExecutableTriggerStateMachine rootTrigger;
+  private final TriggerStateMachineContextFactory<W> contextFactory;
+
+  public TriggerStateMachineRunner(
+      ExecutableTriggerStateMachine rootTrigger,
+      TriggerStateMachineContextFactory<W> contextFactory) {
+    checkState(rootTrigger.getTriggerIndex() == 0);
+    this.rootTrigger = rootTrigger;
+    this.contextFactory = contextFactory;
+  }
+
+  private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // If no trigger in the tree will ever have finished bits, then we don't need to read them.
+      // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
+      // finished) for each trigger in the tree.
+      return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
+    }
+
+    BitSet bitSet = state.read();
+    return bitSet == null
+        ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
+            : FinishedTriggersBitSet.fromBitSet(bitSet);
+  }
+
+
+  private void clearFinishedBits(ValueState<BitSet> state) {
+    if (!isFinishedSetNeeded()) {
+      // Nothing to clear.
+      return;
+    }
+    state.clear();
+  }
+
+  /** Return true if the trigger is closed in the window corresponding to the specified state. */
+  public boolean isClosed(StateAccessor<?> state) {
+    return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
+  }
+
+  public void prefetchForValue(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnElement(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  public void prefetchOnFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  public void prefetchShouldFire(W window, StateAccessor<?> state) {
+    if (isFinishedSetNeeded()) {
+      state.access(FINISHED_BITS_TAG).readLater();
+    }
+    rootTrigger.getSpec().prefetchShouldFire(
+        contextFactory.createStateAccessor(window, rootTrigger));
+  }
+
+  /**
+   * Run the trigger logic to deal with a new value.
+   */
+  public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
+      throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    TriggerStateMachine.OnElementContext triggerContext = contextFactory.createOnElementContext(
+        window, timers, timestamp, rootTrigger, finishedSet);
+    rootTrigger.invokeOnElement(triggerContext);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  public void prefetchForMerge(
+      W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
+    if (isFinishedSetNeeded()) {
+      for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
+        value.readLater();
+      }
+    }
+    rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
+        window, mergingWindows, rootTrigger));
+  }
+
+  /**
+   * Run the trigger merging logic as part of executing the specified merge.
+   */
+  public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
+    // Clone so that we can detect changes and so that changes here don't pollute merging.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+
+    // And read the finished bits in each merging window.
+    ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
+    for (Map.Entry<W, ValueState<BitSet>> entry :
+        state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
+      // Don't need to clone these, since the trigger context doesn't allow modification
+      builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
+      // Clear the underlying finished bits.
+      clearFinishedBits(entry.getValue());
+    }
+    ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
+
+    TriggerStateMachine.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
+        window, timers, rootTrigger, finishedSet, mergingFinishedSets);
+
+    // Run the merge from the trigger
+    rootTrigger.invokeOnMerge(mergeContext);
+
+    persistFinishedSet(state, finishedSet);
+  }
+
+  public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    TriggerStateMachine.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    return rootTrigger.invokeShouldFire(context);
+  }
+
+  public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // shouldFire should be false.
+    // However it is too expensive to assert.
+    FinishedTriggersBitSet finishedSet =
+        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+    TriggerStateMachine.TriggerContext context = contextFactory.base(window, timers,
+        rootTrigger, finishedSet);
+    rootTrigger.invokeOnFire(context);
+    persistFinishedSet(state, finishedSet);
+  }
+
+  private void persistFinishedSet(
+      StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
+    if (!isFinishedSetNeeded()) {
+      return;
+    }
+
+    ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
+    if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
+      if (modifiedFinishedSet.getBitSet().isEmpty()) {
+        finishedSetState.clear();
+      } else {
+        finishedSetState.write(modifiedFinishedSet.getBitSet());
+      }
+    }
+  }
+
+  /**
+   * Clear the finished bits.
+   */
+  public void clearFinished(StateAccessor<?> state) {
+    clearFinishedBits(state.access(FINISHED_BITS_TAG));
+  }
+
+  /**
+   * Clear the state used for executing triggers, but leave the finished set to indicate
+   * the window is closed.
+   */
+  public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
+    // Don't need to clone, because we'll be clearing the finished bits anyways.
+    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
+    rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
+  }
+
+  private boolean isFinishedSetNeeded() {
+    // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
+    // lookup. Right now, we special case this for the DefaultTrigger.
+    return !(rootTrigger.getSpec() instanceof DefaultTriggerStateMachine);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/package-info.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/package-info.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/package-info.java
new file mode 100644
index 0000000..b7c7050
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * State machine implementations for triggers, called "triggers" because
+ * they react to events.
+ */
+package org.apache.beam.runners.core.triggers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
deleted file mode 100644
index b591229..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterAllTest.java
+++ /dev/null
@@ -1,156 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link AfterAll}.
- */
-@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));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterAll.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(1))
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterAll = AfterAll.of(trigger1, trigger2);
-    assertEquals(
-        AfterAll.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
-        afterAll.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterAll.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
-    assertEquals("AfterAll.of(t1, t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
deleted file mode 100644
index c413c6e..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterEachTest.java
+++ /dev/null
@@ -1,132 +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.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}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        AfterEach.inOrder(AfterWatermark.pastEndOfWindow(),
-                          AfterPane.elementCountAtLeast(4))
-            .getWatermarkThatGuaranteesFiring(window));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterEach.inOrder(AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow())
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterEach = AfterEach.inOrder(trigger1, trigger2);
-    assertEquals(
-        Repeatedly.forever(AfterFirst.of(
-            trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger())),
-        afterEach.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterEach.inOrder(
-        StubTrigger.named("t1"),
-        StubTrigger.named("t2"),
-        StubTrigger.named("t3"));
-
-    assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
deleted file mode 100644
index 415060b..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterFirstTest.java
+++ /dev/null
@@ -1,181 +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.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}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        AfterFirst.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4))
-            .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterFirst.of(AfterPane.elementCountAtLeast(2), AfterPane.elementCountAtLeast(1))
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterFirst = AfterFirst.of(trigger1, trigger2);
-    assertEquals(
-        AfterFirst.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
-        afterFirst.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterFirst.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
-    assertEquals("AfterFirst.of(t1, t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
deleted file mode 100644
index 38d030e..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterPaneTest.java
+++ /dev/null
@@ -1,132 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link AfterPane}.
- */
-@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,
-        AfterPane.elementCountAtLeast(1).getWatermarkThatGuaranteesFiring(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(
-        AfterPane.elementCountAtLeast(1),
-        AfterPane.elementCountAtLeast(100).getContinuationTrigger());
-    assertEquals(
-        AfterPane.elementCountAtLeast(1),
-        AfterPane.elementCountAtLeast(100).getContinuationTrigger().getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterPane.elementCountAtLeast(5);
-    assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
deleted file mode 100644
index 13a7acf..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterProcessingTimeTest.java
+++ /dev/null
@@ -1,187 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link AfterProcessingTime}.
- */
-@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,
-        AfterProcessingTime.pastFirstElementInPane().getWatermarkThatGuaranteesFiring(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger firstElementPlus1 =
-        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.standardHours(1));
-    assertEquals(
-        new AfterSynchronizedProcessingTime(),
-        firstElementPlus1.getContinuationTrigger());
-  }
-
-  /**
-   * Basic test of compatibility check between identical triggers.
-   */
-  @Test
-  public void testCompatibilityIdentical() throws Exception {
-    Trigger t1 = AfterProcessingTime.pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(1L));
-    Trigger t2 = AfterProcessingTime.pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(1L));
-    assertTrue(t1.isCompatible(t2));
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
-    assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString());
-  }
-
-  @Test
-  public void testWithDelayToString() {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane()
-        .plusDelayOf(Duration.standardMinutes(5));
-
-    assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)",
-        trigger.toString());
-  }
-
-  @Test
-  public void testBuiltUpToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withLateFirings(AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(10)));
-
-    String expected = "AfterWatermark.pastEndOfWindow()"
-        + ".withLateFirings(AfterProcessingTime"
-        + ".pastFirstElementInPane()"
-        + ".plusDelayOf(10 minutes))";
-
-    assertEquals(expected, trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
deleted file mode 100644
index 7e6e938..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTimeTest.java
+++ /dev/null
@@ -1,121 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link AfterSynchronizedProcessingTime}.
- */
-@RunWith(JUnit4.class)
-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(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(underTest, underTest.getContinuationTrigger());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
deleted file mode 100644
index 084027b..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/AfterWatermarkTest.java
+++ /dev/null
@@ -1,380 +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.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.
- */
-@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();
-    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
-  }
-
-  @Test
-  public void testEarlyFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow().withEarlyFirings(StubTrigger.named("t1"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
-  }
-
-  @Test
-  public void testLateFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow().withLateFirings(StubTrigger.named("t1"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
-  }
-
-  @Test
-  public void testEarlyAndLateFiringsToString() {
-    Trigger trigger =
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(StubTrigger.named("t1"))
-            .withLateFirings(StubTrigger.named("t2"));
-
-    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
-        trigger.toString());
-  }
-
-  @Test
-  public void testToStringExcludesNeverTrigger() {
-    Trigger trigger =
-        AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(Never.ever())
-            .withLateFirings(Never.ever());
-
-    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
deleted file mode 100644
index 673e555..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/DefaultTriggerTest.java
+++ /dev/null
@@ -1,176 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link DefaultTrigger}, which should be equivalent to
- * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
- */
-@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(
-        new IntervalWindow(new Instant(0), new Instant(10))));
-    assertEquals(GlobalWindow.INSTANCE.maxTimestamp(),
-        DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(DefaultTrigger.of(), DefaultTrigger.of().getContinuationTrigger());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
deleted file mode 100644
index 1e3a1ff..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ExecutableTriggerTest.java
+++ /dev/null
@@ -1,127 +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.assertEquals;
-import static org.junit.Assert.assertSame;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ExecutableTrigger}.
- */
-@RunWith(JUnit4.class)
-public class ExecutableTriggerTest {
-
-  @Test
-  public void testIndexAssignmentLeaf() throws Exception {
-    StubTrigger t1 = new StubTrigger();
-    ExecutableTrigger executable = ExecutableTrigger.create(t1);
-    assertEquals(0, executable.getTriggerIndex());
-  }
-
-  @Test
-  public void testIndexAssignmentOneLevel() throws Exception {
-    StubTrigger t1 = new StubTrigger();
-    StubTrigger t2 = new StubTrigger();
-    StubTrigger t = new StubTrigger(t1, t2);
-
-    ExecutableTrigger executable = ExecutableTrigger.create(t);
-
-    assertEquals(0, executable.getTriggerIndex());
-    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
-    assertSame(t1, executable.subTriggers().get(0).getSpec());
-    assertEquals(2, executable.subTriggers().get(1).getTriggerIndex());
-    assertSame(t2, executable.subTriggers().get(1).getSpec());
-  }
-
-  @Test
-  public void testIndexAssignmentTwoLevel() throws Exception {
-    StubTrigger t11 = new StubTrigger();
-    StubTrigger t12 = new StubTrigger();
-    StubTrigger t13 = new StubTrigger();
-    StubTrigger t14 = new StubTrigger();
-    StubTrigger t21 = new StubTrigger();
-    StubTrigger t22 = new StubTrigger();
-    StubTrigger t1 = new StubTrigger(t11, t12, t13, t14);
-    StubTrigger t2 = new StubTrigger(t21, t22);
-    StubTrigger t = new StubTrigger(t1, t2);
-
-    ExecutableTrigger executable = ExecutableTrigger.create(t);
-
-    assertEquals(0, executable.getTriggerIndex());
-    assertEquals(1, executable.subTriggers().get(0).getTriggerIndex());
-    assertEquals(6, executable.subTriggers().get(0).getFirstIndexAfterSubtree());
-    assertEquals(6, executable.subTriggers().get(1).getTriggerIndex());
-
-    assertSame(t1, executable.getSubTriggerContaining(1).getSpec());
-    assertSame(t2, executable.getSubTriggerContaining(6).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(2).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(3).getSpec());
-    assertSame(t1, executable.getSubTriggerContaining(5).getSpec());
-    assertSame(t2, executable.getSubTriggerContaining(7).getSpec());
-  }
-
-  private static class StubTrigger extends Trigger {
-
-    @SafeVarargs
-    protected StubTrigger(Trigger... subTriggers) {
-      super(Arrays.asList(subTriggers));
-    }
-
-    @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;
-    }
-
-    @Override
-    public boolean isCompatible(Trigger other) {
-      return false;
-    }
-
-    @Override
-    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/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSetTest.java
deleted file mode 100644
index 7f74620..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/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())));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersProperties.java
deleted file mode 100644
index a66f74f..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/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/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/FinishedTriggersSetTest.java
deleted file mode 100644
index 072d264..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/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())));
-  }
-}



[14/17] incubator-beam git commit: Restore prior trigger files, for temporary compatibility

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..7e6e938
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterSynchronizedProcessingTime}.
+ */
+@RunWith(JUnit4.class)
+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(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(underTest, underTest.getContinuationTrigger());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..084027b
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
@@ -0,0 +1,380 @@
+/*
+ * 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.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.
+ */
+@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();
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyFiringsToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withEarlyFirings(StubTrigger.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testLateFiringsToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withLateFirings(StubTrigger.named("t1"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
+  }
+
+  @Test
+  public void testEarlyAndLateFiringsToString() {
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(StubTrigger.named("t1"))
+            .withLateFirings(StubTrigger.named("t2"));
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testToStringExcludesNeverTrigger() {
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(Never.ever())
+            .withLateFirings(Never.ever());
+
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..673e555
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link DefaultTrigger}, which should be equivalent to
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+ */
+@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(
+        new IntervalWindow(new Instant(0), new Instant(10))));
+    assertEquals(GlobalWindow.INSTANCE.maxTimestamp(),
+        DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(DefaultTrigger.of(), DefaultTrigger.of().getContinuationTrigger());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..fb2b4d5
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.windowing;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+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}.
+ */
+@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 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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..7289d97
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java
@@ -0,0 +1,215 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link OrFinallyTrigger}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        Repeatedly.forever(AfterWatermark.pastEndOfWindow())
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow())
+        .orFinally(AfterPane.elementCountAtLeast(1))
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9), Repeatedly.forever(AfterPane.elementCountAtLeast(1))
+        .orFinally(AfterWatermark.pastEndOfWindow())
+        .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(new Instant(9),
+        AfterPane.elementCountAtLeast(100)
+            .orFinally(AfterWatermark.pastEndOfWindow())
+            .getWatermarkThatGuaranteesFiring(window));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        Repeatedly.forever(AfterPane.elementCountAtLeast(1))
+        .orFinally(AfterPane.elementCountAtLeast(10))
+        .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger triggerA = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger triggerB = AfterWatermark.pastEndOfWindow();
+    Trigger aOrFinallyB = triggerA.orFinally(triggerB);
+    Trigger bOrFinallyA = triggerB.orFinally(triggerA);
+    assertEquals(
+        Repeatedly.forever(
+            triggerA.getContinuationTrigger().orFinally(triggerB.getContinuationTrigger())),
+        aOrFinallyB.getContinuationTrigger());
+    assertEquals(
+        Repeatedly.forever(
+            triggerB.getContinuationTrigger().orFinally(triggerA.getContinuationTrigger())),
+        bOrFinallyA.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = StubTrigger.named("t1").orFinally(StubTrigger.named("t2"));
+    assertEquals("t1.orFinally(t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..6e8930d
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.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;
+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 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.
+   */
+  @Test
+  public void testFireDeadline() throws Exception {
+    setUp(FixedWindows.of(Duration.millis(10)));
+    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
+    Instant arbitraryInstant = new Instant(34957849);
+
+    when(mockTrigger.getWatermarkThatGuaranteesFiring(Mockito.<IntervalWindow>any()))
+        .thenReturn(arbitraryInstant);
+
+    assertThat(
+        Repeatedly.forever(mockTrigger).getWatermarkThatGuaranteesFiring(window),
+        equalTo(arbitraryInstant));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
+    Trigger repeatedly = Repeatedly.forever(trigger);
+    assertEquals(
+        Repeatedly.forever(trigger.getContinuationTrigger()), repeatedly.getContinuationTrigger());
+    assertEquals(
+        Repeatedly.forever(trigger.getContinuationTrigger().getContinuationTrigger()),
+        repeatedly.getContinuationTrigger().getContinuationTrigger());
+  }
+
+  @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";
+        }
+      });
+
+    assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..b258a79
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
@@ -0,0 +1,70 @@
+/*
+ * 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.windowing;
+
+import com.google.common.collect.Lists;
+import java.util.List;
+import org.joda.time.Instant;
+
+/**
+ * No-op {@link OnceTrigger} implementation for testing.
+ */
+abstract class StubTrigger extends Trigger.OnceTrigger {
+  /**
+   * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}.
+   */
+  static StubTrigger named(final String name) {
+    return new StubTrigger() {
+      @Override
+      public String toString() {
+        return name;
+      }
+    };
+  }
+
+  protected StubTrigger() {
+    super(Lists.<Trigger>newArrayList());
+  }
+
+  @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;
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..cfc03b2
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.windowing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link Trigger}.
+ */
+@RunWith(JUnit4.class)
+public class TriggerTest {
+
+  @Test
+  public void testTriggerToString() throws Exception {
+    assertEquals("AfterWatermark.pastEndOfWindow()", AfterWatermark.pastEndOfWindow().toString());
+    assertEquals("Repeatedly.forever(AfterWatermark.pastEndOfWindow())",
+        Repeatedly.forever(AfterWatermark.pastEndOfWindow()).toString());
+  }
+
+  @Test
+  public void testIsCompatible() throws Exception {
+    assertTrue(new Trigger1(null).isCompatible(new Trigger1(null)));
+    assertTrue(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))
+        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
+
+    assertFalse(new Trigger1(null).isCompatible(new Trigger2(null)));
+    assertFalse(new Trigger1(Arrays.<Trigger>asList(new Trigger1(null)))
+        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
+  }
+
+  private static class Trigger1 extends Trigger {
+
+    private Trigger1(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(Trigger.OnElementContext c) { }
+
+    @Override
+    public void onMerge(Trigger.OnMergeContext c) { }
+
+    @Override
+    protected Trigger getContinuationTrigger(
+        List<Trigger> continuationTriggers) {
+      return null;
+    }
+
+    @Override
+    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 {
+
+    private Trigger2(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public void onElement(Trigger.OnElementContext c) { }
+
+    @Override
+    public void onMerge(Trigger.OnMergeContext c) { }
+
+    @Override
+    protected Trigger getContinuationTrigger(
+        List<Trigger> continuationTriggers) {
+      return null;
+    }
+
+    @Override
+    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/a64acb2f/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
new file mode 100644
index 0000000..5fe17ad
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
@@ -0,0 +1,410 @@
+/*
+ * 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();
+    }
+  }
+}


[04/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
new file mode 100644
index 0000000..2490463
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
@@ -0,0 +1,93 @@
+/*
+ * 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.triggers;
+
+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;
+import org.joda.time.Instant;
+
+/**
+ * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in
+ * the real-time domain.
+ *
+ * <p>The time at which to fire the timer can be adjusted via the methods in {@link
+ * AfterDelayFromFirstElementStateMachine}, such as {@link
+ * AfterDelayFromFirstElementStateMachine#plusDelayOf} or {@link
+ * AfterDelayFromFirstElementStateMachine#alignedTo}.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterProcessingTimeStateMachine extends AfterDelayFromFirstElementStateMachine {
+
+  @Override
+  @Nullable
+  public Instant getCurrentTime(TriggerStateMachine.TriggerContext context) {
+    return context.currentProcessingTime();
+  }
+
+  private AfterProcessingTimeStateMachine(List<SerializableFunction<Instant, Instant>> transforms) {
+    super(TimeDomain.PROCESSING_TIME, transforms);
+  }
+
+  /**
+   * Creates a trigger that fires when the current processing time passes the processing time
+   * at which this trigger saw the first element in a pane.
+   */
+  public static AfterProcessingTimeStateMachine pastFirstElementInPane() {
+    return new AfterProcessingTimeStateMachine(IDENTITY);
+  }
+
+  @Override
+  protected AfterProcessingTimeStateMachine newWith(
+      List<SerializableFunction<Instant, Instant>> transforms) {
+    return new AfterProcessingTimeStateMachine(transforms);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()");
+    for (SerializableFunction<Instant, Instant> delayFn : timestampMappers) {
+      builder
+          .append(".plusDelayOf(")
+          .append(delayFn)
+          .append(")");
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterProcessingTimeStateMachine)) {
+      return false;
+    }
+    AfterProcessingTimeStateMachine that = (AfterProcessingTimeStateMachine) obj;
+    return Objects.equals(this.timestampMappers, that.timestampMappers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), this.timestampMappers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
new file mode 100644
index 0000000..000f6e7
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
@@ -0,0 +1,63 @@
+/*
+ * 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.triggers;
+
+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;
+
+class AfterSynchronizedProcessingTimeStateMachine extends AfterDelayFromFirstElementStateMachine {
+
+  @Override
+  @Nullable
+  public Instant getCurrentTime(TriggerStateMachine.TriggerContext context) {
+    return context.currentSynchronizedProcessingTime();
+  }
+
+  public AfterSynchronizedProcessingTimeStateMachine() {
+    super(TimeDomain.SYNCHRONIZED_PROCESSING_TIME,
+        Collections.<SerializableFunction<Instant, Instant>>emptyList());
+  }
+
+  @Override
+  public String toString() {
+    return "AfterSynchronizedProcessingTime.pastFirstElementInPane()";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return this == obj || obj instanceof AfterSynchronizedProcessingTimeStateMachine;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(AfterSynchronizedProcessingTimeStateMachine.class);
+  }
+
+  @Override
+  protected AfterSynchronizedProcessingTimeStateMachine
+      newWith(List<SerializableFunction<Instant, Instant>> transforms) {
+    // ignore transforms
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
new file mode 100644
index 0000000..5ad6214
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
@@ -0,0 +1,325 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableList;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.util.TimeDomain;
+
+/**
+ * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
+ * lower-bound, sometimes heuristically established, on event times that have been fully processed
+ * by the pipeline.
+ *
+ * <p>For sources that provide non-heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
+ * watermark is a strict guarantee that no data with an event time earlier than
+ * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
+ * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
+ * of the window will be the last pane ever for that window.
+ *
+ * <p>For sources that provide heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
+ * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
+ * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
+ * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
+ * Thus, if absolute correctness over time is important to your use case, you may want to consider
+ * using a trigger that accounts for late data. The default trigger,
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires
+ * once when the watermark passes the end of the window and then immediately therafter when any
+ * late data arrives, is one such example.
+ *
+ * <p>The watermark is the clock that defines {@link TimeDomain#EVENT_TIME}.
+ *
+ * <p>Additionaly firings before or after the watermark can be requested by calling
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterWatermarkStateMachine {
+
+  private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
+
+  // Static factory class.
+  private AfterWatermarkStateMachine() {}
+
+  /**
+   * Creates a trigger that fires when the watermark passes the end of the window.
+   */
+  public static FromEndOfWindow pastEndOfWindow() {
+    return new FromEndOfWindow();
+  }
+
+  /**
+   * @see AfterWatermarkStateMachine
+   */
+  public static class AfterWatermarkEarlyAndLate extends TriggerStateMachine {
+
+    private static final int EARLY_INDEX = 0;
+    private static final int LATE_INDEX = 1;
+
+    private final OnceTriggerStateMachine earlyTrigger;
+    @Nullable
+    private final OnceTriggerStateMachine lateTrigger;
+
+    @SuppressWarnings("unchecked")
+    private AfterWatermarkEarlyAndLate(
+        OnceTriggerStateMachine earlyTrigger, OnceTriggerStateMachine lateTrigger) {
+      super(
+          lateTrigger == null
+              ? ImmutableList.<TriggerStateMachine>of(earlyTrigger)
+              : ImmutableList.<TriggerStateMachine>of(earlyTrigger, lateTrigger));
+      this.earlyTrigger = checkNotNull(earlyTrigger, "earlyTrigger should not be null");
+      this.lateTrigger = lateTrigger;
+    }
+
+    public TriggerStateMachine withEarlyFirings(OnceTriggerStateMachine earlyTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    public TriggerStateMachine withLateFirings(OnceTriggerStateMachine lateTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    @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 (ExecutableTriggerStateMachine 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.
+
+      ExecutableTriggerStateMachine 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) {
+          ExecutableTriggerStateMachine 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);
+        }
+      }
+    }
+
+    private boolean endOfWindowReached(TriggerStateMachine.TriggerContext context) {
+      return context.currentEventTime() != null
+          && context.currentEventTime().isAfter(context.window().maxTimestamp());
+    }
+
+    @Override
+    public boolean shouldFire(TriggerStateMachine.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(TriggerStateMachine.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);
+
+      if (!(earlyTrigger instanceof NeverStateMachine)) {
+        builder
+            .append(".withEarlyFirings(")
+            .append(earlyTrigger)
+            .append(")");
+      }
+
+      if (lateTrigger != null && !(lateTrigger instanceof NeverStateMachine)) {
+        builder
+            .append(".withLateFirings(")
+            .append(lateTrigger)
+            .append(")");
+      }
+
+      return builder.toString();
+    }
+
+    private void onNonLateFiring(TriggerStateMachine.TriggerContext context) throws Exception {
+      // We have not yet transitioned to late firings.
+      ExecutableTriggerStateMachine earlySubtrigger = context.trigger().subTrigger(EARLY_INDEX);
+      TriggerStateMachine.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(TriggerStateMachine.TriggerContext context) throws Exception {
+      // We are firing the late trigger, with implicit repeat
+      ExecutableTriggerStateMachine 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);
+    }
+  }
+
+  /**
+   * A watermark trigger targeted relative to the end of the window.
+   */
+  public static class FromEndOfWindow extends OnceTriggerStateMachine {
+
+    private FromEndOfWindow() {
+      super(null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires before the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTriggerStateMachine earlyFirings) {
+      checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
+      return new AfterWatermarkEarlyAndLate(earlyFirings, null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires after the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withLateFirings(OnceTriggerStateMachine lateFirings) {
+      checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
+      return new AfterWatermarkEarlyAndLate(NeverStateMachine.ever(), lateFirings);
+    }
+
+    @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 String toString() {
+      return TO_STRING;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj instanceof FromEndOfWindow;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(getClass());
+    }
+
+    @Override
+    public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+      return endOfWindowReached(context);
+    }
+
+    private boolean endOfWindowReached(TriggerStateMachine.TriggerContext context) {
+      return context.currentEventTime() != null
+          && context.currentEventTime().isAfter(context.window().maxTimestamp());
+    }
+
+    @Override
+    protected void onOnlyFiring(TriggerStateMachine.TriggerContext context) throws Exception { }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
new file mode 100644
index 0000000..be4dd68
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
@@ -0,0 +1,81 @@
+/*
+ * 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.triggers;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.util.TimeDomain;
+
+/**
+ * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. See
+ * {@link RepeatedlyStateMachine#forever} and {@link AfterWatermarkStateMachine#pastEndOfWindow} for
+ * more details.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class DefaultTriggerStateMachine extends TriggerStateMachine {
+
+  private DefaultTriggerStateMachine() {
+    super(null);
+  }
+
+  /**
+   * Returns the default trigger.
+   */
+  public static DefaultTriggerStateMachine of() {
+    return new DefaultTriggerStateMachine();
+  }
+
+  @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 boolean isCompatible(TriggerStateMachine other) {
+    // Semantically, all default triggers are identical
+    return other instanceof DefaultTriggerStateMachine;
+  }
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    return endOfWindowReached(context);
+  }
+
+  private boolean endOfWindowReached(TriggerStateMachine.TriggerContext context) {
+    return context.currentEventTime() != null
+        && context.currentEventTime().isAfter(context.window().maxTimestamp());
+  }
+
+  @Override
+  public void onFire(TriggerStateMachine.TriggerContext context) throws Exception { }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
new file mode 100644
index 0000000..c4d89c2
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
@@ -0,0 +1,160 @@
+/*
+ * 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.triggers;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+/**
+ * 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).
+ */
+public class ExecutableTriggerStateMachine implements Serializable {
+
+  /** Store the index assigned to this trigger. */
+  private final int triggerIndex;
+  private final int firstIndexAfterSubtree;
+  private final List<ExecutableTriggerStateMachine> subTriggers = new ArrayList<>();
+  private final TriggerStateMachine trigger;
+
+  public static <W extends BoundedWindow> ExecutableTriggerStateMachine create(
+      TriggerStateMachine trigger) {
+    return create(trigger, 0);
+  }
+
+  private static <W extends BoundedWindow> ExecutableTriggerStateMachine create(
+      TriggerStateMachine trigger, int nextUnusedIndex) {
+    if (trigger instanceof OnceTriggerStateMachine) {
+      return new ExecutableOnceTriggerStateMachine(
+          (OnceTriggerStateMachine) trigger, nextUnusedIndex);
+    } else {
+      return new ExecutableTriggerStateMachine(trigger, nextUnusedIndex);
+    }
+  }
+
+  public static <W extends BoundedWindow> ExecutableTriggerStateMachine createForOnceTrigger(
+      OnceTriggerStateMachine trigger, int nextUnusedIndex) {
+    return new ExecutableOnceTriggerStateMachine(trigger, nextUnusedIndex);
+  }
+
+  private ExecutableTriggerStateMachine(TriggerStateMachine trigger, int nextUnusedIndex) {
+    this.trigger = checkNotNull(trigger, "trigger must not be null");
+    this.triggerIndex = nextUnusedIndex++;
+
+    if (trigger.subTriggers() != null) {
+      for (TriggerStateMachine subTrigger : trigger.subTriggers()) {
+        ExecutableTriggerStateMachine subExecutable = create(subTrigger, nextUnusedIndex);
+        subTriggers.add(subExecutable);
+        nextUnusedIndex = subExecutable.firstIndexAfterSubtree;
+      }
+    }
+    firstIndexAfterSubtree = nextUnusedIndex;
+  }
+
+  public List<ExecutableTriggerStateMachine> subTriggers() {
+    return subTriggers;
+  }
+
+  @Override
+  public String toString() {
+    return trigger.toString();
+  }
+
+  /**
+   * Return the underlying trigger specification corresponding to this {@code ExecutableTrigger}.
+   */
+  public TriggerStateMachine getSpec() {
+    return trigger;
+  }
+
+  public int getTriggerIndex() {
+    return triggerIndex;
+  }
+
+  public final int getFirstIndexAfterSubtree() {
+    return firstIndexAfterSubtree;
+  }
+
+  public boolean isCompatible(ExecutableTriggerStateMachine other) {
+    return trigger.isCompatible(other.trigger);
+  }
+
+  public ExecutableTriggerStateMachine getSubTriggerContaining(int index) {
+    checkNotNull(subTriggers);
+    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
+        "Cannot find sub-trigger containing index not in this tree.");
+    ExecutableTriggerStateMachine previous = null;
+    for (ExecutableTriggerStateMachine subTrigger : subTriggers) {
+      if (index < subTrigger.triggerIndex) {
+        return previous;
+      }
+      previous = subTrigger;
+    }
+    return previous;
+  }
+
+  /**
+   * Invoke the {@link TriggerStateMachine#onElement} method for this trigger, ensuring that the
+   * bits are properly updated if the trigger finishes.
+   */
+  public void invokeOnElement(TriggerStateMachine.OnElementContext c) throws Exception {
+    trigger.onElement(c.forTrigger(this));
+  }
+
+  /**
+   * Invoke the {@link TriggerStateMachine#onMerge} method for this trigger, ensuring that the bits
+   * are properly updated.
+   */
+  public void invokeOnMerge(TriggerStateMachine.OnMergeContext c) throws Exception {
+    TriggerStateMachine.OnMergeContext subContext = c.forTrigger(this);
+    trigger.onMerge(subContext);
+  }
+
+  public boolean invokeShouldFire(TriggerStateMachine.TriggerContext c) throws Exception {
+    return trigger.shouldFire(c.forTrigger(this));
+  }
+
+  public void invokeOnFire(TriggerStateMachine.TriggerContext c) throws Exception {
+    trigger.onFire(c.forTrigger(this));
+  }
+
+  /**
+   * Invoke clear for the current this trigger.
+   */
+  public void invokeClear(TriggerStateMachine.TriggerContext c) throws Exception {
+    trigger.clear(c.forTrigger(this));
+  }
+
+  /**
+   * {@link ExecutableTriggerStateMachine} that enforces the fact that the trigger should always
+   * FIRE_AND_FINISH and never just FIRE.
+   */
+  private static class ExecutableOnceTriggerStateMachine extends ExecutableTriggerStateMachine {
+
+    public ExecutableOnceTriggerStateMachine(OnceTriggerStateMachine trigger, int nextUnusedIndex) {
+      super(trigger, nextUnusedIndex);
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSet.java
new file mode 100644
index 0000000..0f74969
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersBitSet.java
@@ -0,0 +1,67 @@
+/*
+ * 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.triggers;
+
+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(ExecutableTriggerStateMachine trigger) {
+    return bitSet.get(trigger.getTriggerIndex());
+  }
+
+  @Override
+  public void setFinished(ExecutableTriggerStateMachine trigger, boolean value) {
+    bitSet.set(trigger.getTriggerIndex(), value);
+  }
+
+  @Override
+  public void clearRecursively(ExecutableTriggerStateMachine 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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersSet.java
new file mode 100644
index 0000000..95c35f2
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/FinishedTriggersSet.java
@@ -0,0 +1,72 @@
+/*
+ * 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.triggers;
+
+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<ExecutableTriggerStateMachine> finishedTriggers;
+
+  private FinishedTriggersSet(Set<ExecutableTriggerStateMachine> finishedTriggers) {
+    this.finishedTriggers = finishedTriggers;
+  }
+
+  public static FinishedTriggersSet fromSet(Set<ExecutableTriggerStateMachine> finishedTriggers) {
+    return new FinishedTriggersSet(finishedTriggers);
+  }
+
+  /**
+   * Returns a mutable {@link Set} of the underlying triggers that are finished.
+   */
+  public Set<ExecutableTriggerStateMachine> getFinishedTriggers() {
+    return finishedTriggers;
+  }
+
+  @Override
+  public boolean isFinished(ExecutableTriggerStateMachine trigger) {
+    return finishedTriggers.contains(trigger);
+  }
+
+  @Override
+  public void setFinished(ExecutableTriggerStateMachine trigger, boolean value) {
+    if (value) {
+      finishedTriggers.add(trigger);
+    } else {
+      finishedTriggers.remove(trigger);
+    }
+  }
+
+  @Override
+  public void clearRecursively(ExecutableTriggerStateMachine trigger) {
+    finishedTriggers.remove(trigger);
+    for (ExecutableTriggerStateMachine 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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/NeverStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/NeverStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/NeverStateMachine.java
new file mode 100644
index 0000000..f32c7a8
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/NeverStateMachine.java
@@ -0,0 +1,60 @@
+/*
+ * 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.triggers;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+/**
+ * A {@link TriggerStateMachine} which never fires.
+ *
+ * <p>Using this trigger will only produce output when the watermark passes the end of the
+ * {@link BoundedWindow window} plus the allowed lateness.
+ */
+public final class NeverStateMachine extends OnceTriggerStateMachine {
+  /**
+   * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
+   * when the {@link BoundedWindow} closes.
+   */
+  public static NeverStateMachine ever() {
+    // NeverTrigger ignores all inputs and is Window-type independent.
+    return new NeverStateMachine();
+  }
+
+  private NeverStateMachine() {
+    super(null);
+  }
+
+  @Override
+  public void onElement(OnElementContext c) {}
+
+  @Override
+  public void onMerge(OnMergeContext c) {}
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) {
+    return false;
+  }
+
+  @Override
+  protected void onOnlyFiring(TriggerStateMachine.TriggerContext context) {
+    throw new UnsupportedOperationException(
+        String.format("%s should never fire", getClass().getSimpleName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
new file mode 100644
index 0000000..f9ec5e7
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
@@ -0,0 +1,85 @@
+/*
+ * 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.triggers;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.Arrays;
+
+/**
+ * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
+ */
+class OrFinallyStateMachine extends TriggerStateMachine {
+
+  private static final int ACTUAL = 0;
+  private static final int UNTIL = 1;
+
+  @VisibleForTesting
+  OrFinallyStateMachine(TriggerStateMachine actual, OnceTriggerStateMachine until) {
+    super(Arrays.asList(actual, until));
+  }
+
+  @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 (ExecutableTriggerStateMachine subTrigger : c.trigger().subTriggers()) {
+      subTrigger.invokeOnMerge(c);
+    }
+    updateFinishedState(c);
+  }
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    return context.trigger().subTrigger(ACTUAL).invokeShouldFire(context)
+        || context.trigger().subTrigger(UNTIL).invokeShouldFire(context);
+  }
+
+  @Override
+  public void onFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    ExecutableTriggerStateMachine actualSubtrigger = context.trigger().subTrigger(ACTUAL);
+    ExecutableTriggerStateMachine 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 (ExecutableTriggerStateMachine 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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
new file mode 100644
index 0000000..35b701c
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
@@ -0,0 +1,88 @@
+/*
+ * 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.triggers;
+
+import java.util.Arrays;
+
+/**
+ * Repeat a trigger, either until some condition is met or forever.
+ *
+ * <p>For example, to fire after the end of the window, and every time late data arrives:
+ * <pre> {@code
+ *     Repeatedly.forever(AfterWatermark.isPastEndOfWindow());
+ * } </pre>
+ *
+ * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite
+ * {@code AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
+ */
+public class RepeatedlyStateMachine extends TriggerStateMachine {
+
+  private static final int REPEATED = 0;
+
+  /**
+   * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each
+   * time it fires and ignoring any indications to finish.
+   *
+   * <p>Unless used with {@link TriggerStateMachine#orFinally} the composite trigger will never
+   * finish.
+   *
+   * @param repeated the trigger to execute repeatedly.
+   */
+  public static RepeatedlyStateMachine forever(TriggerStateMachine repeated) {
+    return new RepeatedlyStateMachine(repeated);
+  }
+
+  private RepeatedlyStateMachine(TriggerStateMachine repeated) {
+    super(Arrays.asList(repeated));
+  }
+
+
+  @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 boolean shouldFire(TriggerStateMachine.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 ExecutableTriggerStateMachine getRepeated(TriggerContext context) {
+    return context.trigger().subTrigger(REPEATED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
new file mode 100644
index 0000000..fc9f203
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
@@ -0,0 +1,50 @@
+/*
+ * 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.triggers;
+
+import org.apache.beam.sdk.util.Reshuffle;
+
+/**
+ * The trigger used with {@link Reshuffle} which triggers on every element
+ * and never buffers state.
+ */
+public class ReshuffleTriggerStateMachine extends TriggerStateMachine {
+
+  public ReshuffleTriggerStateMachine() {
+    super(null);
+  }
+
+  @Override
+  public void onElement(TriggerStateMachine.OnElementContext c) { }
+
+  @Override
+  public void onMerge(TriggerStateMachine.OnMergeContext c) { }
+
+  @Override
+  public boolean shouldFire(TriggerStateMachine.TriggerContext context) throws Exception {
+    return true;
+  }
+
+  @Override
+  public void onFire(TriggerStateMachine.TriggerContext context) throws Exception { }
+
+  @Override
+  public String toString() {
+    return "ReshuffleTriggerStateMachine()";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
new file mode 100644
index 0000000..f8def20
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
@@ -0,0 +1,487 @@
+/*
+ * 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.triggers;
+
+import com.google.common.base.Joiner;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+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.TimeDomain;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+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.
+ *
+ * <p>See {@link org.apache.beam.sdk.transforms.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
+ * elements as they are added. When the root trigger fires, the elements in the current pane will be
+ * 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:
+ *
+ * <ul>
+ * <li> {@link AfterWatermarkStateMachine} 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 AfterProcessingTimeStateMachine} for firing after some amount of processing time has
+ *     elapsed (typically since the first element in a pane).
+ * <li> {@link AfterPaneStateMachine} 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 RepeatedlyStateMachine#forever} to create a trigger that executes forever. Any time
+ *     its argument finishes it gets reset and starts over. Can be combined with {@link
+ *     TriggerStateMachine#orFinally} to specify a condition that causes the repetition to stop.
+ * <li> {@link AfterEachStateMachine#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 AfterFirstStateMachine#of} to create a trigger that fires after at least one of its
+ *     arguments fires. An {@link AfterFirstStateMachine} trigger finishes after it fires once.
+ * <li> {@link AfterAllStateMachine#of} to create a trigger that fires after all least one of its
+ *     arguments have fired at least once. An {@link AfterAllStateMachine} 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.
+ * </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.
+ */
+public abstract class TriggerStateMachine 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 {@link
+     * WindowFn}. If true, the trigger execution needs to keep enough information to support the
+     * possibility of {@link TriggerStateMachine#onMerge} being called. If false, {@link
+     * TriggerStateMachine#onMerge} will never be called.
+     */
+    boolean isMerging();
+
+    /**
+     * Access the executable versions of the sub-triggers of the current trigger.
+     */
+    Iterable<ExecutableTriggerStateMachine> subTriggers();
+
+    /**
+     * Access the executable version of the specified sub-trigger.
+     */
+    ExecutableTriggerStateMachine 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<ExecutableTriggerStateMachine> unfinishedSubTriggers();
+
+    /**
+     * Returns the first unfinished sub-trigger.
+     */
+    ExecutableTriggerStateMachine 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. */
+    public abstract boolean finishedInAnyMergingWindow();
+
+    /** Return true if the trigger is finished in all windows being merged. */
+    public abstract boolean finishedInAllMergingWindows();
+  }
+
+  /**
+   * Information accessible to all operational hooks in this {@code Trigger}.
+   *
+   * <p>Used directly in {@link TriggerStateMachine#shouldFire} and {@link
+   * TriggerStateMachine#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(ExecutableTriggerStateMachine 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(ExecutableTriggerStateMachine 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(ExecutableTriggerStateMachine trigger);
+
+    @Override
+    public abstract MergingStateAccessor<?, ?> state();
+
+    @Override
+    public abstract MergingTriggerInfo trigger();
+  }
+
+  @Nullable
+  protected final List<TriggerStateMachine> subTriggers;
+
+  protected TriggerStateMachine(@Nullable List<TriggerStateMachine> subTriggers) {
+    this.subTriggers = subTriggers;
+  }
+
+
+  /**
+   * 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
+   * ExecutableTriggerStateMachine#invokeOnMerge} on their sub-triggers, and applying appropriate
+   * logic.
+   *
+   * <p>A trigger such as {@link AfterWatermarkStateMachine#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 RepeatedlyStateMachine} 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 (TriggerStateMachine 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 (TriggerStateMachine 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 (TriggerStateMachine 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 (TriggerStateMachine 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 (ExecutableTriggerStateMachine trigger : c.trigger().subTriggers()) {
+        trigger.invokeClear(c);
+      }
+    }
+  }
+
+  public Iterable<TriggerStateMachine> subTriggers() {
+    return subTriggers;
+  }
+
+  /**
+   * Returns whether this performs the same triggering as the given {@code Trigger}.
+   */
+  public boolean isCompatible(TriggerStateMachine other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    if (subTriggers == null) {
+      return other.subTriggers == null;
+    } else if (other.subTriggers == null) {
+      return false;
+    } else if (subTriggers.size() != other.subTriggers.size()) {
+      return false;
+    }
+
+    for (int i = 0; i < subTriggers.size(); i++) {
+      if (!subTriggers.get(i).isCompatible(other.subTriggers.get(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    String simpleName = getClass().getSimpleName();
+    if (getClass().getEnclosingClass() != null) {
+      simpleName = getClass().getEnclosingClass().getSimpleName() + "." + simpleName;
+    }
+    if (subTriggers == null || subTriggers.size() == 0) {
+      return simpleName;
+    } else {
+      return simpleName + "(" + Joiner.on(", ").join(subTriggers) + ")";
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof TriggerStateMachine)) {
+      return false;
+    }
+    TriggerStateMachine that = (TriggerStateMachine) obj;
+    return Objects.equals(getClass(), that.getClass())
+        && Objects.equals(subTriggers, that.subTriggers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), subTriggers);
+  }
+
+  /**
+   * Specify an ending condition for this trigger. If the {@code until} 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>For example the final firing of the following trigger may only have 1 element:
+   *
+   * <pre>{@code
+   * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
+   *     .orFinally(AfterPane.elementCountAtLeast(5))
+   * }
+   * </pre>
+   *
+   * <p>Note that if {@code t1} is {@link OnceTriggerStateMachine}, then {@code t1.orFinally(t2)} is
+   * the same as {@code AfterFirst.of(t1, t2)}.
+   */
+  public TriggerStateMachine orFinally(OnceTriggerStateMachine until) {
+    return new OrFinallyStateMachine(this, until);
+  }
+
+  /**
+   * {@link TriggerStateMachine}s that are guaranteed to fire at most once should extend from this,
+   * rather than the general {@link TriggerStateMachine} class to indicate that behavior.
+   */
+  public abstract static class OnceTriggerStateMachine extends TriggerStateMachine {
+    protected OnceTriggerStateMachine(List<TriggerStateMachine> subTriggers) {
+      super(subTriggers);
+    }
+
+    /**
+     * {@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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
new file mode 100644
index 0000000..1c06e8d
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
@@ -0,0 +1,509 @@
+/*
+ * 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.triggers;
+
+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.runners.core.triggers.TriggerStateMachine.MergingTriggerInfo;
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.TriggerInfo;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.ActiveWindowSet;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timers;
+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 TriggerStateMachine} 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 TriggerStateMachineContextFactory<W extends BoundedWindow> {
+
+  private final WindowFn<?, W> windowFn;
+  private StateInternals<?> stateInternals;
+  private final Coder<W> windowCoder;
+
+  public TriggerStateMachineContextFactory(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 TriggerStateMachine.TriggerContext base(W window, Timers timers,
+      ExecutableTriggerStateMachine rootTrigger, FinishedTriggers finishedSet) {
+    return new TriggerContextImpl(window, timers, rootTrigger, finishedSet);
+  }
+
+  public TriggerStateMachine.OnElementContext createOnElementContext(
+      W window, Timers timers, Instant elementTimestamp,
+      ExecutableTriggerStateMachine rootTrigger, FinishedTriggers finishedSet) {
+    return new OnElementContextImpl(window, timers, rootTrigger, finishedSet, elementTimestamp);
+  }
+
+  public TriggerStateMachine.OnMergeContext createOnMergeContext(W window, Timers timers,
+      ExecutableTriggerStateMachine rootTrigger, FinishedTriggers finishedSet,
+      Map<W, FinishedTriggers> finishedSets) {
+    return new OnMergeContextImpl(window, timers, rootTrigger, finishedSet, finishedSets);
+  }
+
+  public StateAccessor<?> createStateAccessor(W window, ExecutableTriggerStateMachine trigger) {
+    return new StateAccessorImpl(window, trigger);
+  }
+
+  public MergingStateAccessor<?, W> createMergingStateAccessor(
+      W mergeResult, Collection<W> mergingWindows, ExecutableTriggerStateMachine trigger) {
+    return new MergingStateAccessorImpl(trigger, mergingWindows, mergeResult);
+  }
+
+  private class TriggerInfoImpl implements TriggerStateMachine.TriggerInfo {
+
+    protected final ExecutableTriggerStateMachine trigger;
+    protected final FinishedTriggers finishedSet;
+    private final TriggerStateMachine.TriggerContext context;
+
+    public TriggerInfoImpl(ExecutableTriggerStateMachine trigger, FinishedTriggers finishedSet,
+        TriggerStateMachine.TriggerContext context) {
+      this.trigger = trigger;
+      this.finishedSet = finishedSet;
+      this.context = context;
+    }
+
+    @Override
+    public boolean isMerging() {
+      return !windowFn.isNonMerging();
+    }
+
+    @Override
+    public Iterable<ExecutableTriggerStateMachine> subTriggers() {
+      return trigger.subTriggers();
+    }
+
+    @Override
+    public ExecutableTriggerStateMachine 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<ExecutableTriggerStateMachine> unfinishedSubTriggers() {
+      return FluentIterable
+          .from(trigger.subTriggers())
+          .filter(new Predicate<ExecutableTriggerStateMachine>() {
+            @Override
+            public boolean apply(ExecutableTriggerStateMachine trigger) {
+              return !finishedSet.isFinished(trigger);
+            }
+          });
+    }
+
+    @Override
+    public ExecutableTriggerStateMachine firstUnfinishedSubTrigger() {
+      for (ExecutableTriggerStateMachine 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 TriggerStateMachine.MergingTriggerInfo {
+
+    private final Map<W, FinishedTriggers> finishedSets;
+
+    public MergingTriggerInfoImpl(
+        ExecutableTriggerStateMachine trigger,
+        FinishedTriggers finishedSet,
+        TriggerStateMachine.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,
+        ExecutableTriggerStateMachine 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(
+        ExecutableTriggerStateMachine 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 TriggerStateMachine.TriggerContext {
+
+    private final W window;
+    private final StateAccessorImpl state;
+    private final Timers timers;
+    private final TriggerInfoImpl triggerInfo;
+
+    private TriggerContextImpl(
+        W window,
+        Timers timers,
+        ExecutableTriggerStateMachine 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 TriggerStateMachine.TriggerContext forTrigger(ExecutableTriggerStateMachine 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 TriggerStateMachine.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,
+        ExecutableTriggerStateMachine 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 TriggerStateMachine.OnElementContext forTrigger(ExecutableTriggerStateMachine 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 TriggerStateMachine.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,
+        ExecutableTriggerStateMachine 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 TriggerStateMachine.OnMergeContext forTrigger(ExecutableTriggerStateMachine 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();
+    }
+  }
+}



[06/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Posted by ke...@apache.org.
Rename runners-core Trigger to TriggerStateMachine

This is a step in separating the syntax (in the SDK), semantics
(an abstract description), and implementation (runners-core state
machines that react to elements and timers) of triggers.


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

Branch: refs/heads/master
Commit: 69b1efda02c66368ebb7447389d995e5c6975d43
Parents: e4398e1
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 20:31:10 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 13 14:34:34 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/reactors/AfterAll.java    | 122 -----
 .../reactors/AfterDelayFromFirstElement.java    | 335 ------------
 .../beam/runners/core/reactors/AfterEach.java   | 141 -----
 .../beam/runners/core/reactors/AfterFirst.java  | 124 -----
 .../beam/runners/core/reactors/AfterPane.java   | 144 -----
 .../core/reactors/AfterProcessingTime.java      | 102 ----
 .../AfterSynchronizedProcessingTime.java        |  73 ---
 .../runners/core/reactors/AfterWatermark.java   | 355 -------------
 .../runners/core/reactors/DefaultTrigger.java   |  92 ----
 .../core/reactors/ExecutableTrigger.java        | 159 ------
 .../runners/core/reactors/FinishedTriggers.java |  44 --
 .../core/reactors/FinishedTriggersBitSet.java   |  67 ---
 .../core/reactors/FinishedTriggersSet.java      |  72 ---
 .../beam/runners/core/reactors/Never.java       |  75 ---
 .../runners/core/reactors/OrFinallyTrigger.java | 105 ----
 .../beam/runners/core/reactors/Repeatedly.java  | 101 ----
 .../runners/core/reactors/ReshuffleTrigger.java |  66 ---
 .../beam/runners/core/reactors/Trigger.java     | 527 -------------------
 .../core/reactors/TriggerContextFactory.java    | 507 ------------------
 .../runners/core/reactors/TriggerRunner.java    | 247 ---------
 .../core/triggers/AfterAllStateMachine.java     | 104 ++++
 .../AfterDelayFromFirstElementStateMachine.java | 322 +++++++++++
 .../core/triggers/AfterEachStateMachine.java    | 125 +++++
 .../core/triggers/AfterFirstStateMachine.java   | 106 ++++
 .../core/triggers/AfterPaneStateMachine.java    | 132 +++++
 .../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 ++++++++
 .../runners/core/triggers/package-info.java     |  23 +
 .../runners/core/reactors/AfterAllTest.java     | 156 ------
 .../runners/core/reactors/AfterEachTest.java    | 132 -----
 .../runners/core/reactors/AfterFirstTest.java   | 181 -------
 .../runners/core/reactors/AfterPaneTest.java    | 132 -----
 .../core/reactors/AfterProcessingTimeTest.java  | 187 -------
 .../AfterSynchronizedProcessingTimeTest.java    | 121 -----
 .../core/reactors/AfterWatermarkTest.java       | 380 -------------
 .../core/reactors/DefaultTriggerTest.java       | 176 -------
 .../core/reactors/ExecutableTriggerTest.java    | 127 -----
 .../reactors/FinishedTriggersBitSetTest.java    |  55 --
 .../reactors/FinishedTriggersProperties.java    | 110 ----
 .../core/reactors/FinishedTriggersSetTest.java  |  60 ---
 .../beam/runners/core/reactors/NeverTest.java   |  56 --
 .../core/reactors/OrFinallyTriggerTest.java     | 215 --------
 .../runners/core/reactors/RepeatedlyTest.java   | 224 --------
 .../core/reactors/ReshuffleTriggerTest.java     |  67 ---
 .../beam/runners/core/reactors/StubTrigger.java |  70 ---
 .../beam/runners/core/reactors/TriggerTest.java | 118 -----
 .../runners/core/reactors/TriggerTester.java    | 410 ---------------
 .../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 +++++++++++++++
 79 files changed, 6014 insertions(+), 6435 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
deleted file mode 100644
index cc8c97f..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
+++ /dev/null
@@ -1,122 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterAll extends OnceTrigger {
-
-  private AfterAll(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
-   */
-  public static OnceTrigger of(OnceTrigger... triggers) {
-    return new AfterAll(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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;
-    for (Trigger subTrigger : subTriggers) {
-      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
-      if (deadline.isBefore(subDeadline)) {
-        deadline = subDeadline;
-      }
-    }
-    return deadline;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    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(");
-    Joiner.on(", ").appendTo(builder, subTriggers);
-    builder.append(")");
-
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
deleted file mode 100644
index c4bc946..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
+++ /dev/null
@@ -1,335 +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.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;
-import org.apache.beam.sdk.transforms.Min;
-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;
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-import org.joda.time.format.PeriodFormatter;
-
-/**
- * A base class for triggers that happen after a processing time delay from the arrival
- * of the first element in a pane.
- *
- * <p>This class is for internal use only and may change at any time.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public abstract class AfterDelayFromFirstElement extends OnceTrigger {
-
-  protected static final List<SerializableFunction<Instant, Instant>> IDENTITY =
-      ImmutableList.<SerializableFunction<Instant, Instant>>of();
-
-  protected static final StateTag<Object, AccumulatorCombiningState<Instant,
-                                              Combine.Holder<Instant>, Instant>> DELAYED_UNTIL_TAG =
-      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
-          "delayed", InstantCoder.of(), Min.MinFn.<Instant>naturalOrder()));
-
-  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.
-   */
-  protected abstract AfterDelayFromFirstElement newWith(
-      List<SerializableFunction<Instant, Instant>> transform);
-
-  /**
-   * A list of timestampMappers m1, m2, m3, ... m_n considered to be composed in sequence. The
-   * overall mapping for an instance `instance` is `m_n(... m3(m2(m1(instant))`,
-   * implemented via #computeTargetTimestamp
-   */
-  protected final List<SerializableFunction<Instant, Instant>> timestampMappers;
-
-  private final TimeDomain timeDomain;
-
-  public AfterDelayFromFirstElement(
-      TimeDomain timeDomain,
-      List<SerializableFunction<Instant, Instant>> timestampMappers) {
-    super(null);
-    this.timestampMappers = timestampMappers;
-    this.timeDomain = timeDomain;
-  }
-
-  private Instant getTargetTimestamp(OnElementContext c) {
-    return computeTargetTimestamp(c.currentProcessingTime());
-  }
-
-  /**
-   * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
-   * than the timestamp.
-   *
-   * <p>TODO: Consider sharing this with FixedWindows, and bring over the equivalent of
-   * CalendarWindows.
-   */
-  public AfterDelayFromFirstElement alignedTo(final Duration size, final Instant offset) {
-    return newWith(new AlignFn(size, offset));
-  }
-
-  /**
-   * Aligns the time to be the smallest multiple of {@code size} greater than the timestamp
-   * since the epoch.
-   */
-  public AfterDelayFromFirstElement alignedTo(final Duration size) {
-    return alignedTo(size, new Instant(0));
-  }
-
-  /**
-   * Adds some delay to the original target time.
-   *
-   * @param delay the delay to add
-   * @return An updated time trigger that will wait the additional time before firing.
-   */
-  public AfterDelayFromFirstElement plusDelayOf(final Duration delay) {
-    return newWith(new DelayFn(delay));
-  }
-
-  /**
-   * @deprecated This will be removed in the next major version. Please use only
-   *             {@link #plusDelayOf} and {@link #alignedTo}.
-   */
-  @Deprecated
-  public OnceTrigger mappedTo(SerializableFunction<Instant, Instant> timestampMapper) {
-    return newWith(timestampMapper);
-  }
-
-  @Override
-  public boolean isCompatible(Trigger other) {
-    if (!getClass().equals(other.getClass())) {
-      return false;
-    }
-
-    AfterDelayFromFirstElement that = (AfterDelayFromFirstElement) other;
-    return this.timestampMappers.equals(that.timestampMappers);
-  }
-
-
-  private AfterDelayFromFirstElement newWith(
-      SerializableFunction<Instant, Instant> timestampMapper) {
-    return newWith(
-        ImmutableList.<SerializableFunction<Instant, Instant>>builder()
-            .addAll(timestampMappers)
-            .add(timestampMapper)
-            .build());
-  }
-
-  @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) {
-      result = timestampMapper.apply(result);
-    }
-    return result;
-  }
-
-  /**
-   * A {@link SerializableFunction} to delay the timestamp at which this triggers fires.
-   */
-  private static final class DelayFn implements SerializableFunction<Instant, Instant> {
-    private final Duration delay;
-
-    public DelayFn(Duration delay) {
-      this.delay = delay;
-    }
-
-    @Override
-    public Instant apply(Instant input) {
-      return input.plus(delay);
-    }
-
-    @Override
-    public boolean equals(Object object) {
-      if (object == this) {
-        return true;
-      }
-
-      if (!(object instanceof DelayFn)) {
-        return false;
-      }
-
-      return this.delay.equals(((DelayFn) object).delay);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(delay);
-    }
-
-    @Override
-    public String toString() {
-      return PERIOD_FORMATTER.print(delay.toPeriod());
-    }
-  }
-
-  /**
-   * A {@link SerializableFunction} to align an instant to the nearest interval boundary.
-   */
-  static final class AlignFn implements SerializableFunction<Instant, Instant> {
-    private final Duration size;
-    private final Instant offset;
-
-
-    /**
-     * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
-     * than the timestamp.
-     */
-    public AlignFn(Duration size, Instant offset) {
-      this.size = size;
-      this.offset = offset;
-    }
-
-    @Override
-    public Instant apply(Instant point) {
-      long millisSinceStart = new Duration(offset, point).getMillis() % size.getMillis();
-      return millisSinceStart == 0 ? point : point.plus(size).minus(millisSinceStart);
-    }
-
-    @Override
-    public boolean equals(Object object) {
-      if (object == this) {
-        return true;
-      }
-
-      if (!(object instanceof AlignFn)) {
-        return false;
-      }
-
-      AlignFn other = (AlignFn) object;
-      return other.size.equals(this.size)
-          && other.offset.equals(this.offset);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(size, offset);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
deleted file mode 100644
index 629c640..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
+++ /dev/null
@@ -1,141 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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;
-
-/**
- * A composite {@link Trigger} that executes its sub-triggers in order.
- * Only one sub-trigger is executing at a time,
- * and any time it fires the {@code AfterEach} fires. When the currently executing
- * sub-trigger finishes, the {@code AfterEach} starts executing the next sub-trigger.
- *
- * <p>{@code AfterEach.inOrder(t1, t2, ...)} finishes when all of the sub-triggers have finished.
- *
- * <p>The following properties hold:
- * <ul>
- *   <li> {@code AfterEach.inOrder(AfterEach.inOrder(a, b), c)} behaves the same as
- *   {@code AfterEach.inOrder(a, b, c)} and {@code AfterEach.inOrder(a, AfterEach.inOrder(b, c)}.
- *   <li> {@code AfterEach.inOrder(Repeatedly.forever(a), b)} behaves the same as
- *   {@code Repeatedly.forever(a)}, since the repeated trigger never finishes.
- * </ul>
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterEach extends Trigger {
-
-  private AfterEach(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterEach} {@code Trigger} with the given subtriggers.
-   */
-  @SafeVarargs
-  public static Trigger inOrder(Trigger... triggers) {
-    return new AfterEach(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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.
-    return subTriggers.get(0).getWatermarkThatGuaranteesFiring(window);
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return Repeatedly.forever(new AfterFirst(continuationTriggers));
-  }
-
-  @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);
-    builder.append(")");
-
-    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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
deleted file mode 100644
index 6b06cfa..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
+++ /dev/null
@@ -1,124 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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.transforms.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.joda.time.Instant;
-
-/**
- * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have
- * fired.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterFirst extends OnceTrigger {
-
-  AfterFirst(List<Trigger> subTriggers) {
-    super(subTriggers);
-    checkArgument(subTriggers.size() > 1);
-  }
-
-  /**
-   * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
-   */
-  public static OnceTrigger of(OnceTrigger... triggers) {
-    return new AfterFirst(Arrays.<Trigger>asList(triggers));
-  }
-
-  @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;
-    for (Trigger subTrigger : subTriggers) {
-      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
-      if (deadline.isAfter(subDeadline)) {
-        deadline = subDeadline;
-      }
-    }
-    return deadline;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new AfterFirst(continuationTriggers);
-  }
-
-  @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);
-    builder.append(")");
-
-    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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
deleted file mode 100644
index 8c128dd..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
+++ /dev/null
@@ -1,144 +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.windowing;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.List;
-import java.util.Objects;
-import org.apache.beam.sdk.annotations.Experimental;
-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;
-
-/**
- * {@link Trigger}s that fire based on properties of the elements in the current pane.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterPane extends OnceTrigger {
-
-private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
-      ELEMENTS_IN_PANE_TAG =
-      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
-          "count", VarLongCoder.of(), new Sum.SumLongFn()));
-
-  private final int countElems;
-
-  private AfterPane(int countElems) {
-    super(null);
-    this.countElems = countElems;
-  }
-
-  /**
-   * Creates a trigger that fires when the pane contains at least {@code countElems} elements.
-   */
-  public static AfterPane elementCountAtLeast(int countElems) {
-    return new AfterPane(countElems);
-  }
-
-  @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);
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return AfterPane.elementCountAtLeast(1);
-  }
-
-  @Override
-  public String toString() {
-    return "AfterPane.elementCountAtLeast(" + countElems + ")";
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof AfterPane)) {
-      return false;
-    }
-    AfterPane that = (AfterPane) obj;
-    return this.countElems == that.countElems;
-  }
-
-  @Override
-  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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
deleted file mode 100644
index f551118..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
+++ /dev/null
@@ -1,102 +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.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;
-import org.joda.time.Instant;
-
-/**
- * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in
- * the real-time domain.
- *
- * <p>The time at which to fire the timer can be adjusted via the methods in
- * {@link AfterDelayFromFirstElement}, such as {@link AfterDelayFromFirstElement#plusDelayOf} or
- * {@link AfterDelayFromFirstElement#alignedTo}.
- */
-@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);
-  }
-
-  /**
-   * Creates a trigger that fires when the current processing time passes the processing time
-   * at which this trigger saw the first element in a pane.
-   */
-  public static AfterProcessingTime pastFirstElementInPane() {
-    return new AfterProcessingTime(IDENTITY);
-  }
-
-  @Override
-  protected AfterProcessingTime newWith(
-      List<SerializableFunction<Instant, Instant>> transforms) {
-    return new AfterProcessingTime(transforms);
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return new AfterSynchronizedProcessingTime();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()");
-    for (SerializableFunction<Instant, Instant> delayFn : timestampMappers) {
-      builder
-          .append(".plusDelayOf(")
-          .append(delayFn)
-          .append(")");
-    }
-
-    return builder.toString();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof AfterProcessingTime)) {
-      return false;
-    }
-    AfterProcessingTime that = (AfterProcessingTime) obj;
-    return Objects.equals(this.timestampMappers, that.timestampMappers);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(getClass(), this.timestampMappers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
deleted file mode 100644
index 59ece10..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
+++ /dev/null
@@ -1,73 +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.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;
-
-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());
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return this;
-  }
-
-  @Override
-  public String toString() {
-    return "AfterSynchronizedProcessingTime.pastFirstElementInPane()";
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return this == obj || obj instanceof AfterSynchronizedProcessingTime;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(AfterSynchronizedProcessingTime.class);
-  }
-
-  @Override
-  protected AfterSynchronizedProcessingTime
-      newWith(List<SerializableFunction<Instant, Instant>> transforms) {
-    // ignore transforms
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
deleted file mode 100644
index e2463d8..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
+++ /dev/null
@@ -1,355 +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.windowing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.ImmutableList;
-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.windowing.Trigger.OnceTrigger;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.joda.time.Instant;
-
-/**
- * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
- * lower-bound, sometimes heuristically established, on event times that have been fully processed
- * by the pipeline.
- *
- * <p>For sources that provide non-heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
- * watermark is a strict guarantee that no data with an event time earlier than
- * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
- * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
- * of the window will be the last pane ever for that window.
- *
- * <p>For sources that provide heuristic watermarks (e.g.
- * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
- * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
- * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
- * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
- * Thus, if absolute correctness over time is important to your use case, you may want to consider
- * using a trigger that accounts for late data. The default trigger,
- * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires
- * once when the watermark passes the end of the window and then immediately therafter when any
- * late data arrives, is one such example.
- *
- * <p>The watermark is the clock that defines {@link TimeDomain#EVENT_TIME}.
- *
- * <p>Additionaly firings before or after the watermark can be requested by calling
- * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or
- * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class AfterWatermark {
-
-  private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
-
-  // Static factory class.
-  private AfterWatermark() {}
-
-  /**
-   * Creates a trigger that fires when the watermark passes the end of the window.
-   */
-  public static FromEndOfWindow pastEndOfWindow() {
-    return new FromEndOfWindow();
-  }
-
-  /**
-   * @see AfterWatermark
-   */
-  public static class AfterWatermarkEarlyAndLate extends Trigger {
-
-    private static final int EARLY_INDEX = 0;
-    private static final int LATE_INDEX = 1;
-
-    private final OnceTrigger earlyTrigger;
-    @Nullable
-    private final OnceTrigger lateTrigger;
-
-    @SuppressWarnings("unchecked")
-    public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
-      super(lateTrigger == null
-          ? ImmutableList.<Trigger>of(earlyTrigger)
-          : ImmutableList.<Trigger>of(earlyTrigger, lateTrigger));
-      this.earlyTrigger = checkNotNull(earlyTrigger, "earlyTrigger should not be null");
-      this.lateTrigger = lateTrigger;
-    }
-
-    public Trigger withEarlyFirings(OnceTrigger earlyTrigger) {
-      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
-    }
-
-    public Trigger withLateFirings(OnceTrigger lateTrigger) {
-      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
-    }
-
-    @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(),
-          lateTrigger == null ? null : lateTrigger.getContinuationTrigger());
-    }
-
-    @Override
-    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-      throw new UnsupportedOperationException(
-          "Should not call getContinuationTrigger(List<Trigger>)");
-    }
-
-    @Override
-    public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-      // Even without an early or late trigger, we'll still produce a firing at the watermark.
-      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);
-
-      if (!(earlyTrigger instanceof Never.NeverTrigger)) {
-        builder
-            .append(".withEarlyFirings(")
-            .append(earlyTrigger)
-            .append(")");
-      }
-
-      if (lateTrigger != null && !(lateTrigger instanceof Never.NeverTrigger)) {
-        builder
-            .append(".withLateFirings(")
-            .append(lateTrigger)
-            .append(")");
-      }
-
-      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);
-    }
-  }
-
-  /**
-   * A watermark trigger targeted relative to the end of the window.
-   */
-  public static class FromEndOfWindow extends OnceTrigger {
-
-    private FromEndOfWindow() {
-      super(null);
-    }
-
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires before the watermark has passed the end of the window.
-     */
-    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTrigger earlyFirings) {
-      checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
-      return new AfterWatermarkEarlyAndLate(earlyFirings, null);
-    }
-
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires after the watermark has passed the end of the window.
-     */
-    public AfterWatermarkEarlyAndLate withLateFirings(OnceTrigger lateFirings) {
-      checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
-      return new AfterWatermarkEarlyAndLate(Never.ever(), lateFirings);
-    }
-
-    @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();
-    }
-
-    @Override
-    public FromEndOfWindow getContinuationTrigger(List<Trigger> continuationTriggers) {
-      return this;
-    }
-
-    @Override
-    public String toString() {
-      return TO_STRING;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return obj instanceof FromEndOfWindow;
-    }
-
-    @Override
-    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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
deleted file mode 100644
index d6b72ef..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
+++ /dev/null
@@ -1,92 +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.windowing;
-
-import java.util.List;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.joda.time.Instant;
-
-/**
- * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
- * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details.
- */
-@Experimental(Experimental.Kind.TRIGGER)
-public class DefaultTrigger extends Trigger{
-
-  private DefaultTrigger() {
-    super(null);
-  }
-
-  /**
-   * Returns the default trigger.
-   */
-  public static DefaultTrigger of() {
-    return new DefaultTrigger();
-  }
-
-  @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();
-  }
-
-  @Override
-  public boolean isCompatible(Trigger other) {
-    // Semantically, all default triggers are identical
-    return other instanceof DefaultTrigger;
-  }
-
-  @Override
-  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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
deleted file mode 100644
index 088c499..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
+++ /dev/null
@@ -1,159 +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.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-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).
- */
-public class ExecutableTrigger implements Serializable {
-
-  /** Store the index assigned to this trigger. */
-  private final int triggerIndex;
-  private final int firstIndexAfterSubtree;
-  private final List<ExecutableTrigger> subTriggers = new ArrayList<>();
-  private final Trigger trigger;
-
-  public static <W extends BoundedWindow> ExecutableTrigger create(Trigger trigger) {
-    return create(trigger, 0);
-  }
-
-  private static <W extends BoundedWindow> ExecutableTrigger create(
-      Trigger trigger, int nextUnusedIndex) {
-    if (trigger instanceof OnceTrigger) {
-      return new ExecutableOnceTrigger((OnceTrigger) trigger, nextUnusedIndex);
-    } else {
-      return new ExecutableTrigger(trigger, nextUnusedIndex);
-    }
-  }
-
-  public static <W extends BoundedWindow> ExecutableTrigger createForOnceTrigger(
-      OnceTrigger trigger, int nextUnusedIndex) {
-    return new ExecutableOnceTrigger(trigger, nextUnusedIndex);
-  }
-
-  private ExecutableTrigger(Trigger trigger, int nextUnusedIndex) {
-    this.trigger = checkNotNull(trigger, "trigger must not be null");
-    this.triggerIndex = nextUnusedIndex++;
-
-    if (trigger.subTriggers() != null) {
-      for (Trigger subTrigger : trigger.subTriggers()) {
-        ExecutableTrigger subExecutable = create(subTrigger, nextUnusedIndex);
-        subTriggers.add(subExecutable);
-        nextUnusedIndex = subExecutable.firstIndexAfterSubtree;
-      }
-    }
-    firstIndexAfterSubtree = nextUnusedIndex;
-  }
-
-  public List<ExecutableTrigger> subTriggers() {
-    return subTriggers;
-  }
-
-  @Override
-  public String toString() {
-    return trigger.toString();
-  }
-
-  /**
-   * Return the underlying trigger specification corresponding to this {@code ExecutableTrigger}.
-   */
-  public Trigger getSpec() {
-    return trigger;
-  }
-
-  public int getTriggerIndex() {
-    return triggerIndex;
-  }
-
-  public final int getFirstIndexAfterSubtree() {
-    return firstIndexAfterSubtree;
-  }
-
-  public boolean isCompatible(ExecutableTrigger other) {
-    return trigger.isCompatible(other.trigger);
-  }
-
-  public ExecutableTrigger getSubTriggerContaining(int index) {
-    checkNotNull(subTriggers);
-    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
-        "Cannot find sub-trigger containing index not in this tree.");
-    ExecutableTrigger previous = null;
-    for (ExecutableTrigger subTrigger : subTriggers) {
-      if (index < subTrigger.triggerIndex) {
-        return previous;
-      }
-      previous = subTrigger;
-    }
-    return previous;
-  }
-
-  /**
-   * 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.
-   */
-  private static class ExecutableOnceTrigger extends ExecutableTrigger {
-
-    public ExecutableOnceTrigger(OnceTrigger trigger, int nextUnusedIndex) {
-      super(trigger, nextUnusedIndex);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java
deleted file mode 100644
index 6666ab9..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/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.
-   */
-  public boolean isFinished(ExecutableTrigger trigger);
-
-  /**
-   * Sets the fact that the trigger is finished.
-   */
-  public void setFinished(ExecutableTrigger trigger, boolean value);
-
-  /**
-   * Sets the trigger and all of its subtriggers to unfinished.
-   */
-  public void clearRecursively(ExecutableTrigger trigger);
-
-  /**
-   * Create an independent copy of this mutable {@link FinishedTriggers}.
-   */
-  public FinishedTriggers copy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java
deleted file mode 100644
index 4cd617f..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersSet.java
deleted file mode 100644
index a9feb73..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
deleted file mode 100644
index 5f20465..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/Never.java
+++ /dev/null
@@ -1,75 +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.windowing;
-
-import java.util.List;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
-import org.joda.time.Instant;
-
-/**
- * A trigger which never fires.
- *
- * <p>Using this trigger will only produce output when the watermark passes the end of the
- * {@link BoundedWindow window} plus the {@link Window#withAllowedLateness allowed
- * lateness}.
- */
-public final class Never {
-  /**
-   * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
-   * when the {@link BoundedWindow} closes.
-   */
-  public static OnceTrigger ever() {
-    // NeverTrigger ignores all inputs and is Window-type independent.
-    return new NeverTrigger();
-  }
-
-  // package-private in order to check identity for string formatting.
-  static class NeverTrigger extends OnceTrigger {
-    protected NeverTrigger() {
-      super(null);
-    }
-
-    @Override
-    public void onElement(OnElementContext c) {}
-
-    @Override
-    public void onMerge(OnMergeContext c) {}
-
-    @Override
-    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-      return this;
-    }
-
-    @Override
-    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/69b1efda/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
deleted file mode 100644
index 25b7b34..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/OrFinallyTrigger.java
+++ /dev/null
@@ -1,105 +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.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;
-
-/**
- * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
- */
-class OrFinallyTrigger extends Trigger {
-
-  private static final int ACTUAL = 0;
-  private static final int UNTIL = 1;
-
-  @VisibleForTesting OrFinallyTrigger(Trigger actual, Trigger.OnceTrigger until) {
-    super(Arrays.asList(actual, until));
-  }
-
-  @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);
-    Instant untilDeadline = subTriggers.get(UNTIL).getWatermarkThatGuaranteesFiring(window);
-    return actualDeadline.isBefore(untilDeadline) ? actualDeadline : untilDeadline;
-  }
-
-  @Override
-  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    // Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL
-    // may not be a OnceTrigger.
-    return Repeatedly.forever(
-        new OrFinallyTrigger(
-            continuationTriggers.get(ACTUAL),
-            (Trigger.OnceTrigger) continuationTriggers.get(UNTIL)));
-  }
-
-  @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);
-  }
-}


[13/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
Move triggers to runners-core

This commit is effectively a "git hint" that all the files
in runners-core are moved from prior files. The moved files
will be replaced.


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

Branch: refs/heads/master
Commit: e4398e1e7b08ca5098e422eb62c2d8cb139d906b
Parents: 7322616
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 20:05:27 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 13 14:34:34 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/TriggerRunner.java | 247 ---------
 .../beam/runners/core/reactors/AfterAll.java    | 122 +++++
 .../reactors/AfterDelayFromFirstElement.java    | 335 ++++++++++++
 .../beam/runners/core/reactors/AfterEach.java   | 141 +++++
 .../beam/runners/core/reactors/AfterFirst.java  | 124 +++++
 .../beam/runners/core/reactors/AfterPane.java   | 144 +++++
 .../core/reactors/AfterProcessingTime.java      | 102 ++++
 .../AfterSynchronizedProcessingTime.java        |  73 +++
 .../runners/core/reactors/AfterWatermark.java   | 355 +++++++++++++
 .../runners/core/reactors/DefaultTrigger.java   |  92 ++++
 .../core/reactors/ExecutableTrigger.java        | 159 ++++++
 .../runners/core/reactors/FinishedTriggers.java |  44 ++
 .../core/reactors/FinishedTriggersBitSet.java   |  67 +++
 .../core/reactors/FinishedTriggersSet.java      |  72 +++
 .../beam/runners/core/reactors/Never.java       |  75 +++
 .../runners/core/reactors/OrFinallyTrigger.java | 105 ++++
 .../beam/runners/core/reactors/Repeatedly.java  | 101 ++++
 .../runners/core/reactors/ReshuffleTrigger.java |  66 +++
 .../beam/runners/core/reactors/Trigger.java     | 527 +++++++++++++++++++
 .../core/reactors/TriggerContextFactory.java    | 507 ++++++++++++++++++
 .../runners/core/reactors/TriggerRunner.java    | 247 +++++++++
 .../runners/core/reactors/AfterAllTest.java     | 156 ++++++
 .../runners/core/reactors/AfterEachTest.java    | 132 +++++
 .../runners/core/reactors/AfterFirstTest.java   | 181 +++++++
 .../runners/core/reactors/AfterPaneTest.java    | 132 +++++
 .../core/reactors/AfterProcessingTimeTest.java  | 187 +++++++
 .../AfterSynchronizedProcessingTimeTest.java    | 121 +++++
 .../core/reactors/AfterWatermarkTest.java       | 380 +++++++++++++
 .../core/reactors/DefaultTriggerTest.java       | 176 +++++++
 .../core/reactors/ExecutableTriggerTest.java    | 127 +++++
 .../reactors/FinishedTriggersBitSetTest.java    |  55 ++
 .../reactors/FinishedTriggersProperties.java    | 110 ++++
 .../core/reactors/FinishedTriggersSetTest.java  |  60 +++
 .../beam/runners/core/reactors/NeverTest.java   |  56 ++
 .../core/reactors/OrFinallyTriggerTest.java     | 215 ++++++++
 .../runners/core/reactors/RepeatedlyTest.java   | 224 ++++++++
 .../core/reactors/ReshuffleTriggerTest.java     |  67 +++
 .../beam/runners/core/reactors/StubTrigger.java |  70 +++
 .../beam/runners/core/reactors/TriggerTest.java | 118 +++++
 .../runners/core/reactors/TriggerTester.java    | 410 +++++++++++++++
 .../beam/sdk/transforms/windowing/AfterAll.java | 122 -----
 .../windowing/AfterDelayFromFirstElement.java   | 335 ------------
 .../sdk/transforms/windowing/AfterEach.java     | 141 -----
 .../sdk/transforms/windowing/AfterFirst.java    | 124 -----
 .../sdk/transforms/windowing/AfterPane.java     | 144 -----
 .../windowing/AfterProcessingTime.java          | 102 ----
 .../AfterSynchronizedProcessingTime.java        |  73 ---
 .../transforms/windowing/AfterWatermark.java    | 355 -------------
 .../transforms/windowing/DefaultTrigger.java    |  92 ----
 .../beam/sdk/transforms/windowing/Never.java    |  75 ---
 .../transforms/windowing/OrFinallyTrigger.java  | 105 ----
 .../sdk/transforms/windowing/Repeatedly.java    | 101 ----
 .../beam/sdk/transforms/windowing/Trigger.java  | 527 -------------------
 .../apache/beam/sdk/util/ExecutableTrigger.java | 159 ------
 .../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  |  66 ---
 .../beam/sdk/util/TriggerContextFactory.java    | 507 ------------------
 .../sdk/transforms/windowing/AfterAllTest.java  | 156 ------
 .../sdk/transforms/windowing/AfterEachTest.java | 132 -----
 .../transforms/windowing/AfterFirstTest.java    | 181 -------
 .../sdk/transforms/windowing/AfterPaneTest.java | 132 -----
 .../windowing/AfterProcessingTimeTest.java      | 187 -------
 .../AfterSynchronizedProcessingTimeTest.java    | 121 -----
 .../windowing/AfterWatermarkTest.java           | 380 -------------
 .../windowing/DefaultTriggerTest.java           | 176 -------
 .../sdk/transforms/windowing/NeverTest.java     |  56 --
 .../windowing/OrFinallyTriggerTest.java         | 215 --------
 .../transforms/windowing/RepeatedlyTest.java    | 224 --------
 .../sdk/transforms/windowing/StubTrigger.java   |  70 ---
 .../sdk/transforms/windowing/TriggerTest.java   | 118 -----
 .../beam/sdk/util/ExecutableTriggerTest.java    | 127 -----
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 --
 .../sdk/util/FinishedTriggersProperties.java    | 110 ----
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ---
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  67 ---
 .../org/apache/beam/sdk/util/TriggerTester.java | 410 ---------------
 78 files changed, 6435 insertions(+), 6435 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
deleted file mode 100644
index 8d0f322..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java
+++ /dev/null
@@ -1,247 +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.core;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.Map;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.BitSetCoder;
-import org.apache.beam.sdk.util.ExecutableTrigger;
-import org.apache.beam.sdk.util.FinishedTriggers;
-import org.apache.beam.sdk.util.FinishedTriggersBitSet;
-import org.apache.beam.sdk.util.Timers;
-import org.apache.beam.sdk.util.TriggerContextFactory;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.joda.time.Instant;
-
-/**
- * Executes a trigger while managing persistence of information about which subtriggers are
- * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
- *
- * <p>Specifically, the responsibilities are:
- *
- * <ul>
- *   <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
- *       constructing the appropriate trigger contexts.</li>
- *   <li>Committing a record of which subtriggers are finished to persistent state.</li>
- *   <li>Restoring the record of which subtriggers are finished from persistent state.</li>
- *   <li>Clearing out the persisted finished set when a caller indicates
- *       (via {#link #clearFinished}) that it is no longer needed.</li>
- * </ul>
- *
- * <p>These responsibilities are intertwined: trigger contexts include mutable information about
- * which subtriggers are finished. This class provides the information when building the contexts
- * and commits the information when the method of the {@link ExecutableTrigger} returns.
- *
- * @param <W> The kind of windows being processed.
- */
-public class TriggerRunner<W extends BoundedWindow> {
-  @VisibleForTesting
-  static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
-      StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
-
-  private final ExecutableTrigger rootTrigger;
-  private final TriggerContextFactory<W> contextFactory;
-
-  public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
-    checkState(rootTrigger.getTriggerIndex() == 0);
-    this.rootTrigger = rootTrigger;
-    this.contextFactory = contextFactory;
-  }
-
-  private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
-    if (!isFinishedSetNeeded()) {
-      // If no trigger in the tree will ever have finished bits, then we don't need to read them.
-      // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
-      // finished) for each trigger in the tree.
-      return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
-    }
-
-    BitSet bitSet = state.read();
-    return bitSet == null
-        ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
-            : FinishedTriggersBitSet.fromBitSet(bitSet);
-  }
-
-
-  private void clearFinishedBits(ValueState<BitSet> state) {
-    if (!isFinishedSetNeeded()) {
-      // Nothing to clear.
-      return;
-    }
-    state.clear();
-  }
-
-  /** Return true if the trigger is closed in the window corresponding to the specified state. */
-  public boolean isClosed(StateAccessor<?> state) {
-    return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchForValue(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchOnElement(
-        contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchOnFire(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchShouldFire(W window, StateAccessor<?> state) {
-    if (isFinishedSetNeeded()) {
-      state.access(FINISHED_BITS_TAG).readLater();
-    }
-    rootTrigger.getSpec().prefetchShouldFire(
-        contextFactory.createStateAccessor(window, rootTrigger));
-  }
-
-  /**
-   * Run the trigger logic to deal with a new value.
-   */
-  public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
-      throws Exception {
-    // Clone so that we can detect changes and so that changes here don't pollute merging.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
-        window, timers, timestamp, rootTrigger, finishedSet);
-    rootTrigger.invokeOnElement(triggerContext);
-    persistFinishedSet(state, finishedSet);
-  }
-
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-      justification = "prefetch side effect")
-  public void prefetchForMerge(
-      W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
-    if (isFinishedSetNeeded()) {
-      for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
-        value.readLater();
-      }
-    }
-    rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
-        window, mergingWindows, rootTrigger));
-  }
-
-  /**
-   * Run the trigger merging logic as part of executing the specified merge.
-   */
-  public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
-    // Clone so that we can detect changes and so that changes here don't pollute merging.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-
-    // And read the finished bits in each merging window.
-    ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
-    for (Map.Entry<W, ValueState<BitSet>> entry :
-        state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
-      // Don't need to clone these, since the trigger context doesn't allow modification
-      builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
-      // Clear the underlying finished bits.
-      clearFinishedBits(entry.getValue());
-    }
-    ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
-
-    Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
-        window, timers, rootTrigger, finishedSet, mergingFinishedSets);
-
-    // Run the merge from the trigger
-    rootTrigger.invokeOnMerge(mergeContext);
-
-    persistFinishedSet(state, finishedSet);
-  }
-
-  public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.TriggerContext context = contextFactory.base(window, timers,
-        rootTrigger, finishedSet);
-    return rootTrigger.invokeShouldFire(context);
-  }
-
-  public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    // shouldFire should be false.
-    // However it is too expensive to assert.
-    FinishedTriggersBitSet finishedSet =
-        readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-    Trigger.TriggerContext context = contextFactory.base(window, timers,
-        rootTrigger, finishedSet);
-    rootTrigger.invokeOnFire(context);
-    persistFinishedSet(state, finishedSet);
-  }
-
-  private void persistFinishedSet(
-      StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
-    if (!isFinishedSetNeeded()) {
-      return;
-    }
-
-    ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
-    if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
-      if (modifiedFinishedSet.getBitSet().isEmpty()) {
-        finishedSetState.clear();
-      } else {
-        finishedSetState.write(modifiedFinishedSet.getBitSet());
-      }
-    }
-  }
-
-  /**
-   * Clear the finished bits.
-   */
-  public void clearFinished(StateAccessor<?> state) {
-    clearFinishedBits(state.access(FINISHED_BITS_TAG));
-  }
-
-  /**
-   * Clear the state used for executing triggers, but leave the finished set to indicate
-   * the window is closed.
-   */
-  public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
-    // Don't need to clone, because we'll be clearing the finished bits anyways.
-    FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
-    rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
-  }
-
-  private boolean isFinishedSetNeeded() {
-    // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
-    // lookup. Right now, we special case this for the DefaultTrigger.
-    return !(rootTrigger.getSpec() instanceof DefaultTrigger);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
new file mode 100644
index 0000000..cc8c97f
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterAll.java
@@ -0,0 +1,122 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterAll extends OnceTrigger {
+
+  private AfterAll(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
+   */
+  public static OnceTrigger of(OnceTrigger... triggers) {
+    return new AfterAll(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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;
+    for (Trigger subTrigger : subTriggers) {
+      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
+      if (deadline.isBefore(subDeadline)) {
+        deadline = subDeadline;
+      }
+    }
+    return deadline;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    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(");
+    Joiner.on(", ").appendTo(builder, subTriggers);
+    builder.append(")");
+
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
new file mode 100644
index 0000000..c4bc946
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterDelayFromFirstElement.java
@@ -0,0 +1,335 @@
+/*
+ * 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.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;
+import org.apache.beam.sdk.transforms.Min;
+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;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+import org.joda.time.format.PeriodFormatter;
+
+/**
+ * A base class for triggers that happen after a processing time delay from the arrival
+ * of the first element in a pane.
+ *
+ * <p>This class is for internal use only and may change at any time.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public abstract class AfterDelayFromFirstElement extends OnceTrigger {
+
+  protected static final List<SerializableFunction<Instant, Instant>> IDENTITY =
+      ImmutableList.<SerializableFunction<Instant, Instant>>of();
+
+  protected static final StateTag<Object, AccumulatorCombiningState<Instant,
+                                              Combine.Holder<Instant>, Instant>> DELAYED_UNTIL_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "delayed", InstantCoder.of(), Min.MinFn.<Instant>naturalOrder()));
+
+  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.
+   */
+  protected abstract AfterDelayFromFirstElement newWith(
+      List<SerializableFunction<Instant, Instant>> transform);
+
+  /**
+   * A list of timestampMappers m1, m2, m3, ... m_n considered to be composed in sequence. The
+   * overall mapping for an instance `instance` is `m_n(... m3(m2(m1(instant))`,
+   * implemented via #computeTargetTimestamp
+   */
+  protected final List<SerializableFunction<Instant, Instant>> timestampMappers;
+
+  private final TimeDomain timeDomain;
+
+  public AfterDelayFromFirstElement(
+      TimeDomain timeDomain,
+      List<SerializableFunction<Instant, Instant>> timestampMappers) {
+    super(null);
+    this.timestampMappers = timestampMappers;
+    this.timeDomain = timeDomain;
+  }
+
+  private Instant getTargetTimestamp(OnElementContext c) {
+    return computeTargetTimestamp(c.currentProcessingTime());
+  }
+
+  /**
+   * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+   * than the timestamp.
+   *
+   * <p>TODO: Consider sharing this with FixedWindows, and bring over the equivalent of
+   * CalendarWindows.
+   */
+  public AfterDelayFromFirstElement alignedTo(final Duration size, final Instant offset) {
+    return newWith(new AlignFn(size, offset));
+  }
+
+  /**
+   * Aligns the time to be the smallest multiple of {@code size} greater than the timestamp
+   * since the epoch.
+   */
+  public AfterDelayFromFirstElement alignedTo(final Duration size) {
+    return alignedTo(size, new Instant(0));
+  }
+
+  /**
+   * Adds some delay to the original target time.
+   *
+   * @param delay the delay to add
+   * @return An updated time trigger that will wait the additional time before firing.
+   */
+  public AfterDelayFromFirstElement plusDelayOf(final Duration delay) {
+    return newWith(new DelayFn(delay));
+  }
+
+  /**
+   * @deprecated This will be removed in the next major version. Please use only
+   *             {@link #plusDelayOf} and {@link #alignedTo}.
+   */
+  @Deprecated
+  public OnceTrigger mappedTo(SerializableFunction<Instant, Instant> timestampMapper) {
+    return newWith(timestampMapper);
+  }
+
+  @Override
+  public boolean isCompatible(Trigger other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    AfterDelayFromFirstElement that = (AfterDelayFromFirstElement) other;
+    return this.timestampMappers.equals(that.timestampMappers);
+  }
+
+
+  private AfterDelayFromFirstElement newWith(
+      SerializableFunction<Instant, Instant> timestampMapper) {
+    return newWith(
+        ImmutableList.<SerializableFunction<Instant, Instant>>builder()
+            .addAll(timestampMappers)
+            .add(timestampMapper)
+            .build());
+  }
+
+  @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) {
+      result = timestampMapper.apply(result);
+    }
+    return result;
+  }
+
+  /**
+   * A {@link SerializableFunction} to delay the timestamp at which this triggers fires.
+   */
+  private static final class DelayFn implements SerializableFunction<Instant, Instant> {
+    private final Duration delay;
+
+    public DelayFn(Duration delay) {
+      this.delay = delay;
+    }
+
+    @Override
+    public Instant apply(Instant input) {
+      return input.plus(delay);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof DelayFn)) {
+        return false;
+      }
+
+      return this.delay.equals(((DelayFn) object).delay);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(delay);
+    }
+
+    @Override
+    public String toString() {
+      return PERIOD_FORMATTER.print(delay.toPeriod());
+    }
+  }
+
+  /**
+   * A {@link SerializableFunction} to align an instant to the nearest interval boundary.
+   */
+  static final class AlignFn implements SerializableFunction<Instant, Instant> {
+    private final Duration size;
+    private final Instant offset;
+
+
+    /**
+     * Aligns timestamps to the smallest multiple of {@code size} since the {@code offset} greater
+     * than the timestamp.
+     */
+    public AlignFn(Duration size, Instant offset) {
+      this.size = size;
+      this.offset = offset;
+    }
+
+    @Override
+    public Instant apply(Instant point) {
+      long millisSinceStart = new Duration(offset, point).getMillis() % size.getMillis();
+      return millisSinceStart == 0 ? point : point.plus(size).minus(millisSinceStart);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+      if (object == this) {
+        return true;
+      }
+
+      if (!(object instanceof AlignFn)) {
+        return false;
+      }
+
+      AlignFn other = (AlignFn) object;
+      return other.size.equals(this.size)
+          && other.offset.equals(this.offset);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(size, offset);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
new file mode 100644
index 0000000..629c640
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterEach.java
@@ -0,0 +1,141 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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;
+
+/**
+ * A composite {@link Trigger} that executes its sub-triggers in order.
+ * Only one sub-trigger is executing at a time,
+ * and any time it fires the {@code AfterEach} fires. When the currently executing
+ * sub-trigger finishes, the {@code AfterEach} starts executing the next sub-trigger.
+ *
+ * <p>{@code AfterEach.inOrder(t1, t2, ...)} finishes when all of the sub-triggers have finished.
+ *
+ * <p>The following properties hold:
+ * <ul>
+ *   <li> {@code AfterEach.inOrder(AfterEach.inOrder(a, b), c)} behaves the same as
+ *   {@code AfterEach.inOrder(a, b, c)} and {@code AfterEach.inOrder(a, AfterEach.inOrder(b, c)}.
+ *   <li> {@code AfterEach.inOrder(Repeatedly.forever(a), b)} behaves the same as
+ *   {@code Repeatedly.forever(a)}, since the repeated trigger never finishes.
+ * </ul>
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterEach extends Trigger {
+
+  private AfterEach(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterEach} {@code Trigger} with the given subtriggers.
+   */
+  @SafeVarargs
+  public static Trigger inOrder(Trigger... triggers) {
+    return new AfterEach(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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.
+    return subTriggers.get(0).getWatermarkThatGuaranteesFiring(window);
+  }
+
+  @Override
+  public Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return Repeatedly.forever(new AfterFirst(continuationTriggers));
+  }
+
+  @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);
+    builder.append(")");
+
+    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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
new file mode 100644
index 0000000..6b06cfa
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterFirst.java
@@ -0,0 +1,124 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.joda.time.Instant;
+
+/**
+ * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have
+ * fired.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterFirst extends OnceTrigger {
+
+  AfterFirst(List<Trigger> subTriggers) {
+    super(subTriggers);
+    checkArgument(subTriggers.size() > 1);
+  }
+
+  /**
+   * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
+   */
+  public static OnceTrigger of(OnceTrigger... triggers) {
+    return new AfterFirst(Arrays.<Trigger>asList(triggers));
+  }
+
+  @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;
+    for (Trigger subTrigger : subTriggers) {
+      Instant subDeadline = subTrigger.getWatermarkThatGuaranteesFiring(window);
+      if (deadline.isAfter(subDeadline)) {
+        deadline = subDeadline;
+      }
+    }
+    return deadline;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new AfterFirst(continuationTriggers);
+  }
+
+  @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);
+    builder.append(")");
+
+    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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
new file mode 100644
index 0000000..8c128dd
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterPane.java
@@ -0,0 +1,144 @@
+/*
+ * 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.windowing;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.annotations.Experimental;
+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;
+
+/**
+ * {@link Trigger}s that fire based on properties of the elements in the current pane.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterPane extends OnceTrigger {
+
+private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
+      ELEMENTS_IN_PANE_TAG =
+      StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+          "count", VarLongCoder.of(), new Sum.SumLongFn()));
+
+  private final int countElems;
+
+  private AfterPane(int countElems) {
+    super(null);
+    this.countElems = countElems;
+  }
+
+  /**
+   * Creates a trigger that fires when the pane contains at least {@code countElems} elements.
+   */
+  public static AfterPane elementCountAtLeast(int countElems) {
+    return new AfterPane(countElems);
+  }
+
+  @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);
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  public OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return AfterPane.elementCountAtLeast(1);
+  }
+
+  @Override
+  public String toString() {
+    return "AfterPane.elementCountAtLeast(" + countElems + ")";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterPane)) {
+      return false;
+    }
+    AfterPane that = (AfterPane) obj;
+    return this.countElems == that.countElems;
+  }
+
+  @Override
+  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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
new file mode 100644
index 0000000..f551118
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterProcessingTime.java
@@ -0,0 +1,102 @@
+/*
+ * 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.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;
+import org.joda.time.Instant;
+
+/**
+ * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in
+ * the real-time domain.
+ *
+ * <p>The time at which to fire the timer can be adjusted via the methods in
+ * {@link AfterDelayFromFirstElement}, such as {@link AfterDelayFromFirstElement#plusDelayOf} or
+ * {@link AfterDelayFromFirstElement#alignedTo}.
+ */
+@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);
+  }
+
+  /**
+   * Creates a trigger that fires when the current processing time passes the processing time
+   * at which this trigger saw the first element in a pane.
+   */
+  public static AfterProcessingTime pastFirstElementInPane() {
+    return new AfterProcessingTime(IDENTITY);
+  }
+
+  @Override
+  protected AfterProcessingTime newWith(
+      List<SerializableFunction<Instant, Instant>> transforms) {
+    return new AfterProcessingTime(transforms);
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return new AfterSynchronizedProcessingTime();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()");
+    for (SerializableFunction<Instant, Instant> delayFn : timestampMappers) {
+      builder
+          .append(".plusDelayOf(")
+          .append(delayFn)
+          .append(")");
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof AfterProcessingTime)) {
+      return false;
+    }
+    AfterProcessingTime that = (AfterProcessingTime) obj;
+    return Objects.equals(this.timestampMappers, that.timestampMappers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), this.timestampMappers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
new file mode 100644
index 0000000..59ece10
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterSynchronizedProcessingTime.java
@@ -0,0 +1,73 @@
+/*
+ * 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.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;
+
+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());
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return this;
+  }
+
+  @Override
+  public String toString() {
+    return "AfterSynchronizedProcessingTime.pastFirstElementInPane()";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return this == obj || obj instanceof AfterSynchronizedProcessingTime;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(AfterSynchronizedProcessingTime.class);
+  }
+
+  @Override
+  protected AfterSynchronizedProcessingTime
+      newWith(List<SerializableFunction<Instant, Instant>> transforms) {
+    // ignore transforms
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
new file mode 100644
index 0000000..e2463d8
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/AfterWatermark.java
@@ -0,0 +1,355 @@
+/*
+ * 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.windowing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableList;
+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.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.joda.time.Instant;
+
+/**
+ * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
+ * lower-bound, sometimes heuristically established, on event times that have been fully processed
+ * by the pipeline.
+ *
+ * <p>For sources that provide non-heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using arrival times as event times), the
+ * watermark is a strict guarantee that no data with an event time earlier than
+ * that watermark will ever be observed in the pipeline. In this case, it's safe to assume that any
+ * pane triggered by an {@code AfterWatermark} trigger with a reference point at or beyond the end
+ * of the window will be the last pane ever for that window.
+ *
+ * <p>For sources that provide heuristic watermarks (e.g.
+ * {@link org.apache.beam.sdk.io.PubsubIO} when using user-supplied event times), the
+ * watermark itself becomes an <i>estimate</i> that no data with an event time earlier than that
+ * watermark (i.e. "late data") will ever be observed in the pipeline. These heuristics can
+ * often be quite accurate, but the chance of seeing late data for any given window is non-zero.
+ * Thus, if absolute correctness over time is important to your use case, you may want to consider
+ * using a trigger that accounts for late data. The default trigger,
+ * {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}, which fires
+ * once when the watermark passes the end of the window and then immediately therafter when any
+ * late data arrives, is one such example.
+ *
+ * <p>The watermark is the clock that defines {@link TimeDomain#EVENT_TIME}.
+ *
+ * <p>Additionaly firings before or after the watermark can be requested by calling
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or
+ * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class AfterWatermark {
+
+  private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
+
+  // Static factory class.
+  private AfterWatermark() {}
+
+  /**
+   * Creates a trigger that fires when the watermark passes the end of the window.
+   */
+  public static FromEndOfWindow pastEndOfWindow() {
+    return new FromEndOfWindow();
+  }
+
+  /**
+   * @see AfterWatermark
+   */
+  public static class AfterWatermarkEarlyAndLate extends Trigger {
+
+    private static final int EARLY_INDEX = 0;
+    private static final int LATE_INDEX = 1;
+
+    private final OnceTrigger earlyTrigger;
+    @Nullable
+    private final OnceTrigger lateTrigger;
+
+    @SuppressWarnings("unchecked")
+    public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
+      super(lateTrigger == null
+          ? ImmutableList.<Trigger>of(earlyTrigger)
+          : ImmutableList.<Trigger>of(earlyTrigger, lateTrigger));
+      this.earlyTrigger = checkNotNull(earlyTrigger, "earlyTrigger should not be null");
+      this.lateTrigger = lateTrigger;
+    }
+
+    public Trigger withEarlyFirings(OnceTrigger earlyTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    public Trigger withLateFirings(OnceTrigger lateTrigger) {
+      return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
+    }
+
+    @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(),
+          lateTrigger == null ? null : lateTrigger.getContinuationTrigger());
+    }
+
+    @Override
+    protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+      throw new UnsupportedOperationException(
+          "Should not call getContinuationTrigger(List<Trigger>)");
+    }
+
+    @Override
+    public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+      // Even without an early or late trigger, we'll still produce a firing at the watermark.
+      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);
+
+      if (!(earlyTrigger instanceof Never.NeverTrigger)) {
+        builder
+            .append(".withEarlyFirings(")
+            .append(earlyTrigger)
+            .append(")");
+      }
+
+      if (lateTrigger != null && !(lateTrigger instanceof Never.NeverTrigger)) {
+        builder
+            .append(".withLateFirings(")
+            .append(lateTrigger)
+            .append(")");
+      }
+
+      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);
+    }
+  }
+
+  /**
+   * A watermark trigger targeted relative to the end of the window.
+   */
+  public static class FromEndOfWindow extends OnceTrigger {
+
+    private FromEndOfWindow() {
+      super(null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires before the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTrigger earlyFirings) {
+      checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
+      return new AfterWatermarkEarlyAndLate(earlyFirings, null);
+    }
+
+    /**
+     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
+     * the given {@code Trigger} fires after the watermark has passed the end of the window.
+     */
+    public AfterWatermarkEarlyAndLate withLateFirings(OnceTrigger lateFirings) {
+      checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
+      return new AfterWatermarkEarlyAndLate(Never.ever(), lateFirings);
+    }
+
+    @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();
+    }
+
+    @Override
+    public FromEndOfWindow getContinuationTrigger(List<Trigger> continuationTriggers) {
+      return this;
+    }
+
+    @Override
+    public String toString() {
+      return TO_STRING;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj instanceof FromEndOfWindow;
+    }
+
+    @Override
+    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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
new file mode 100644
index 0000000..d6b72ef
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/DefaultTrigger.java
@@ -0,0 +1,92 @@
+/*
+ * 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.windowing;
+
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.joda.time.Instant;
+
+/**
+ * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+ * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details.
+ */
+@Experimental(Experimental.Kind.TRIGGER)
+public class DefaultTrigger extends Trigger{
+
+  private DefaultTrigger() {
+    super(null);
+  }
+
+  /**
+   * Returns the default trigger.
+   */
+  public static DefaultTrigger of() {
+    return new DefaultTrigger();
+  }
+
+  @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();
+  }
+
+  @Override
+  public boolean isCompatible(Trigger other) {
+    // Semantically, all default triggers are identical
+    return other instanceof DefaultTrigger;
+  }
+
+  @Override
+  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/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
new file mode 100644
index 0000000..088c499
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/ExecutableTrigger.java
@@ -0,0 +1,159 @@
+/*
+ * 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.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+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).
+ */
+public class ExecutableTrigger implements Serializable {
+
+  /** Store the index assigned to this trigger. */
+  private final int triggerIndex;
+  private final int firstIndexAfterSubtree;
+  private final List<ExecutableTrigger> subTriggers = new ArrayList<>();
+  private final Trigger trigger;
+
+  public static <W extends BoundedWindow> ExecutableTrigger create(Trigger trigger) {
+    return create(trigger, 0);
+  }
+
+  private static <W extends BoundedWindow> ExecutableTrigger create(
+      Trigger trigger, int nextUnusedIndex) {
+    if (trigger instanceof OnceTrigger) {
+      return new ExecutableOnceTrigger((OnceTrigger) trigger, nextUnusedIndex);
+    } else {
+      return new ExecutableTrigger(trigger, nextUnusedIndex);
+    }
+  }
+
+  public static <W extends BoundedWindow> ExecutableTrigger createForOnceTrigger(
+      OnceTrigger trigger, int nextUnusedIndex) {
+    return new ExecutableOnceTrigger(trigger, nextUnusedIndex);
+  }
+
+  private ExecutableTrigger(Trigger trigger, int nextUnusedIndex) {
+    this.trigger = checkNotNull(trigger, "trigger must not be null");
+    this.triggerIndex = nextUnusedIndex++;
+
+    if (trigger.subTriggers() != null) {
+      for (Trigger subTrigger : trigger.subTriggers()) {
+        ExecutableTrigger subExecutable = create(subTrigger, nextUnusedIndex);
+        subTriggers.add(subExecutable);
+        nextUnusedIndex = subExecutable.firstIndexAfterSubtree;
+      }
+    }
+    firstIndexAfterSubtree = nextUnusedIndex;
+  }
+
+  public List<ExecutableTrigger> subTriggers() {
+    return subTriggers;
+  }
+
+  @Override
+  public String toString() {
+    return trigger.toString();
+  }
+
+  /**
+   * Return the underlying trigger specification corresponding to this {@code ExecutableTrigger}.
+   */
+  public Trigger getSpec() {
+    return trigger;
+  }
+
+  public int getTriggerIndex() {
+    return triggerIndex;
+  }
+
+  public final int getFirstIndexAfterSubtree() {
+    return firstIndexAfterSubtree;
+  }
+
+  public boolean isCompatible(ExecutableTrigger other) {
+    return trigger.isCompatible(other.trigger);
+  }
+
+  public ExecutableTrigger getSubTriggerContaining(int index) {
+    checkNotNull(subTriggers);
+    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
+        "Cannot find sub-trigger containing index not in this tree.");
+    ExecutableTrigger previous = null;
+    for (ExecutableTrigger subTrigger : subTriggers) {
+      if (index < subTrigger.triggerIndex) {
+        return previous;
+      }
+      previous = subTrigger;
+    }
+    return previous;
+  }
+
+  /**
+   * 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.
+   */
+  private static class ExecutableOnceTrigger extends ExecutableTrigger {
+
+    public ExecutableOnceTrigger(OnceTrigger trigger, int nextUnusedIndex) {
+      super(trigger, nextUnusedIndex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java
new file mode 100644
index 0000000..6666ab9
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggers.java
@@ -0,0 +1,44 @@
+/*
+ * 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.
+   */
+  public boolean isFinished(ExecutableTrigger trigger);
+
+  /**
+   * Sets the fact that the trigger is finished.
+   */
+  public void setFinished(ExecutableTrigger trigger, boolean value);
+
+  /**
+   * Sets the trigger and all of its subtriggers to unfinished.
+   */
+  public void clearRecursively(ExecutableTrigger trigger);
+
+  /**
+   * Create an independent copy of this mutable {@link FinishedTriggers}.
+   */
+  public FinishedTriggers copy();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java
new file mode 100644
index 0000000..4cd617f
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/reactors/FinishedTriggersBitSet.java
@@ -0,0 +1,67 @@
+/*
+ * 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());
+  }
+}


[02/17] incubator-beam git commit: Rename runners-core Trigger to TriggerStateMachine

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
deleted file mode 100644
index fb2b4d5..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/NeverTest.java
+++ /dev/null
@@ -1,56 +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.windowing;
-
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-
-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}.
- */
-@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 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));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
deleted file mode 100644
index 7289d97..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/OrFinallyTriggerTest.java
+++ /dev/null
@@ -1,215 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link OrFinallyTrigger}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        Repeatedly.forever(AfterWatermark.pastEndOfWindow())
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow())
-        .orFinally(AfterPane.elementCountAtLeast(1))
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9), Repeatedly.forever(AfterPane.elementCountAtLeast(1))
-        .orFinally(AfterWatermark.pastEndOfWindow())
-        .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(new Instant(9),
-        AfterPane.elementCountAtLeast(100)
-            .orFinally(AfterWatermark.pastEndOfWindow())
-            .getWatermarkThatGuaranteesFiring(window));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        Repeatedly.forever(AfterPane.elementCountAtLeast(1))
-        .orFinally(AfterPane.elementCountAtLeast(10))
-        .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger triggerA = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger triggerB = AfterWatermark.pastEndOfWindow();
-    Trigger aOrFinallyB = triggerA.orFinally(triggerB);
-    Trigger bOrFinallyA = triggerB.orFinally(triggerA);
-    assertEquals(
-        Repeatedly.forever(
-            triggerA.getContinuationTrigger().orFinally(triggerB.getContinuationTrigger())),
-        aOrFinallyB.getContinuationTrigger());
-    assertEquals(
-        Repeatedly.forever(
-            triggerB.getContinuationTrigger().orFinally(triggerA.getContinuationTrigger())),
-        bOrFinallyA.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = StubTrigger.named("t1").orFinally(StubTrigger.named("t2"));
-    assertEquals("t1.orFinally(t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
deleted file mode 100644
index 6e8930d..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/RepeatedlyTest.java
+++ /dev/null
@@ -1,224 +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.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;
-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 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.
-   */
-  @Test
-  public void testFireDeadline() throws Exception {
-    setUp(FixedWindows.of(Duration.millis(10)));
-    IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
-    Instant arbitraryInstant = new Instant(34957849);
-
-    when(mockTrigger.getWatermarkThatGuaranteesFiring(Mockito.<IntervalWindow>any()))
-        .thenReturn(arbitraryInstant);
-
-    assertThat(
-        Repeatedly.forever(mockTrigger).getWatermarkThatGuaranteesFiring(window),
-        equalTo(arbitraryInstant));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
-    Trigger repeatedly = Repeatedly.forever(trigger);
-    assertEquals(
-        Repeatedly.forever(trigger.getContinuationTrigger()), repeatedly.getContinuationTrigger());
-    assertEquals(
-        Repeatedly.forever(trigger.getContinuationTrigger().getContinuationTrigger()),
-        repeatedly.getContinuationTrigger().getContinuationTrigger());
-  }
-
-  @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";
-        }
-      });
-
-    assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.java
deleted file mode 100644
index 83077f4..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/ReshuffleTriggerTest.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 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;
-
-/**
- * Tests for {@link ReshuffleTrigger}.
- */
-@RunWith(JUnit4.class)
-public class ReshuffleTriggerTest {
-
-  /** Public so that other tests can instantiate {@link ReshuffleTrigger}. */
-  public static <W extends BoundedWindow> ReshuffleTrigger<W> forTest() {
-    return new ReshuffleTrigger<>();
-  }
-
-  @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/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
deleted file mode 100644
index b258a79..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/StubTrigger.java
+++ /dev/null
@@ -1,70 +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.windowing;
-
-import com.google.common.collect.Lists;
-import java.util.List;
-import org.joda.time.Instant;
-
-/**
- * No-op {@link OnceTrigger} implementation for testing.
- */
-abstract class StubTrigger extends Trigger.OnceTrigger {
-  /**
-   * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}.
-   */
-  static StubTrigger named(final String name) {
-    return new StubTrigger() {
-      @Override
-      public String toString() {
-        return name;
-      }
-    };
-  }
-
-  protected StubTrigger() {
-    super(Lists.<Trigger>newArrayList());
-  }
-
-  @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;
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
deleted file mode 100644
index cfc03b2..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTest.java
+++ /dev/null
@@ -1,118 +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.windowing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Arrays;
-import java.util.List;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link Trigger}.
- */
-@RunWith(JUnit4.class)
-public class TriggerTest {
-
-  @Test
-  public void testTriggerToString() throws Exception {
-    assertEquals("AfterWatermark.pastEndOfWindow()", AfterWatermark.pastEndOfWindow().toString());
-    assertEquals("Repeatedly.forever(AfterWatermark.pastEndOfWindow())",
-        Repeatedly.forever(AfterWatermark.pastEndOfWindow()).toString());
-  }
-
-  @Test
-  public void testIsCompatible() throws Exception {
-    assertTrue(new Trigger1(null).isCompatible(new Trigger1(null)));
-    assertTrue(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))
-        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
-
-    assertFalse(new Trigger1(null).isCompatible(new Trigger2(null)));
-    assertFalse(new Trigger1(Arrays.<Trigger>asList(new Trigger1(null)))
-        .isCompatible(new Trigger1(Arrays.<Trigger>asList(new Trigger2(null)))));
-  }
-
-  private static class Trigger1 extends Trigger {
-
-    private Trigger1(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
-    protected Trigger getContinuationTrigger(
-        List<Trigger> continuationTriggers) {
-      return null;
-    }
-
-    @Override
-    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 {
-
-    private Trigger2(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public void onElement(Trigger.OnElementContext c) { }
-
-    @Override
-    public void onMerge(Trigger.OnMergeContext c) { }
-
-    @Override
-    protected Trigger getContinuationTrigger(
-        List<Trigger> continuationTriggers) {
-      return null;
-    }
-
-    @Override
-    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/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/TriggerTester.java
deleted file mode 100644
index 5fe17ad..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/reactors/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();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
new file mode 100644
index 0000000..907292c
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterAllStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterAllStateMachineTest {
+
+  private SimpleTriggerStateMachineTester<IntervalWindow> tester;
+
+  @Test
+  public void testT1FiresFirst() throws Exception {
+    tester = TriggerStateMachineTester.forTrigger(
+        AfterAllStateMachine.of(
+            AfterPaneStateMachine.elementCountAtLeast(1),
+            AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterAllStateMachine.of(
+            AfterPaneStateMachine.elementCountAtLeast(2),
+            AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            AfterAllStateMachine.of(
+                AfterWatermarkStateMachine.pastEndOfWindow(),
+                AfterPaneStateMachine.elementCountAtLeast(1)),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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 testToString() {
+    TriggerStateMachine trigger =
+        AfterAllStateMachine.of(
+            StubTriggerStateMachine.named("t1"), StubTriggerStateMachine.named("t2"));
+    assertEquals("AfterAll.of(t1, t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
new file mode 100644
index 0000000..4fae8f1
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+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 AfterEachStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterEachStateMachineTest {
+
+  private SimpleTriggerStateMachineTester<IntervalWindow> tester;
+
+  @Before
+  public void initMocks() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  /**
+   * Tests that the {@link AfterEachStateMachine} trigger fires and finishes the first trigger then
+   * the second.
+   */
+  @Test
+  public void testAfterEachInSequence() throws Exception {
+    tester = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.elementCountAtLeast(2))
+                .orFinally(AfterPaneStateMachine.elementCountAtLeast(3)),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.elementCountAtLeast(5))
+                .orFinally(AfterWatermarkStateMachine.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 testToString() {
+    TriggerStateMachine trigger = AfterEachStateMachine.inOrder(
+        StubTriggerStateMachine.named("t1"),
+        StubTriggerStateMachine.named("t2"),
+        StubTriggerStateMachine.named("t3"));
+
+    assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
new file mode 100644
index 0000000..453c8ff
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine;
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+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 AfterFirstStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterFirstStateMachineTest {
+
+  @Mock private OnceTriggerStateMachine mockTrigger1;
+  @Mock private OnceTriggerStateMachine mockTrigger2;
+  private SimpleTriggerStateMachineTester<IntervalWindow> tester;
+  private static TriggerStateMachine.TriggerContext anyTriggerContext() {
+    return Mockito.<TriggerStateMachine.TriggerContext>any();
+  }
+
+  @Before
+  public void initMocks() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testNeitherShouldFireFixedWindows() throws Exception {
+    tester =
+        TriggerStateMachineTester.forTrigger(
+            AfterFirstStateMachine.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 =
+        TriggerStateMachineTester.forTrigger(
+            AfterFirstStateMachine.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 = TriggerStateMachineTester.forTrigger(
+    AfterFirstStateMachine.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 = TriggerStateMachineTester.forTrigger(
+    AfterFirstStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterEachStateMachine.inOrder(
+            AfterFirstStateMachine.of(
+                AfterPaneStateMachine.elementCountAtLeast(5),
+                AfterWatermarkStateMachine.pastEndOfWindow()),
+            RepeatedlyStateMachine.forever(AfterPaneStateMachine.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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
new file mode 100644
index 0000000..4240174
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterPaneStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterPaneStateMachineTest {
+
+  SimpleTriggerStateMachineTester<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 = TriggerStateMachineTester.forTrigger(
+        AfterPaneStateMachine.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 {
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterPaneStateMachine.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 = TriggerStateMachineTester.forTrigger(
+        AfterPaneStateMachine.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 testToString() {
+    TriggerStateMachine trigger = AfterPaneStateMachine.elementCountAtLeast(5);
+    assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java
new file mode 100644
index 0000000..9fbf801
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterProcessingTimeStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterProcessingTimeStateMachineTest {
+
+  /**
+   * 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);
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterProcessingTimeStateMachine
+            .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 {
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterProcessingTimeStateMachine
+            .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 {
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterProcessingTimeStateMachine
+            .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));
+  }
+
+  /**
+   * Basic test of compatibility check between identical triggers.
+   */
+  @Test
+  public void testCompatibilityIdentical() throws Exception {
+    TriggerStateMachine t1 = AfterProcessingTimeStateMachine.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    TriggerStateMachine t2 = AfterProcessingTimeStateMachine.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    assertTrue(t1.isCompatible(t2));
+  }
+
+  @Test
+  public void testToString() {
+    TriggerStateMachine trigger = AfterProcessingTimeStateMachine.pastFirstElementInPane();
+    assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString());
+  }
+
+  @Test
+  public void testWithDelayToString() {
+    TriggerStateMachine trigger = AfterProcessingTimeStateMachine.pastFirstElementInPane()
+        .plusDelayOf(Duration.standardMinutes(5));
+
+    assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testBuiltUpToString() {
+    TriggerStateMachine trigger = AfterWatermarkStateMachine.pastEndOfWindow()
+        .withLateFirings(AfterProcessingTimeStateMachine
+            .pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(10)));
+
+    String expected = "AfterWatermark.pastEndOfWindow()"
+        + ".withLateFirings(AfterProcessingTime"
+        + ".pastFirstElementInPane()"
+        + ".plusDelayOf(10 minutes))";
+
+    assertEquals(expected, trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69b1efda/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
new file mode 100644
index 0000000..140bd62
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.triggers;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.core.triggers.TriggerStateMachineTester.SimpleTriggerStateMachineTester;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterSynchronizedProcessingTimeStateMachine}.
+ */
+@RunWith(JUnit4.class)
+public class AfterSynchronizedProcessingTimeStateMachineTest {
+
+  private TriggerStateMachine underTest = new AfterSynchronizedProcessingTimeStateMachine();
+
+  @Test
+  public void testAfterProcessingTimeWithFixedWindows() throws Exception {
+    Duration windowDuration = Duration.millis(10);
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterProcessingTimeStateMachine
+            .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);
+    SimpleTriggerStateMachineTester<IntervalWindow> tester = TriggerStateMachineTester.forTrigger(
+        AfterProcessingTimeStateMachine
+            .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));
+  }
+}


[09/17] incubator-beam git commit: Move triggers to runners-core

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index a960aa4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
+++ /dev/null
@@ -1,527 +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.windowing;
-
-import com.google.common.base.Joiner;
-import java.io.Serializable;
-import java.util.ArrayList;
-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.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.
- *
- * <p>See {@link org.apache.beam.sdk.transforms.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
- * elements as they are added. When the root trigger fires, the elements in the current pane will be
- * 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:
- * <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.
- * </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.
- * </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. */
-    public abstract boolean finishedInAnyMergingWindow();
-
-    /** Return true if the trigger is finished in all windows being merged. */
-    public abstract 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();
-  }
-
-  @Nullable
-  protected final List<Trigger> subTriggers;
-
-  protected Trigger(@Nullable List<Trigger> subTriggers) {
-    this.subTriggers = subTriggers;
-  }
-
-
-  /**
-   * 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 Iterable<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
-   * continue firing once (or multiple times).
-   */
-  public Trigger getContinuationTrigger() {
-    if (subTriggers == null) {
-      return getContinuationTrigger(null);
-    }
-
-    List<Trigger> subTriggerContinuations = new ArrayList<>();
-    for (Trigger subTrigger : subTriggers) {
-      subTriggerContinuations.add(subTrigger.getContinuationTrigger());
-    }
-    return getContinuationTrigger(subTriggerContinuations);
-  }
-
-  /**
-   * Return the {@link #getContinuationTrigger} of this {@code 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.
-   *
-   * <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.
-   */
-  public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window);
-
-  /**
-   * Returns whether this performs the same triggering as the given {@code Trigger}.
-   */
-  public boolean isCompatible(Trigger other) {
-    if (!getClass().equals(other.getClass())) {
-      return false;
-    }
-
-    if (subTriggers == null) {
-      return other.subTriggers == null;
-    } else if (other.subTriggers == null) {
-      return false;
-    } else if (subTriggers.size() != other.subTriggers.size()) {
-      return false;
-    }
-
-    for (int i = 0; i < subTriggers.size(); i++) {
-      if (!subTriggers.get(i).isCompatible(other.subTriggers.get(i))) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    String simpleName = getClass().getSimpleName();
-    if (getClass().getEnclosingClass() != null) {
-      simpleName = getClass().getEnclosingClass().getSimpleName() + "." + simpleName;
-    }
-    if (subTriggers == null || subTriggers.size() == 0) {
-      return simpleName;
-    } else {
-      return simpleName + "(" + Joiner.on(", ").join(subTriggers) + ")";
-    }
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!(obj instanceof Trigger)) {
-      return false;
-    }
-    Trigger that = (Trigger) obj;
-    return Objects.equals(getClass(), that.getClass())
-        && Objects.equals(subTriggers, that.subTriggers);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(getClass(), subTriggers);
-  }
-
-  /**
-   * Specify an ending condition for this trigger. If the {@code until} 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>For example the final firing of the following trigger may only have 1 element:
-   * <pre> {@code
-   * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
-   *     .orFinally(AfterPane.elementCountAtLeast(5))
-   * } </pre>
-   *
-   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same
-   * as {@code AfterFirst.of(t1, t2)}.
-   */
-  public Trigger 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.
-   */
-  public abstract static class OnceTrigger extends Trigger {
-    protected OnceTrigger(List<Trigger> subTriggers) {
-      super(subTriggers);
-    }
-
-    @Override
-    public final OnceTrigger getContinuationTrigger() {
-      Trigger continuation = super.getContinuationTrigger();
-      if (!(continuation instanceof OnceTrigger)) {
-        throw new IllegalStateException("Continuation of a OnceTrigger must be a OnceTrigger");
-      }
-      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/e4398e1e/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
deleted file mode 100644
index 088c499..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
+++ /dev/null
@@ -1,159 +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.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-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).
- */
-public class ExecutableTrigger implements Serializable {
-
-  /** Store the index assigned to this trigger. */
-  private final int triggerIndex;
-  private final int firstIndexAfterSubtree;
-  private final List<ExecutableTrigger> subTriggers = new ArrayList<>();
-  private final Trigger trigger;
-
-  public static <W extends BoundedWindow> ExecutableTrigger create(Trigger trigger) {
-    return create(trigger, 0);
-  }
-
-  private static <W extends BoundedWindow> ExecutableTrigger create(
-      Trigger trigger, int nextUnusedIndex) {
-    if (trigger instanceof OnceTrigger) {
-      return new ExecutableOnceTrigger((OnceTrigger) trigger, nextUnusedIndex);
-    } else {
-      return new ExecutableTrigger(trigger, nextUnusedIndex);
-    }
-  }
-
-  public static <W extends BoundedWindow> ExecutableTrigger createForOnceTrigger(
-      OnceTrigger trigger, int nextUnusedIndex) {
-    return new ExecutableOnceTrigger(trigger, nextUnusedIndex);
-  }
-
-  private ExecutableTrigger(Trigger trigger, int nextUnusedIndex) {
-    this.trigger = checkNotNull(trigger, "trigger must not be null");
-    this.triggerIndex = nextUnusedIndex++;
-
-    if (trigger.subTriggers() != null) {
-      for (Trigger subTrigger : trigger.subTriggers()) {
-        ExecutableTrigger subExecutable = create(subTrigger, nextUnusedIndex);
-        subTriggers.add(subExecutable);
-        nextUnusedIndex = subExecutable.firstIndexAfterSubtree;
-      }
-    }
-    firstIndexAfterSubtree = nextUnusedIndex;
-  }
-
-  public List<ExecutableTrigger> subTriggers() {
-    return subTriggers;
-  }
-
-  @Override
-  public String toString() {
-    return trigger.toString();
-  }
-
-  /**
-   * Return the underlying trigger specification corresponding to this {@code ExecutableTrigger}.
-   */
-  public Trigger getSpec() {
-    return trigger;
-  }
-
-  public int getTriggerIndex() {
-    return triggerIndex;
-  }
-
-  public final int getFirstIndexAfterSubtree() {
-    return firstIndexAfterSubtree;
-  }
-
-  public boolean isCompatible(ExecutableTrigger other) {
-    return trigger.isCompatible(other.trigger);
-  }
-
-  public ExecutableTrigger getSubTriggerContaining(int index) {
-    checkNotNull(subTriggers);
-    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
-        "Cannot find sub-trigger containing index not in this tree.");
-    ExecutableTrigger previous = null;
-    for (ExecutableTrigger subTrigger : subTriggers) {
-      if (index < subTrigger.triggerIndex) {
-        return previous;
-      }
-      previous = subTrigger;
-    }
-    return previous;
-  }
-
-  /**
-   * 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.
-   */
-  private static class ExecutableOnceTrigger extends ExecutableTrigger {
-
-    public ExecutableOnceTrigger(OnceTrigger trigger, int nextUnusedIndex) {
-      super(trigger, nextUnusedIndex);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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 6666ab9..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.
-   */
-  public boolean isFinished(ExecutableTrigger trigger);
-
-  /**
-   * Sets the fact that the trigger is finished.
-   */
-  public void setFinished(ExecutableTrigger trigger, boolean value);
-
-  /**
-   * Sets the trigger and all of its subtriggers to unfinished.
-   */
-  public void clearRecursively(ExecutableTrigger trigger);
-
-  /**
-   * Create an independent copy of this mutable {@link FinishedTriggers}.
-   */
-  public FinishedTriggers copy();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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/e4398e1e/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/e4398e1e/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
deleted file mode 100644
index 9e2c27d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
+++ /dev/null
@@ -1,66 +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.List;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.joda.time.Instant;
-
-/**
- * The trigger used with {@link Reshuffle} which triggers on every element
- * and never buffers state.
- *
- * @param <W> The kind of window that is being reshuffled.
- */
-public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
-
-  public ReshuffleTrigger() {
-    super(null);
-  }
-
-  @Override
-  public void onElement(Trigger.OnElementContext c) { }
-
-  @Override
-  public void onMerge(Trigger.OnMergeContext c) { }
-
-  @Override
-  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
-    return this;
-  }
-
-  @Override
-  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
-    throw new UnsupportedOperationException(
-        "ReshuffleTrigger should not be used outside of Reshuffle");
-  }
-
-  @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()";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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/e4398e1e/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
deleted file mode 100644
index b591229..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
+++ /dev/null
@@ -1,156 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link AfterAll}.
- */
-@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));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterAll.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(1))
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterAll = AfterAll.of(trigger1, trigger2);
-    assertEquals(
-        AfterAll.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
-        afterAll.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterAll.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
-    assertEquals("AfterAll.of(t1, t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index c413c6e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
+++ /dev/null
@@ -1,132 +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.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}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        AfterEach.inOrder(AfterWatermark.pastEndOfWindow(),
-                          AfterPane.elementCountAtLeast(4))
-            .getWatermarkThatGuaranteesFiring(window));
-
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterEach.inOrder(AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow())
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterEach = AfterEach.inOrder(trigger1, trigger2);
-    assertEquals(
-        Repeatedly.forever(AfterFirst.of(
-            trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger())),
-        afterEach.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterEach.inOrder(
-        StubTrigger.named("t1"),
-        StubTrigger.named("t2"),
-        StubTrigger.named("t3"));
-
-    assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 415060b..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
+++ /dev/null
@@ -1,181 +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.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}.
- */
-@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));
-
-    assertEquals(new Instant(9),
-        AfterFirst.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4))
-            .getWatermarkThatGuaranteesFiring(window));
-    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
-        AfterFirst.of(AfterPane.elementCountAtLeast(2), AfterPane.elementCountAtLeast(1))
-            .getWatermarkThatGuaranteesFiring(window));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
-    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
-    Trigger afterFirst = AfterFirst.of(trigger1, trigger2);
-    assertEquals(
-        AfterFirst.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
-        afterFirst.getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterFirst.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
-    assertEquals("AfterFirst.of(t1, t2)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 38d030e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
+++ /dev/null
@@ -1,132 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link AfterPane}.
- */
-@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,
-        AfterPane.elementCountAtLeast(1).getWatermarkThatGuaranteesFiring(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    assertEquals(
-        AfterPane.elementCountAtLeast(1),
-        AfterPane.elementCountAtLeast(100).getContinuationTrigger());
-    assertEquals(
-        AfterPane.elementCountAtLeast(1),
-        AfterPane.elementCountAtLeast(100).getContinuationTrigger().getContinuationTrigger());
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterPane.elementCountAtLeast(5);
-    assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e4398e1e/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
deleted file mode 100644
index 13a7acf..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
+++ /dev/null
@@ -1,187 +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.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;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests the {@link AfterProcessingTime}.
- */
-@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,
-        AfterProcessingTime.pastFirstElementInPane().getWatermarkThatGuaranteesFiring(
-            new IntervalWindow(new Instant(0), new Instant(10))));
-  }
-
-  @Test
-  public void testContinuation() throws Exception {
-    OnceTrigger firstElementPlus1 =
-        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.standardHours(1));
-    assertEquals(
-        new AfterSynchronizedProcessingTime(),
-        firstElementPlus1.getContinuationTrigger());
-  }
-
-  /**
-   * Basic test of compatibility check between identical triggers.
-   */
-  @Test
-  public void testCompatibilityIdentical() throws Exception {
-    Trigger t1 = AfterProcessingTime.pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(1L));
-    Trigger t2 = AfterProcessingTime.pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(1L));
-    assertTrue(t1.isCompatible(t2));
-  }
-
-  @Test
-  public void testToString() {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
-    assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString());
-  }
-
-  @Test
-  public void testWithDelayToString() {
-    Trigger trigger = AfterProcessingTime.pastFirstElementInPane()
-        .plusDelayOf(Duration.standardMinutes(5));
-
-    assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)",
-        trigger.toString());
-  }
-
-  @Test
-  public void testBuiltUpToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withLateFirings(AfterProcessingTime
-            .pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(10)));
-
-    String expected = "AfterWatermark.pastEndOfWindow()"
-        + ".withLateFirings(AfterProcessingTime"
-        + ".pastFirstElementInPane()"
-        + ".plusDelayOf(10 minutes))";
-
-    assertEquals(expected, trigger.toString());
-  }
-}


[17/17] incubator-beam git commit: This closes #1083

Posted by ke...@apache.org.
This closes #1083


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

Branch: refs/heads/master
Commit: e969f3d388dda4b991d36e5aa37d5ada71f593e3
Parents: 7322616 a64acb2
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 13 15:22:08 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 13 15:22:08 2016 -0700

----------------------------------------------------------------------
 .../core/triggers/AfterAllStateMachine.java     | 104 ++++
 .../AfterDelayFromFirstElementStateMachine.java | 322 ++++++++++++
 .../core/triggers/AfterEachStateMachine.java    | 125 +++++
 .../core/triggers/AfterFirstStateMachine.java   | 106 ++++
 .../core/triggers/AfterPaneStateMachine.java    | 132 +++++
 .../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 +++++++++
 .../runners/core/triggers/package-info.java     |  23 +
 .../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 ++++++++++++++++
 .../beam/sdk/util/ExecutableTriggerTest.java    | 127 -----
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 --
 .../sdk/util/FinishedTriggersProperties.java    | 110 ----
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ---
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  67 ---
 45 files changed, 6014 insertions(+), 419 deletions(-)
----------------------------------------------------------------------



[15/17] incubator-beam git commit: Restore prior trigger files, for temporary compatibility

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..a960aa4
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
@@ -0,0 +1,527 @@
+/*
+ * 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.windowing;
+
+import com.google.common.base.Joiner;
+import java.io.Serializable;
+import java.util.ArrayList;
+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.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.
+ *
+ * <p>See {@link org.apache.beam.sdk.transforms.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
+ * elements as they are added. When the root trigger fires, the elements in the current pane will be
+ * 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:
+ * <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.
+ * </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.
+ * </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. */
+    public abstract boolean finishedInAnyMergingWindow();
+
+    /** Return true if the trigger is finished in all windows being merged. */
+    public abstract 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();
+  }
+
+  @Nullable
+  protected final List<Trigger> subTriggers;
+
+  protected Trigger(@Nullable List<Trigger> subTriggers) {
+    this.subTriggers = subTriggers;
+  }
+
+
+  /**
+   * 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 Iterable<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
+   * continue firing once (or multiple times).
+   */
+  public Trigger getContinuationTrigger() {
+    if (subTriggers == null) {
+      return getContinuationTrigger(null);
+    }
+
+    List<Trigger> subTriggerContinuations = new ArrayList<>();
+    for (Trigger subTrigger : subTriggers) {
+      subTriggerContinuations.add(subTrigger.getContinuationTrigger());
+    }
+    return getContinuationTrigger(subTriggerContinuations);
+  }
+
+  /**
+   * Return the {@link #getContinuationTrigger} of this {@code 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.
+   *
+   * <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.
+   */
+  public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window);
+
+  /**
+   * Returns whether this performs the same triggering as the given {@code Trigger}.
+   */
+  public boolean isCompatible(Trigger other) {
+    if (!getClass().equals(other.getClass())) {
+      return false;
+    }
+
+    if (subTriggers == null) {
+      return other.subTriggers == null;
+    } else if (other.subTriggers == null) {
+      return false;
+    } else if (subTriggers.size() != other.subTriggers.size()) {
+      return false;
+    }
+
+    for (int i = 0; i < subTriggers.size(); i++) {
+      if (!subTriggers.get(i).isCompatible(other.subTriggers.get(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    String simpleName = getClass().getSimpleName();
+    if (getClass().getEnclosingClass() != null) {
+      simpleName = getClass().getEnclosingClass().getSimpleName() + "." + simpleName;
+    }
+    if (subTriggers == null || subTriggers.size() == 0) {
+      return simpleName;
+    } else {
+      return simpleName + "(" + Joiner.on(", ").join(subTriggers) + ")";
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof Trigger)) {
+      return false;
+    }
+    Trigger that = (Trigger) obj;
+    return Objects.equals(getClass(), that.getClass())
+        && Objects.equals(subTriggers, that.subTriggers);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getClass(), subTriggers);
+  }
+
+  /**
+   * Specify an ending condition for this trigger. If the {@code until} 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>For example the final firing of the following trigger may only have 1 element:
+   * <pre> {@code
+   * Repeatedly.forever(AfterPane.elementCountAtLeast(2))
+   *     .orFinally(AfterPane.elementCountAtLeast(5))
+   * } </pre>
+   *
+   * <p>Note that if {@code t1} is {@link OnceTrigger}, then {@code t1.orFinally(t2)} is the same
+   * as {@code AfterFirst.of(t1, t2)}.
+   */
+  public Trigger 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.
+   */
+  public abstract static class OnceTrigger extends Trigger {
+    protected OnceTrigger(List<Trigger> subTriggers) {
+      super(subTriggers);
+    }
+
+    @Override
+    public final OnceTrigger getContinuationTrigger() {
+      Trigger continuation = super.getContinuationTrigger();
+      if (!(continuation instanceof OnceTrigger)) {
+        throw new IllegalStateException("Continuation of a OnceTrigger must be a OnceTrigger");
+      }
+      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/a64acb2f/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
new file mode 100644
index 0000000..088c499
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
@@ -0,0 +1,159 @@
+/*
+ * 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.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+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).
+ */
+public class ExecutableTrigger implements Serializable {
+
+  /** Store the index assigned to this trigger. */
+  private final int triggerIndex;
+  private final int firstIndexAfterSubtree;
+  private final List<ExecutableTrigger> subTriggers = new ArrayList<>();
+  private final Trigger trigger;
+
+  public static <W extends BoundedWindow> ExecutableTrigger create(Trigger trigger) {
+    return create(trigger, 0);
+  }
+
+  private static <W extends BoundedWindow> ExecutableTrigger create(
+      Trigger trigger, int nextUnusedIndex) {
+    if (trigger instanceof OnceTrigger) {
+      return new ExecutableOnceTrigger((OnceTrigger) trigger, nextUnusedIndex);
+    } else {
+      return new ExecutableTrigger(trigger, nextUnusedIndex);
+    }
+  }
+
+  public static <W extends BoundedWindow> ExecutableTrigger createForOnceTrigger(
+      OnceTrigger trigger, int nextUnusedIndex) {
+    return new ExecutableOnceTrigger(trigger, nextUnusedIndex);
+  }
+
+  private ExecutableTrigger(Trigger trigger, int nextUnusedIndex) {
+    this.trigger = checkNotNull(trigger, "trigger must not be null");
+    this.triggerIndex = nextUnusedIndex++;
+
+    if (trigger.subTriggers() != null) {
+      for (Trigger subTrigger : trigger.subTriggers()) {
+        ExecutableTrigger subExecutable = create(subTrigger, nextUnusedIndex);
+        subTriggers.add(subExecutable);
+        nextUnusedIndex = subExecutable.firstIndexAfterSubtree;
+      }
+    }
+    firstIndexAfterSubtree = nextUnusedIndex;
+  }
+
+  public List<ExecutableTrigger> subTriggers() {
+    return subTriggers;
+  }
+
+  @Override
+  public String toString() {
+    return trigger.toString();
+  }
+
+  /**
+   * Return the underlying trigger specification corresponding to this {@code ExecutableTrigger}.
+   */
+  public Trigger getSpec() {
+    return trigger;
+  }
+
+  public int getTriggerIndex() {
+    return triggerIndex;
+  }
+
+  public final int getFirstIndexAfterSubtree() {
+    return firstIndexAfterSubtree;
+  }
+
+  public boolean isCompatible(ExecutableTrigger other) {
+    return trigger.isCompatible(other.trigger);
+  }
+
+  public ExecutableTrigger getSubTriggerContaining(int index) {
+    checkNotNull(subTriggers);
+    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
+        "Cannot find sub-trigger containing index not in this tree.");
+    ExecutableTrigger previous = null;
+    for (ExecutableTrigger subTrigger : subTriggers) {
+      if (index < subTrigger.triggerIndex) {
+        return previous;
+      }
+      previous = subTrigger;
+    }
+    return previous;
+  }
+
+  /**
+   * 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.
+   */
+  private static class ExecutableOnceTrigger extends ExecutableTrigger {
+
+    public ExecutableOnceTrigger(OnceTrigger trigger, int nextUnusedIndex) {
+      super(trigger, nextUnusedIndex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..6666ab9
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggers.java
@@ -0,0 +1,44 @@
+/*
+ * 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.
+   */
+  public boolean isFinished(ExecutableTrigger trigger);
+
+  /**
+   * Sets the fact that the trigger is finished.
+   */
+  public void setFinished(ExecutableTrigger trigger, boolean value);
+
+  /**
+   * Sets the trigger and all of its subtriggers to unfinished.
+   */
+  public void clearRecursively(ExecutableTrigger trigger);
+
+  /**
+   * Create an independent copy of this mutable {@link FinishedTriggers}.
+   */
+  public FinishedTriggers copy();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..4cd617f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
@@ -0,0 +1,67 @@
+/*
+ * 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/a64acb2f/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
new file mode 100644
index 0000000..a9feb73
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java
@@ -0,0 +1,72 @@
+/*
+ * 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/a64acb2f/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
new file mode 100644
index 0000000..9e2c27d
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java
@@ -0,0 +1,66 @@
+/*
+ * 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.List;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.joda.time.Instant;
+
+/**
+ * The trigger used with {@link Reshuffle} which triggers on every element
+ * and never buffers state.
+ *
+ * @param <W> The kind of window that is being reshuffled.
+ */
+public class ReshuffleTrigger<W extends BoundedWindow> extends Trigger {
+
+  public ReshuffleTrigger() {
+    super(null);
+  }
+
+  @Override
+  public void onElement(Trigger.OnElementContext c) { }
+
+  @Override
+  public void onMerge(Trigger.OnMergeContext c) { }
+
+  @Override
+  protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
+    return this;
+  }
+
+  @Override
+  public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
+    throw new UnsupportedOperationException(
+        "ReshuffleTrigger should not be used outside of Reshuffle");
+  }
+
+  @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()";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..e09aac2
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
@@ -0,0 +1,507 @@
+/*
+ * 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/a64acb2f/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
new file mode 100644
index 0000000..b591229
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterAll}.
+ */
+@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));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterAll.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(1))
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterAll = AfterAll.of(trigger1, trigger2);
+    assertEquals(
+        AfterAll.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
+        afterAll.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterAll.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
+    assertEquals("AfterAll.of(t1, t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..c413c6e
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.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}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        AfterEach.inOrder(AfterWatermark.pastEndOfWindow(),
+                          AfterPane.elementCountAtLeast(4))
+            .getWatermarkThatGuaranteesFiring(window));
+
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterEach.inOrder(AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow())
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterEach = AfterEach.inOrder(trigger1, trigger2);
+    assertEquals(
+        Repeatedly.forever(AfterFirst.of(
+            trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger())),
+        afterEach.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterEach.inOrder(
+        StubTrigger.named("t1"),
+        StubTrigger.named("t2"),
+        StubTrigger.named("t3"));
+
+    assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..415060b
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.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}.
+ */
+@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));
+
+    assertEquals(new Instant(9),
+        AfterFirst.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4))
+            .getWatermarkThatGuaranteesFiring(window));
+    assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE,
+        AfterFirst.of(AfterPane.elementCountAtLeast(2), AfterPane.elementCountAtLeast(1))
+            .getWatermarkThatGuaranteesFiring(window));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger trigger1 = AfterProcessingTime.pastFirstElementInPane();
+    OnceTrigger trigger2 = AfterWatermark.pastEndOfWindow();
+    Trigger afterFirst = AfterFirst.of(trigger1, trigger2);
+    assertEquals(
+        AfterFirst.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()),
+        afterFirst.getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterFirst.of(StubTrigger.named("t1"), StubTrigger.named("t2"));
+    assertEquals("AfterFirst.of(t1, t2)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..38d030e
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link AfterPane}.
+ */
+@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,
+        AfterPane.elementCountAtLeast(1).getWatermarkThatGuaranteesFiring(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    assertEquals(
+        AfterPane.elementCountAtLeast(1),
+        AfterPane.elementCountAtLeast(100).getContinuationTrigger());
+    assertEquals(
+        AfterPane.elementCountAtLeast(1),
+        AfterPane.elementCountAtLeast(100).getContinuationTrigger().getContinuationTrigger());
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterPane.elementCountAtLeast(5);
+    assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a64acb2f/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
new file mode 100644
index 0000000..13a7acf
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.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;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests the {@link AfterProcessingTime}.
+ */
+@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,
+        AfterProcessingTime.pastFirstElementInPane().getWatermarkThatGuaranteesFiring(
+            new IntervalWindow(new Instant(0), new Instant(10))));
+  }
+
+  @Test
+  public void testContinuation() throws Exception {
+    OnceTrigger firstElementPlus1 =
+        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.standardHours(1));
+    assertEquals(
+        new AfterSynchronizedProcessingTime(),
+        firstElementPlus1.getContinuationTrigger());
+  }
+
+  /**
+   * Basic test of compatibility check between identical triggers.
+   */
+  @Test
+  public void testCompatibilityIdentical() throws Exception {
+    Trigger t1 = AfterProcessingTime.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    Trigger t2 = AfterProcessingTime.pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(1L));
+    assertTrue(t1.isCompatible(t2));
+  }
+
+  @Test
+  public void testToString() {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane();
+    assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString());
+  }
+
+  @Test
+  public void testWithDelayToString() {
+    Trigger trigger = AfterProcessingTime.pastFirstElementInPane()
+        .plusDelayOf(Duration.standardMinutes(5));
+
+    assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)",
+        trigger.toString());
+  }
+
+  @Test
+  public void testBuiltUpToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow()
+        .withLateFirings(AfterProcessingTime
+            .pastFirstElementInPane()
+            .plusDelayOf(Duration.standardMinutes(10)));
+
+    String expected = "AfterWatermark.pastEndOfWindow()"
+        + ".withLateFirings(AfterProcessingTime"
+        + ".pastFirstElementInPane()"
+        + ".plusDelayOf(10 minutes))";
+
+    assertEquals(expected, trigger.toString());
+  }
+}