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/04/29 23:55:59 UTC

[01/17] incubator-beam git commit: Move InProcessRunner to its own module

Repository: incubator-beam
Updated Branches:
  refs/heads/master bba4c64d3 -> b9116ac42


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
deleted file mode 100644
index dfd857e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,324 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TupleTag;
-
-import org.hamcrest.Matchers;
-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 java.io.Serializable;
-
-/**
- * Tests for {@link ParDoSingleEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ParDoSingleEvaluatorFactoryTest implements Serializable {
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void testParDoInMemoryTransformEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-    PCollection<Integer> collection =
-        input.apply(
-            ParDo.of(
-                new DoFn<String, Integer>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.output(c.element().length());
-                  }
-                }));
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
-    when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
-    when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoSingleEvaluatorFactory()
-            .forApplication(
-                collection.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    assertThat(result.getCounters(), equalTo(counters));
-
-    assertThat(
-        outputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<Integer>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow(3),
-            WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
-            WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-  }
-
-  @Test
-  public void testSideOutputToUndeclaredSideOutputSucceeds() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-    final TupleTag<Integer> sideOutputTag = new TupleTag<Integer>() {};
-    PCollection<Integer> collection =
-        input.apply(
-            ParDo.of(
-                new DoFn<String, Integer>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.sideOutput(sideOutputTag, c.element().length());
-                  }
-                }));
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
-    when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
-    when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    TransformEvaluator<String> evaluator =
-        new ParDoSingleEvaluatorFactory()
-            .forApplication(
-                collection.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getOutputBundles(), Matchers.<UncommittedBundle<?>>containsInAnyOrder(outputBundle));
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    assertThat(result.getCounters(), equalTo(counters));
-  }
-
-  @Test
-  public void finishBundleWithStatePutsStateInResult() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
-        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
-    final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
-    final StateNamespace windowNs =
-        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
-    ParDo.Bound<String, KV<String, Integer>> pardo =
-        ParDo.of(
-            new DoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(StateNamespaces.global(), watermarkTag)
-                    .add(new Instant(124443L - c.element().length()));
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(
-                        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                        bagTag)
-                    .add(c.element());
-              }
-            });
-    PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, "myKey", null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoSingleEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
-    assertThat(result.getState(), not(nullValue()));
-    assertThat(
-        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
-        equalTo(new Instant(124438L)));
-    assertThat(
-        result.getState().state(windowNs, bagTag).read(),
-        containsInAnyOrder("foo", "bara", "bazam"));
-  }
-
-  @Test
-  public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    final TimerData addedTimer =
-        TimerData.of(
-            StateNamespaces.window(
-                IntervalWindow.getCoder(),
-                new IntervalWindow(
-                    new Instant(0).plus(Duration.standardMinutes(5)),
-                    new Instant(1)
-                        .plus(Duration.standardMinutes(5))
-                        .plus(Duration.standardHours(1)))),
-            new Instant(54541L),
-            TimeDomain.EVENT_TIME);
-    final TimerData deletedTimer =
-        TimerData.of(
-            StateNamespaces.window(
-                IntervalWindow.getCoder(),
-                new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
-            new Instant(3400000),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-
-    ParDo.Bound<String, KV<String, Integer>> pardo =
-        ParDo.of(
-            new DoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals().stateInternals();
-                c.windowingInternals()
-                    .timerInternals()
-                    .setTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0).plus(Duration.standardMinutes(5)),
-                                    new Instant(1)
-                                        .plus(Duration.standardMinutes(5))
-                                        .plus(Duration.standardHours(1)))),
-                            new Instant(54541L),
-                            TimeDomain.EVENT_TIME));
-                c.windowingInternals()
-                    .timerInternals()
-                    .deleteTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0),
-                                    new Instant(0).plus(Duration.standardHours(1)))),
-                            new Instant(3400000),
-                            TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
-              }
-            });
-    PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, "myKey", null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    TransformEvaluator<String> evaluator =
-        new ParDoSingleEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getTimerUpdate(),
-        equalTo(
-            TimerUpdate.builder("myKey").setTimer(addedTimer).deletedTimer(deletedTimer).build()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
deleted file mode 100644
index 239ce27..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,112 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIOTest;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link TextIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class TextIOShardedWriteFactoryTest {
-  @Rule public TemporaryFolder tmp = new TemporaryFolder();
-  private TextIOShardedWriteFactory factory;
-
-  @Before
-  public void setup() {
-    factory = new TextIOShardedWriteFactory();
-  }
-
-  @Test
-  public void originalWithoutShardingReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        TextIO.Write.to(file.getAbsolutePath()).withoutSharding();
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original = TextIO.Write.to(file.getAbsolutePath());
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    TextIO.Write.Bound<String> original =
-        TextIO.Write.to(file.getAbsolutePath()).withNumShards(1);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    TestPipeline p = TestPipeline.create();
-    String[] elems = new String[] {"foo", "bar", "baz"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-
-    p.run();
-    TextIOTest.assertOutputFiles(
-        elems, StringUtf8Coder.of(), 1, tmp, "foo", original.getShardNameTemplate());
-  }
-
-  @Test
-  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    TextIO.Write.Bound<String> original = TextIO.Write.to(file.getAbsolutePath()).withNumShards(3);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    TestPipeline p = TestPipeline.create();
-    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-    p.run();
-    TextIOTest.assertOutputFiles(
-        elems, StringUtf8Coder.of(), 3, tmp, "foo", original.getShardNameTemplate());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
deleted file mode 100644
index 33dbbdc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServicesTest.java
+++ /dev/null
@@ -1,136 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.any;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-
-/**
- * Tests for {@link TransformExecutorServices}.
- */
-@RunWith(JUnit4.class)
-public class TransformExecutorServicesTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private ExecutorService executorService;
-  private Map<TransformExecutor<?>, Boolean> scheduled;
-
-  @Before
-  public void setup() {
-    executorService = MoreExecutors.newDirectExecutorService();
-    scheduled = new ConcurrentHashMap<>();
-  }
-
-  @Test
-  public void parallelScheduleMultipleSchedulesBothImmediately() {
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> first = mock(TransformExecutor.class);
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> second = mock(TransformExecutor.class);
-
-    TransformExecutorService parallel =
-        TransformExecutorServices.parallel(executorService, scheduled);
-    parallel.schedule(first);
-    parallel.schedule(second);
-
-    verify(first).call();
-    verify(second).call();
-    assertThat(
-        scheduled,
-        Matchers.allOf(
-            Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true),
-            Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true)));
-
-    parallel.complete(first);
-    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
-    assertThat(
-        scheduled,
-        not(
-            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
-                Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
-    parallel.complete(second);
-    assertThat(scheduled.isEmpty(), is(true));
-  }
-
-  @Test
-  public void serialScheduleTwoWaitsForFirstToComplete() {
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> first = mock(TransformExecutor.class);
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> second = mock(TransformExecutor.class);
-
-    TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
-    serial.schedule(first);
-    verify(first).call();
-
-    serial.schedule(second);
-    verify(second, never()).call();
-
-    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true));
-    assertThat(
-        scheduled,
-        not(
-            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
-                Matchers.<TransformExecutor<?>>equalTo(second), any(Boolean.class))));
-
-    serial.complete(first);
-    verify(second).call();
-    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
-    assertThat(
-        scheduled,
-        not(
-            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
-                Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
-
-    serial.complete(second);
-  }
-
-  @Test
-  public void serialCompleteNotExecutingTaskThrows() {
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> first = mock(TransformExecutor.class);
-    @SuppressWarnings("unchecked")
-    TransformExecutor<Object> second = mock(TransformExecutor.class);
-
-    TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
-    serial.schedule(first);
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("unexpected currently executing");
-
-    serial.complete(second);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
deleted file mode 100644
index 31cb29a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorTest.java
+++ /dev/null
@@ -1,538 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.isA;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Tests for {@link TransformExecutor}.
- */
-@RunWith(JUnit4.class)
-public class TransformExecutorTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private PCollection<String> created;
-  private PCollection<KV<Integer, String>> downstream;
-
-  private CountDownLatch evaluatorCompleted;
-
-  private RegisteringCompletionCallback completionCallback;
-  private TransformExecutorService transformEvaluationState;
-  private BundleFactory bundleFactory;
-  @Mock private InProcessEvaluationContext evaluationContext;
-  @Mock private TransformEvaluatorRegistry registry;
-  private Map<TransformExecutor<?>, Boolean> scheduled;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-
-    bundleFactory = InProcessBundleFactory.create();
-
-    scheduled = new HashMap<>();
-    transformEvaluationState =
-        TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService(), scheduled);
-
-    evaluatorCompleted = new CountDownLatch(1);
-    completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
-
-    TestPipeline p = TestPipeline.create();
-    created = p.apply(Create.of("foo", "spam", "third"));
-    downstream = created.apply(WithKeys.<Integer, String>of(3));
-  }
-
-  @Test
-  public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
-    final AtomicBoolean finishCalled = new AtomicBoolean(false);
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-            throw new IllegalArgumentException("Shouldn't be called");
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            finishCalled.set(true);
-            return result;
-          }
-        };
-
-    when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<Object> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>emptyList(),
-            evaluationContext,
-            null,
-            created.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-    executor.call();
-
-    assertThat(finishCalled.get(), is(true));
-    assertThat(completionCallback.handledResult, equalTo(result));
-    assertThat(completionCallback.handledThrowable, is(nullValue()));
-    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
-  }
-
-  @Test
-  public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
-    final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {
-            elementsProcessed.add(element);
-            return;
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
-    WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
-    WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(created).add(foo).add(spam).add(third).commit(Instant.now());
-    when(
-            registry.<String>forApplication(
-                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<String> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>emptyList(),
-            evaluationContext,
-            inputBundle,
-            downstream.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-
-    Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-
-    assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
-    assertThat(completionCallback.handledResult, equalTo(result));
-    assertThat(completionCallback.handledThrowable, is(nullValue()));
-    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
-  }
-
-  @Test
-  public void processElementThrowsExceptionCallsback() throws Exception {
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
-    final Exception exception = new Exception();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {
-            throw exception;
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(created).add(foo).commit(Instant.now());
-    when(
-            registry.<String>forApplication(
-                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<String> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>emptyList(),
-            evaluationContext,
-            inputBundle,
-            downstream.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-    Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-
-    assertThat(completionCallback.handledResult, is(nullValue()));
-    assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
-    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
-  }
-
-  @Test
-  public void finishBundleThrowsExceptionCallsback() throws Exception {
-    final Exception exception = new Exception();
-    TransformEvaluator<String> evaluator =
-        new TransformEvaluator<String>() {
-          @Override
-          public void processElement(WindowedValue<String> element) throws Exception {}
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            throw exception;
-          }
-        };
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(created).commit(Instant.now());
-    when(
-            registry.<String>forApplication(
-                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<String> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>emptyList(),
-            evaluationContext,
-            inputBundle,
-            downstream.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-    Executors.newSingleThreadExecutor().submit(executor);
-
-    evaluatorCompleted.await();
-
-    assertThat(completionCallback.handledResult, is(nullValue()));
-    assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
-    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
-  }
-
-  @Test
-  public void duringCallGetThreadIsNonNull() throws Exception {
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
-    final CountDownLatch testLatch = new CountDownLatch(1);
-    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-            throw new IllegalArgumentException("Shouldn't be called");
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            testLatch.countDown();
-            evaluatorLatch.await();
-            return result;
-          }
-        };
-
-    when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<String> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>emptyList(),
-            evaluationContext,
-            null,
-            created.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-
-    Executors.newSingleThreadExecutor().submit(executor);
-    testLatch.await();
-    assertThat(executor.getThread(), not(nullValue()));
-
-    // Finish the execution so everything can get closed down cleanly.
-    evaluatorLatch.countDown();
-  }
-
-  @Test
-  public void callWithEnforcementAppliesEnforcement() throws Exception {
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
-
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<String> fooElem = WindowedValue.valueInGlobalWindow("foo");
-    WindowedValue<String> barElem = WindowedValue.valueInGlobalWindow("bar");
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(created).add(fooElem).add(barElem).commit(Instant.now());
-    when(
-            registry.forApplication(
-                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TestEnforcementFactory enforcement = new TestEnforcementFactory();
-    TransformExecutor<String> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>singleton(enforcement),
-            evaluationContext,
-            inputBundle,
-            downstream.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-
-    executor.call();
-    TestEnforcement<?> testEnforcement = enforcement.instance;
-    assertThat(
-        testEnforcement.beforeElements,
-        Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
-    assertThat(
-        testEnforcement.afterElements,
-        Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
-    assertThat(testEnforcement.finishedBundles, contains(result));
-  }
-
-  @Test
-  public void callWithEnforcementThrowsOnFinishPropagates() throws Exception {
-    PCollection<byte[]> pcBytes =
-        created.apply(
-            new PTransform<PCollection<String>, PCollection<byte[]>>() {
-              @Override
-              public PCollection<byte[]> apply(PCollection<String> input) {
-                return PCollection.<byte[]>createPrimitiveOutputInternal(
-                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-                    .setCoder(ByteArrayCoder.of());
-              }
-            });
-
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
-    final CountDownLatch testLatch = new CountDownLatch(1);
-    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
-
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {}
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            testLatch.countDown();
-            evaluatorLatch.await();
-            return result;
-          }
-        };
-
-    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
-    CommittedBundle<byte[]> inputBundle =
-        bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
-    when(
-            registry.forApplication(
-                pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<byte[]> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
-            evaluationContext,
-            inputBundle,
-            pcBytes.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-
-    Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
-    testLatch.await();
-    fooBytes.getValue()[0] = 'b';
-    evaluatorLatch.countDown();
-
-    thrown.expectCause(isA(IllegalMutationException.class));
-    task.get();
-  }
-
-  @Test
-  public void callWithEnforcementThrowsOnElementPropagates() throws Exception {
-    PCollection<byte[]> pcBytes =
-        created.apply(
-            new PTransform<PCollection<String>, PCollection<byte[]>>() {
-              @Override
-              public PCollection<byte[]> apply(PCollection<String> input) {
-                return PCollection.<byte[]>createPrimitiveOutputInternal(
-                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-                    .setCoder(ByteArrayCoder.of());
-              }
-            });
-
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
-    final CountDownLatch testLatch = new CountDownLatch(1);
-    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
-
-    TransformEvaluator<Object> evaluator =
-        new TransformEvaluator<Object>() {
-          @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-            testLatch.countDown();
-            evaluatorLatch.await();
-          }
-
-          @Override
-          public InProcessTransformResult finishBundle() throws Exception {
-            return result;
-          }
-        };
-
-    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
-    CommittedBundle<byte[]> inputBundle =
-        bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
-    when(
-            registry.forApplication(
-                pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
-        .thenReturn(evaluator);
-
-    TransformExecutor<byte[]> executor =
-        TransformExecutor.create(
-            registry,
-            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
-            evaluationContext,
-            inputBundle,
-            pcBytes.getProducingTransformInternal(),
-            completionCallback,
-            transformEvaluationState);
-
-    Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
-    testLatch.await();
-    fooBytes.getValue()[0] = 'b';
-    evaluatorLatch.countDown();
-
-    thrown.expectCause(isA(IllegalMutationException.class));
-    task.get();
-  }
-
-  private static class RegisteringCompletionCallback implements CompletionCallback {
-    private InProcessTransformResult handledResult = null;
-    private Throwable handledThrowable = null;
-    private final CountDownLatch onMethod;
-
-    private RegisteringCompletionCallback(CountDownLatch onMethod) {
-      this.onMethod = onMethod;
-    }
-
-    @Override
-    public CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
-      handledResult = result;
-      onMethod.countDown();
-      return CommittedResult.create(result, Collections.<CommittedBundle<?>>emptyList());
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
-      handledThrowable = t;
-      onMethod.countDown();
-    }
-  }
-
-  private static class TestEnforcementFactory implements ModelEnforcementFactory {
-    private TestEnforcement<?> instance;
-    @Override
-    public <T> TestEnforcement<T> forBundle(
-        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
-      TestEnforcement<T> newEnforcement = new TestEnforcement<>();
-      instance = newEnforcement;
-      return newEnforcement;
-    }
-  }
-
-  private static class TestEnforcement<T> implements ModelEnforcement<T> {
-    private final List<WindowedValue<T>> beforeElements = new ArrayList<>();
-    private final List<WindowedValue<T>> afterElements = new ArrayList<>();
-    private final List<InProcessTransformResult> finishedBundles = new ArrayList<>();
-
-    @Override
-    public void beforeElement(WindowedValue<T> element) {
-      beforeElements.add(element);
-    }
-
-    @Override
-    public void afterElement(WindowedValue<T> element) {
-      afterElements.add(element);
-    }
-
-    @Override
-    public void afterFinish(
-        CommittedBundle<T> input,
-        InProcessTransformResult result,
-        Iterable<? extends CommittedBundle<?>> outputs) {
-      finishedBundles.add(result);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
deleted file mode 100644
index 82657c0..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,334 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasSize;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.io.CountingSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.DateTime;
-import org.joda.time.Instant;
-import org.joda.time.ReadableInstant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-/**
- * Tests for {@link UnboundedReadEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class UnboundedReadEvaluatorFactoryTest {
-  private PCollection<Long> longs;
-  private TransformEvaluatorFactory factory;
-  private InProcessEvaluationContext context;
-  private UncommittedBundle<Long> output;
-
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Before
-  public void setup() {
-    UnboundedSource<Long, ?> source =
-        CountingSource.unboundedWithTimestampFn(new LongToInstantFn());
-    TestPipeline p = TestPipeline.create();
-    longs = p.apply(Read.from(source));
-
-    factory = new UnboundedReadEvaluatorFactory();
-    context = mock(InProcessEvaluationContext.class);
-    output = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(output);
-  }
-
-  @Test
-  public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
-    assertThat(
-        output.commit(Instant.now()).getElements(),
-        containsInAnyOrder(
-            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
-            tgw(0L)));
-  }
-
-  /**
-   * Demonstrate that multiple sequential creations will produce additional elements if the source
-   * can provide them.
-   */
-  @Test
-  public void unboundedSourceInMemoryTransformEvaluatorMultipleSequentialCalls() throws Exception {
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
-    assertThat(
-        output.commit(Instant.now()).getElements(),
-        containsInAnyOrder(
-            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
-            tgw(0L)));
-
-    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(secondOutput);
-    TransformEvaluator<?> secondEvaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
-    assertThat(
-        secondResult.getWatermarkHold(),
-        Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
-    assertThat(
-        secondOutput.commit(Instant.now()).getElements(),
-        containsInAnyOrder(tgw(11L), tgw(12L), tgw(14L), tgw(18L), tgw(19L), tgw(17L), tgw(16L),
-            tgw(15L), tgw(13L), tgw(10L)));
-  }
-
-  @Test
-  public void boundedSourceEvaluatorClosesReader() throws Exception {
-    TestUnboundedSource<Long> source =
-        new TestUnboundedSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
-
-    TestPipeline p = TestPipeline.create();
-    PCollection<Long> pcollection = p.apply(Read.from(source));
-    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
-    when(context.createRootBundle(pcollection)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
-    evaluator.finishBundle();
-    CommittedBundle<Long> committed = output.commit(Instant.now());
-    assertThat(ImmutableList.copyOf(committed.getElements()), hasSize(3));
-    assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
-  }
-
-  @Test
-  public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
-    TestUnboundedSource<Long> source = new TestUnboundedSource<>(BigEndianLongCoder.of());
-
-    TestPipeline p = TestPipeline.create();
-    PCollection<Long> pcollection = p.apply(Read.from(source));
-    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
-    when(context.createRootBundle(pcollection)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
-    evaluator.finishBundle();
-    CommittedBundle<Long> committed = output.commit(Instant.now());
-    assertThat(committed.getElements(), emptyIterable());
-    assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
-  }
-
-  // TODO: Once the source is split into multiple sources before evaluating, this test will have to
-  // be updated.
-  /**
-   * Demonstrate that only a single unfinished instance of TransformEvaluator can be created at a
-   * time, with other calls returning an empty evaluator.
-   */
-  @Test
-  public void unboundedSourceWithMultipleSimultaneousEvaluatorsIndependent() throws Exception {
-    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
-
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
-    TransformEvaluator<?> secondEvaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
-    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
-    InProcessTransformResult result = evaluator.finishBundle();
-
-    assertThat(
-        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
-    assertThat(
-        output.commit(Instant.now()).getElements(),
-        containsInAnyOrder(
-            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
-            tgw(0L)));
-
-    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-    assertThat(secondOutput.commit(Instant.now()).getElements(), emptyIterable());
-  }
-
-  /**
-   * A terse alias for producing timestamped longs in the {@link GlobalWindow}, where
-   * the timestamp is the epoch offset by the value of the element.
-   */
-  private static WindowedValue<Long> tgw(Long elem) {
-    return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem));
-  }
-
-  private static class LongToInstantFn implements SerializableFunction<Long, Instant> {
-    @Override
-    public Instant apply(Long input) {
-      return new Instant(input);
-    }
-  }
-
-  private static class TestUnboundedSource<T> extends UnboundedSource<T, TestCheckpointMark> {
-    static int readerClosedCount;
-    private final Coder<T> coder;
-    private final List<T> elems;
-
-    public TestUnboundedSource(Coder<T> coder, T... elems) {
-      readerClosedCount = 0;
-      this.coder = coder;
-      this.elems = Arrays.asList(elems);
-    }
-
-    @Override
-    public List<? extends UnboundedSource<T, TestCheckpointMark>> generateInitialSplits(
-        int desiredNumSplits, PipelineOptions options) throws Exception {
-      return ImmutableList.of(this);
-    }
-
-    @Override
-    public UnboundedSource.UnboundedReader<T> createReader(
-        PipelineOptions options, TestCheckpointMark checkpointMark) {
-      return new TestUnboundedReader(elems);
-    }
-
-    @Override
-    @Nullable
-    public Coder<TestCheckpointMark> getCheckpointMarkCoder() {
-      return new TestCheckpointMark.Coder();
-    }
-
-    @Override
-    public void validate() {}
-
-    @Override
-    public Coder<T> getDefaultOutputCoder() {
-      return coder;
-    }
-
-    private class TestUnboundedReader extends UnboundedReader<T> {
-      private final List<T> elems;
-      private int index;
-
-      public TestUnboundedReader(List<T> elems) {
-        this.elems = elems;
-        this.index = -1;
-      }
-
-      @Override
-      public boolean start() throws IOException {
-        return advance();
-      }
-
-      @Override
-      public boolean advance() throws IOException {
-        if (index + 1 < elems.size()) {
-          index++;
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public Instant getWatermark() {
-        return Instant.now();
-      }
-
-      @Override
-      public CheckpointMark getCheckpointMark() {
-        return new TestCheckpointMark();
-      }
-
-      @Override
-      public UnboundedSource<T, ?> getCurrentSource() {
-        TestUnboundedSource<T> source = TestUnboundedSource.this;
-        return source;
-      }
-
-      @Override
-      public T getCurrent() throws NoSuchElementException {
-        return elems.get(index);
-      }
-
-      @Override
-      public Instant getCurrentTimestamp() throws NoSuchElementException {
-        return Instant.now();
-      }
-
-      @Override
-      public void close() throws IOException {
-        readerClosedCount++;
-      }
-    }
-  }
-
-  private static class TestCheckpointMark implements CheckpointMark {
-    @Override
-    public void finalizeCheckpoint() throws IOException {}
-
-    public static class Coder extends AtomicCoder<TestCheckpointMark> {
-      @Override
-      public void encode(
-          TestCheckpointMark value,
-          OutputStream outStream,
-          org.apache.beam.sdk.coders.Coder.Context context)
-          throws CoderException, IOException {}
-
-      @Override
-      public TestCheckpointMark decode(
-          InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-          throws CoderException, IOException {
-        return new TestCheckpointMark();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.java
deleted file mode 100644
index 05346dc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactoryTest.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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ViewEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ViewEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void testInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bar"));
-    CreatePCollectionView<String, Iterable<String>> createView =
-        CreatePCollectionView.of(
-            PCollectionViews.iterableView(p, input.getWindowingStrategy(), StringUtf8Coder.of()));
-    PCollection<Iterable<String>> concat =
-        input.apply(WithKeys.<Void, String>of((Void) null))
-            .setCoder(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of()))
-            .apply(GroupByKey.<Void, String>create())
-            .apply(Values.<Iterable<String>>create());
-    PCollectionView<Iterable<String>> view =
-        concat.apply(new ViewEvaluatorFactory.WriteView<>(createView));
-
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
-    TestViewWriter<String, Iterable<String>> viewWriter = new TestViewWriter<>();
-    when(context.createPCollectionViewWriter(concat, view)).thenReturn(viewWriter);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-    TransformEvaluator<Iterable<String>> evaluator =
-        new ViewEvaluatorFactory()
-            .forApplication(view.getProducingTransformInternal(), inputBundle, context);
-
-    evaluator.processElement(
-        WindowedValue.<Iterable<String>>valueInGlobalWindow(ImmutableList.of("foo", "bar")));
-    assertThat(viewWriter.latest, nullValue());
-
-    evaluator.finishBundle();
-    assertThat(
-        viewWriter.latest,
-        containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar")));
-  }
-
-  private static class TestViewWriter<ElemT, ViewT> implements PCollectionViewWriter<ElemT, ViewT> {
-    private Iterable<WindowedValue<ElemT>> latest;
-
-    @Override
-    public void add(Iterable<WindowedValue<ElemT>> values) {
-      latest = values;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
deleted file mode 100644
index 3b36bc5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutorTest.java
+++ /dev/null
@@ -1,128 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollection;
-
-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 java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for {@link WatermarkCallbackExecutor}.
- */
-@RunWith(JUnit4.class)
-public class WatermarkCallbackExecutorTest {
-  private WatermarkCallbackExecutor executor = WatermarkCallbackExecutor.create();
-  private AppliedPTransform<?, ?, ?> create;
-  private AppliedPTransform<?, ?, ?> sum;
-
-  @Before
-  public void setup() {
-    TestPipeline p = TestPipeline.create();
-    PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
-    create = created.getProducingTransformInternal();
-    sum = created.apply(Sum.integersGlobally()).getProducingTransformInternal();
-  }
-
-  @Test
-  public void onGuaranteedFiringFiresAfterTrigger() throws Exception {
-    CountDownLatch latch = new CountDownLatch(1);
-    executor.callOnGuaranteedFiring(
-        create,
-        GlobalWindow.INSTANCE,
-        WindowingStrategy.globalDefault(),
-        new CountDownLatchCallback(latch));
-
-    executor.fireForWatermark(create, BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
-  }
-
-  @Test
-  public void multipleCallbacksShouldFireFires() throws Exception {
-    CountDownLatch latch = new CountDownLatch(2);
-    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
-    IntervalWindow window =
-        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
-    executor.callOnGuaranteedFiring(
-        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-    executor.callOnGuaranteedFiring(
-        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
-    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(10)));
-    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
-  }
-
-  @Test
-  public void noCallbacksShouldFire() throws Exception {
-    CountDownLatch latch = new CountDownLatch(1);
-    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
-    IntervalWindow window =
-        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
-    executor.callOnGuaranteedFiring(
-        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
-    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(5)));
-    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
-  }
-
-  @Test
-  public void unrelatedStepShouldNotFire() throws Exception {
-    CountDownLatch latch = new CountDownLatch(1);
-    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
-    IntervalWindow window =
-        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
-    executor.callOnGuaranteedFiring(
-        sum, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
-
-    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(20)));
-    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
-  }
-
-  private static class CountDownLatchCallback implements Runnable {
-    private final CountDownLatch latch;
-
-    public CountDownLatchCallback(CountDownLatch latch) {
-      this.latch = latch;
-    }
-
-    @Override
-    public void run() {
-      latch.countDown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
deleted file mode 100644
index d41825d..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,222 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-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.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.Window.Bound;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-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.MockitoAnnotations;
-
-/**
- * Tests for {@link WindowEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class WindowEvaluatorFactoryTest {
-  private static final Instant EPOCH = new Instant(0);
-
-  private PCollection<Long> input;
-  private WindowEvaluatorFactory factory;
-
-  @Mock private InProcessEvaluationContext evaluationContext;
-
-  private BundleFactory bundleFactory;
-
-  private WindowedValue<Long> first =
-      WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L));
-  private WindowedValue<Long> second =
-      WindowedValue.timestampedValueInGlobalWindow(
-          Long.valueOf(1L), EPOCH.plus(Duration.standardDays(3)));
-  private WindowedValue<Long> third =
-      WindowedValue.of(
-          Long.valueOf(2L),
-          new Instant(-10L),
-          new IntervalWindow(new Instant(-100), EPOCH),
-          PaneInfo.NO_FIRING);
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    TestPipeline p = TestPipeline.create();
-    input = p.apply(Create.of(1L, 2L, 3L));
-
-    bundleFactory = InProcessBundleFactory.create();
-    factory = new WindowEvaluatorFactory();
-  }
-
-  @Test
-  public void nullWindowFunSucceeds() throws Exception {
-    Bound<Long> transform =
-        Window.<Long>triggering(
-                AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1)))
-            .accumulatingFiredPanes();
-    PCollection<Long> triggering = input.apply(transform);
-
-    CommittedBundle<Long> inputBundle = createInputBundle();
-
-    UncommittedBundle<Long> outputBundle = createOutputBundle(triggering, inputBundle);
-
-    InProcessTransformResult result = runEvaluator(triggering, inputBundle, transform);
-
-    assertThat(
-        Iterables.getOnlyElement(result.getOutputBundles()),
-        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
-    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(third, first, second));
-  }
-
-  @Test
-  public void singleWindowFnSucceeds() throws Exception {
-    Duration windowDuration = Duration.standardDays(7);
-    Bound<Long> transform = Window.<Long>into(FixedWindows.of(windowDuration));
-    PCollection<Long> windowed = input.apply(transform);
-
-    CommittedBundle<Long> inputBundle = createInputBundle();
-
-    UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
-
-    BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
-    BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
-
-    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
-
-    assertThat(
-        Iterables.getOnlyElement(result.getOutputBundles()),
-        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
-    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
-
-    WindowedValue<Long> expectedNewFirst =
-        WindowedValue.of(3L, new Instant(2L), firstSecondWindow, PaneInfo.NO_FIRING);
-    WindowedValue<Long> expectedNewSecond =
-        WindowedValue.of(
-            1L, EPOCH.plus(Duration.standardDays(3)), firstSecondWindow, PaneInfo.NO_FIRING);
-    WindowedValue<Long> expectedNewThird =
-        WindowedValue.of(2L, new Instant(-10L), thirdWindow, PaneInfo.NO_FIRING);
-    assertThat(
-        committed.getElements(),
-        containsInAnyOrder(expectedNewFirst, expectedNewSecond, expectedNewThird));
-  }
-
-  @Test
-  public void multipleWindowsWindowFnSucceeds() throws Exception {
-    Duration windowDuration = Duration.standardDays(6);
-    Duration slidingBy = Duration.standardDays(3);
-    Bound<Long> transform = Window.into(SlidingWindows.of(windowDuration).every(slidingBy));
-    PCollection<Long> windowed = input.apply(transform);
-
-    CommittedBundle<Long> inputBundle = createInputBundle();
-    UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
-
-    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
-
-    assertThat(
-        Iterables.getOnlyElement(result.getOutputBundles()),
-        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
-    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
-
-    BoundedWindow w1 = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
-    BoundedWindow w2 =
-        new IntervalWindow(EPOCH.plus(slidingBy), EPOCH.plus(slidingBy).plus(windowDuration));
-    BoundedWindow wMinus1 = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
-    BoundedWindow wMinusSlide =
-        new IntervalWindow(EPOCH.minus(windowDuration).plus(slidingBy), EPOCH.plus(slidingBy));
-
-    WindowedValue<Long> expectedFirst =
-        WindowedValue.of(
-            first.getValue(),
-            first.getTimestamp(),
-            ImmutableSet.of(w1, wMinusSlide),
-            PaneInfo.NO_FIRING);
-    WindowedValue<Long> expectedSecond =
-        WindowedValue.of(
-            second.getValue(), second.getTimestamp(), ImmutableSet.of(w1, w2), PaneInfo.NO_FIRING);
-    WindowedValue<Long> expectedThird =
-        WindowedValue.of(
-            third.getValue(),
-            third.getTimestamp(),
-            ImmutableSet.of(wMinus1, wMinusSlide),
-            PaneInfo.NO_FIRING);
-
-    assertThat(
-        committed.getElements(), containsInAnyOrder(expectedFirst, expectedSecond, expectedThird));
-  }
-
-  private CommittedBundle<Long> createInputBundle() {
-    CommittedBundle<Long> inputBundle =
-        bundleFactory
-            .createRootBundle(input)
-            .add(first)
-            .add(second)
-            .add(third)
-            .commit(Instant.now());
-    return inputBundle;
-  }
-
-  private UncommittedBundle<Long> createOutputBundle(
-      PCollection<Long> output, CommittedBundle<Long> inputBundle) {
-    UncommittedBundle<Long> outputBundle = bundleFactory.createBundle(inputBundle, output);
-    when(evaluationContext.createBundle(inputBundle, output)).thenReturn(outputBundle);
-    return outputBundle;
-  }
-
-  private InProcessTransformResult runEvaluator(
-      PCollection<Long> windowed,
-      CommittedBundle<Long> inputBundle,
-      Window.Bound<Long> windowTransform /* Required while Window.Bound is a composite */)
-      throws Exception {
-    TransformEvaluator<Long> evaluator =
-        factory.forApplication(
-            AppliedPTransform.of("Window", input, windowed, windowTransform),
-            inputBundle,
-            evaluationContext);
-
-    evaluator.processElement(first);
-    evaluator.processElement(second);
-    evaluator.processElement(third);
-    InProcessTransformResult result = evaluator.finishBundle();
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index a8972c2..54c841e 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -56,5 +56,4 @@
       </modules>
     </profile>
   </profiles>
-
 </project>


[13/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
new file mode 100644
index 0000000..1c51738
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.DoFnRunner;
+import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class ParDoInProcessEvaluator<T> implements TransformEvaluator<T> {
+  public static <InputT, OutputT> ParDoInProcessEvaluator<InputT> create(
+      InProcessEvaluationContext evaluationContext,
+      CommittedBundle<InputT> inputBundle,
+      AppliedPTransform<PCollection<InputT>, ?, ?> application,
+      DoFn<InputT, OutputT> fn,
+      List<PCollectionView<?>> sideInputs,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      Map<TupleTag<?>, PCollection<?>> outputs) {
+    InProcessExecutionContext executionContext =
+        evaluationContext.getExecutionContext(application, inputBundle.getKey());
+    String stepName = evaluationContext.getStepName(application);
+    InProcessStepContext stepContext =
+        executionContext.getOrCreateStepContext(stepName, stepName);
+
+    CounterSet counters = evaluationContext.createCounterSet();
+
+    Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
+    for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
+      outputBundles.put(
+          outputEntry.getKey(),
+          evaluationContext.createBundle(inputBundle, outputEntry.getValue()));
+    }
+
+    DoFnRunner<InputT, OutputT> runner =
+        DoFnRunners.createDefault(
+            evaluationContext.getPipelineOptions(),
+            SerializableUtils.clone(fn),
+            evaluationContext.createSideInputReader(sideInputs),
+            BundleOutputManager.create(outputBundles),
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            counters.getAddCounterMutator(),
+            application.getInput().getWindowingStrategy());
+
+    try {
+      runner.startBundle();
+    } catch (Exception e) {
+      throw UserCodeException.wrap(e);
+    }
+
+    return new ParDoInProcessEvaluator<>(
+        runner, application, counters, outputBundles.values(), stepContext);
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private final DoFnRunner<T, ?> fnRunner;
+  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
+  private final CounterSet counters;
+  private final Collection<UncommittedBundle<?>> outputBundles;
+  private final InProcessStepContext stepContext;
+
+  private ParDoInProcessEvaluator(
+      DoFnRunner<T, ?> fnRunner,
+      AppliedPTransform<PCollection<T>, ?, ?> transform,
+      CounterSet counters,
+      Collection<UncommittedBundle<?>> outputBundles,
+      InProcessStepContext stepContext) {
+    this.fnRunner = fnRunner;
+    this.transform = transform;
+    this.counters = counters;
+    this.outputBundles = outputBundles;
+    this.stepContext = stepContext;
+  }
+
+  @Override
+  public void processElement(WindowedValue<T> element) {
+    try {
+      fnRunner.processElement(element);
+    } catch (Exception e) {
+      throw UserCodeException.wrap(e);
+    }
+  }
+
+  @Override
+  public InProcessTransformResult finishBundle() {
+    try {
+      fnRunner.finishBundle();
+    } catch (Exception e) {
+      throw UserCodeException.wrap(e);
+    }
+    StepTransformResult.Builder resultBuilder;
+    CopyOnAccessInMemoryStateInternals<?> state = stepContext.commitState();
+    if (state != null) {
+      resultBuilder =
+          StepTransformResult.withHold(transform, state.getEarliestWatermarkHold())
+              .withState(state);
+    } else {
+      resultBuilder = StepTransformResult.withoutHold(transform);
+    }
+    return resultBuilder
+        .addOutput(outputBundles)
+        .withTimerUpdate(stepContext.getTimerUpdate())
+        .withCounters(counters)
+        .build();
+  }
+
+  static class BundleOutputManager implements OutputManager {
+    private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
+    private final Map<TupleTag<?>, List<?>> undeclaredOutputs;
+
+    public static BundleOutputManager create(Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
+      return new BundleOutputManager(outputBundles);
+    }
+
+    private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
+      this.bundles = bundles;
+      undeclaredOutputs = new HashMap<>();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      @SuppressWarnings("rawtypes")
+      UncommittedBundle bundle = bundles.get(tag);
+      if (bundle == null) {
+        List undeclaredContents = undeclaredOutputs.get(tag);
+        if (undeclaredContents == null) {
+          undeclaredContents = new ArrayList<T>();
+          undeclaredOutputs.put(tag, undeclaredContents);
+        }
+        undeclaredContents.add(output);
+      } else {
+        bundle.add(output);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
new file mode 100644
index 0000000..ae8ac6f
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.Map;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link BoundMulti} primitive {@link PTransform}.
+ */
+class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <T> TransformEvaluator<T> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    TransformEvaluator<T> evaluator =
+        createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
+    return evaluator;
+  }
+
+  private static <InT, OuT> ParDoInProcessEvaluator<InT> createMultiEvaluator(
+      AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
+      CommittedBundle<InT> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    Map<TupleTag<?>, PCollection<?>> outputs = application.getOutput().getAll();
+    DoFn<InT, OuT> fn = application.getTransform().getFn();
+
+    return ParDoInProcessEvaluator.create(
+        evaluationContext,
+        inputBundle,
+        application,
+        fn,
+        application.getTransform().getSideInputs(),
+        application.getTransform().getMainOutputTag(),
+        application.getTransform().getSideOutputTags().getAll(),
+        outputs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
new file mode 100644
index 0000000..989ae51
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.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.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo.Bound;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Collections;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link Bound ParDo.Bound} primitive {@link PTransform}.
+ */
+class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <T> TransformEvaluator<T> forApplication(
+      final AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    TransformEvaluator<T> evaluator =
+        createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
+    return evaluator;
+  }
+
+  private static <InputT, OutputT> ParDoInProcessEvaluator<InputT> createSingleEvaluator(
+      @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
+          Bound<InputT, OutputT>> application,
+      CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
+    TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
+
+    return ParDoInProcessEvaluator.create(
+        evaluationContext,
+        inputBundle,
+        application,
+        application.getTransform().getFn(),
+        application.getTransform().getSideInputs(),
+        mainOutputTag,
+        Collections.<TupleTag<?>>emptyList(),
+        ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
new file mode 100644
index 0000000..aef62b2
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+
+class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
+  public static <InputT> PassthroughTransformEvaluator<InputT> create(
+      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
+    return new PassthroughTransformEvaluator<>(transform, output);
+  }
+
+  private final AppliedPTransform<?, ?, ?> transform;
+  private final UncommittedBundle<InputT> output;
+
+  private PassthroughTransformEvaluator(
+      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
+    this.transform = transform;
+    this.output = output;
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> element) throws Exception {
+    output.add(element);
+  }
+
+  @Override
+  public InProcessTransformResult finishBundle() throws Exception {
+    return StepTransformResult.withoutHold(transform).addOutput(output).build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
new file mode 100644
index 0000000..4687f85
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.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.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Partition;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PDone;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * A write that explicitly controls its number of output shards.
+ */
+abstract class ShardControlledWrite<InputT>
+    extends ForwardingPTransform<PCollection<InputT>, PDone> {
+  @Override
+  public PDone apply(PCollection<InputT> input) {
+    int numShards = getNumShards();
+    checkArgument(
+        numShards >= 1,
+        "%s should only be applied if the output has a controlled number of shards (> 1); got %s",
+        getClass().getSimpleName(),
+        getNumShards());
+    PCollectionList<InputT> shards =
+        input.apply(
+            "PartitionInto" + numShards + "Shards",
+            Partition.of(getNumShards(), new RandomSeedPartitionFn<InputT>()));
+    for (int i = 0; i < shards.size(); i++) {
+      PCollection<InputT> shard = shards.get(i);
+      PTransform<? super PCollection<InputT>, PDone> writeShard = getSingleShardTransform(i);
+      shard.apply(String.format("%s(Shard:%s)", writeShard.getName(), i), writeShard);
+    }
+    return PDone.in(input.getPipeline());
+  }
+
+  /**
+   * Returns the number of shards this {@link PTransform} should write to.
+   */
+  abstract int getNumShards();
+
+  /**
+   * Returns a {@link PTransform} that performs a write to the shard with the specified shard
+   * number.
+   *
+   * <p>This method will be called n times, where n is the value of {@link #getNumShards()}, for
+   * shard numbers {@code [0...n)}.
+   */
+  abstract PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum);
+
+  private static class RandomSeedPartitionFn<T> implements Partition.PartitionFn<T> {
+    int nextPartition = -1;
+    @Override
+    public int partitionFor(T elem, int numPartitions) {
+      if (nextPartition < 0) {
+        nextPartition = ThreadLocalRandom.current().nextInt(numPartitions);
+      }
+      nextPartition++;
+      nextPartition %= numPartitions;
+      return nextPartition;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
new file mode 100644
index 0000000..1c7cf6c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import com.google.common.base.MoreObjects;
+
+import java.util.Objects;
+
+/**
+ * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
+ * per-step in a keyed manner (e.g. State).
+ */
+final class StepAndKey {
+  private final AppliedPTransform<?, ?, ?> step;
+  private final Object key;
+
+  /**
+   * Create a new {@link StepAndKey} with the provided step and key.
+   */
+  public static StepAndKey of(AppliedPTransform<?, ?, ?> step, Object key) {
+    return new StepAndKey(step, key);
+  }
+
+  private StepAndKey(AppliedPTransform<?, ?, ?> step, Object key) {
+    this.step = step;
+    this.key = key;
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(StepAndKey.class)
+        .add("step", step.getFullName())
+        .add("key", key)
+        .toString();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(step, key);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == this) {
+      return true;
+    } else if (!(other instanceof StepAndKey)) {
+      return false;
+    } else {
+      StepAndKey that = (StepAndKey) other;
+      return Objects.equals(this.step, that.step)
+          && Objects.equals(this.key, that.key);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
new file mode 100644
index 0000000..46e7d04
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.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.direct;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * An immutable {@link InProcessTransformResult}.
+ */
+public class StepTransformResult implements InProcessTransformResult {
+  private final AppliedPTransform<?, ?, ?> transform;
+  private final Iterable<? extends UncommittedBundle<?>> bundles;
+  @Nullable private final CopyOnAccessInMemoryStateInternals<?> state;
+  private final TimerUpdate timerUpdate;
+  @Nullable private final CounterSet counters;
+  private final Instant watermarkHold;
+
+  private StepTransformResult(
+      AppliedPTransform<?, ?, ?> transform,
+      Iterable<? extends UncommittedBundle<?>> outputBundles,
+      CopyOnAccessInMemoryStateInternals<?> state,
+      TimerUpdate timerUpdate,
+      CounterSet counters,
+      Instant watermarkHold) {
+    this.transform = checkNotNull(transform);
+    this.bundles = checkNotNull(outputBundles);
+    this.state = state;
+    this.timerUpdate = checkNotNull(timerUpdate);
+    this.counters = counters;
+    this.watermarkHold = checkNotNull(watermarkHold);
+  }
+
+  @Override
+  public Iterable<? extends UncommittedBundle<?>> getOutputBundles() {
+    return bundles;
+  }
+
+  @Override
+  public CounterSet getCounters() {
+    return counters;
+  }
+
+  @Override
+  public AppliedPTransform<?, ?, ?> getTransform() {
+    return transform;
+  }
+
+  @Override
+  public Instant getWatermarkHold() {
+    return watermarkHold;
+  }
+
+  @Nullable
+  @Override
+  public CopyOnAccessInMemoryStateInternals<?> getState() {
+    return state;
+  }
+
+  @Override
+  public TimerUpdate getTimerUpdate() {
+    return timerUpdate;
+  }
+
+  public static Builder withHold(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
+    return new Builder(transform, watermarkHold);
+  }
+
+  public static Builder withoutHold(AppliedPTransform<?, ?, ?> transform) {
+    return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(StepTransformResult.class)
+        .add("transform", transform)
+        .toString();
+  }
+
+  /**
+   * A builder for creating instances of {@link StepTransformResult}.
+   */
+  public static class Builder {
+    private final AppliedPTransform<?, ?, ?> transform;
+    private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
+    private CopyOnAccessInMemoryStateInternals<?> state;
+    private TimerUpdate timerUpdate;
+    private CounterSet counters;
+    private final Instant watermarkHold;
+
+    private Builder(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
+      this.transform = transform;
+      this.watermarkHold = watermarkHold;
+      this.bundlesBuilder = ImmutableList.builder();
+      this.timerUpdate = TimerUpdate.builder(null).build();
+    }
+
+    public StepTransformResult build() {
+      return new StepTransformResult(
+          transform,
+          bundlesBuilder.build(),
+          state,
+          timerUpdate,
+          counters,
+          watermarkHold);
+    }
+
+    public Builder withCounters(CounterSet counters) {
+      this.counters = counters;
+      return this;
+    }
+
+    public Builder withState(CopyOnAccessInMemoryStateInternals<?> state) {
+      this.state = state;
+      return this;
+    }
+
+    public Builder withTimerUpdate(TimerUpdate timerUpdate) {
+      this.timerUpdate = timerUpdate;
+      return this;
+    }
+
+    public Builder addOutput(
+        UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
+      bundlesBuilder.add(outputBundle);
+      bundlesBuilder.add(outputBundles);
+      return this;
+    }
+
+    public Builder addOutput(Collection<UncommittedBundle<?>> outputBundles) {
+      bundlesBuilder.addAll(outputBundles);
+      return this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
new file mode 100644
index 0000000..be1bf18
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.TextIO.Write.Bound;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+class TextIOShardedWriteFactory implements PTransformOverrideFactory {
+
+  @Override
+  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+      PTransform<InputT, OutputT> transform) {
+    if (transform instanceof TextIO.Write.Bound) {
+      @SuppressWarnings("unchecked")
+      TextIO.Write.Bound<InputT> originalWrite = (TextIO.Write.Bound<InputT>) transform;
+      if (originalWrite.getNumShards() > 1
+          || (originalWrite.getNumShards() == 1
+              && !"".equals(originalWrite.getShardNameTemplate()))) {
+        @SuppressWarnings("unchecked")
+        PTransform<InputT, OutputT> override =
+            (PTransform<InputT, OutputT>) new TextIOShardedWrite<InputT>(originalWrite);
+        return override;
+      }
+    }
+    return transform;
+  }
+
+  private static class TextIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
+    private final TextIO.Write.Bound<InputT> initial;
+
+    private TextIOShardedWrite(Bound<InputT> initial) {
+      this.initial = initial;
+    }
+
+    @Override
+    int getNumShards() {
+      return initial.getNumShards();
+    }
+
+    @Override
+    PTransform<PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
+      String shardName =
+          IOChannelUtils.constructName(
+              initial.getFilenamePrefix(),
+              initial.getShardTemplate(),
+              initial.getFilenameSuffix(),
+              shardNum,
+              getNumShards());
+      return TextIO.Write.withCoder(initial.getCoder()).to(shardName).withoutSharding();
+    }
+
+    @Override
+    protected PTransform<PCollection<InputT>, PDone> delegate() {
+      return initial;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java
new file mode 100644
index 0000000..ba9815b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * An evaluator of a specific application of a transform. Will be used for at least one
+ * {@link CommittedBundle}.
+ *
+ * @param <InputT> the type of elements that will be passed to {@link #processElement}
+ */
+public interface TransformEvaluator<InputT> {
+  /**
+   * Process an element in the input {@link CommittedBundle}.
+   *
+   * @param element the element to process
+   */
+  void processElement(WindowedValue<InputT> element) throws Exception;
+
+  /**
+   * Finish processing the bundle of this {@link TransformEvaluator}.
+   *
+   * After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
+   * and no more elements will be processed.
+   *
+   * @return an {@link InProcessTransformResult} containing the results of this bundle evaluation.
+   */
+  InProcessTransformResult finishBundle() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java
new file mode 100644
index 0000000..8f8d84c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.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.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import javax.annotation.Nullable;
+
+/**
+ * A factory for creating instances of {@link TransformEvaluator} for the application of a
+ * {@link PTransform}.
+ */
+public interface TransformEvaluatorFactory {
+  /**
+   * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
+   *
+   * Any work that must be done before input elements are processed (such as calling
+   * {@link DoFn#startBundle(DoFn.Context)}) must be done before the {@link TransformEvaluator} is
+   * made available to the caller.
+   *
+   * @throws Exception whenever constructing the underlying evaluator throws an exception
+   */
+  <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
new file mode 100644
index 0000000..f449731
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
+ * implementations based on the type of {@link PTransform} of the application.
+ */
+class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
+  public static TransformEvaluatorRegistry defaultRegistry() {
+    @SuppressWarnings("rawtypes")
+    ImmutableMap<Class<? extends PTransform>, TransformEvaluatorFactory> primitives =
+        ImmutableMap.<Class<? extends PTransform>, TransformEvaluatorFactory>builder()
+            .put(Read.Bounded.class, new BoundedReadEvaluatorFactory())
+            .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory())
+            .put(ParDo.Bound.class, new ParDoSingleEvaluatorFactory())
+            .put(ParDo.BoundMulti.class, new ParDoMultiEvaluatorFactory())
+            .put(
+                GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly.class,
+                new GroupByKeyEvaluatorFactory())
+            .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory())
+            .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
+            .put(Window.Bound.class, new WindowEvaluatorFactory())
+            .build();
+    return new TransformEvaluatorRegistry(primitives);
+  }
+
+  // the TransformEvaluatorFactories can construct instances of all generic types of transform,
+  // so all instances of a primitive can be handled with the same evaluator factory.
+  @SuppressWarnings("rawtypes")
+  private final Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories;
+
+  private TransformEvaluatorRegistry(
+      @SuppressWarnings("rawtypes")
+      Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories) {
+    this.factories = factories;
+  }
+
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      @Nullable CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext)
+      throws Exception {
+    TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
+    return factory.forApplication(application, inputBundle, evaluationContext);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java
new file mode 100644
index 0000000..8346e89
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.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.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+
+import com.google.common.base.Throwables;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a
+ * {@link TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering
+ * the result using a registered {@link CompletionCallback}.
+ *
+ * <p>A {@link TransformExecutor} that is currently executing also provides access to the thread
+ * that it is being executed on.
+ */
+class TransformExecutor<T> implements Callable<InProcessTransformResult> {
+  public static <T> TransformExecutor<T> create(
+      TransformEvaluatorFactory factory,
+      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
+      InProcessEvaluationContext evaluationContext,
+      CommittedBundle<T> inputBundle,
+      AppliedPTransform<?, ?, ?> transform,
+      CompletionCallback completionCallback,
+      TransformExecutorService transformEvaluationState) {
+    return new TransformExecutor<>(
+        factory,
+        modelEnforcements,
+        evaluationContext,
+        inputBundle,
+        transform,
+        completionCallback,
+        transformEvaluationState);
+  }
+
+  private final TransformEvaluatorFactory evaluatorFactory;
+  private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
+
+  private final InProcessEvaluationContext evaluationContext;
+
+  /** The transform that will be evaluated. */
+  private final AppliedPTransform<?, ?, ?> transform;
+  /** The inputs this {@link TransformExecutor} will deliver to the transform. */
+  private final CommittedBundle<T> inputBundle;
+
+  private final CompletionCallback onComplete;
+  private final TransformExecutorService transformEvaluationState;
+
+  private final AtomicReference<Thread> thread;
+
+  private TransformExecutor(
+      TransformEvaluatorFactory factory,
+      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
+      InProcessEvaluationContext evaluationContext,
+      CommittedBundle<T> inputBundle,
+      AppliedPTransform<?, ?, ?> transform,
+      CompletionCallback completionCallback,
+      TransformExecutorService transformEvaluationState) {
+    this.evaluatorFactory = factory;
+    this.modelEnforcements = modelEnforcements;
+    this.evaluationContext = evaluationContext;
+
+    this.inputBundle = inputBundle;
+    this.transform = transform;
+
+    this.onComplete = completionCallback;
+
+    this.transformEvaluationState = transformEvaluationState;
+    this.thread = new AtomicReference<>();
+  }
+
+  @Override
+  public InProcessTransformResult call() {
+    checkState(
+        thread.compareAndSet(null, Thread.currentThread()),
+        "Tried to execute %s for %s on thread %s, but is already executing on thread %s",
+        TransformExecutor.class.getSimpleName(),
+        transform.getFullName(),
+        Thread.currentThread(),
+        thread.get());
+    try {
+      Collection<ModelEnforcement<T>> enforcements = new ArrayList<>();
+      for (ModelEnforcementFactory enforcementFactory : modelEnforcements) {
+        ModelEnforcement<T> enforcement = enforcementFactory.forBundle(inputBundle, transform);
+        enforcements.add(enforcement);
+      }
+      TransformEvaluator<T> evaluator =
+          evaluatorFactory.forApplication(transform, inputBundle, evaluationContext);
+
+      processElements(evaluator, enforcements);
+
+      InProcessTransformResult result = finishBundle(evaluator, enforcements);
+      return result;
+    } catch (Throwable t) {
+      onComplete.handleThrowable(inputBundle, t);
+      throw Throwables.propagate(t);
+    } finally {
+      transformEvaluationState.complete(this);
+    }
+  }
+
+  /**
+   * Processes all the elements in the input bundle using the transform evaluator, applying any
+   * necessary {@link ModelEnforcement ModelEnforcements}.
+   */
+  private void processElements(
+      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
+      throws Exception {
+    if (inputBundle != null) {
+      for (WindowedValue<T> value : inputBundle.getElements()) {
+        for (ModelEnforcement<T> enforcement : enforcements) {
+          enforcement.beforeElement(value);
+        }
+
+        evaluator.processElement(value);
+
+        for (ModelEnforcement<T> enforcement : enforcements) {
+          enforcement.afterElement(value);
+        }
+      }
+    }
+  }
+
+  /**
+   * Finishes processing the input bundle and commit the result using the
+   * {@link CompletionCallback}, applying any {@link ModelEnforcement} if necessary.
+   *
+   * @return the {@link InProcessTransformResult} produced by
+   *         {@link TransformEvaluator#finishBundle()}
+   */
+  private InProcessTransformResult finishBundle(
+      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
+      throws Exception {
+    InProcessTransformResult result = evaluator.finishBundle();
+    CommittedResult outputs = onComplete.handleResult(inputBundle, result);
+    for (ModelEnforcement<T> enforcement : enforcements) {
+      enforcement.afterFinish(inputBundle, result, outputs.getOutputs());
+    }
+    return result;
+  }
+
+  /**
+   * If this {@link TransformExecutor} is currently executing, return the thread it is executing in.
+   * Otherwise, return null.
+   */
+  @Nullable
+  public Thread getThread() {
+    return thread.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
new file mode 100644
index 0000000..837b858
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorService.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+/**
+ * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
+ * appropriate for the {@link StepAndKey} the executor exists for.
+ */
+interface TransformExecutorService {
+  /**
+   * Schedule the provided work to be eventually executed.
+   */
+  void schedule(TransformExecutor<?> work);
+
+  /**
+   * Finish executing the provided work. This may cause additional
+   * {@link TransformExecutor TransformExecutors} to be evaluated.
+   */
+  void complete(TransformExecutor<?> completed);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
new file mode 100644
index 0000000..087b7c2
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import com.google.common.base.MoreObjects;
+
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Static factory methods for constructing instances of {@link TransformExecutorService}.
+ */
+final class TransformExecutorServices {
+  private TransformExecutorServices() {
+    // Do not instantiate
+  }
+
+  /**
+   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
+   * parallel.
+   */
+  public static TransformExecutorService parallel(
+      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+    return new ParallelEvaluationState(executor, scheduled);
+  }
+
+  /**
+   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
+   * serial.
+   */
+  public static TransformExecutorService serial(
+      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+    return new SerialEvaluationState(executor, scheduled);
+  }
+
+  /**
+   * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
+   * scheduled will be immediately submitted to the {@link ExecutorService}.
+   *
+   * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
+   * processed in parallel.
+   */
+  private static class ParallelEvaluationState implements TransformExecutorService {
+    private final ExecutorService executor;
+    private final Map<TransformExecutor<?>, Boolean> scheduled;
+
+    private ParallelEvaluationState(
+        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+      this.executor = executor;
+      this.scheduled = scheduled;
+    }
+
+    @Override
+    public void schedule(TransformExecutor<?> work) {
+      executor.submit(work);
+      scheduled.put(work, true);
+    }
+
+    @Override
+    public void complete(TransformExecutor<?> completed) {
+      scheduled.remove(completed);
+    }
+  }
+
+  /**
+   * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
+   * scheduled will be placed on the work queue. Only one item of work will be submitted to the
+   * {@link ExecutorService} at any time.
+   *
+   * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair.
+   * Keyed computations are processed serially per step.
+   */
+  private static class SerialEvaluationState implements TransformExecutorService {
+    private final ExecutorService executor;
+    private final Map<TransformExecutor<?>, Boolean> scheduled;
+
+    private AtomicReference<TransformExecutor<?>> currentlyEvaluating;
+    private final Queue<TransformExecutor<?>> workQueue;
+
+    private SerialEvaluationState(
+        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
+      this.scheduled = scheduled;
+      this.executor = executor;
+      this.currentlyEvaluating = new AtomicReference<>();
+      this.workQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    /**
+     * Schedules the work, adding it to the work queue if there is a bundle currently being
+     * evaluated and scheduling it immediately otherwise.
+     */
+    @Override
+    public void schedule(TransformExecutor<?> work) {
+      workQueue.offer(work);
+      updateCurrentlyEvaluating();
+    }
+
+    @Override
+    public void complete(TransformExecutor<?> completed) {
+      if (!currentlyEvaluating.compareAndSet(completed, null)) {
+        throw new IllegalStateException(
+            "Finished work "
+                + completed
+                + " but could not complete due to unexpected currently executing "
+                + currentlyEvaluating.get());
+      }
+      scheduled.remove(completed);
+      updateCurrentlyEvaluating();
+    }
+
+    private void updateCurrentlyEvaluating() {
+      if (currentlyEvaluating.get() == null) {
+        // Only synchronize if we need to update what's currently evaluating
+        synchronized (this) {
+          TransformExecutor<?> newWork = workQueue.poll();
+          if (newWork != null) {
+            if (currentlyEvaluating.compareAndSet(null, newWork)) {
+              scheduled.put(newWork, true);
+              executor.submit(newWork);
+            } else {
+              workQueue.offer(newWork);
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(SerialEvaluationState.class)
+          .add("currentlyEvaluating", currentlyEvaluating)
+          .add("workQueue", workQueue)
+          .toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
new file mode 100644
index 0000000..7a95c9f
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.io.Read.Unbounded;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.io.IOException;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
+ * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}.
+ */
+class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
+  /*
+   * An evaluator for a Source is stateful, to ensure the CheckpointMark is properly persisted.
+   * Evaluators are cached here to ensure that the checkpoint mark is appropriately reused
+   * and any splits are honored.
+   */
+  private final ConcurrentMap<EvaluatorKey, Queue<? extends UnboundedReadEvaluator<?>>>
+      sourceEvaluators = new ConcurrentHashMap<>();
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
+      @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
+    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
+  }
+
+  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
+      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+      final InProcessEvaluationContext evaluationContext) {
+    UnboundedReadEvaluator<?> currentEvaluator =
+        getTransformEvaluatorQueue(transform, evaluationContext).poll();
+    if (currentEvaluator == null) {
+      return EmptyTransformEvaluator.create(transform);
+    }
+    return currentEvaluator;
+  }
+
+  /**
+   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
+   * provided application of {@link Unbounded Read.Unbounded}, initializing it if required.
+   *
+   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
+   * already done so.
+   */
+  @SuppressWarnings("unchecked")
+  private <OutputT> Queue<UnboundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
+      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+      final InProcessEvaluationContext evaluationContext) {
+    // Key by the application and the context the evaluation is occurring in (which call to
+    // Pipeline#run).
+    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
+    @SuppressWarnings("unchecked")
+    Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue =
+        (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+    if (evaluatorQueue == null) {
+      evaluatorQueue = new ConcurrentLinkedQueue<>();
+      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
+        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
+        // factory for this transform
+        UnboundedSource<OutputT, ?> source = transform.getTransform().getSource();
+        UnboundedReadEvaluator<OutputT> evaluator =
+            new UnboundedReadEvaluator<OutputT>(
+                transform, evaluationContext, source, evaluatorQueue);
+        evaluatorQueue.offer(evaluator);
+      } else {
+        // otherwise return the existing Queue that arrived before us
+        evaluatorQueue = (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+      }
+    }
+    return evaluatorQueue;
+  }
+
+  /**
+   * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource},
+   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
+   * creates the {@link UnboundedReader} and consumes some currently available input.
+   *
+   * <p>Calls to {@link UnboundedReadEvaluator} are not internally thread-safe, and should only be
+   * used by a single thread at a time. Each {@link UnboundedReadEvaluator} maintains its own
+   * checkpoint, and constructs its reader from the current checkpoint in each call to
+   * {@link #finishBundle()}.
+   */
+  private static class UnboundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
+    private static final int ARBITRARY_MAX_ELEMENTS = 10;
+    private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
+    private final InProcessEvaluationContext evaluationContext;
+    private final Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue;
+    /**
+     * The source being read from by this {@link UnboundedReadEvaluator}. This may not be the same
+     * source as derived from {@link #transform} due to splitting.
+     */
+    private final UnboundedSource<OutputT, ?> source;
+    private CheckpointMark checkpointMark;
+
+    public UnboundedReadEvaluator(
+        AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
+        InProcessEvaluationContext evaluationContext,
+        UnboundedSource<OutputT, ?> source,
+        Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue) {
+      this.transform = transform;
+      this.evaluationContext = evaluationContext;
+      this.evaluatorQueue = evaluatorQueue;
+      this.source = source;
+      this.checkpointMark = null;
+    }
+
+    @Override
+    public void processElement(WindowedValue<Object> element) {}
+
+    @Override
+    public InProcessTransformResult finishBundle() throws IOException {
+      UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
+      try (UnboundedReader<OutputT> reader =
+              createReader(source, evaluationContext.getPipelineOptions());) {
+        int numElements = 0;
+        if (reader.start()) {
+          do {
+            output.add(
+                WindowedValue.timestampedValueInGlobalWindow(
+                    reader.getCurrent(), reader.getCurrentTimestamp()));
+            numElements++;
+          } while (numElements < ARBITRARY_MAX_ELEMENTS && reader.advance());
+        }
+        checkpointMark = reader.getCheckpointMark();
+        checkpointMark.finalizeCheckpoint();
+        // TODO: When exercising create initial splits, make this the minimum watermark across all
+        // existing readers
+        StepTransformResult result =
+            StepTransformResult.withHold(transform, reader.getWatermark())
+                .addOutput(output)
+                .build();
+        evaluatorQueue.offer(this);
+        return result;
+      }
+    }
+
+    private <CheckpointMarkT extends CheckpointMark> UnboundedReader<OutputT> createReader(
+        UnboundedSource<OutputT, CheckpointMarkT> source, PipelineOptions options) {
+      @SuppressWarnings("unchecked")
+      CheckpointMarkT mark = (CheckpointMarkT) checkpointMark;
+      return source.createReader(options, mark);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
new file mode 100644
index 0000000..ffaf3fa
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link CreatePCollectionView} primitive {@link PTransform}.
+ *
+ * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
+ * the {@link WriteView} {@link PTransform}, which is part of the
+ * {@link InProcessCreatePCollectionView} composite transform. This transform is an override for the
+ * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
+ * written.
+ */
+class ViewEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <T> TransformEvaluator<T> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      InProcessPipelineRunner.CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+    TransformEvaluator<T> evaluator = createEvaluator(
+            (AppliedPTransform) application, evaluationContext);
+    return evaluator;
+  }
+
+  private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
+      final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
+          application,
+      InProcessEvaluationContext context) {
+    PCollection<Iterable<InT>> input = application.getInput();
+    final PCollectionViewWriter<InT, OuT> writer =
+        context.createPCollectionViewWriter(input, application.getOutput());
+    return new TransformEvaluator<Iterable<InT>>() {
+      private final List<WindowedValue<InT>> elements = new ArrayList<>();
+
+      @Override
+      public void processElement(WindowedValue<Iterable<InT>> element) {
+        for (InT input : element.getValue()) {
+          elements.add(element.withValue(input));
+        }
+      }
+
+      @Override
+      public InProcessTransformResult finishBundle() {
+        writer.add(elements);
+        return StepTransformResult.withoutHold(application).build();
+      }
+    };
+  }
+
+  public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
+    @Override
+    public <InputT extends PInput, OutputT extends POutput>
+        PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
+      if (transform instanceof CreatePCollectionView) {
+
+      }
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      PTransform<InputT, OutputT> createView =
+          (PTransform<InputT, OutputT>)
+              new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
+      return createView;
+    }
+  }
+
+  /**
+   * An in-process override for {@link CreatePCollectionView}.
+   */
+  private static class InProcessCreatePCollectionView<ElemT, ViewT>
+      extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
+    private final CreatePCollectionView<ElemT, ViewT> og;
+
+    private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
+      this.og = og;
+    }
+
+    @Override
+    public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
+      return input.apply(WithKeys.<Void, ElemT>of((Void) null))
+          .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()))
+          .apply(GroupByKey.<Void, ElemT>create())
+          .apply(Values.<Iterable<ElemT>>create())
+          .apply(new WriteView<ElemT, ViewT>(og));
+    }
+
+    @Override
+    protected PTransform<PCollection<ElemT>, PCollectionView<ViewT>> delegate() {
+      return og;
+    }
+  }
+
+  /**
+   * An in-process implementation of the {@link CreatePCollectionView} primitive.
+   *
+   * This implementation requires the input {@link PCollection} to be an iterable, which is provided
+   * to {@link PCollectionView#fromIterableInternal(Iterable)}.
+   */
+  public static final class WriteView<ElemT, ViewT>
+      extends PTransform<PCollection<Iterable<ElemT>>, PCollectionView<ViewT>> {
+    private final CreatePCollectionView<ElemT, ViewT> og;
+
+    WriteView(CreatePCollectionView<ElemT, ViewT> og) {
+      this.og = og;
+    }
+
+    @Override
+    public PCollectionView<ViewT> apply(PCollection<Iterable<ElemT>> input) {
+      return og.getView();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
new file mode 100644
index 0000000..4a3a517
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy;
+
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Ordering;
+
+import org.joda.time.Instant;
+
+import java.util.PriorityQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Executes callbacks that occur based on the progression of the watermark per-step.
+ *
+ * <p>Callbacks are registered by calls to
+ * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)},
+ * and are executed after a call to {@link #fireForWatermark(AppliedPTransform, Instant)} with the
+ * same {@link AppliedPTransform} and a watermark sufficient to ensure that the trigger for the
+ * windowing strategy would have been produced.
+ *
+ * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
+ * {@link AppliedPTransform} - any call to
+ * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)}
+ * that could have potentially already fired should be followed by a call to
+ * {@link #fireForWatermark(AppliedPTransform, Instant)} for the same transform with the current
+ * value of the watermark.
+ */
+class WatermarkCallbackExecutor {
+  /**
+   * Create a new {@link WatermarkCallbackExecutor}.
+   */
+  public static WatermarkCallbackExecutor create() {
+    return new WatermarkCallbackExecutor();
+  }
+
+  private final ConcurrentMap<AppliedPTransform<?, ?, ?>, PriorityQueue<WatermarkCallback>>
+      callbacks;
+  private final ExecutorService executor;
+
+  private WatermarkCallbackExecutor() {
+    this.callbacks = new ConcurrentHashMap<>();
+    this.executor = Executors.newSingleThreadExecutor();
+  }
+
+  /**
+   * Execute the provided {@link Runnable} after the next call to
+   * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window is guaranteed to have
+   * produced output.
+   */
+  public void callOnGuaranteedFiring(
+      AppliedPTransform<?, ?, ?> step,
+      BoundedWindow window,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Runnable runnable) {
+    WatermarkCallback callback =
+        WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
+
+    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
+    if (callbackQueue == null) {
+      callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
+      if (callbacks.putIfAbsent(step, callbackQueue) != null) {
+        callbackQueue = callbacks.get(step);
+      }
+    }
+
+    synchronized (callbackQueue) {
+      callbackQueue.offer(callback);
+    }
+  }
+
+  /**
+   * Schedule all pending callbacks that must have produced output by the time of the provided
+   * watermark.
+   */
+  public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark) {
+    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
+    if (callbackQueue == null) {
+      return;
+    }
+    synchronized (callbackQueue) {
+      while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
+        executor.submit(callbackQueue.poll().getCallback());
+      }
+    }
+  }
+
+  private static class WatermarkCallback {
+    public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
+        BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
+      @SuppressWarnings("unchecked")
+      Instant firingAfter =
+          strategy.getTrigger().getSpec().getWatermarkThatGuaranteesFiring((W) window);
+      return new WatermarkCallback(firingAfter, callback);
+    }
+
+    private final Instant fireAfter;
+    private final Runnable callback;
+
+    private WatermarkCallback(Instant fireAfter, Runnable callback) {
+      this.fireAfter = fireAfter;
+      this.callback = callback;
+    }
+
+    public boolean shouldFire(Instant currentWatermark) {
+      return currentWatermark.isAfter(fireAfter)
+          || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    }
+
+    public Runnable getCallback() {
+      return callback;
+    }
+  }
+
+  private static class CallbackOrdering extends Ordering<WatermarkCallback> {
+    @Override
+    public int compare(WatermarkCallback left, WatermarkCallback right) {
+      return ComparisonChain.start()
+          .compare(left.fireAfter, right.fireAfter)
+          .compare(left.callback, right.callback, Ordering.arbitrary())
+          .result();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
new file mode 100644
index 0000000..628f94d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * {@link Bound Window.Bound} primitive {@link PTransform}.
+ */
+class WindowEvaluatorFactory implements TransformEvaluatorFactory {
+
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      @Nullable CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext)
+      throws Exception {
+    return createTransformEvaluator(
+        (AppliedPTransform) application, inputBundle, evaluationContext);
+  }
+
+  private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
+      AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
+      CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
+    UncommittedBundle<InputT> outputBundle =
+        evaluationContext.createBundle(inputBundle, transform.getOutput());
+    if (fn == null) {
+      return PassthroughTransformEvaluator.create(transform, outputBundle);
+    }
+    return new WindowIntoEvaluator<>(transform, fn, outputBundle);
+  }
+
+  private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
+    private final AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>>
+        transform;
+    private final WindowFn<InputT, ?> windowFn;
+    private final UncommittedBundle<InputT> outputBundle;
+
+    @SuppressWarnings("unchecked")
+    public WindowIntoEvaluator(
+        AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
+        WindowFn<? super InputT, ?> windowFn,
+        UncommittedBundle<InputT> outputBundle) {
+      this.outputBundle = outputBundle;
+      this.transform = transform;
+      // Safe contravariant cast
+      this.windowFn = (WindowFn<InputT, ?>) windowFn;
+    }
+
+    @Override
+    public void processElement(WindowedValue<InputT> element) throws Exception {
+      Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
+      outputBundle.add(
+          WindowedValue.<InputT>of(
+              element.getValue(), element.getTimestamp(), windows, PaneInfo.NO_FIRING));
+    }
+
+    private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
+        WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
+      WindowFn<InputT, W>.AssignContext assignContext =
+          new InProcessAssignContext<>(windowFn, element);
+      Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
+      return windows;
+    }
+
+    @Override
+    public InProcessTransformResult finishBundle() throws Exception {
+      return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
+    }
+  }
+
+  private static class InProcessAssignContext<InputT, W extends BoundedWindow>
+      extends WindowFn<InputT, W>.AssignContext {
+    private final WindowedValue<InputT> value;
+
+    public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
+      fn.super();
+      this.value = value;
+    }
+
+    @Override
+    public InputT element() {
+      return value.getValue();
+    }
+
+    @Override
+    public Instant timestamp() {
+      return value.getTimestamp();
+    }
+
+    @Override
+    public Collection<? extends BoundedWindow> windows() {
+      return value.getWindows();
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
new file mode 100644
index 0000000..d290a4b
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.AvroIOTest;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+
+/**
+ * Tests for {@link AvroIOShardedWriteFactory}.
+ */
+@RunWith(JUnit4.class)
+public class AvroIOShardedWriteFactoryTest {
+
+  @Rule public TemporaryFolder tmp = new TemporaryFolder();
+  private AvroIOShardedWriteFactory factory;
+
+  @Before
+  public void setup() {
+    factory = new AvroIOShardedWriteFactory();
+  }
+
+  @Test
+  public void originalWithoutShardingReturnsOriginal() throws Exception {
+    File file = tmp.newFile("foo");
+    PTransform<PCollection<String>, PDone> original =
+        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withoutSharding();
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, theInstance(original));
+  }
+
+  @Test
+  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
+    File file = tmp.newFile("foo");
+    PTransform<PCollection<String>, PDone> original =
+        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath());
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, theInstance(original));
+  }
+
+  @Test
+  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
+    File file = tmp.newFile("foo");
+    AvroIO.Write.Bound<String> original =
+        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(1);
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+    TestPipeline p = TestPipeline.create();
+    String[] elems = new String[] {"foo", "bar", "baz"};
+    p.apply(Create.<String>of(elems)).apply(overridden);
+
+    file.delete();
+
+    p.run();
+    AvroIOTest.assertTestOutputs(elems, 1, file.getAbsolutePath(), original.getShardNameTemplate());
+  }
+
+  @Test
+  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
+    File file = tmp.newFile("foo");
+    AvroIO.Write.Bound<String> original =
+        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(3);
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+    TestPipeline p = TestPipeline.create();
+    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
+    p.apply(Create.<String>of(elems)).apply(overridden);
+
+    file.delete();
+    p.run();
+    AvroIOTest.assertTestOutputs(elems, 3, file.getAbsolutePath(), original.getShardNameTemplate());
+  }
+}



[03/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
deleted file mode 100644
index 3c4503f..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManagerTest.java
+++ /dev/null
@@ -1,1168 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.joda.time.Instant;
-import org.joda.time.ReadableInstant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Tests for {@link InMemoryWatermarkManager}.
- */
-@RunWith(JUnit4.class)
-public class InMemoryWatermarkManagerTest implements Serializable {
-  private transient MockClock clock;
-
-  private transient PCollection<Integer> createdInts;
-
-  private transient PCollection<Integer> filtered;
-  private transient PCollection<Integer> filteredTimesTwo;
-  private transient PCollection<KV<String, Integer>> keyed;
-
-  private transient PCollection<Integer> intsToFlatten;
-  private transient PCollection<Integer> flattened;
-
-  private transient InMemoryWatermarkManager manager;
-  private transient BundleFactory bundleFactory;
-
-  @Before
-  public void setup() {
-    TestPipeline p = TestPipeline.create();
-
-    createdInts = p.apply("createdInts", Create.of(1, 2, 3));
-
-    filtered = createdInts.apply("filtered", Filter.greaterThan(1));
-    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
-      @Override
-      public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
-        c.output(c.element() * 2);
-      }
-    }));
-
-    keyed = createdInts.apply("keyed", WithKeys.<String, Integer>of("MyKey"));
-
-    intsToFlatten = p.apply("intsToFlatten", Create.of(-1, 256, 65535));
-    PCollectionList<Integer> preFlatten = PCollectionList.of(createdInts).and(intsToFlatten);
-    flattened = preFlatten.apply("flattened", Flatten.<Integer>pCollections());
-
-    Collection<AppliedPTransform<?, ?, ?>> rootTransforms =
-        ImmutableList.<AppliedPTransform<?, ?, ?>>of(
-            createdInts.getProducingTransformInternal(),
-            intsToFlatten.getProducingTransformInternal());
-
-    Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers = new HashMap<>();
-    consumers.put(
-        createdInts,
-        ImmutableList.<AppliedPTransform<?, ?, ?>>of(filtered.getProducingTransformInternal(),
-            keyed.getProducingTransformInternal(), flattened.getProducingTransformInternal()));
-    consumers.put(
-        filtered,
-        Collections.<AppliedPTransform<?, ?, ?>>singleton(
-            filteredTimesTwo.getProducingTransformInternal()));
-    consumers.put(filteredTimesTwo, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
-    consumers.put(keyed, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
-
-    consumers.put(
-        intsToFlatten,
-        Collections.<AppliedPTransform<?, ?, ?>>singleton(
-            flattened.getProducingTransformInternal()));
-    consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
-
-    clock = MockClock.fromInstant(new Instant(1000));
-
-    manager = InMemoryWatermarkManager.create(clock, rootTransforms, consumers);
-    bundleFactory = InProcessBundleFactory.create();
-  }
-
-  /**
-   * Demonstrates that getWatermark, when called on an {@link AppliedPTransform} that has not
-   * processed any elements, returns the {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
-   */
-  @Test
-  public void getWatermarkForUntouchedTransform() {
-    TransformWatermarks watermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-
-    assertThat(watermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-    assertThat(watermarks.getOutputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-  }
-
-  /**
-   * Demonstrates that getWatermark for a transform that consumes no input uses the Watermark
-   * Hold value provided to it as the output watermark.
-   */
-  @Test
-  public void getWatermarkForUpdatedSourceTransform() {
-    CommittedBundle<Integer> output = multiWindowedBundle(createdInts, 1);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(output), new Instant(8000L));
-    TransformWatermarks updatedSourceWatermark =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-
-    assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L)));
-  }
-
-  /**
-   * Demonstrates that getWatermark for a transform that takes multiple inputs is held to the
-   * minimum watermark across all of its inputs.
-   */
-  @Test
-  public void getWatermarkForMultiInputTransform() {
-    CommittedBundle<Integer> secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1);
-
-    manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    // We didn't do anything for the first source, so we shouldn't have progressed the watermark
-    TransformWatermarks firstSourceWatermark =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(
-        firstSourceWatermark.getOutputWatermark(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
-    // the Second Source output all of the elements so it should be done (with a watermark at the
-    // end of time).
-    TransformWatermarks secondSourceWatermark =
-        manager.getWatermarks(intsToFlatten.getProducingTransformInternal());
-    assertThat(
-        secondSourceWatermark.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    // We haven't consumed anything yet, so our watermark should be at the beginning of time
-    TransformWatermarks transformWatermark =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    assertThat(
-        transformWatermark.getInputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-    assertThat(
-        transformWatermark.getOutputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
-    CommittedBundle<Integer> flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1);
-    // We have finished processing the bundle from the second PCollection, but we haven't consumed
-    // anything from the first PCollection yet; so our watermark shouldn't advance
-    manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
-        null);
-    TransformWatermarks transformAfterProcessing =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
-        null);
-    assertThat(
-        transformAfterProcessing.getInputWatermark(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-    assertThat(
-        transformAfterProcessing.getOutputWatermark(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
-    Instant firstCollectionTimestamp = new Instant(10000);
-    CommittedBundle<Integer> firstPcollectionBundle =
-        timestampedBundle(createdInts, TimestampedValue.<Integer>of(5, firstCollectionTimestamp));
-    // the source is done, but elements are still buffered. The source output watermark should be
-    // past the end of the global window
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle),
-        new Instant(Long.MAX_VALUE));
-    TransformWatermarks firstSourceWatermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(
-        firstSourceWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    // We still haven't consumed any of the first source's input, so the watermark should still not
-    // progress
-    TransformWatermarks flattenAfterSourcesProduced =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    assertThat(
-        flattenAfterSourcesProduced.getInputWatermark(), not(laterThan(firstCollectionTimestamp)));
-    assertThat(
-        flattenAfterSourcesProduced.getOutputWatermark(), not(laterThan(firstCollectionTimestamp)));
-
-    // We have buffered inputs, but since the PCollection has all of the elements (has a WM past the
-    // end of the global window), we should have a watermark equal to the min among buffered
-    // elements
-    TransformWatermarks withBufferedElements =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    assertThat(withBufferedElements.getInputWatermark(), equalTo(firstCollectionTimestamp));
-    assertThat(withBufferedElements.getOutputWatermark(), equalTo(firstCollectionTimestamp));
-
-    CommittedBundle<?> completedFlattenBundle =
-        bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(completedFlattenBundle),
-        null);
-    TransformWatermarks afterConsumingAllInput =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    assertThat(
-        afterConsumingAllInput.getInputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(
-        afterConsumingAllInput.getOutputWatermark(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-  }
-
-  /**
-   * Demonstrates that pending elements are independent among
-   * {@link AppliedPTransform AppliedPTransforms} that consume the same input {@link PCollection}.
-   */
-  @Test
-  public void getWatermarkForMultiConsumedCollection() {
-    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
-        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
-        TimestampedValue.of(3, new Instant(-1000L)));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
-    TransformWatermarks createdAfterProducing =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(
-        createdAfterProducing.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    CommittedBundle<KV<String, Integer>> keyBundle =
-        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
-            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
-            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
-    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
-    TransformWatermarks keyedWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(
-        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(
-        keyedWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    TransformWatermarks filteredWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(filteredWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
-    assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
-
-    CommittedBundle<Integer> filteredBundle =
-        timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L)));
-    manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filteredBundle), null);
-    TransformWatermarks filteredProcessedWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        filteredProcessedWatermarks.getInputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(
-        filteredProcessedWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-  }
-
-  /**
-   * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
-   * watermark hold.
-   */
-  @Test
-  public void updateWatermarkWithWatermarkHolds() {
-    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
-        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
-        TimestampedValue.of(3, new Instant(-1000L)));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
-
-    CommittedBundle<KV<String, Integer>> keyBundle =
-        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
-            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
-            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
-    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
-        new Instant(500L));
-    TransformWatermarks keyedWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(
-        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
-  }
-
-  /**
-   * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
-   * watermark hold.
-   */
-  @Test
-  public void updateWatermarkWithKeyedWatermarkHolds() {
-    CommittedBundle<Integer> firstKeyBundle =
-        bundleFactory.createKeyedBundle(null, "Odd", createdInts)
-            .add(WindowedValue.timestampedValueInGlobalWindow(1, new Instant(1_000_000L)))
-            .add(WindowedValue.timestampedValueInGlobalWindow(3, new Instant(-1000L)))
-            .commit(clock.now());
-
-    CommittedBundle<Integer> secondKeyBundle =
-        bundleFactory.createKeyedBundle(null, "Even", createdInts)
-            .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
-            .commit(clock.now());
-
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(-1000L));
-    manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(1234L));
-
-    TransformWatermarks filteredWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        filteredWatermarks.getInputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
-
-    CommittedBundle<Integer> fauxFirstKeyTimerBundle =
-        bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now());
-    manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
-
-    CommittedBundle<Integer> fauxSecondKeyTimerBundle =
-        bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now());
-    manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(5678L));
-    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
-
-    manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(
-        filteredWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-  }
-
-  /**
-   * Demonstrates that updated output watermarks are monotonic in the presence of late data, when
-   * called on an {@link AppliedPTransform} that consumes no input.
-   */
-  @Test
-  public void updateOutputWatermarkShouldBeMonotonic() {
-    CommittedBundle<?> firstInput =
-        bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(firstInput), new Instant(0L));
-    TransformWatermarks firstWatermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L)));
-
-    CommittedBundle<?> secondInput =
-        bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(secondInput), new Instant(-250L));
-    TransformWatermarks secondWatermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L))));
-  }
-
-  /**
-   * Demonstrates that updated output watermarks are monotonic in the presence of watermark holds
-   * that become earlier than a previous watermark hold.
-   */
-  @Test
-  public void updateWatermarkWithHoldsShouldBeMonotonic() {
-    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
-        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
-        TimestampedValue.of(3, new Instant(-1000L)));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
-
-    CommittedBundle<KV<String, Integer>> keyBundle =
-        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
-            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
-            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
-    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
-        new Instant(500L));
-    TransformWatermarks keyedWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(
-        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
-    Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark();
-
-    TransformWatermarks updatedWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(
-        updatedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    // We added a hold prior to the old watermark; we shouldn't progress (due to the earlier hold)
-    // but the watermark is monotonic and should not backslide to the new, earlier hold
-    assertThat(updatedWatermarks.getOutputWatermark(), equalTo(oldOutputWatermark));
-  }
-
-  /**
-   * Demonstrates that updateWatermarks in the presence of late data is monotonic.
-   */
-  @Test
-  public void updateWatermarkWithLateData() {
-    Instant sourceWatermark = new Instant(1_000_000L);
-    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
-        TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L)));
-
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createdBundle), sourceWatermark);
-
-    CommittedBundle<KV<String, Integer>> keyBundle =
-        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark),
-            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)));
-
-    // Finish processing the on-time data. The watermarks should progress to be equal to the source
-    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
-    TransformWatermarks onTimeWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark));
-    assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark));
-
-    CommittedBundle<Integer> lateDataBundle =
-        timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L)));
-    // the late data arrives in a downstream PCollection after its watermark has advanced past it;
-    // we don't advance the watermark past the current watermark until we've consumed the late data
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(lateDataBundle), new Instant(2_000_000L));
-    TransformWatermarks bufferedLateWm =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L)));
-
-    // The input watermark should be held to its previous value (not advanced due to late data; not
-    // moved backwards in the presence of watermarks due to monotonicity).
-    TransformWatermarks lateDataBufferedWatermark =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(lateDataBufferedWatermark.getInputWatermark(), not(earlierThan(sourceWatermark)));
-    assertThat(lateDataBufferedWatermark.getOutputWatermark(), not(earlierThan(sourceWatermark)));
-
-    CommittedBundle<KV<String, Integer>> lateKeyedBundle =
-        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
-    manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(lateKeyedBundle), null);
-  }
-
-  public void updateWatermarkWithDifferentWindowedValueInstances() {
-    manager.updateWatermarks(
-        null,
-        createdInts.getProducingTransformInternal(),
-        TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(
-            bundleFactory
-                .createRootBundle(createdInts)
-                .add(WindowedValue.valueInGlobalWindow(1))
-                .commit(Instant.now())),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    manager.updateWatermarks(
-        bundleFactory
-            .createRootBundle(createdInts)
-            .add(WindowedValue.valueInGlobalWindow(1))
-            .commit(Instant.now()),
-        keyed.getProducingTransformInternal(),
-        TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>emptyList(),
-        null);
-    TransformWatermarks onTimeWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    assertThat(onTimeWatermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-  }
-
-  /**
-   * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
-   * produced, the watermarks of a downstream process are advanced.
-   */
-  @Test
-  public void getWatermarksAfterOnlyEmptyOutput() {
-    CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks updatedSourceWatermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-
-    assertThat(
-        updatedSourceWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    TransformWatermarks finishedFilterWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        finishedFilterWatermarks.getInputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(
-        finishedFilterWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-  }
-
-  /**
-   * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
-   * produced, and the downstream transform has a watermark hold, the watermark is held to the hold.
-   */
-  @Test
-  public void getWatermarksAfterHoldAndEmptyOutput() {
-    CommittedBundle<Integer> firstCreateOutput = multiWindowedBundle(createdInts, 1, 2);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(firstCreateOutput), new Instant(12_000L));
-
-    CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
-    manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(firstFilterOutput),
-        new Instant(10_000L));
-    TransformWatermarks firstFilterWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(firstFilterWatermarks.getInputWatermark(), not(earlierThan(new Instant(12_000L))));
-    assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
-
-    CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks updatedSourceWatermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-
-    assertThat(
-        updatedSourceWatermarks.getOutputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-
-    TransformWatermarks finishedFilterWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        finishedFilterWatermarks.getInputWatermark(),
-        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
-    assertThat(finishedFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
-  }
-
-  @Test
-  public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() {
-    TransformWatermarks watermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-    assertThat(
-        watermarks.getSynchronizedProcessingOutputTime(),
-        equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-
-    TransformWatermarks filteredWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    // Non-root processing watermarks don't progress until data has been processed
-    assertThat(
-        filteredWatermarks.getSynchronizedProcessingInputTime(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-    assertThat(
-        filteredWatermarks.getSynchronizedProcessingOutputTime(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
-    CommittedBundle<Integer> createOutput =
-        bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
-
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks createAfterUpdate =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-
-    TransformWatermarks filterAfterProduced =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        filterAfterProduced.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
-    assertThat(
-        filterAfterProduced.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
-
-    clock.set(new Instant(1500L));
-    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-    assertThat(
-        filterAfterProduced.getSynchronizedProcessingInputTime(),
-        not(laterThan(new Instant(1250L))));
-    assertThat(
-        filterAfterProduced.getSynchronizedProcessingOutputTime(),
-        not(laterThan(new Instant(1250L))));
-
-    CommittedBundle<?> filterOutputBundle =
-        bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L));
-    manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filterOutputBundle),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks filterAfterConsumed =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    assertThat(
-        filterAfterConsumed.getSynchronizedProcessingInputTime(),
-        not(laterThan(createAfterUpdate.getSynchronizedProcessingOutputTime())));
-    assertThat(
-        filterAfterConsumed.getSynchronizedProcessingOutputTime(),
-        not(laterThan(filterAfterConsumed.getSynchronizedProcessingInputTime())));
-  }
-
-  /**
-   * Demonstrates that the Synchronized Processing Time output watermark cannot progress past
-   * pending timers in the same set. This propagates to all downstream SynchronizedProcessingTimes.
-   *
-   * <p>Also demonstrate that the result is monotonic.
-   */
-  //  @Test
-  public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() {
-    CommittedBundle<Integer> createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(1248L));
-
-    TransformWatermarks filteredWms =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    TransformWatermarks filteredDoubledWms =
-        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
-    Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
-    Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
-
-    CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 8);
-    TimerData pastTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(250L), TimeDomain.PROCESSING_TIME);
-    TimerData futureTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME);
-    TimerUpdate timers =
-        TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build();
-    manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers,
-        Collections.<CommittedBundle<?>>singleton(filteredBundle),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    Instant startTime = clock.now();
-    clock.set(startTime.plus(250L));
-    // We're held based on the past timer
-    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
-    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
-    // And we're monotonic
-    assertThat(
-        filteredWms.getSynchronizedProcessingOutputTime(), not(earlierThan(initialFilteredWm)));
-    assertThat(
-        filteredDoubledWms.getSynchronizedProcessingOutputTime(),
-        not(earlierThan(initialFilteredDoubledWm)));
-
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firedTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firedTimers.get(filtered.getProducingTransformInternal())
-            .get("key")
-            .getTimers(TimeDomain.PROCESSING_TIME),
-        contains(pastTimer));
-    // Our timer has fired, but has not been completed, so it holds our synchronized processing WM
-    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
-    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
-
-    CommittedBundle<Integer> filteredTimerBundle =
-        bundleFactory
-            .createKeyedBundle(null, "key", filtered)
-            .commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    CommittedBundle<Integer> filteredTimerResult =
-        bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo)
-            .commit(filteredWms.getSynchronizedProcessingOutputTime());
-    // Complete the processing time timer
-    manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(),
-        TimerUpdate.builder("key")
-            .withCompletedTimers(Collections.<TimerData>singleton(pastTimer))
-            .build(),
-        Collections.<CommittedBundle<?>>singleton(filteredTimerResult),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    clock.set(startTime.plus(500L));
-    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
-    // filtered should be held to the time at which the filteredTimerResult fired
-    assertThat(
-        filteredDoubledWms.getSynchronizedProcessingOutputTime(),
-        not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark())));
-
-    manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(),
-        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-
-    clock.set(new Instant(Long.MAX_VALUE));
-    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
-    assertThat(
-        filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
-  }
-
-  /**
-   * Demonstrates that if any earlier processing holds appear in the synchronized processing time
-   * output hold the result is monotonic.
-   */
-  @Test
-  public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() {
-    Instant startTime = clock.now();
-    TransformWatermarks watermarks =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(startTime));
-
-    TransformWatermarks filteredWatermarks =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    // Non-root processing watermarks don't progress until data has been processed
-    assertThat(
-        filteredWatermarks.getSynchronizedProcessingInputTime(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-    assertThat(
-        filteredWatermarks.getSynchronizedProcessingOutputTime(),
-        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
-
-    CommittedBundle<Integer> createOutput =
-        bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
-
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks createAfterUpdate =
-        manager.getWatermarks(createdInts.getProducingTransformInternal());
-    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
-    assertThat(
-        createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
-
-    CommittedBundle<Integer> createSecondOutput =
-        bundleFactory.createRootBundle(createdInts).commit(new Instant(750L));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(createSecondOutput),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-  }
-
-  @Test
-  public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
-    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(created), new Instant(40_900L));
-
-    CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 4);
-    Instant upstreamHold = new Instant(2048L);
-    TimerData upstreamProcessingTimer =
-        TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME);
-    manager.updateWatermarks(created, filtered.getProducingTransformInternal(),
-        TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(),
-        Collections.<CommittedBundle<?>>singleton(filteredBundle),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    TransformWatermarks downstreamWms =
-        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-
-    clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
-
-    manager.extractFiredTimers();
-    // Pending processing time timers that have been fired but aren't completed hold the
-    // synchronized processing time
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
-
-    CommittedBundle<Integer> otherCreated = multiWindowedBundle(createdInts, 4, 8, 12);
-    manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(),
-        TimerUpdate.builder("key")
-            .withCompletedTimers(Collections.singleton(upstreamProcessingTimer))
-            .build(),
-        Collections.<CommittedBundle<?>>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
-  }
-
-  @Test
-  public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
-    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
-    manager.updateWatermarks(
-        null,
-        createdInts.getProducingTransformInternal(),
-        TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(created),
-        new Instant(29_919_235L));
-
-    Instant upstreamHold = new Instant(2048L);
-    CommittedBundle<Integer> filteredBundle =
-        bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold);
-    manager.updateWatermarks(
-        created,
-        filtered.getProducingTransformInternal(),
-        TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>singleton(filteredBundle),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    TransformWatermarks downstreamWms =
-        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
-
-    clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
-  }
-
-  @Test
-  public void extractFiredTimersReturnsFiredEventTimeTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
-        manager.extractFiredTimers();
-    // Watermarks haven't advanced
-    assertThat(initialTimers.entrySet(), emptyIterable());
-
-    // Advance WM of keyed past the first timer, but ahead of the second and third
-    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.singleton(createdBundle), new Instant(1500L));
-
-    TimerData earliestTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME);
-    TimerData middleTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.EVENT_TIME);
-    TimerData lastTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.EVENT_TIME);
-    Object key = new Object();
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .setTimer(lastTimer)
-            .build();
-
-    manager.updateWatermarks(
-        createdBundle,
-        filtered.getProducingTransformInternal(),
-        update,
-        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
-        new Instant(1000L));
-
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> firstFilteredTimers =
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(firstFilteredTimers.get(key), not(nullValue()));
-    FiredTimers firstFired = firstFilteredTimers.get(key);
-    assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer));
-
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> secondFilteredTimers =
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(secondFilteredTimers.get(key), not(nullValue()));
-    FiredTimers secondFired = secondFilteredTimers.get(key);
-    // Contains, in order, middleTimer and then lastTimer
-    assertThat(secondFired.getTimers(TimeDomain.EVENT_TIME), contains(middleTimer, lastTimer));
-  }
-
-  @Test
-  public void extractFiredTimersReturnsFiredProcessingTimeTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
-        manager.extractFiredTimers();
-    // Watermarks haven't advanced
-    assertThat(initialTimers.entrySet(), emptyIterable());
-
-    // Advance WM of keyed past the first timer, but ahead of the second and third
-    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.singleton(createdBundle), new Instant(1500L));
-
-    TimerData earliestTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME);
-    TimerData middleTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.PROCESSING_TIME);
-    TimerData lastTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.PROCESSING_TIME);
-    Object key = new Object();
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(lastTimer)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .build();
-
-    manager.updateWatermarks(
-        createdBundle,
-        filtered.getProducingTransformInternal(),
-        update,
-        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
-        new Instant(1000L));
-
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> firstFilteredTimers =
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(firstFilteredTimers.get(key), not(nullValue()));
-    FiredTimers firstFired = firstFilteredTimers.get(key);
-    assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer));
-
-    clock.set(new Instant(50_000L));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> secondFilteredTimers =
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(secondFilteredTimers.get(key), not(nullValue()));
-    FiredTimers secondFired = secondFilteredTimers.get(key);
-    // Contains, in order, middleTimer and then lastTimer
-    assertThat(secondFired.getTimers(TimeDomain.PROCESSING_TIME), contains(middleTimer, lastTimer));
-  }
-
-  @Test
-  public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
-        manager.extractFiredTimers();
-    // Watermarks haven't advanced
-    assertThat(initialTimers.entrySet(), emptyIterable());
-
-    // Advance WM of keyed past the first timer, but ahead of the second and third
-    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.singleton(createdBundle), new Instant(1500L));
-
-    TimerData earliestTimer = TimerData.of(
-        StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    TimerData middleTimer = TimerData.of(
-        StateNamespaces.global(), new Instant(5000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    TimerData lastTimer = TimerData.of(
-        StateNamespaces.global(), new Instant(10000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    Object key = new Object();
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(lastTimer)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .build();
-
-    manager.updateWatermarks(
-        createdBundle,
-        filtered.getProducingTransformInternal(),
-        update,
-        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
-        new Instant(1000L));
-
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> firstFilteredTimers =
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(firstFilteredTimers.get(key), not(nullValue()));
-    FiredTimers firstFired = firstFilteredTimers.get(key);
-    assertThat(
-        firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer));
-
-    clock.set(new Instant(50_000L));
-    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
-        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<Object, FiredTimers> secondFilteredTimers =
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
-    assertThat(secondFilteredTimers.get(key), not(nullValue()));
-    FiredTimers secondFired = secondFilteredTimers.get(key);
-    // Contains, in order, middleTimer and then lastTimer
-    assertThat(
-        secondFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME),
-        contains(middleTimer, lastTimer));
-  }
-
-  @Test
-  public void timerUpdateBuilderBuildAddsAllAddedTimers() {
-    TimerData set = TimerData.of(StateNamespaces.global(), new Instant(10L), TimeDomain.EVENT_TIME);
-    TimerData deleted =
-        TimerData.of(StateNamespaces.global(), new Instant(24L), TimeDomain.PROCESSING_TIME);
-    TimerData completedOne = TimerData.of(
-        StateNamespaces.global(), new Instant(1024L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    TimerData completedTwo =
-        TimerData.of(StateNamespaces.global(), new Instant(2048L), TimeDomain.EVENT_TIME);
-
-    TimerUpdate update =
-        TimerUpdate.builder("foo")
-            .withCompletedTimers(ImmutableList.of(completedOne, completedTwo))
-            .setTimer(set)
-            .deletedTimer(deleted)
-            .build();
-
-    assertThat(update.getCompletedTimers(), containsInAnyOrder(completedOne, completedTwo));
-    assertThat(update.getSetTimers(), contains(set));
-    assertThat(update.getDeletedTimers(), contains(deleted));
-  }
-
-  @Test
-  public void timerUpdateBuilderWithSetThenDeleteHasOnlyDeleted() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.setTimer(timer).deletedTimer(timer).build();
-
-    assertThat(built.getSetTimers(), emptyIterable());
-    assertThat(built.getDeletedTimers(), contains(timer));
-  }
-
-  @Test
-  public void timerUpdateBuilderWithDeleteThenSetHasOnlySet() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.deletedTimer(timer).setTimer(timer).build();
-
-    assertThat(built.getSetTimers(), contains(timer));
-    assertThat(built.getDeletedTimers(), emptyIterable());
-  }
-
-  @Test
-  public void timerUpdateBuilderWithSetAfterBuildNotAddedToBuilt() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.build();
-    builder.setTimer(timer);
-    assertThat(built.getSetTimers(), emptyIterable());
-    builder.build();
-    assertThat(built.getSetTimers(), emptyIterable());
-  }
-
-  @Test
-  public void timerUpdateBuilderWithDeleteAfterBuildNotAddedToBuilt() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.build();
-    builder.deletedTimer(timer);
-    assertThat(built.getDeletedTimers(), emptyIterable());
-    builder.build();
-    assertThat(built.getDeletedTimers(), emptyIterable());
-  }
-
-  @Test
-  public void timerUpdateBuilderWithCompletedAfterBuildNotAddedToBuilt() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.build();
-    builder.withCompletedTimers(ImmutableList.of(timer));
-    assertThat(built.getCompletedTimers(), emptyIterable());
-    builder.build();
-    assertThat(built.getCompletedTimers(), emptyIterable());
-  }
-
-  @Test
-  public void timerUpdateWithCompletedTimersNotAddedToExisting() {
-    TimerUpdateBuilder builder = TimerUpdate.builder(null);
-    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
-
-    TimerUpdate built = builder.build();
-    assertThat(built.getCompletedTimers(), emptyIterable());
-    assertThat(
-        built.withCompletedTimers(ImmutableList.of(timer)).getCompletedTimers(), contains(timer));
-    assertThat(built.getCompletedTimers(), emptyIterable());
-  }
-
-  private static Matcher<Instant> earlierThan(final Instant laterInstant) {
-    return new BaseMatcher<Instant>() {
-      @Override
-      public boolean matches(Object item) {
-        ReadableInstant instant = (ReadableInstant) item;
-        return instant.isBefore(laterInstant);
-      }
-
-      @Override
-      public void describeTo(Description description) {
-        description.appendText("earlier than ").appendValue(laterInstant);
-      }
-    };
-  }
-
-  private static Matcher<Instant> laterThan(final Instant shouldBeEarlier) {
-    return new BaseMatcher<Instant>() {
-      @Override
-      public boolean matches(Object item) {
-        ReadableInstant instant = (ReadableInstant) item;
-        return instant.isAfter(shouldBeEarlier);
-      }
-
-      @Override
-      public void describeTo(Description description) {
-        description.appendText("later than ").appendValue(shouldBeEarlier);
-      }
-    };
-  }
-
-  @SafeVarargs
-  private final <T> CommittedBundle<T> timestampedBundle(
-      PCollection<T> pc, TimestampedValue<T>... values) {
-    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
-    for (TimestampedValue<T> value : values) {
-      bundle.add(
-          WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp()));
-    }
-    return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-
-  @SafeVarargs
-  private final <T> CommittedBundle<T> multiWindowedBundle(PCollection<T> pc, T... values) {
-    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
-    Collection<BoundedWindow> windows =
-        ImmutableList.of(
-            GlobalWindow.INSTANCE,
-            new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0)));
-    for (T value : values) {
-      bundle.add(
-          WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING));
-    }
-    return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
deleted file mode 100644
index 3c7c2c6..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactoryTest.java
+++ /dev/null
@@ -1,223 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * Tests for {@link InProcessBundleFactory}.
- */
-@RunWith(JUnit4.class)
-public class InProcessBundleFactoryTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private InProcessBundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  private PCollection<Integer> created;
-  private PCollection<KV<String, Integer>> downstream;
-
-  @Before
-  public void setup() {
-    TestPipeline p = TestPipeline.create();
-    created = p.apply(Create.of(1, 2, 3));
-    downstream = created.apply(WithKeys.<String, Integer>of("foo"));
-  }
-
-  @Test
-  public void createRootBundleShouldCreateWithNullKey() {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
-
-    UncommittedBundle<Integer> inFlightBundle = bundleFactory.createRootBundle(pcollection);
-
-    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
-
-    assertThat(bundle.getKey(), nullValue());
-  }
-
-  private void createKeyedBundle(Object key) {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
-
-    UncommittedBundle<Integer> inFlightBundle =
-        bundleFactory.createKeyedBundle(null, key, pcollection);
-
-    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
-    assertThat(bundle.getKey(), equalTo(key));
-  }
-
-  @Test
-  public void keyedWithNullKeyShouldCreateKeyedBundle() {
-    createKeyedBundle(null);
-  }
-
-  @Test
-  public void keyedWithKeyShouldCreateKeyedBundle() {
-    createKeyedBundle(new Object());
-  }
-
-  private <T> CommittedBundle<T>
-  afterCommitGetElementsShouldHaveAddedElements(Iterable<WindowedValue<T>> elems) {
-    PCollection<T> pcollection = TestPipeline.create().apply(Create.<T>of());
-
-    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pcollection);
-    Collection<Matcher<? super WindowedValue<T>>> expectations = new ArrayList<>();
-    for (WindowedValue<T> elem : elems) {
-      bundle.add(elem);
-      expectations.add(equalTo(elem));
-    }
-    Matcher<Iterable<? extends WindowedValue<T>>> containsMatcher =
-        Matchers.<WindowedValue<T>>containsInAnyOrder(expectations);
-    CommittedBundle<T> committed = bundle.commit(Instant.now());
-    assertThat(committed.getElements(), containsMatcher);
-
-    return committed;
-  }
-
-  @Test
-  public void getElementsBeforeAddShouldReturnEmptyIterable() {
-    afterCommitGetElementsShouldHaveAddedElements(Collections.<WindowedValue<Integer>>emptyList());
-  }
-
-  @Test
-  public void getElementsAfterAddShouldReturnAddedElements() {
-    WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
-    WindowedValue<Integer> secondValue =
-        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
-
-    afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void withElementsShouldReturnIndependentBundle() {
-    WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
-    WindowedValue<Integer> secondValue =
-        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
-
-    CommittedBundle<Integer> committed =
-        afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
-
-    WindowedValue<Integer> firstReplacement =
-        WindowedValue.of(
-            9,
-            new Instant(2048L),
-            new IntervalWindow(new Instant(2044L), Instant.now()),
-            PaneInfo.NO_FIRING);
-    WindowedValue<Integer> secondReplacement =
-        WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now());
-    CommittedBundle<Integer> withed =
-        committed.withElements(ImmutableList.of(firstReplacement, secondReplacement));
-
-    assertThat(withed.getElements(), containsInAnyOrder(firstReplacement, secondReplacement));
-    assertThat(committed.getElements(), containsInAnyOrder(firstValue, secondValue));
-    assertThat(withed.getKey(), equalTo(committed.getKey()));
-    assertThat(withed.getPCollection(), equalTo(committed.getPCollection()));
-    assertThat(
-        withed.getSynchronizedProcessingOutputWatermark(),
-        equalTo(committed.getSynchronizedProcessingOutputWatermark()));
-  }
-
-  @Test
-  public void addAfterCommitShouldThrowException() {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
-
-    UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
-    bundle.add(WindowedValue.valueInGlobalWindow(1));
-    CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
-    assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("3");
-    thrown.expectMessage("committed");
-
-    bundle.add(WindowedValue.valueInGlobalWindow(3));
-  }
-
-  @Test
-  public void commitAfterCommitShouldThrowException() {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
-
-    UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
-    bundle.add(WindowedValue.valueInGlobalWindow(1));
-    CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
-    assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("committed");
-
-    bundle.commit(Instant.now());
-  }
-
-  @Test
-  public void createBundleUnkeyedResultUnkeyed() {
-    CommittedBundle<KV<String, Integer>> newBundle =
-        bundleFactory
-            .createBundle(bundleFactory.createRootBundle(created).commit(Instant.now()), downstream)
-            .commit(Instant.now());
-  }
-
-  @Test
-  public void createBundleKeyedResultPropagatesKey() {
-    CommittedBundle<KV<String, Integer>> newBundle =
-        bundleFactory
-            .createBundle(
-                bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
-                downstream)
-            .commit(Instant.now());
-    assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
-  }
-
-  @Test
-  public void createKeyedBundleKeyed() {
-    CommittedBundle<KV<String, Integer>> keyedBundle =
-        bundleFactory
-            .createKeyedBundle(
-                bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
-            .commit(Instant.now());
-    assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
-  }
-}


[04/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
deleted file mode 100644
index 8ed2684..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactoryTest.java
+++ /dev/null
@@ -1,257 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.isA;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-
-/**
- * Tests for {@link EncodabilityEnforcementFactory}.
- */
-@RunWith(JUnit4.class)
-public class EncodabilityEnforcementFactoryTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private EncodabilityEnforcementFactory factory = EncodabilityEnforcementFactory.create();
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void encodeFailsThrows() {
-    WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
-
-    ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoEncodeCoder(), record);
-
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(CoderException.class));
-    thrown.expectMessage("Encode not allowed");
-    enforcement.beforeElement(record);
-  }
-
-  @Test
-  public void decodeFailsThrows() {
-    WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
-
-    ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoDecodeCoder(), record);
-
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(CoderException.class));
-    thrown.expectMessage("Decode not allowed");
-    enforcement.beforeElement(record);
-  }
-
-  @Test
-  public void consistentWithEqualsStructuralValueNotEqualThrows() {
-    WindowedValue<Record> record =
-        WindowedValue.<Record>valueInGlobalWindow(
-            new Record() {
-              @Override
-              public String toString() {
-                return "OriginalRecord";
-              }
-            });
-
-    ModelEnforcement<Record> enforcement =
-        createEnforcement(new RecordStructuralValueCoder(), record);
-
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(IllegalArgumentException.class));
-    thrown.expectMessage("does not maintain structural value equality");
-    thrown.expectMessage(RecordStructuralValueCoder.class.getSimpleName());
-    thrown.expectMessage("OriginalRecord");
-    enforcement.beforeElement(record);
-  }
-
-  @Test
-  public void notConsistentWithEqualsStructuralValueNotEqualSucceeds() {
-    TestPipeline p = TestPipeline.create();
-    PCollection<Record> unencodable =
-        p.apply(
-            Create.of(new Record())
-                .withCoder(new RecordNotConsistentWithEqualsStructuralValueCoder()));
-    AppliedPTransform<?, ?, ?> consumer =
-        unencodable.apply(Count.<Record>globally()).getProducingTransformInternal();
-
-    WindowedValue<Record> record = WindowedValue.<Record>valueInGlobalWindow(new Record());
-
-    CommittedBundle<Record> input =
-        bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
-    ModelEnforcement<Record> enforcement = factory.forBundle(input, consumer);
-
-    enforcement.beforeElement(record);
-    enforcement.afterElement(record);
-    enforcement.afterFinish(
-        input,
-        StepTransformResult.withoutHold(consumer).build(),
-        Collections.<CommittedBundle<?>>emptyList());
-  }
-
-  private <T> ModelEnforcement<T> createEnforcement(Coder<T> coder, WindowedValue<T> record) {
-    TestPipeline p = TestPipeline.create();
-    PCollection<T> unencodable = p.apply(Create.<T>of().withCoder(coder));
-    AppliedPTransform<?, ?, ?> consumer =
-        unencodable.apply(Count.<T>globally()).getProducingTransformInternal();
-    CommittedBundle<T> input =
-        bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
-    ModelEnforcement<T> enforcement = factory.forBundle(input, consumer);
-    return enforcement;
-  }
-
-  @Test
-  public void structurallyEqualResultsSucceeds() {
-    TestPipeline p = TestPipeline.create();
-    PCollection<Integer> unencodable = p.apply(Create.of(1).withCoder(VarIntCoder.of()));
-    AppliedPTransform<?, ?, ?> consumer =
-        unencodable.apply(Count.<Integer>globally()).getProducingTransformInternal();
-
-    WindowedValue<Integer> value = WindowedValue.valueInGlobalWindow(1);
-
-    CommittedBundle<Integer> input =
-        bundleFactory.createRootBundle(unencodable).add(value).commit(Instant.now());
-    ModelEnforcement<Integer> enforcement = factory.forBundle(input, consumer);
-
-    enforcement.beforeElement(value);
-    enforcement.afterElement(value);
-    enforcement.afterFinish(
-        input,
-        StepTransformResult.withoutHold(consumer).build(),
-        Collections.<CommittedBundle<?>>emptyList());
-  }
-
-  private static class Record {}
-  private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
-
-    @Override
-    public void encode(
-        Record value,
-        OutputStream outStream,
-        org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {
-      throw new CoderException("Encode not allowed");
-    }
-
-    @Override
-    public Record decode(
-        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {
-      return null;
-    }
-  }
-
-  private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
-    @Override
-    public void encode(
-        Record value,
-        OutputStream outStream,
-        org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {}
-
-    @Override
-    public Record decode(
-        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {
-      throw new CoderException("Decode not allowed");
-    }
-  }
-
-  private static class RecordStructuralValueCoder extends AtomicCoder<Record> {
-    @Override
-    public void encode(
-        Record value,
-        OutputStream outStream,
-        org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {}
-
-    @Override
-    public Record decode(
-        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {
-      return new Record() {
-        @Override
-        public String toString() {
-          return "DecodedRecord";
-        }
-      };
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      return true;
-    }
-
-    @Override
-    public Object structuralValue(Record value) {
-      return value;
-    }
-  }
-
-  private static class RecordNotConsistentWithEqualsStructuralValueCoder
-      extends AtomicCoder<Record> {
-    @Override
-    public void encode(
-        Record value,
-        OutputStream outStream,
-        org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {}
-
-    @Override
-    public Record decode(
-        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-        throws CoderException, IOException {
-      return new Record() {
-        @Override
-        public String toString() {
-          return "DecodedRecord";
-        }
-      };
-    }
-
-    @Override
-    public boolean consistentWithEquals() {
-      return false;
-    }
-
-    @Override
-    public Object structuralValue(Record value) {
-      return value;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.java
deleted file mode 100644
index 5c1da14..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactoryTest.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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link FlattenEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class FlattenEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
-  @Test
-  public void testFlattenInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
-    PCollection<Integer> left = p.apply("left", Create.of(1, 2, 4));
-    PCollection<Integer> right = p.apply("right", Create.of(-1, 2, -4));
-    PCollectionList<Integer> list = PCollectionList.of(left).and(right);
-
-    PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
-
-    CommittedBundle<Integer> leftBundle =
-        bundleFactory.createRootBundle(left).commit(Instant.now());
-    CommittedBundle<Integer> rightBundle =
-        bundleFactory.createRootBundle(right).commit(Instant.now());
-
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
-
-    UncommittedBundle<Integer> flattenedLeftBundle = bundleFactory.createRootBundle(flattened);
-    UncommittedBundle<Integer> flattenedRightBundle = bundleFactory.createRootBundle(flattened);
-
-    when(context.createBundle(leftBundle, flattened)).thenReturn(flattenedLeftBundle);
-    when(context.createBundle(rightBundle, flattened)).thenReturn(flattenedRightBundle);
-
-    FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
-    TransformEvaluator<Integer> leftSideEvaluator =
-        factory.forApplication(flattened.getProducingTransformInternal(), leftBundle, context);
-    TransformEvaluator<Integer> rightSideEvaluator =
-        factory.forApplication(
-            flattened.getProducingTransformInternal(),
-            rightBundle,
-            context);
-
-    leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
-    rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
-    leftSideEvaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
-    leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING));
-    rightSideEvaluator.processElement(
-        WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
-    rightSideEvaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
-
-    InProcessTransformResult rightSideResult = rightSideEvaluator.finishBundle();
-    InProcessTransformResult leftSideResult = leftSideEvaluator.finishBundle();
-
-    assertThat(
-        rightSideResult.getOutputBundles(),
-        Matchers.<UncommittedBundle<?>>contains(flattenedRightBundle));
-    assertThat(
-        rightSideResult.getTransform(),
-        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
-    assertThat(
-        leftSideResult.getOutputBundles(),
-        Matchers.<UncommittedBundle<?>>contains(flattenedLeftBundle));
-    assertThat(
-        leftSideResult.getTransform(),
-        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
-
-    assertThat(
-        flattenedLeftBundle.commit(Instant.now()).getElements(),
-        containsInAnyOrder(
-            WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
-            WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING),
-            WindowedValue.valueInGlobalWindow(1)));
-    assertThat(
-        flattenedRightBundle.commit(Instant.now()).getElements(),
-        containsInAnyOrder(
-            WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
-            WindowedValue.valueInGlobalWindow(-1)));
-  }
-
-  @Test
-  public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception {
-    TestPipeline p = TestPipeline.create();
-    PCollectionList<Integer> list = PCollectionList.empty(p);
-
-    PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
-
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
-
-    FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
-    TransformEvaluator<Integer> emptyEvaluator =
-        factory.forApplication(flattened.getProducingTransformInternal(), null, context);
-
-    InProcessTransformResult leftSideResult = emptyEvaluator.finishBundle();
-
-    assertThat(leftSideResult.getOutputBundles(), emptyIterable());
-    assertThat(
-        leftSideResult.getTransform(),
-        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
deleted file mode 100644
index 366dfc5..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransformTest.java
+++ /dev/null
@@ -1,112 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/**
- * Tests for {@link ForwardingPTransform}.
- */
-@RunWith(JUnit4.class)
-public class ForwardingPTransformTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  @Mock private PTransform<PCollection<Integer>, PCollection<String>> delegate;
-
-  private ForwardingPTransform<PCollection<Integer>, PCollection<String>> forwarding;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    forwarding =
-        new ForwardingPTransform<PCollection<Integer>, PCollection<String>>() {
-          @Override
-          protected PTransform<PCollection<Integer>, PCollection<String>> delegate() {
-            return delegate;
-          }
-        };
-  }
-
-  @Test
-  public void applyDelegates() {
-    @SuppressWarnings("unchecked")
-    PCollection<Integer> collection = mock(PCollection.class);
-    @SuppressWarnings("unchecked")
-    PCollection<String> output = mock(PCollection.class);
-    when(delegate.apply(collection)).thenReturn(output);
-    PCollection<String> result = forwarding.apply(collection);
-    assertThat(result, equalTo(output));
-  }
-
-  @Test
-  public void getNameDelegates() {
-    String name = "My_forwardingptransform-name;for!thisTest";
-    when(delegate.getName()).thenReturn(name);
-    assertThat(forwarding.getName(), equalTo(name));
-  }
-
-  @Test
-  public void validateDelegates() {
-    @SuppressWarnings("unchecked")
-    PCollection<Integer> input = mock(PCollection.class);
-    doThrow(RuntimeException.class).when(delegate).validate(input);
-
-    thrown.expect(RuntimeException.class);
-    forwarding.validate(input);
-  }
-
-  @Test
-  public void getDefaultOutputCoderDelegates() throws Exception {
-    @SuppressWarnings("unchecked")
-    PCollection<Integer> input = mock(PCollection.class);
-    @SuppressWarnings("unchecked")
-    PCollection<String> output = mock(PCollection.class);
-    @SuppressWarnings("unchecked")
-    Coder<String> outputCoder = mock(Coder.class);
-
-    when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder);
-    assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder));
-  }
-
-  @Test
-  public void populateDisplayDataDelegates() {
-    DisplayData.Builder builder = mock(DisplayData.Builder.class);
-    doThrow(RuntimeException.class).when(delegate).populateDisplayData(builder);
-
-    thrown.expect(RuntimeException.class);
-    forwarding.populateDisplayData(builder);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
deleted file mode 100644
index b7ce169..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,183 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Multiset;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link GroupByKeyEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class GroupByKeyEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void testInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
-    KV<String, Integer> firstFoo = KV.of("foo", -1);
-    KV<String, Integer> secondFoo = KV.of("foo", 1);
-    KV<String, Integer> thirdFoo = KV.of("foo", 3);
-    KV<String, Integer> firstBar = KV.of("bar", 22);
-    KV<String, Integer> secondBar = KV.of("bar", 12);
-    KV<String, Integer> firstBaz = KV.of("baz", Integer.MAX_VALUE);
-    PCollection<KV<String, Integer>> values =
-        p.apply(Create.of(firstFoo, firstBar, secondFoo, firstBaz, secondBar, thirdFoo));
-    PCollection<KV<String, WindowedValue<Integer>>> kvs =
-        values.apply(new ReifyTimestampsAndWindows<String, Integer>());
-    PCollection<KeyedWorkItem<String, Integer>> groupedKvs =
-        kvs.apply(new GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly<String, Integer>());
-
-    CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
-        bundleFactory.createRootBundle(kvs).commit(Instant.now());
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-
-    UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle =
-        bundleFactory.createKeyedBundle(null, "foo", groupedKvs);
-    UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle =
-        bundleFactory.createKeyedBundle(null, "bar", groupedKvs);
-    UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle =
-        bundleFactory.createKeyedBundle(null, "baz", groupedKvs);
-
-    when(evaluationContext.createKeyedBundle(inputBundle, "foo", groupedKvs)).thenReturn(fooBundle);
-    when(evaluationContext.createKeyedBundle(inputBundle, "bar", groupedKvs)).thenReturn(barBundle);
-    when(evaluationContext.createKeyedBundle(inputBundle, "baz", groupedKvs)).thenReturn(bazBundle);
-
-    // The input to a GroupByKey is assumed to be a KvCoder
-    @SuppressWarnings("unchecked")
-    Coder<String> keyCoder =
-        ((KvCoder<String, WindowedValue<Integer>>) kvs.getCoder()).getKeyCoder();
-    TransformEvaluator<KV<String, WindowedValue<Integer>>> evaluator =
-        new GroupByKeyEvaluatorFactory()
-            .forApplication(
-                groupedKvs.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstFoo)));
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondFoo)));
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(thirdFoo)));
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBar)));
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondBar)));
-    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBaz)));
-
-    evaluator.finishBundle();
-
-    assertThat(
-        fooBundle.commit(Instant.now()).getElements(),
-        contains(
-            new KeyedWorkItemMatcher<String, Integer>(
-                KeyedWorkItems.elementsWorkItem(
-                    "foo",
-                    ImmutableSet.of(
-                        WindowedValue.valueInGlobalWindow(-1),
-                        WindowedValue.valueInGlobalWindow(1),
-                        WindowedValue.valueInGlobalWindow(3))),
-                keyCoder)));
-    assertThat(
-        barBundle.commit(Instant.now()).getElements(),
-        contains(
-            new KeyedWorkItemMatcher<String, Integer>(
-                KeyedWorkItems.elementsWorkItem(
-                    "bar",
-                    ImmutableSet.of(
-                        WindowedValue.valueInGlobalWindow(12),
-                        WindowedValue.valueInGlobalWindow(22))),
-                keyCoder)));
-    assertThat(
-        bazBundle.commit(Instant.now()).getElements(),
-        contains(
-            new KeyedWorkItemMatcher<String, Integer>(
-                KeyedWorkItems.elementsWorkItem(
-                    "baz",
-                    ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))),
-                keyCoder)));
-  }
-
-  private <K, V> KV<K, WindowedValue<V>> gwValue(KV<K, V> kv) {
-    return KV.of(kv.getKey(), WindowedValue.valueInGlobalWindow(kv.getValue()));
-  }
-
-  private static class KeyedWorkItemMatcher<K, V>
-      extends BaseMatcher<WindowedValue<KeyedWorkItem<K, V>>> {
-    private final KeyedWorkItem<K, V> myWorkItem;
-    private final Coder<K> keyCoder;
-
-    public KeyedWorkItemMatcher(KeyedWorkItem<K, V> myWorkItem, Coder<K> keyCoder) {
-      this.myWorkItem = myWorkItem;
-      this.keyCoder = keyCoder;
-    }
-
-    @Override
-    public boolean matches(Object item) {
-      if (item == null || !(item instanceof WindowedValue)) {
-        return false;
-      }
-      WindowedValue<KeyedWorkItem<K, V>> that = (WindowedValue<KeyedWorkItem<K, V>>) item;
-      Multiset<WindowedValue<V>> myValues = HashMultiset.create();
-      Multiset<WindowedValue<V>> thatValues = HashMultiset.create();
-      for (WindowedValue<V> value : myWorkItem.elementsIterable()) {
-        myValues.add(value);
-      }
-      for (WindowedValue<V> value : that.getValue().elementsIterable()) {
-        thatValues.add(value);
-      }
-      try {
-        return myValues.equals(thatValues)
-            && keyCoder
-                .structuralValue(myWorkItem.key())
-                .equals(keyCoder.structuralValue(that.getValue().key()));
-      } catch (Exception e) {
-        return false;
-      }
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description
-          .appendText("KeyedWorkItem<K, V> containing key ")
-          .appendValue(myWorkItem.key())
-          .appendText(" and values ")
-          .appendValueList("[", ", ", "]", myWorkItem.elementsIterable());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
deleted file mode 100644
index 386eacc..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactoryTest.java
+++ /dev/null
@@ -1,220 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ImmutabilityCheckingBundleFactory}.
- */
-@RunWith(JUnit4.class)
-public class ImmutabilityCheckingBundleFactoryTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-  private ImmutabilityCheckingBundleFactory factory;
-  private PCollection<byte[]> created;
-  private PCollection<byte[]> transformed;
-
-  @Before
-  public void setup() {
-    TestPipeline p = TestPipeline.create();
-    created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
-    transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
-    factory = ImmutabilityCheckingBundleFactory.create(InProcessBundleFactory.create());
-  }
-
-  @Test
-  public void noMutationRootBundleSucceeds() {
-    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
-    byte[] array = new byte[] {0, 1, 2};
-    root.add(WindowedValue.valueInGlobalWindow(array));
-    CommittedBundle<byte[]> committed = root.commit(Instant.now());
-
-    assertThat(
-        committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
-  }
-
-  @Test
-  public void noMutationKeyedBundleSucceeds() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            new byte[] {4, 8, 12},
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    keyed.add(windowedArray);
-
-    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
-  }
-
-  @Test
-  public void noMutationCreateBundleSucceeds() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            new byte[] {4, 8, 12},
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    intermediate.add(windowedArray);
-
-    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
-  }
-
-  @Test
-  public void mutationBeforeAddRootBundleSucceeds() {
-    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
-    byte[] array = new byte[] {0, 1, 2};
-    array[1] = 2;
-    root.add(WindowedValue.valueInGlobalWindow(array));
-    CommittedBundle<byte[]> committed = root.commit(Instant.now());
-
-    assertThat(
-        committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
-  }
-
-  @Test
-  public void mutationBeforeAddKeyedBundleSucceeds() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
-    byte[] array = new byte[] {4, 8, 12};
-    array[0] = Byte.MAX_VALUE;
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            array,
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    keyed.add(windowedArray);
-
-    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
-  }
-
-  @Test
-  public void mutationBeforeAddCreateBundleSucceeds() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
-    byte[] array = new byte[] {4, 8, 12};
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            array,
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    array[2] = -3;
-    intermediate.add(windowedArray);
-
-    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
-  }
-
-  @Test
-  public void mutationAfterAddRootBundleThrows() {
-    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
-    byte[] array = new byte[] {0, 1, 2};
-    root.add(WindowedValue.valueInGlobalWindow(array));
-
-    array[1] = 2;
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(IllegalMutationException.class));
-    thrown.expectMessage("Values must not be mutated in any way after being output");
-    CommittedBundle<byte[]> committed = root.commit(Instant.now());
-  }
-
-  @Test
-  public void mutationAfterAddKeyedBundleThrows() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
-
-    byte[] array = new byte[] {4, 8, 12};
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            array,
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    keyed.add(windowedArray);
-
-    array[0] = Byte.MAX_VALUE;
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(IllegalMutationException.class));
-    thrown.expectMessage("Values must not be mutated in any way after being output");
-    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
-  }
-
-  @Test
-  public void mutationAfterAddCreateBundleThrows() {
-    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
-    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
-
-    byte[] array = new byte[] {4, 8, 12};
-    WindowedValue<byte[]> windowedArray =
-        WindowedValue.of(
-            array,
-            new Instant(891L),
-            new IntervalWindow(new Instant(0), new Instant(1000)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    intermediate.add(windowedArray);
-
-    array[2] = -3;
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(isA(IllegalMutationException.class));
-    thrown.expectMessage("Values must not be mutated in any way after being output");
-    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
-  }
-
-  private static class IdentityDoFn<T> extends DoFn<T, T> {
-    @Override
-    public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
-      c.output(c.element());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
deleted file mode 100644
index 16633ed..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactoryTest.java
+++ /dev/null
@@ -1,128 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collections;
-
-/**
- * Tests for {@link ImmutabilityEnforcementFactory}.
- */
-@RunWith(JUnit4.class)
-public class ImmutabilityEnforcementFactoryTest implements Serializable {
-  @Rule public transient ExpectedException thrown = ExpectedException.none();
-  private transient ImmutabilityEnforcementFactory factory;
-  private transient BundleFactory bundleFactory;
-  private transient PCollection<byte[]> pcollection;
-  private transient AppliedPTransform<?, ?, ?> consumer;
-
-  @Before
-  public void setup() {
-    factory = new ImmutabilityEnforcementFactory();
-    bundleFactory = InProcessBundleFactory.create();
-    TestPipeline p = TestPipeline.create();
-    pcollection =
-        p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
-            .apply(
-                ParDo.of(
-                    new DoFn<byte[], byte[]>() {
-                      @Override
-                      public void processElement(DoFn<byte[], byte[]>.ProcessContext c)
-                          throws Exception {
-                        c.element()[0] = 'b';
-                      }
-                    }));
-    consumer = pcollection.apply(Count.<byte[]>globally()).getProducingTransformInternal();
-  }
-
-  @Test
-  public void unchangedSucceeds() {
-    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
-    CommittedBundle<byte[]> elements =
-        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
-    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
-    enforcement.beforeElement(element);
-    enforcement.afterElement(element);
-    enforcement.afterFinish(
-        elements,
-        StepTransformResult.withoutHold(consumer).build(),
-        Collections.<CommittedBundle<?>>emptyList());
-  }
-
-  @Test
-  public void mutatedDuringProcessElementThrows() {
-    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
-    CommittedBundle<byte[]> elements =
-        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
-    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
-    enforcement.beforeElement(element);
-    element.getValue()[0] = 'f';
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage(consumer.getFullName());
-    thrown.expectMessage("illegaly mutated");
-    thrown.expectMessage("Input values must not be mutated");
-    enforcement.afterElement(element);
-    enforcement.afterFinish(
-        elements,
-        StepTransformResult.withoutHold(consumer).build(),
-        Collections.<CommittedBundle<?>>emptyList());
-  }
-
-  @Test
-  public void mutatedAfterProcessElementFails() {
-
-    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
-    CommittedBundle<byte[]> elements =
-        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
-
-    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
-    enforcement.beforeElement(element);
-    enforcement.afterElement(element);
-
-    element.getValue()[0] = 'f';
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage(consumer.getFullName());
-    thrown.expectMessage("illegaly mutated");
-    thrown.expectMessage("Input values must not be mutated");
-    enforcement.afterFinish(
-        elements,
-        StepTransformResult.withoutHold(consumer).build(),
-        Collections.<CommittedBundle<?>>emptyList());
-  }
-}



[11/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
new file mode 100644
index 0000000..2880ade
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
@@ -0,0 +1,1168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.joda.time.Instant;
+import org.joda.time.ReadableInstant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for {@link InMemoryWatermarkManager}.
+ */
+@RunWith(JUnit4.class)
+public class InMemoryWatermarkManagerTest implements Serializable {
+  private transient MockClock clock;
+
+  private transient PCollection<Integer> createdInts;
+
+  private transient PCollection<Integer> filtered;
+  private transient PCollection<Integer> filteredTimesTwo;
+  private transient PCollection<KV<String, Integer>> keyed;
+
+  private transient PCollection<Integer> intsToFlatten;
+  private transient PCollection<Integer> flattened;
+
+  private transient InMemoryWatermarkManager manager;
+  private transient BundleFactory bundleFactory;
+
+  @Before
+  public void setup() {
+    TestPipeline p = TestPipeline.create();
+
+    createdInts = p.apply("createdInts", Create.of(1, 2, 3));
+
+    filtered = createdInts.apply("filtered", Filter.greaterThan(1));
+    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
+      @Override
+      public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
+        c.output(c.element() * 2);
+      }
+    }));
+
+    keyed = createdInts.apply("keyed", WithKeys.<String, Integer>of("MyKey"));
+
+    intsToFlatten = p.apply("intsToFlatten", Create.of(-1, 256, 65535));
+    PCollectionList<Integer> preFlatten = PCollectionList.of(createdInts).and(intsToFlatten);
+    flattened = preFlatten.apply("flattened", Flatten.<Integer>pCollections());
+
+    Collection<AppliedPTransform<?, ?, ?>> rootTransforms =
+        ImmutableList.<AppliedPTransform<?, ?, ?>>of(
+            createdInts.getProducingTransformInternal(),
+            intsToFlatten.getProducingTransformInternal());
+
+    Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers = new HashMap<>();
+    consumers.put(
+        createdInts,
+        ImmutableList.<AppliedPTransform<?, ?, ?>>of(filtered.getProducingTransformInternal(),
+            keyed.getProducingTransformInternal(), flattened.getProducingTransformInternal()));
+    consumers.put(
+        filtered,
+        Collections.<AppliedPTransform<?, ?, ?>>singleton(
+            filteredTimesTwo.getProducingTransformInternal()));
+    consumers.put(filteredTimesTwo, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+    consumers.put(keyed, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+
+    consumers.put(
+        intsToFlatten,
+        Collections.<AppliedPTransform<?, ?, ?>>singleton(
+            flattened.getProducingTransformInternal()));
+    consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
+
+    clock = MockClock.fromInstant(new Instant(1000));
+
+    manager = InMemoryWatermarkManager.create(clock, rootTransforms, consumers);
+    bundleFactory = InProcessBundleFactory.create();
+  }
+
+  /**
+   * Demonstrates that getWatermark, when called on an {@link AppliedPTransform} that has not
+   * processed any elements, returns the {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
+   */
+  @Test
+  public void getWatermarkForUntouchedTransform() {
+    TransformWatermarks watermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+    assertThat(watermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+    assertThat(watermarks.getOutputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+  }
+
+  /**
+   * Demonstrates that getWatermark for a transform that consumes no input uses the Watermark
+   * Hold value provided to it as the output watermark.
+   */
+  @Test
+  public void getWatermarkForUpdatedSourceTransform() {
+    CommittedBundle<Integer> output = multiWindowedBundle(createdInts, 1);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(output), new Instant(8000L));
+    TransformWatermarks updatedSourceWatermark =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+    assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L)));
+  }
+
+  /**
+   * Demonstrates that getWatermark for a transform that takes multiple inputs is held to the
+   * minimum watermark across all of its inputs.
+   */
+  @Test
+  public void getWatermarkForMultiInputTransform() {
+    CommittedBundle<Integer> secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1);
+
+    manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    // We didn't do anything for the first source, so we shouldn't have progressed the watermark
+    TransformWatermarks firstSourceWatermark =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(
+        firstSourceWatermark.getOutputWatermark(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+    // the Second Source output all of the elements so it should be done (with a watermark at the
+    // end of time).
+    TransformWatermarks secondSourceWatermark =
+        manager.getWatermarks(intsToFlatten.getProducingTransformInternal());
+    assertThat(
+        secondSourceWatermark.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    // We haven't consumed anything yet, so our watermark should be at the beginning of time
+    TransformWatermarks transformWatermark =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    assertThat(
+        transformWatermark.getInputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+    assertThat(
+        transformWatermark.getOutputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+    CommittedBundle<Integer> flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1);
+    // We have finished processing the bundle from the second PCollection, but we haven't consumed
+    // anything from the first PCollection yet; so our watermark shouldn't advance
+    manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
+        null);
+    TransformWatermarks transformAfterProcessing =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate),
+        null);
+    assertThat(
+        transformAfterProcessing.getInputWatermark(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+    assertThat(
+        transformAfterProcessing.getOutputWatermark(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+    Instant firstCollectionTimestamp = new Instant(10000);
+    CommittedBundle<Integer> firstPcollectionBundle =
+        timestampedBundle(createdInts, TimestampedValue.<Integer>of(5, firstCollectionTimestamp));
+    // the source is done, but elements are still buffered. The source output watermark should be
+    // past the end of the global window
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle),
+        new Instant(Long.MAX_VALUE));
+    TransformWatermarks firstSourceWatermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(
+        firstSourceWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    // We still haven't consumed any of the first source's input, so the watermark should still not
+    // progress
+    TransformWatermarks flattenAfterSourcesProduced =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    assertThat(
+        flattenAfterSourcesProduced.getInputWatermark(), not(laterThan(firstCollectionTimestamp)));
+    assertThat(
+        flattenAfterSourcesProduced.getOutputWatermark(), not(laterThan(firstCollectionTimestamp)));
+
+    // We have buffered inputs, but since the PCollection has all of the elements (has a WM past the
+    // end of the global window), we should have a watermark equal to the min among buffered
+    // elements
+    TransformWatermarks withBufferedElements =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    assertThat(withBufferedElements.getInputWatermark(), equalTo(firstCollectionTimestamp));
+    assertThat(withBufferedElements.getOutputWatermark(), equalTo(firstCollectionTimestamp));
+
+    CommittedBundle<?> completedFlattenBundle =
+        bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(completedFlattenBundle),
+        null);
+    TransformWatermarks afterConsumingAllInput =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    assertThat(
+        afterConsumingAllInput.getInputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(
+        afterConsumingAllInput.getOutputWatermark(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+  }
+
+  /**
+   * Demonstrates that pending elements are independent among
+   * {@link AppliedPTransform AppliedPTransforms} that consume the same input {@link PCollection}.
+   */
+  @Test
+  public void getWatermarkForMultiConsumedCollection() {
+    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+        TimestampedValue.of(3, new Instant(-1000L)));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+    TransformWatermarks createdAfterProducing =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(
+        createdAfterProducing.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    CommittedBundle<KV<String, Integer>> keyBundle =
+        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
+    TransformWatermarks keyedWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(
+        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(
+        keyedWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    TransformWatermarks filteredWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(filteredWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
+    assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
+
+    CommittedBundle<Integer> filteredBundle =
+        timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L)));
+    manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filteredBundle), null);
+    TransformWatermarks filteredProcessedWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        filteredProcessedWatermarks.getInputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(
+        filteredProcessedWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+  }
+
+  /**
+   * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
+   * watermark hold.
+   */
+  @Test
+  public void updateWatermarkWithWatermarkHolds() {
+    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+        TimestampedValue.of(3, new Instant(-1000L)));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+
+    CommittedBundle<KV<String, Integer>> keyBundle =
+        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
+        new Instant(500L));
+    TransformWatermarks keyedWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(
+        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
+  }
+
+  /**
+   * Demonstrates that the watermark of an {@link AppliedPTransform} is held to the provided
+   * watermark hold.
+   */
+  @Test
+  public void updateWatermarkWithKeyedWatermarkHolds() {
+    CommittedBundle<Integer> firstKeyBundle =
+        bundleFactory.createKeyedBundle(null, "Odd", createdInts)
+            .add(WindowedValue.timestampedValueInGlobalWindow(1, new Instant(1_000_000L)))
+            .add(WindowedValue.timestampedValueInGlobalWindow(3, new Instant(-1000L)))
+            .commit(clock.now());
+
+    CommittedBundle<Integer> secondKeyBundle =
+        bundleFactory.createKeyedBundle(null, "Even", createdInts)
+            .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
+            .commit(clock.now());
+
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(-1000L));
+    manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(1234L));
+
+    TransformWatermarks filteredWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        filteredWatermarks.getInputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L))));
+
+    CommittedBundle<Integer> fauxFirstKeyTimerBundle =
+        bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now());
+    manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
+
+    CommittedBundle<Integer> fauxSecondKeyTimerBundle =
+        bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now());
+    manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(), new Instant(5678L));
+    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
+
+    manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    assertThat(
+        filteredWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+  }
+
+  /**
+   * Demonstrates that updated output watermarks are monotonic in the presence of late data, when
+   * called on an {@link AppliedPTransform} that consumes no input.
+   */
+  @Test
+  public void updateOutputWatermarkShouldBeMonotonic() {
+    CommittedBundle<?> firstInput =
+        bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(firstInput), new Instant(0L));
+    TransformWatermarks firstWatermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L)));
+
+    CommittedBundle<?> secondInput =
+        bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(secondInput), new Instant(-250L));
+    TransformWatermarks secondWatermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L))));
+  }
+
+  /**
+   * Demonstrates that updated output watermarks are monotonic in the presence of watermark holds
+   * that become earlier than a previous watermark hold.
+   */
+  @Test
+  public void updateWatermarkWithHoldsShouldBeMonotonic() {
+    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+        TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)),
+        TimestampedValue.of(3, new Instant(-1000L)));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(Long.MAX_VALUE));
+
+    CommittedBundle<KV<String, Integer>> keyBundle =
+        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)),
+            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)),
+            TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle),
+        new Instant(500L));
+    TransformWatermarks keyedWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(
+        keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L))));
+    Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark();
+
+    TransformWatermarks updatedWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(
+        updatedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    // We added a hold prior to the old watermark; we shouldn't progress (due to the earlier hold)
+    // but the watermark is monotonic and should not backslide to the new, earlier hold
+    assertThat(updatedWatermarks.getOutputWatermark(), equalTo(oldOutputWatermark));
+  }
+
+  /**
+   * Demonstrates that updateWatermarks in the presence of late data is monotonic.
+   */
+  @Test
+  public void updateWatermarkWithLateData() {
+    Instant sourceWatermark = new Instant(1_000_000L);
+    CommittedBundle<Integer> createdBundle = timestampedBundle(createdInts,
+        TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L)));
+
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createdBundle), sourceWatermark);
+
+    CommittedBundle<KV<String, Integer>> keyBundle =
+        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark),
+            TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)));
+
+    // Finish processing the on-time data. The watermarks should progress to be equal to the source
+    manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(keyBundle), null);
+    TransformWatermarks onTimeWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark));
+    assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark));
+
+    CommittedBundle<Integer> lateDataBundle =
+        timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L)));
+    // the late data arrives in a downstream PCollection after its watermark has advanced past it;
+    // we don't advance the watermark past the current watermark until we've consumed the late data
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(lateDataBundle), new Instant(2_000_000L));
+    TransformWatermarks bufferedLateWm =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L)));
+
+    // The input watermark should be held to its previous value (not advanced due to late data; not
+    // moved backwards in the presence of watermarks due to monotonicity).
+    TransformWatermarks lateDataBufferedWatermark =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(lateDataBufferedWatermark.getInputWatermark(), not(earlierThan(sourceWatermark)));
+    assertThat(lateDataBufferedWatermark.getOutputWatermark(), not(earlierThan(sourceWatermark)));
+
+    CommittedBundle<KV<String, Integer>> lateKeyedBundle =
+        timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L)));
+    manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(lateKeyedBundle), null);
+  }
+
+  public void updateWatermarkWithDifferentWindowedValueInstances() {
+    manager.updateWatermarks(
+        null,
+        createdInts.getProducingTransformInternal(),
+        TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(
+            bundleFactory
+                .createRootBundle(createdInts)
+                .add(WindowedValue.valueInGlobalWindow(1))
+                .commit(Instant.now())),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    manager.updateWatermarks(
+        bundleFactory
+            .createRootBundle(createdInts)
+            .add(WindowedValue.valueInGlobalWindow(1))
+            .commit(Instant.now()),
+        keyed.getProducingTransformInternal(),
+        TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>emptyList(),
+        null);
+    TransformWatermarks onTimeWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    assertThat(onTimeWatermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+  }
+
+  /**
+   * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
+   * produced, the watermarks of a downstream process are advanced.
+   */
+  @Test
+  public void getWatermarksAfterOnlyEmptyOutput() {
+    CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks updatedSourceWatermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+    assertThat(
+        updatedSourceWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    TransformWatermarks finishedFilterWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        finishedFilterWatermarks.getInputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(
+        finishedFilterWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+  }
+
+  /**
+   * Demonstrates that after watermarks of an upstream transform are updated, but no output has been
+   * produced, and the downstream transform has a watermark hold, the watermark is held to the hold.
+   */
+  @Test
+  public void getWatermarksAfterHoldAndEmptyOutput() {
+    CommittedBundle<Integer> firstCreateOutput = multiWindowedBundle(createdInts, 1, 2);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(firstCreateOutput), new Instant(12_000L));
+
+    CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
+    manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(firstFilterOutput),
+        new Instant(10_000L));
+    TransformWatermarks firstFilterWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(firstFilterWatermarks.getInputWatermark(), not(earlierThan(new Instant(12_000L))));
+    assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
+
+    CommittedBundle<Integer> emptyCreateOutput = multiWindowedBundle(createdInts);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(emptyCreateOutput),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks updatedSourceWatermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+
+    assertThat(
+        updatedSourceWatermarks.getOutputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+
+    TransformWatermarks finishedFilterWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        finishedFilterWatermarks.getInputWatermark(),
+        not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE)));
+    assertThat(finishedFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L))));
+  }
+
+  @Test
+  public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() {
+    TransformWatermarks watermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+    assertThat(
+        watermarks.getSynchronizedProcessingOutputTime(),
+        equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+    TransformWatermarks filteredWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    // Non-root processing watermarks don't progress until data has been processed
+    assertThat(
+        filteredWatermarks.getSynchronizedProcessingInputTime(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+    assertThat(
+        filteredWatermarks.getSynchronizedProcessingOutputTime(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+    CommittedBundle<Integer> createOutput =
+        bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
+
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks createAfterUpdate =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+
+    TransformWatermarks filterAfterProduced =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        filterAfterProduced.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
+    assertThat(
+        filterAfterProduced.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+
+    clock.set(new Instant(1500L));
+    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+    assertThat(
+        filterAfterProduced.getSynchronizedProcessingInputTime(),
+        not(laterThan(new Instant(1250L))));
+    assertThat(
+        filterAfterProduced.getSynchronizedProcessingOutputTime(),
+        not(laterThan(new Instant(1250L))));
+
+    CommittedBundle<?> filterOutputBundle =
+        bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L));
+    manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>singleton(filterOutputBundle),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks filterAfterConsumed =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    assertThat(
+        filterAfterConsumed.getSynchronizedProcessingInputTime(),
+        not(laterThan(createAfterUpdate.getSynchronizedProcessingOutputTime())));
+    assertThat(
+        filterAfterConsumed.getSynchronizedProcessingOutputTime(),
+        not(laterThan(filterAfterConsumed.getSynchronizedProcessingInputTime())));
+  }
+
+  /**
+   * Demonstrates that the Synchronized Processing Time output watermark cannot progress past
+   * pending timers in the same set. This propagates to all downstream SynchronizedProcessingTimes.
+   *
+   * <p>Also demonstrate that the result is monotonic.
+   */
+  //  @Test
+  public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() {
+    CommittedBundle<Integer> createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createdBundle), new Instant(1248L));
+
+    TransformWatermarks filteredWms =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    TransformWatermarks filteredDoubledWms =
+        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+    Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
+    Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
+
+    CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 8);
+    TimerData pastTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(250L), TimeDomain.PROCESSING_TIME);
+    TimerData futureTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME);
+    TimerUpdate timers =
+        TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build();
+    manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers,
+        Collections.<CommittedBundle<?>>singleton(filteredBundle),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    Instant startTime = clock.now();
+    clock.set(startTime.plus(250L));
+    // We're held based on the past timer
+    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+    // And we're monotonic
+    assertThat(
+        filteredWms.getSynchronizedProcessingOutputTime(), not(earlierThan(initialFilteredWm)));
+    assertThat(
+        filteredDoubledWms.getSynchronizedProcessingOutputTime(),
+        not(earlierThan(initialFilteredDoubledWm)));
+
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firedTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firedTimers.get(filtered.getProducingTransformInternal())
+            .get("key")
+            .getTimers(TimeDomain.PROCESSING_TIME),
+        contains(pastTimer));
+    // Our timer has fired, but has not been completed, so it holds our synchronized processing WM
+    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(laterThan(startTime)));
+
+    CommittedBundle<Integer> filteredTimerBundle =
+        bundleFactory
+            .createKeyedBundle(null, "key", filtered)
+            .commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    CommittedBundle<Integer> filteredTimerResult =
+        bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo)
+            .commit(filteredWms.getSynchronizedProcessingOutputTime());
+    // Complete the processing time timer
+    manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(),
+        TimerUpdate.builder("key")
+            .withCompletedTimers(Collections.<TimerData>singleton(pastTimer))
+            .build(),
+        Collections.<CommittedBundle<?>>singleton(filteredTimerResult),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    clock.set(startTime.plus(500L));
+    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+    // filtered should be held to the time at which the filteredTimerResult fired
+    assertThat(
+        filteredDoubledWms.getSynchronizedProcessingOutputTime(),
+        not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark())));
+
+    manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(),
+        TimerUpdate.empty(), Collections.<CommittedBundle<?>>emptyList(),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+
+    clock.set(new Instant(Long.MAX_VALUE));
+    assertThat(filteredWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
+    assertThat(
+        filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(new Instant(4096)));
+  }
+
+  /**
+   * Demonstrates that if any earlier processing holds appear in the synchronized processing time
+   * output hold the result is monotonic.
+   */
+  @Test
+  public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() {
+    Instant startTime = clock.now();
+    TransformWatermarks watermarks =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(startTime));
+
+    TransformWatermarks filteredWatermarks =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    // Non-root processing watermarks don't progress until data has been processed
+    assertThat(
+        filteredWatermarks.getSynchronizedProcessingInputTime(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+    assertThat(
+        filteredWatermarks.getSynchronizedProcessingOutputTime(),
+        not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE)));
+
+    CommittedBundle<Integer> createOutput =
+        bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L));
+
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks createAfterUpdate =
+        manager.getWatermarks(createdInts.getProducingTransformInternal());
+    assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now())));
+    assertThat(
+        createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now())));
+
+    CommittedBundle<Integer> createSecondOutput =
+        bundleFactory.createRootBundle(createdInts).commit(new Instant(750L));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(createSecondOutput),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+  }
+
+  @Test
+  public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
+    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(created), new Instant(40_900L));
+
+    CommittedBundle<Integer> filteredBundle = multiWindowedBundle(filtered, 2, 4);
+    Instant upstreamHold = new Instant(2048L);
+    TimerData upstreamProcessingTimer =
+        TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME);
+    manager.updateWatermarks(created, filtered.getProducingTransformInternal(),
+        TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(),
+        Collections.<CommittedBundle<?>>singleton(filteredBundle),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    TransformWatermarks downstreamWms =
+        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+
+    clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+
+    manager.extractFiredTimers();
+    // Pending processing time timers that have been fired but aren't completed hold the
+    // synchronized processing time
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+
+    CommittedBundle<Integer> otherCreated = multiWindowedBundle(createdInts, 4, 8, 12);
+    manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(),
+        TimerUpdate.builder("key")
+            .withCompletedTimers(Collections.singleton(upstreamProcessingTimer))
+            .build(),
+        Collections.<CommittedBundle<?>>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
+  }
+
+  @Test
+  public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
+    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+    manager.updateWatermarks(
+        null,
+        createdInts.getProducingTransformInternal(),
+        TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(created),
+        new Instant(29_919_235L));
+
+    Instant upstreamHold = new Instant(2048L);
+    CommittedBundle<Integer> filteredBundle =
+        bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold);
+    manager.updateWatermarks(
+        created,
+        filtered.getProducingTransformInternal(),
+        TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>singleton(filteredBundle),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    TransformWatermarks downstreamWms =
+        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now()));
+
+    clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE);
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold));
+  }
+
+  @Test
+  public void extractFiredTimersReturnsFiredEventTimeTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+        manager.extractFiredTimers();
+    // Watermarks haven't advanced
+    assertThat(initialTimers.entrySet(), emptyIterable());
+
+    // Advance WM of keyed past the first timer, but ahead of the second and third
+    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.singleton(createdBundle), new Instant(1500L));
+
+    TimerData earliestTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME);
+    TimerData middleTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.EVENT_TIME);
+    TimerData lastTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.EVENT_TIME);
+    Object key = new Object();
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .setTimer(lastTimer)
+            .build();
+
+    manager.updateWatermarks(
+        createdBundle,
+        filtered.getProducingTransformInternal(),
+        update,
+        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+        new Instant(1000L));
+
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> firstFilteredTimers =
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(firstFilteredTimers.get(key), not(nullValue()));
+    FiredTimers firstFired = firstFilteredTimers.get(key);
+    assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer));
+
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> secondFilteredTimers =
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(secondFilteredTimers.get(key), not(nullValue()));
+    FiredTimers secondFired = secondFilteredTimers.get(key);
+    // Contains, in order, middleTimer and then lastTimer
+    assertThat(secondFired.getTimers(TimeDomain.EVENT_TIME), contains(middleTimer, lastTimer));
+  }
+
+  @Test
+  public void extractFiredTimersReturnsFiredProcessingTimeTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+        manager.extractFiredTimers();
+    // Watermarks haven't advanced
+    assertThat(initialTimers.entrySet(), emptyIterable());
+
+    // Advance WM of keyed past the first timer, but ahead of the second and third
+    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.singleton(createdBundle), new Instant(1500L));
+
+    TimerData earliestTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME);
+    TimerData middleTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(5000L), TimeDomain.PROCESSING_TIME);
+    TimerData lastTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(10000L), TimeDomain.PROCESSING_TIME);
+    Object key = new Object();
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(lastTimer)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .build();
+
+    manager.updateWatermarks(
+        createdBundle,
+        filtered.getProducingTransformInternal(),
+        update,
+        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+        new Instant(1000L));
+
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> firstFilteredTimers =
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(firstFilteredTimers.get(key), not(nullValue()));
+    FiredTimers firstFired = firstFilteredTimers.get(key);
+    assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer));
+
+    clock.set(new Instant(50_000L));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> secondFilteredTimers =
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(secondFilteredTimers.get(key), not(nullValue()));
+    FiredTimers secondFired = secondFilteredTimers.get(key);
+    // Contains, in order, middleTimer and then lastTimer
+    assertThat(secondFired.getTimers(TimeDomain.PROCESSING_TIME), contains(middleTimer, lastTimer));
+  }
+
+  @Test
+  public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> initialTimers =
+        manager.extractFiredTimers();
+    // Watermarks haven't advanced
+    assertThat(initialTimers.entrySet(), emptyIterable());
+
+    // Advance WM of keyed past the first timer, but ahead of the second and third
+    CommittedBundle<Integer> createdBundle = multiWindowedBundle(filtered);
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.singleton(createdBundle), new Instant(1500L));
+
+    TimerData earliestTimer = TimerData.of(
+        StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    TimerData middleTimer = TimerData.of(
+        StateNamespaces.global(), new Instant(5000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    TimerData lastTimer = TimerData.of(
+        StateNamespaces.global(), new Instant(10000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    Object key = new Object();
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(lastTimer)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .build();
+
+    manager.updateWatermarks(
+        createdBundle,
+        filtered.getProducingTransformInternal(),
+        update,
+        Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten)),
+        new Instant(1000L));
+
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> firstFilteredTimers =
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(firstFilteredTimers.get(key), not(nullValue()));
+    FiredTimers firstFired = firstFilteredTimers.get(key);
+    assertThat(
+        firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer));
+
+    clock.set(new Instant(50_000L));
+    manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(),
+        Collections.<CommittedBundle<?>>emptyList(), new Instant(50_000L));
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<Object, FiredTimers> secondFilteredTimers =
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal());
+    assertThat(secondFilteredTimers.get(key), not(nullValue()));
+    FiredTimers secondFired = secondFilteredTimers.get(key);
+    // Contains, in order, middleTimer and then lastTimer
+    assertThat(
+        secondFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME),
+        contains(middleTimer, lastTimer));
+  }
+
+  @Test
+  public void timerUpdateBuilderBuildAddsAllAddedTimers() {
+    TimerData set = TimerData.of(StateNamespaces.global(), new Instant(10L), TimeDomain.EVENT_TIME);
+    TimerData deleted =
+        TimerData.of(StateNamespaces.global(), new Instant(24L), TimeDomain.PROCESSING_TIME);
+    TimerData completedOne = TimerData.of(
+        StateNamespaces.global(), new Instant(1024L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    TimerData completedTwo =
+        TimerData.of(StateNamespaces.global(), new Instant(2048L), TimeDomain.EVENT_TIME);
+
+    TimerUpdate update =
+        TimerUpdate.builder("foo")
+            .withCompletedTimers(ImmutableList.of(completedOne, completedTwo))
+            .setTimer(set)
+            .deletedTimer(deleted)
+            .build();
+
+    assertThat(update.getCompletedTimers(), containsInAnyOrder(completedOne, completedTwo));
+    assertThat(update.getSetTimers(), contains(set));
+    assertThat(update.getDeletedTimers(), contains(deleted));
+  }
+
+  @Test
+  public void timerUpdateBuilderWithSetThenDeleteHasOnlyDeleted() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.setTimer(timer).deletedTimer(timer).build();
+
+    assertThat(built.getSetTimers(), emptyIterable());
+    assertThat(built.getDeletedTimers(), contains(timer));
+  }
+
+  @Test
+  public void timerUpdateBuilderWithDeleteThenSetHasOnlySet() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.deletedTimer(timer).setTimer(timer).build();
+
+    assertThat(built.getSetTimers(), contains(timer));
+    assertThat(built.getDeletedTimers(), emptyIterable());
+  }
+
+  @Test
+  public void timerUpdateBuilderWithSetAfterBuildNotAddedToBuilt() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.build();
+    builder.setTimer(timer);
+    assertThat(built.getSetTimers(), emptyIterable());
+    builder.build();
+    assertThat(built.getSetTimers(), emptyIterable());
+  }
+
+  @Test
+  public void timerUpdateBuilderWithDeleteAfterBuildNotAddedToBuilt() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.build();
+    builder.deletedTimer(timer);
+    assertThat(built.getDeletedTimers(), emptyIterable());
+    builder.build();
+    assertThat(built.getDeletedTimers(), emptyIterable());
+  }
+
+  @Test
+  public void timerUpdateBuilderWithCompletedAfterBuildNotAddedToBuilt() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.build();
+    builder.withCompletedTimers(ImmutableList.of(timer));
+    assertThat(built.getCompletedTimers(), emptyIterable());
+    builder.build();
+    assertThat(built.getCompletedTimers(), emptyIterable());
+  }
+
+  @Test
+  public void timerUpdateWithCompletedTimersNotAddedToExisting() {
+    TimerUpdateBuilder builder = TimerUpdate.builder(null);
+    TimerData timer = TimerData.of(StateNamespaces.global(), Instant.now(), TimeDomain.EVENT_TIME);
+
+    TimerUpdate built = builder.build();
+    assertThat(built.getCompletedTimers(), emptyIterable());
+    assertThat(
+        built.withCompletedTimers(ImmutableList.of(timer)).getCompletedTimers(), contains(timer));
+    assertThat(built.getCompletedTimers(), emptyIterable());
+  }
+
+  private static Matcher<Instant> earlierThan(final Instant laterInstant) {
+    return new BaseMatcher<Instant>() {
+      @Override
+      public boolean matches(Object item) {
+        ReadableInstant instant = (ReadableInstant) item;
+        return instant.isBefore(laterInstant);
+      }
+
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("earlier than ").appendValue(laterInstant);
+      }
+    };
+  }
+
+  private static Matcher<Instant> laterThan(final Instant shouldBeEarlier) {
+    return new BaseMatcher<Instant>() {
+      @Override
+      public boolean matches(Object item) {
+        ReadableInstant instant = (ReadableInstant) item;
+        return instant.isAfter(shouldBeEarlier);
+      }
+
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("later than ").appendValue(shouldBeEarlier);
+      }
+    };
+  }
+
+  @SafeVarargs
+  private final <T> CommittedBundle<T> timestampedBundle(
+      PCollection<T> pc, TimestampedValue<T>... values) {
+    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
+    for (TimestampedValue<T> value : values) {
+      bundle.add(
+          WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp()));
+    }
+    return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+  }
+
+  @SafeVarargs
+  private final <T> CommittedBundle<T> multiWindowedBundle(PCollection<T> pc, T... values) {
+    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pc);
+    Collection<BoundedWindow> windows =
+        ImmutableList.of(
+            GlobalWindow.INSTANCE,
+            new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0)));
+    for (T value : values) {
+      bundle.add(
+          WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING));
+    }
+    return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
new file mode 100644
index 0000000..1809dc6
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Tests for {@link InProcessBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessBundleFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private InProcessBundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  private PCollection<Integer> created;
+  private PCollection<KV<String, Integer>> downstream;
+
+  @Before
+  public void setup() {
+    TestPipeline p = TestPipeline.create();
+    created = p.apply(Create.of(1, 2, 3));
+    downstream = created.apply(WithKeys.<String, Integer>of("foo"));
+  }
+
+  @Test
+  public void createRootBundleShouldCreateWithNullKey() {
+    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+
+    UncommittedBundle<Integer> inFlightBundle = bundleFactory.createRootBundle(pcollection);
+
+    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
+
+    assertThat(bundle.getKey(), nullValue());
+  }
+
+  private void createKeyedBundle(Object key) {
+    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+
+    UncommittedBundle<Integer> inFlightBundle =
+        bundleFactory.createKeyedBundle(null, key, pcollection);
+
+    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
+    assertThat(bundle.getKey(), equalTo(key));
+  }
+
+  @Test
+  public void keyedWithNullKeyShouldCreateKeyedBundle() {
+    createKeyedBundle(null);
+  }
+
+  @Test
+  public void keyedWithKeyShouldCreateKeyedBundle() {
+    createKeyedBundle(new Object());
+  }
+
+  private <T> CommittedBundle<T>
+  afterCommitGetElementsShouldHaveAddedElements(Iterable<WindowedValue<T>> elems) {
+    PCollection<T> pcollection = TestPipeline.create().apply(Create.<T>of());
+
+    UncommittedBundle<T> bundle = bundleFactory.createRootBundle(pcollection);
+    Collection<Matcher<? super WindowedValue<T>>> expectations = new ArrayList<>();
+    for (WindowedValue<T> elem : elems) {
+      bundle.add(elem);
+      expectations.add(equalTo(elem));
+    }
+    Matcher<Iterable<? extends WindowedValue<T>>> containsMatcher =
+        Matchers.<WindowedValue<T>>containsInAnyOrder(expectations);
+    CommittedBundle<T> committed = bundle.commit(Instant.now());
+    assertThat(committed.getElements(), containsMatcher);
+
+    return committed;
+  }
+
+  @Test
+  public void getElementsBeforeAddShouldReturnEmptyIterable() {
+    afterCommitGetElementsShouldHaveAddedElements(Collections.<WindowedValue<Integer>>emptyList());
+  }
+
+  @Test
+  public void getElementsAfterAddShouldReturnAddedElements() {
+    WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
+    WindowedValue<Integer> secondValue =
+        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
+
+    afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void withElementsShouldReturnIndependentBundle() {
+    WindowedValue<Integer> firstValue = WindowedValue.valueInGlobalWindow(1);
+    WindowedValue<Integer> secondValue =
+        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L));
+
+    CommittedBundle<Integer> committed =
+        afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue));
+
+    WindowedValue<Integer> firstReplacement =
+        WindowedValue.of(
+            9,
+            new Instant(2048L),
+            new IntervalWindow(new Instant(2044L), Instant.now()),
+            PaneInfo.NO_FIRING);
+    WindowedValue<Integer> secondReplacement =
+        WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now());
+    CommittedBundle<Integer> withed =
+        committed.withElements(ImmutableList.of(firstReplacement, secondReplacement));
+
+    assertThat(withed.getElements(), containsInAnyOrder(firstReplacement, secondReplacement));
+    assertThat(committed.getElements(), containsInAnyOrder(firstValue, secondValue));
+    assertThat(withed.getKey(), equalTo(committed.getKey()));
+    assertThat(withed.getPCollection(), equalTo(committed.getPCollection()));
+    assertThat(
+        withed.getSynchronizedProcessingOutputWatermark(),
+        equalTo(committed.getSynchronizedProcessingOutputWatermark()));
+  }
+
+  @Test
+  public void addAfterCommitShouldThrowException() {
+    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
+
+    UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
+    bundle.add(WindowedValue.valueInGlobalWindow(1));
+    CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
+    assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("3");
+    thrown.expectMessage("committed");
+
+    bundle.add(WindowedValue.valueInGlobalWindow(3));
+  }
+
+  @Test
+  public void commitAfterCommitShouldThrowException() {
+    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.<Integer>of());
+
+    UncommittedBundle<Integer> bundle = bundleFactory.createRootBundle(pcollection);
+    bundle.add(WindowedValue.valueInGlobalWindow(1));
+    CommittedBundle<Integer> firstCommit = bundle.commit(Instant.now());
+    assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1)));
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("committed");
+
+    bundle.commit(Instant.now());
+  }
+
+  @Test
+  public void createBundleUnkeyedResultUnkeyed() {
+    CommittedBundle<KV<String, Integer>> newBundle =
+        bundleFactory
+            .createBundle(bundleFactory.createRootBundle(created).commit(Instant.now()), downstream)
+            .commit(Instant.now());
+  }
+
+  @Test
+  public void createBundleKeyedResultPropagatesKey() {
+    CommittedBundle<KV<String, Integer>> newBundle =
+        bundleFactory
+            .createBundle(
+                bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
+                downstream)
+            .commit(Instant.now());
+    assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
+  }
+
+  @Test
+  public void createKeyedBundleKeyed() {
+    CommittedBundle<KV<String, Integer>> keyedBundle =
+        bundleFactory
+            .createKeyedBundle(
+                bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
+            .commit(Instant.now());
+    assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
+  }
+}


[14/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
new file mode 100644
index 0000000..f374f99
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleOutputManager.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.Map;
+
+/**
+ * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
+ * {@link InProcessPipelineRunner}.
+ */
+public class InProcessBundleOutputManager implements OutputManager {
+  private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
+
+  public static InProcessBundleOutputManager create(
+      Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
+    return new InProcessBundleOutputManager(outputBundles);
+  }
+
+  public InProcessBundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
+    this.bundles = bundles;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+    @SuppressWarnings("rawtypes")
+    UncommittedBundle bundle = bundles.get(tag);
+    bundle.add(output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
new file mode 100644
index 0000000..d9a7ff0
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * The evaluation context for a specific pipeline being executed by the
+ * {@link InProcessPipelineRunner}. Contains state shared within the execution across all
+ * transforms.
+ *
+ * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
+ * {@link InProcessPipelineRunner} that can be used while evaluating a {@link PTransform}. This
+ * consists of views into underlying state and watermark implementations, access to read and write
+ * {@link PCollectionView PCollectionViews}, and constructing {@link CounterSet CounterSets} and
+ * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when
+ * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and
+ * known to be empty).
+ *
+ * <p>{@link InProcessEvaluationContext} also handles results by committing finalizing bundles based
+ * on the current global state and updating the global state appropriately. This includes updating
+ * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that
+ * can be executed.
+ */
+class InProcessEvaluationContext {
+  /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
+  private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+
+  /** The options that were used to create this {@link Pipeline}. */
+  private final InProcessPipelineOptions options;
+
+  private final BundleFactory bundleFactory;
+  /** The current processing time and event time watermarks and timers. */
+  private final InMemoryWatermarkManager watermarkManager;
+
+  /** Executes callbacks based on the progression of the watermark. */
+  private final WatermarkCallbackExecutor callbackExecutor;
+
+  /** The stateInternals of the world, by applied PTransform and key. */
+  private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals<?>>
+      applicationStateInternals;
+
+  private final InProcessSideInputContainer sideInputContainer;
+
+  private final CounterSet mergedCounters;
+
+  public static InProcessEvaluationContext create(
+      InProcessPipelineOptions options,
+      BundleFactory bundleFactory,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
+      Collection<PCollectionView<?>> views) {
+    return new InProcessEvaluationContext(
+        options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
+  }
+
+  private InProcessEvaluationContext(
+      InProcessPipelineOptions options,
+      BundleFactory bundleFactory,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
+      Collection<PCollectionView<?>> views) {
+    this.options = checkNotNull(options);
+    this.bundleFactory = checkNotNull(bundleFactory);
+    checkNotNull(rootTransforms);
+    checkNotNull(valueToConsumers);
+    checkNotNull(stepNames);
+    checkNotNull(views);
+    this.stepNames = stepNames;
+
+    this.watermarkManager =
+        InMemoryWatermarkManager.create(
+            NanosOffsetClock.create(), rootTransforms, valueToConsumers);
+    this.sideInputContainer = InProcessSideInputContainer.create(this, views);
+
+    this.applicationStateInternals = new ConcurrentHashMap<>();
+    this.mergedCounters = new CounterSet();
+
+    this.callbackExecutor = WatermarkCallbackExecutor.create();
+  }
+
+  /**
+   * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided
+   * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
+   *
+   * <p>The result is the output of running the transform contained in the
+   * {@link InProcessTransformResult} on the contents of the provided bundle.
+   *
+   * @param completedBundle the bundle that was processed to produce the result. Potentially
+   *                        {@code null} if the transform that produced the result is a root
+   *                        transform
+   * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
+   *                        or an empty iterable if no timers were delivered
+   * @param result the result of evaluating the input bundle
+   * @return the committed bundles contained within the handled {@code result}
+   */
+  public synchronized CommittedResult handleResult(
+      @Nullable CommittedBundle<?> completedBundle,
+      Iterable<TimerData> completedTimers,
+      InProcessTransformResult result) {
+    Iterable<? extends CommittedBundle<?>> committedBundles =
+        commitBundles(result.getOutputBundles());
+    // Update watermarks and timers
+    watermarkManager.updateWatermarks(
+        completedBundle,
+        result.getTransform(),
+        result.getTimerUpdate().withCompletedTimers(completedTimers),
+        committedBundles,
+        result.getWatermarkHold());
+    fireAllAvailableCallbacks();
+    // Update counters
+    if (result.getCounters() != null) {
+      mergedCounters.merge(result.getCounters());
+    }
+    // Update state internals
+    CopyOnAccessInMemoryStateInternals<?> theirState = result.getState();
+    if (theirState != null) {
+      CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit();
+      StepAndKey stepAndKey =
+          StepAndKey.of(
+              result.getTransform(), completedBundle == null ? null : completedBundle.getKey());
+      if (!committedState.isEmpty()) {
+        applicationStateInternals.put(stepAndKey, committedState);
+      } else {
+        applicationStateInternals.remove(stepAndKey);
+      }
+    }
+    return CommittedResult.create(result, committedBundles);
+  }
+
+  private Iterable<? extends CommittedBundle<?>> commitBundles(
+      Iterable<? extends UncommittedBundle<?>> bundles) {
+    ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
+    for (UncommittedBundle<?> inProgress : bundles) {
+      AppliedPTransform<?, ?, ?> producing =
+          inProgress.getPCollection().getProducingTransformInternal();
+      TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
+      CommittedBundle<?> committed =
+          inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
+      // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
+      // filter them out
+      if (!Iterables.isEmpty(committed.getElements())) {
+        completed.add(committed);
+      }
+    }
+    return completed.build();
+  }
+
+  private void fireAllAvailableCallbacks() {
+    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+      fireAvailableCallbacks(transform);
+    }
+  }
+
+  private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) {
+    TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
+    callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark());
+  }
+
+  /**
+   * Create a {@link UncommittedBundle} for use by a source.
+   */
+  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+    return bundleFactory.createRootBundle(output);
+  }
+
+  /**
+   * Create a {@link UncommittedBundle} whose elements belong to the specified {@link
+   * PCollection}.
+   */
+  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+    return bundleFactory.createBundle(input, output);
+  }
+
+  /**
+   * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
+   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}.
+   */
+  public <T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, Object key, PCollection<T> output) {
+    return bundleFactory.createKeyedBundle(input, key, output);
+  }
+
+  /**
+   * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided
+   * {@link PCollectionView}.
+   */
+  public <ElemT, ViewT> PCollectionViewWriter<ElemT, ViewT> createPCollectionViewWriter(
+      PCollection<Iterable<ElemT>> input, final PCollectionView<ViewT> output) {
+    return new PCollectionViewWriter<ElemT, ViewT>() {
+      @Override
+      public void add(Iterable<WindowedValue<ElemT>> values) {
+        sideInputContainer.write(output, values);
+      }
+    };
+  }
+
+  /**
+   * Schedule a callback to be executed after output would be produced for the given window
+   * if there had been input.
+   *
+   * <p>Output would be produced when the watermark for a {@link PValue} passes the point at
+   * which the trigger for the specified window (with the specified windowing strategy) must have
+   * fired from the perspective of that {@link PValue}, as specified by the value of
+   * {@link Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the
+   * {@link WindowingStrategy}. When the callback has fired, either values will have been produced
+   * for a key in that window, the window is empty, or all elements in the window are late. The
+   * callback will be executed regardless of whether values have been produced.
+   */
+  public void scheduleAfterOutputWouldBeProduced(
+      PValue value,
+      BoundedWindow window,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Runnable runnable) {
+    AppliedPTransform<?, ?, ?> producing = getProducing(value);
+    callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
+
+    fireAvailableCallbacks(lookupProducing(value));
+  }
+
+  private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
+    if (value.getProducingTransformInternal() != null) {
+      return value.getProducingTransformInternal();
+    }
+    return lookupProducing(value);
+  }
+
+  private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
+    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+      if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
+        return transform;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get the options used by this {@link Pipeline}.
+   */
+  public InProcessPipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  /**
+   * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
+   */
+  public InProcessExecutionContext getExecutionContext(
+      AppliedPTransform<?, ?, ?> application, Object key) {
+    StepAndKey stepAndKey = StepAndKey.of(application, key);
+    return new InProcessExecutionContext(
+        options.getClock(),
+        key,
+        (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey),
+        watermarkManager.getWatermarks(application));
+  }
+
+  /**
+   * Get all of the steps used in this {@link Pipeline}.
+   */
+  public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
+    return stepNames.keySet();
+  }
+
+  /**
+   * Get the Step Name for the provided application.
+   */
+  public String getStepName(AppliedPTransform<?, ?, ?> application) {
+    return stepNames.get(application);
+  }
+
+  /**
+   * Returns a {@link ReadyCheckingSideInputReader} capable of reading the provided
+   * {@link PCollectionView PCollectionViews}.
+   *
+   * @param sideInputs the {@link PCollectionView PCollectionViews} the result should be able to
+   * read
+   * @return a {@link SideInputReader} that can read all of the provided {@link PCollectionView
+   * PCollectionViews}
+   */
+  public ReadyCheckingSideInputReader createSideInputReader(
+      final List<PCollectionView<?>> sideInputs) {
+    return sideInputContainer.createReaderForViews(sideInputs);
+  }
+
+  /**
+   * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has
+   * had its contents set in a window.
+   */
+  static interface ReadyCheckingSideInputReader extends SideInputReader {
+    /**
+     * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}.
+     */
+    boolean isReady(PCollectionView<?> view, BoundedWindow window);
+  }
+
+  /**
+   * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent
+   * of all other {@link CounterSet CounterSets} created by this call.
+   *
+   * The {@link InProcessEvaluationContext} is responsible for unifying the counters present in
+   * all created {@link CounterSet CounterSets} when the transforms that call this method
+   * complete.
+   */
+  public CounterSet createCounterSet() {
+    return new CounterSet();
+  }
+
+  /**
+   * Returns all of the counters that have been merged into this context via calls to
+   * {@link CounterSet#merge(CounterSet)}.
+   */
+  public CounterSet getCounters() {
+    return mergedCounters;
+  }
+
+  /**
+   * Extracts all timers that have been fired and have not already been extracted.
+   *
+   * <p>This is a destructive operation. Timers will only appear in the result of this method once
+   * for each time they are set.
+   */
+  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired =
+        watermarkManager.extractFiredTimers();
+    return fired;
+  }
+
+  /**
+   * Returns true if the step will not produce additional output.
+   *
+   * <p>If the provided transform produces only {@link IsBounded#BOUNDED}
+   * {@link PCollection PCollections}, returns true if the watermark is at
+   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE positive infinity}.
+   *
+   * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
+   * {@link PCollection PCollections}, returns the value of
+   * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
+   */
+  public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
+    // if the PTransform's watermark isn't at the max value, it isn't done
+    if (watermarkManager
+        .getWatermarks(transform)
+        .getOutputWatermark()
+        .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
+      return false;
+    }
+    // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
+    // the PTransform may produce additional output. It is not done.
+    for (PValue output : transform.getOutput().expand()) {
+      if (output instanceof PCollection) {
+        IsBounded bounded = ((PCollection<?>) output).isBounded();
+        if (bounded.equals(IsBounded.UNBOUNDED)
+            && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
+          return false;
+        }
+      }
+    }
+    // The PTransform's watermark was at positive infinity and all of its outputs are known to be
+    // done. It is done.
+    return true;
+  }
+
+  /**
+   * Returns true if all steps are done.
+   */
+  public boolean isDone() {
+    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+      if (!isDone(transform)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.java
new file mode 100644
index 0000000..44d8bd9
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutionContext.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.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.BaseExecutionContext;
+import org.apache.beam.sdk.util.ExecutionContext;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+/**
+ * Execution Context for the {@link InProcessPipelineRunner}.
+ *
+ * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
+ * for each thread that requires it.
+ */
+class InProcessExecutionContext
+    extends BaseExecutionContext<InProcessExecutionContext.InProcessStepContext> {
+  private final Clock clock;
+  private final Object key;
+  private final CopyOnAccessInMemoryStateInternals<Object> existingState;
+  private final TransformWatermarks watermarks;
+
+  public InProcessExecutionContext(Clock clock, Object key,
+      CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
+    this.clock = clock;
+    this.key = key;
+    this.existingState = existingState;
+    this.watermarks = watermarks;
+  }
+
+  @Override
+  protected InProcessStepContext createStepContext(String stepName, String transformName) {
+    return new InProcessStepContext(this, stepName, transformName);
+  }
+
+  /**
+   * Step Context for the {@link InProcessPipelineRunner}.
+   */
+  public class InProcessStepContext
+      extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
+    private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
+    private InProcessTimerInternals timerInternals;
+
+    public InProcessStepContext(
+        ExecutionContext executionContext, String stepName, String transformName) {
+      super(executionContext, stepName, transformName);
+    }
+
+    @Override
+    public CopyOnAccessInMemoryStateInternals<Object> stateInternals() {
+      if (stateInternals == null) {
+        stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
+      }
+      return stateInternals;
+    }
+
+    @Override
+    public InProcessTimerInternals timerInternals() {
+      if (timerInternals == null) {
+        timerInternals =
+            InProcessTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
+      }
+      return timerInternals;
+    }
+
+    /**
+     * Commits the state of this step, and returns the committed state. If the step has not
+     * accessed any state, return null.
+     */
+    public CopyOnAccessInMemoryStateInternals<?> commitState() {
+      if (stateInternals != null) {
+        return stateInternals.commit();
+      }
+      return null;
+    }
+
+    /**
+     * Gets the timer update of the {@link TimerInternals} of this {@link InProcessStepContext},
+     * which is empty if the {@link TimerInternals} were never accessed.
+     */
+    public TimerUpdate getTimerUpdate() {
+      if (timerInternals == null) {
+        return TimerUpdate.empty();
+      }
+      return timerInternals.getTimerUpdate();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
new file mode 100644
index 0000000..d811e1b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessExecutor.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import java.util.Collection;
+
+/**
+ * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both
+ * source and intermediate {@link PTransform PTransforms}.
+ */
+interface InProcessExecutor {
+  /**
+   * Starts this executor. The provided collection is the collection of root transforms to
+   * initially schedule.
+   *
+   * @param rootTransforms
+   */
+  void start(Collection<AppliedPTransform<?, ?, ?>> rootTransforms);
+
+  /**
+   * Blocks until the job being executed enters a terminal state. A job is completed after all
+   * root {@link AppliedPTransform AppliedPTransforms} have completed, and all
+   * {@link CommittedBundle Bundles} have been consumed. Jobs may also terminate abnormally.
+   *
+   * @throws Throwable whenever an executor thread throws anything, transfers the throwable to the
+   *                   waiting thread and rethrows it
+   */
+  void awaitCompletion() throws Throwable;
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.java
new file mode 100644
index 0000000..512b3bd
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineOptions.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.runners.direct;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.Validation.Required;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * Options that can be used to configure the {@link InProcessPipelineRunner}.
+ */
+public interface InProcessPipelineOptions extends PipelineOptions, ApplicationNameOptions {
+  /**
+   * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService}
+   * to execute {@link PTransform PTransforms}.
+   *
+   * <p>Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that
+   * it cannot enter a state in which it will not schedule additional pending work unless currently
+   * scheduled work completes, as this may cause the {@link Pipeline} to cease processing.
+   *
+   * <p>Defaults to a {@link CachedThreadPoolExecutorServiceFactory}, which produces instances of
+   * {@link Executors#newCachedThreadPool()}.
+   */
+  @JsonIgnore
+  @Required
+  @Hidden
+  @Default.InstanceFactory(CachedThreadPoolExecutorServiceFactory.class)
+  ExecutorServiceFactory getExecutorServiceFactory();
+
+  void setExecutorServiceFactory(ExecutorServiceFactory executorService);
+
+  /**
+   * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the
+   * system time when time values are required by the evaluator.
+   */
+  @Default.InstanceFactory(NanosOffsetClock.Factory.class)
+  @JsonIgnore
+  @Required
+  @Hidden
+  @Description(
+      "The processing time source used by the pipeline. When the current time is "
+          + "needed by the evaluator, the result of clock#now() is used.")
+  Clock getClock();
+
+  void setClock(Clock clock);
+
+  @Default.Boolean(false)
+  @Description(
+      "If the pipeline should shut down producers which have reached the maximum "
+          + "representable watermark. If this is set to true, a pipeline in which all PTransforms "
+          + "have reached the maximum watermark will be shut down, even if there are unbounded "
+          + "sources that could produce additional (late) data. By default, if the pipeline "
+          + "contains any unbounded PCollections, it will run until explicitly shut down.")
+  boolean isShutdownUnboundedProducersWithMaxWatermark();
+
+  void setShutdownUnboundedProducersWithMaxWatermark(boolean shutdown);
+
+  @Default.Boolean(true)
+  @Description(
+      "If the pipeline should block awaiting completion of the pipeline. If set to true, "
+          + "a call to Pipeline#run() will block until all PTransforms are complete. Otherwise, "
+          + "the Pipeline will execute asynchronously. If set to false, the completion of the "
+          + "pipeline can be awaited on by use of InProcessPipelineResult#awaitCompletion().")
+  boolean isBlockOnRun();
+
+  void setBlockOnRun(boolean b);
+
+  @Default.Boolean(true)
+  @Description(
+      "Controls whether the runner should ensure that all of the elements of every "
+          + "PCollection are not mutated. PTransforms are not permitted to mutate input elements "
+          + "at any point, or output elements after they are output.")
+  boolean isTestImmutability();
+
+  void setTestImmutability(boolean test);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
new file mode 100644
index 0000000..bb8c0de
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java
@@ -0,0 +1,370 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
+import org.apache.beam.runners.direct.ViewEvaluatorFactory.InProcessViewOverrideFactory;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import javax.annotation.Nullable;
+
+/**
+ * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
+ * {@link PCollection PCollections}.
+ */
+@Experimental
+public class InProcessPipelineRunner
+    extends PipelineRunner<InProcessPipelineRunner.InProcessPipelineResult> {
+  /**
+   * The default set of transform overrides to use in the {@link InProcessPipelineRunner}.
+   *
+   * <p>A transform override must have a single-argument constructor that takes an instance of the
+   * type of transform it is overriding.
+   */
+  @SuppressWarnings("rawtypes")
+  private static Map<Class<? extends PTransform>, PTransformOverrideFactory>
+      defaultTransformOverrides =
+          ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
+              .put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
+              .put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
+              .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
+              .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
+              .build();
+
+  /**
+   * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
+   * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is
+   * a part of at a later point. This is an uncommitted bundle and can have elements added to it.
+   *
+   * @param <T> the type of elements that can be added to this bundle
+   */
+  public static interface UncommittedBundle<T> {
+    /**
+     * Returns the PCollection that the elements of this {@link UncommittedBundle} belong to.
+     */
+    PCollection<T> getPCollection();
+
+    /**
+     * Outputs an element to this bundle.
+     *
+     * @param element the element to add to this bundle
+     * @return this bundle
+     */
+    UncommittedBundle<T> add(WindowedValue<T> element);
+
+    /**
+     * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
+     * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
+     * will throw an {@link IllegalStateException} if called after a call to commit.
+     * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
+     *                                   committed
+     */
+    CommittedBundle<T> commit(Instant synchronizedProcessingTime);
+  }
+
+  /**
+   * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
+   * eventually committed. Committed elements are executed by the {@link PTransform PTransforms}
+   * that consume the {@link PCollection} this bundle is
+   * a part of at a later point.
+   * @param <T> the type of elements contained within this bundle
+   */
+  public static interface CommittedBundle<T> {
+    /**
+     * Returns the PCollection that the elements of this bundle belong to.
+     */
+    PCollection<T> getPCollection();
+
+    /**
+     * Returns the (possibly null) key that was output in the most recent {@link GroupByKey} in the
+     * execution of this bundle.
+     */
+    @Nullable
+    Object getKey();
+
+    /**
+     * Returns an {@link Iterable} containing all of the elements that have been added to this
+     * {@link CommittedBundle}.
+     */
+    Iterable<WindowedValue<T>> getElements();
+
+    /**
+     * Returns the processing time output watermark at the time the producing {@link PTransform}
+     * committed this bundle. Downstream synchronized processing time watermarks cannot progress
+     * past this point before consuming this bundle.
+     *
+     * <p>This value is no greater than the earliest incomplete processing time or synchronized
+     * processing time {@link TimerData timer} at the time this bundle was committed, including any
+     * timers that fired to produce this bundle.
+     */
+    Instant getSynchronizedProcessingOutputWatermark();
+
+    /**
+     * Return a new {@link CommittedBundle} that is like this one, except calls to
+     * {@link #getElements()} will return the provided elements. This bundle is unchanged.
+     *
+     * <p>
+     * The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
+     * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
+     * the current bundle. This is used to ensure a {@link PTransform} that could not complete
+     * processing on input elements properly holds the synchronized processing time to the
+     * appropriate value.
+     */
+    CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
+  }
+
+  /**
+   * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to
+   * a storage mechanism that can be read from while constructing a {@link PCollectionView}.
+   * @param <ElemT> the type of elements the input {@link PCollection} contains.
+   * @param <ViewT> the type of the PCollectionView this writer writes to.
+   */
+  public static interface PCollectionViewWriter<ElemT, ViewT> {
+    void add(Iterable<WindowedValue<ElemT>> values);
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+  private final InProcessPipelineOptions options;
+
+  public static InProcessPipelineRunner fromOptions(PipelineOptions options) {
+    return new InProcessPipelineRunner(options.as(InProcessPipelineOptions.class));
+  }
+
+  private InProcessPipelineRunner(InProcessPipelineOptions options) {
+    this.options = options;
+  }
+
+  /**
+   * Returns the {@link PipelineOptions} used to create this {@link InProcessPipelineRunner}.
+   */
+  public InProcessPipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass());
+    if (overrideFactory != null) {
+      PTransform<InputT, OutputT> customTransform = overrideFactory.override(transform);
+
+      return super.apply(customTransform, input);
+    }
+    // If there is no override, or we should not apply the override, apply the original transform
+    return super.apply(transform, input);
+  }
+
+  @Override
+  public InProcessPipelineResult run(Pipeline pipeline) {
+    ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
+    pipeline.traverseTopologically(consumerTrackingVisitor);
+    for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
+      unfinalized.finishSpecifying();
+    }
+    @SuppressWarnings("rawtypes")
+    KeyedPValueTrackingVisitor keyedPValueVisitor =
+        KeyedPValueTrackingVisitor.create(
+            ImmutableSet.<Class<? extends PTransform>>of(
+                GroupByKey.class, InProcessGroupByKeyOnly.class));
+    pipeline.traverseTopologically(keyedPValueVisitor);
+
+    InProcessEvaluationContext context =
+        InProcessEvaluationContext.create(
+            getPipelineOptions(),
+            createBundleFactory(getPipelineOptions()),
+            consumerTrackingVisitor.getRootTransforms(),
+            consumerTrackingVisitor.getValueToConsumers(),
+            consumerTrackingVisitor.getStepNames(),
+            consumerTrackingVisitor.getViews());
+
+    // independent executor service for each run
+    ExecutorService executorService =
+        context.getPipelineOptions().getExecutorServiceFactory().create();
+    InProcessExecutor executor =
+        ExecutorServiceParallelExecutor.create(
+            executorService,
+            consumerTrackingVisitor.getValueToConsumers(),
+            keyedPValueVisitor.getKeyedPValues(),
+            TransformEvaluatorRegistry.defaultRegistry(),
+            defaultModelEnforcements(options),
+            context);
+    executor.start(consumerTrackingVisitor.getRootTransforms());
+
+    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
+        new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
+    InProcessPipelineResult result =
+        new InProcessPipelineResult(executor, context, aggregatorSteps);
+    if (options.isBlockOnRun()) {
+      try {
+        result.awaitCompletion();
+      } catch (UserCodeException userException) {
+        throw new PipelineExecutionException(userException.getCause());
+      } catch (Throwable t) {
+        Throwables.propagate(t);
+      }
+    }
+    return result;
+  }
+
+  private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+      defaultModelEnforcements(InProcessPipelineOptions options) {
+    ImmutableMap.Builder<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+        enforcements = ImmutableMap.builder();
+    Collection<ModelEnforcementFactory> parDoEnforcements = createParDoEnforcements(options);
+    enforcements.put(ParDo.Bound.class, parDoEnforcements);
+    enforcements.put(ParDo.BoundMulti.class, parDoEnforcements);
+    return enforcements.build();
+  }
+
+  private Collection<ModelEnforcementFactory> createParDoEnforcements(
+      InProcessPipelineOptions options) {
+    ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
+    if (options.isTestImmutability()) {
+      enforcements.add(ImmutabilityEnforcementFactory.create());
+    }
+    return enforcements.build();
+  }
+
+  private BundleFactory createBundleFactory(InProcessPipelineOptions pipelineOptions) {
+    BundleFactory bundleFactory = InProcessBundleFactory.create();
+    if (pipelineOptions.isTestImmutability()) {
+      bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
+    }
+    return bundleFactory;
+  }
+
+  /**
+   * The result of running a {@link Pipeline} with the {@link InProcessPipelineRunner}.
+   *
+   * Throws {@link UnsupportedOperationException} for all methods.
+   */
+  public static class InProcessPipelineResult implements PipelineResult {
+    private final InProcessExecutor executor;
+    private final InProcessEvaluationContext evaluationContext;
+    private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
+    private State state;
+
+    private InProcessPipelineResult(
+        InProcessExecutor executor,
+        InProcessEvaluationContext evaluationContext,
+        Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
+      this.executor = executor;
+      this.evaluationContext = evaluationContext;
+      this.aggregatorSteps = aggregatorSteps;
+      // Only ever constructed after the executor has started.
+      this.state = State.RUNNING;
+    }
+
+    @Override
+    public State getState() {
+      return state;
+    }
+
+    @Override
+    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
+        throws AggregatorRetrievalException {
+      CounterSet counters = evaluationContext.getCounters();
+      Collection<PTransform<?, ?>> steps = aggregatorSteps.get(aggregator);
+      Map<String, T> stepValues = new HashMap<>();
+      for (AppliedPTransform<?, ?, ?> transform : evaluationContext.getSteps()) {
+        if (steps.contains(transform.getTransform())) {
+          String stepName =
+              String.format(
+                  "user-%s-%s", evaluationContext.getStepName(transform), aggregator.getName());
+          Counter<T> counter = (Counter<T>) counters.getExistingCounter(stepName);
+          if (counter != null) {
+            stepValues.put(transform.getFullName(), counter.getAggregate());
+          }
+        }
+      }
+      return new MapAggregatorValues<>(stepValues);
+    }
+
+    /**
+     * Blocks until the {@link Pipeline} execution represented by this
+     * {@link InProcessPipelineResult} is complete, returning the terminal state.
+     *
+     * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
+     * exception. Future calls to {@link #getState()} will return
+     * {@link org.apache.beam.sdk.PipelineResult.State#FAILED}.
+     *
+     * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
+     * {@link PCollection}, and the {@link PipelineRunner} was created with
+     * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
+     * this method will never return.
+     *
+     * See also {@link InProcessExecutor#awaitCompletion()}.
+     */
+    public State awaitCompletion() throws Throwable {
+      if (!state.isTerminal()) {
+        try {
+          executor.awaitCompletion();
+          state = State.DONE;
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw e;
+        } catch (Throwable t) {
+          state = State.FAILED;
+          throw t;
+        }
+      }
+      return state;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.java
new file mode 100644
index 0000000..4a09de7
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessRegistrar.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.direct;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
+ * {@link InProcessPipelineRunner}.
+ */
+public class InProcessRegistrar {
+  private InProcessRegistrar() {}
+  /**
+   * Registers the {@link InProcessPipelineRunner}.
+   */
+  @AutoService(PipelineRunnerRegistrar.class)
+  public static class InProcessRunner implements PipelineRunnerRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(InProcessPipelineRunner.class);
+    }
+  }
+
+  /**
+   * Registers the {@link InProcessPipelineOptions}.
+   */
+  @AutoService(PipelineOptionsRegistrar.class)
+  public static class InProcessOptions implements PipelineOptionsRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+      return ImmutableList.<Class<? extends PipelineOptions>>of(InProcessPipelineOptions.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
new file mode 100644
index 0000000..f4980ef
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.PCollectionViewWindow;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import javax.annotation.Nullable;
+
+/**
+ * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for
+ * constructing {@link SideInputReader SideInputReaders} which block until a side input is
+ * available and writing to a {@link PCollectionView}.
+ */
+class InProcessSideInputContainer {
+  private final InProcessEvaluationContext evaluationContext;
+  private final Collection<PCollectionView<?>> containedViews;
+  private final LoadingCache<PCollectionViewWindow<?>,
+      SettableFuture<Iterable<? extends WindowedValue<?>>>> viewByWindows;
+
+  /**
+   * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
+   * context.
+   */
+  public static InProcessSideInputContainer create(
+      InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
+    CacheLoader<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+        loader = new CacheLoader<PCollectionViewWindow<?>,
+            SettableFuture<Iterable<? extends WindowedValue<?>>>>() {
+          @Override
+          public SettableFuture<Iterable<? extends WindowedValue<?>>> load(
+              PCollectionViewWindow<?> view) {
+            return SettableFuture.create();
+          }
+        };
+    LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+        viewByWindows = CacheBuilder.newBuilder().build(loader);
+    return new InProcessSideInputContainer(context, containedViews, viewByWindows);
+  }
+
+  private InProcessSideInputContainer(InProcessEvaluationContext context,
+      Collection<PCollectionView<?>> containedViews,
+      LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
+      viewByWindows) {
+    this.evaluationContext = context;
+    this.containedViews = ImmutableSet.copyOf(containedViews);
+    this.viewByWindows = viewByWindows;
+  }
+
+  /**
+   * Return a view of this {@link InProcessSideInputContainer} that contains only the views in the
+   * provided argument. The returned {@link InProcessSideInputContainer} is unmodifiable without
+   * casting, but will change as this {@link InProcessSideInputContainer} is modified.
+   */
+  public ReadyCheckingSideInputReader createReaderForViews(
+      Collection<PCollectionView<?>> newContainedViews) {
+    if (!containedViews.containsAll(newContainedViews)) {
+      Set<PCollectionView<?>> currentlyContained = ImmutableSet.copyOf(containedViews);
+      Set<PCollectionView<?>> newRequested = ImmutableSet.copyOf(newContainedViews);
+      throw new IllegalArgumentException("Can't create a SideInputReader with unknown views "
+          + Sets.difference(newRequested, currentlyContained));
+    }
+    return new SideInputContainerSideInputReader(newContainedViews);
+  }
+
+  /**
+   * Write the provided values to the provided view.
+   *
+   * <p>The windowed values are first exploded, then for each window the pane is determined. For
+   * each window, if the pane is later than the current pane stored within this container, write
+   * all of the values to the container as the new values of the {@link PCollectionView}.
+   *
+   * <p>The provided iterable is expected to contain only a single window and pane.
+   */
+  public void write(PCollectionView<?> view, Iterable<? extends WindowedValue<?>> values) {
+    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow =
+        indexValuesByWindow(values);
+    for (Map.Entry<BoundedWindow, Collection<WindowedValue<?>>> windowValues :
+        valuesPerWindow.entrySet()) {
+      updatePCollectionViewWindowValues(view, windowValues.getKey(), windowValues.getValue());
+    }
+  }
+
+  /**
+   * Index the provided values by all {@link BoundedWindow windows} in which they appear.
+   */
+  private Map<BoundedWindow, Collection<WindowedValue<?>>> indexValuesByWindow(
+      Iterable<? extends WindowedValue<?>> values) {
+    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow = new HashMap<>();
+    for (WindowedValue<?> value : values) {
+      for (BoundedWindow window : value.getWindows()) {
+        Collection<WindowedValue<?>> windowValues = valuesPerWindow.get(window);
+        if (windowValues == null) {
+          windowValues = new ArrayList<>();
+          valuesPerWindow.put(window, windowValues);
+        }
+        windowValues.add(value);
+      }
+    }
+    return valuesPerWindow;
+  }
+
+  /**
+   * Set the value of the {@link PCollectionView} in the {@link BoundedWindow} to be based on the
+   * specified values, if the values are part of a later pane than currently exist within the
+   * {@link PCollectionViewWindow}.
+   */
+  private void updatePCollectionViewWindowValues(
+      PCollectionView<?> view, BoundedWindow window, Collection<WindowedValue<?>> windowValues) {
+    PCollectionViewWindow<?> windowedView = PCollectionViewWindow.of(view, window);
+    SettableFuture<Iterable<? extends WindowedValue<?>>> future = null;
+    try {
+      future = viewByWindows.get(windowedView);
+      if (future.isDone()) {
+        Iterator<? extends WindowedValue<?>> existingValues = future.get().iterator();
+        PaneInfo newPane = windowValues.iterator().next().getPane();
+        // The current value may have no elements, if no elements were produced for the window,
+        // but we are recieving late data.
+        if (!existingValues.hasNext()
+            || newPane.getIndex() > existingValues.next().getPane().getIndex()) {
+          viewByWindows.invalidate(windowedView);
+          viewByWindows.get(windowedView).set(windowValues);
+        }
+      } else {
+        future.set(windowValues);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      if (future != null && !future.isDone()) {
+        future.set(Collections.<WindowedValue<?>>emptyList());
+      }
+    } catch (ExecutionException e) {
+      Throwables.propagate(e.getCause());
+    }
+  }
+
+  private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
+    private final Collection<PCollectionView<?>> readerViews;
+
+    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
+      this.readerViews = ImmutableSet.copyOf(readerViews);
+    }
+
+    @Override
+    public boolean isReady(final PCollectionView<?> view, final BoundedWindow window) {
+      checkArgument(
+          readerViews.contains(view),
+          "Tried to check if view %s was ready in a SideInputReader that does not contain it. "
+              + "Contained views; %s",
+          view,
+          readerViews);
+      return getViewFuture(view, window).isDone();
+    }
+
+    @Override
+    @Nullable
+    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
+      checkArgument(
+          readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
+      try {
+        final Future<Iterable<? extends WindowedValue<?>>> future = getViewFuture(view, window);
+        // Safe covariant cast
+        @SuppressWarnings("unchecked")
+        Iterable<WindowedValue<?>> values = (Iterable<WindowedValue<?>>) future.get();
+        return view.fromIterableInternal(values);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return null;
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    /**
+     * Gets the future containing the contents of the provided {@link PCollectionView} in the
+     * provided {@link BoundedWindow}, setting up a callback to populate the future with empty
+     * contents if necessary.
+     */
+    private <T> Future<Iterable<? extends WindowedValue<?>>> getViewFuture(
+        final PCollectionView<T> view, final BoundedWindow window)  {
+      PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
+      final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
+          viewByWindows.getUnchecked(windowedView);
+
+      WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
+      evaluationContext.scheduleAfterOutputWouldBeProduced(
+          view, window, windowingStrategy, new WriteEmptyViewContents(view, window, future));
+      return future;
+    }
+
+    @Override
+    public <T> boolean contains(PCollectionView<T> view) {
+      return readerViews.contains(view);
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return readerViews.isEmpty();
+    }
+  }
+
+  private static class WriteEmptyViewContents implements Runnable {
+    private final PCollectionView<?> view;
+    private final BoundedWindow window;
+    private final SettableFuture<Iterable<? extends WindowedValue<?>>> future;
+
+    private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
+        SettableFuture<Iterable<? extends WindowedValue<?>>> future) {
+      this.future = future;
+      this.view = view;
+      this.window = window;
+    }
+
+    @Override
+    public void run() {
+      // The requested window has closed without producing elements, so reflect that in
+      // the PCollectionView. If set has already been called, will do nothing.
+      future.set(Collections.<WindowedValue<?>>emptyList());
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("view", view)
+          .add("window", window)
+          .toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
new file mode 100644
index 0000000..cd54f59
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTimerInternals.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimerInternals;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * An implementation of {@link TimerInternals} where all relevant data exists in memory.
+ */
+public class InProcessTimerInternals implements TimerInternals {
+  private final Clock processingTimeClock;
+  private final TransformWatermarks watermarks;
+  private final TimerUpdateBuilder timerUpdateBuilder;
+
+  public static InProcessTimerInternals create(
+      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
+    return new InProcessTimerInternals(clock, watermarks, timerUpdateBuilder);
+  }
+
+  private InProcessTimerInternals(
+      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
+    this.processingTimeClock = clock;
+    this.watermarks = watermarks;
+    this.timerUpdateBuilder = timerUpdateBuilder;
+  }
+
+  @Override
+  public void setTimer(TimerData timerKey) {
+    timerUpdateBuilder.setTimer(timerKey);
+  }
+
+  @Override
+  public void deleteTimer(TimerData timerKey) {
+    timerUpdateBuilder.deletedTimer(timerKey);
+  }
+
+  public TimerUpdate getTimerUpdate() {
+    return timerUpdateBuilder.build();
+  }
+
+  @Override
+  public Instant currentProcessingTime() {
+    return processingTimeClock.now();
+  }
+
+  @Override
+  @Nullable
+  public Instant currentSynchronizedProcessingTime() {
+    return watermarks.getSynchronizedProcessingInputTime();
+  }
+
+  @Override
+  public Instant currentInputWatermarkTime() {
+    return watermarks.getInputWatermark();
+  }
+
+  @Override
+  @Nullable
+  public Instant currentOutputWatermarkTime() {
+    return watermarks.getOutputWatermark();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
new file mode 100644
index 0000000..a132c33
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}.
+ */
+public interface InProcessTransformResult {
+  /**
+   * Returns the {@link AppliedPTransform} that produced this result.
+   */
+  AppliedPTransform<?, ?, ?> getTransform();
+
+  /**
+   * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
+   * will be committed by the evaluation context as part of completing this result.
+   */
+  Iterable<? extends UncommittedBundle<?>> getOutputBundles();
+
+  /**
+   * Returns the {@link CounterSet} used by this {@link PTransform}, or null if this transform did
+   * not use a {@link CounterSet}.
+   */
+  @Nullable CounterSet getCounters();
+
+  /**
+   * Returns the Watermark Hold for the transform at the time this result was produced.
+   *
+   * If the transform does not set any watermark hold, returns
+   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
+   */
+  Instant getWatermarkHold();
+
+  /**
+   * Returns the State used by the transform.
+   *
+   * If this evaluation did not access state, this may return null.
+   */
+  @Nullable
+  CopyOnAccessInMemoryStateInternals<?> getState();
+
+  /**
+   * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
+   * evaluation was triggered due to the delivery of one or more timers, those timers must be added
+   * to the builder before it is complete.
+   *
+   * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
+   */
+  TimerUpdate getTimerUpdate();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
new file mode 100644
index 0000000..b7c755e
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
+ * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that
+ * produces keyed outputs is assumed to colocate output elements that share a key.
+ *
+ * <p>All {@link GroupByKey} transforms, or their runner-specific implementation primitive, produce
+ * keyed output.
+ */
+// TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
+// unkeyed
+class KeyedPValueTrackingVisitor implements PipelineVisitor {
+  @SuppressWarnings("rawtypes")
+  private final Set<Class<? extends PTransform>> producesKeyedOutputs;
+  private final Set<PValue> keyedValues;
+  private boolean finalized;
+
+  public static KeyedPValueTrackingVisitor create(
+      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
+    return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
+  }
+
+  private KeyedPValueTrackingVisitor(
+      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
+    this.producesKeyedOutputs = producesKeyedOutputs;
+    this.keyedValues = new HashSet<>();
+  }
+
+  @Override
+  public void enterCompositeTransform(TransformTreeNode node) {
+    checkState(
+        !finalized,
+        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
+        KeyedPValueTrackingVisitor.class.getSimpleName(),
+        node);
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformTreeNode node) {
+    checkState(
+        !finalized,
+        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
+        KeyedPValueTrackingVisitor.class.getSimpleName(),
+        node);
+    if (node.isRootNode()) {
+      finalized = true;
+    } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
+      keyedValues.addAll(node.getExpandedOutputs());
+    }
+  }
+
+  @Override
+  public void visitTransform(TransformTreeNode node) {}
+
+  @Override
+  public void visitValue(PValue value, TransformTreeNode producer) {
+    if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
+      keyedValues.addAll(value.expand());
+    }
+  }
+
+  public Set<PValue> getKeyedPValues() {
+    checkState(
+        finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
+    return keyedValues;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java
new file mode 100644
index 0000000..cc9b6da
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.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.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Enforcement tools that verify that executing code conforms to the model.
+ *
+ * <p>ModelEnforcement is performed on a per-element and per-bundle basis. The
+ * {@link ModelEnforcement} is provided with the input bundle as part of
+ * {@link ModelEnforcementFactory#forBundle(CommittedBundle, AppliedPTransform)}, each element
+ * before and after that element is provided to an underlying {@link TransformEvaluator}, and the
+ * output {@link InProcessTransformResult} and committed output bundles after the
+ * {@link TransformEvaluator} has completed.
+ *
+ * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
+ * of the input {@link PCollection} on construction, and then enforce per-element behavior
+ * (such as the immutability of input elements). When the element is output or the bundle is
+ * completed, the required conditions can be enforced across all elements.
+ */
+public interface ModelEnforcement<T> {
+  /**
+   * Called before a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
+   * provided {@link WindowedValue}.
+   */
+  void beforeElement(WindowedValue<T> element);
+
+  /**
+   * Called after a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
+   * provided {@link WindowedValue}.
+   */
+  void afterElement(WindowedValue<T> element);
+
+  /**
+   * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
+   * called, producing the provided {@link InProcessTransformResult} and
+   * {@link CommittedBundle output bundles}.
+   */
+  void afterFinish(
+      CommittedBundle<T> input,
+      InProcessTransformResult result,
+      Iterable<? extends CommittedBundle<?>> outputs);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
new file mode 100644
index 0000000..6162ba0
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+/**
+ * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input
+ * {@link CommittedBundle bundle}. {@link ModelEnforcement} instances are created before the
+ * {@link TransformEvaluator} is created.
+ */
+public interface ModelEnforcementFactory {
+  <T> ModelEnforcement<T> forBundle(CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java
new file mode 100644
index 0000000..ffdee9d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.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.direct;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import org.joda.time.Instant;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time.
+ */
+public class NanosOffsetClock implements Clock {
+  private final long baseMillis;
+  private final long nanosAtBaseMillis;
+
+  public static NanosOffsetClock create() {
+    return new NanosOffsetClock();
+  }
+
+  private NanosOffsetClock() {
+    baseMillis = System.currentTimeMillis();
+    nanosAtBaseMillis = System.nanoTime();
+  }
+
+  @Override
+  public Instant now() {
+    return new Instant(
+        baseMillis + (TimeUnit.MILLISECONDS.convert(
+            System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)));
+  }
+
+  /**
+   * Creates instances of {@link NanosOffsetClock}.
+   */
+  public static class Factory implements DefaultValueFactory<Clock> {
+    @Override
+    public Clock create(PipelineOptions options) {
+      return new NanosOffsetClock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
new file mode 100644
index 0000000..81e4863
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+interface PTransformOverrideFactory {
+  /**
+   * Create a {@link PTransform} override for the provided {@link PTransform} if applicable.
+   * Otherwise, return the input {@link PTransform}.
+   *
+   * <p>The returned PTransform must be semantically equivalent to the input {@link PTransform}.
+   */
+  <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+      PTransform<InputT, OutputT> transform);
+}


[02/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
deleted file mode 100644
index d1ea51a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContextTest.java
+++ /dev/null
@@ -1,526 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.WithKeys;
-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.PaneInfo.Timing;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.common.Counter;
-import org.apache.beam.sdk.util.common.Counter.AggregationKind;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for {@link InProcessEvaluationContext}.
- */
-@RunWith(JUnit4.class)
-public class InProcessEvaluationContextTest {
-  private TestPipeline p;
-  private InProcessEvaluationContext context;
-
-  private PCollection<Integer> created;
-  private PCollection<KV<String, Integer>> downstream;
-  private PCollectionView<Iterable<Integer>> view;
-  private PCollection<Long> unbounded;
-  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms;
-  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
-
-  private BundleFactory bundleFactory;
-
-  @Before
-  public void setup() {
-    InProcessPipelineRunner runner =
-        InProcessPipelineRunner.fromOptions(PipelineOptionsFactory.create());
-
-    p = TestPipeline.create();
-
-    created = p.apply(Create.of(1, 2, 3));
-    downstream = created.apply(WithKeys.<String, Integer>of("foo"));
-    view = created.apply(View.<Integer>asIterable());
-    unbounded = p.apply(CountingInput.unbounded());
-
-    ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
-    p.traverseTopologically(cVis);
-    rootTransforms = cVis.getRootTransforms();
-    valueToConsumers = cVis.getValueToConsumers();
-
-    bundleFactory = InProcessBundleFactory.create();
-
-    context =
-        InProcessEvaluationContext.create(
-            runner.getPipelineOptions(),
-            InProcessBundleFactory.create(),
-            rootTransforms,
-            valueToConsumers,
-            cVis.getStepNames(),
-            cVis.getViews());
-  }
-
-  @Test
-  public void writeToViewWriterThenReadReads() {
-    PCollectionViewWriter<Integer, Iterable<Integer>> viewWriter =
-        context.createPCollectionViewWriter(
-            PCollection.<Iterable<Integer>>createPrimitiveOutputInternal(
-                p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED),
-            view);
-    BoundedWindow window = new TestBoundedWindow(new Instant(1024L));
-    BoundedWindow second = new TestBoundedWindow(new Instant(899999L));
-    WindowedValue<Integer> firstValue =
-        WindowedValue.of(1, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    WindowedValue<Integer> secondValue =
-        WindowedValue.of(
-            2, new Instant(8766L), second, PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0));
-    Iterable<WindowedValue<Integer>> values = ImmutableList.of(firstValue, secondValue);
-    viewWriter.add(values);
-
-    SideInputReader reader =
-        context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
-    assertThat(reader.get(view, window), containsInAnyOrder(1));
-    assertThat(reader.get(view, second), containsInAnyOrder(2));
-
-    WindowedValue<Integer> overrittenSecondValue =
-        WindowedValue.of(
-            4444, new Instant(8677L), second, PaneInfo.createPane(false, true, Timing.LATE, 1, 1));
-    viewWriter.add(Collections.singleton(overrittenSecondValue));
-    assertThat(reader.get(view, second), containsInAnyOrder(4444));
-  }
-
-  @Test
-  public void getExecutionContextSameStepSameKeyState() {
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
-
-    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    InProcessStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1");
-    stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
-    context.handleResult(
-        InProcessBundleFactory.create()
-            .createKeyedBundle(null, "foo", created)
-            .commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(created.getProducingTransformInternal())
-            .withState(stepContext.commitState())
-            .build());
-
-    InProcessExecutionContext secondFooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
-    assertThat(
-        secondFooContext
-            .getOrCreateStepContext("s1", "s1")
-            .stateInternals()
-            .state(StateNamespaces.global(), intBag)
-            .read(),
-        contains(1));
-  }
-
-
-  @Test
-  public void getExecutionContextDifferentKeysIndependentState() {
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
-
-    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    fooContext
-        .getOrCreateStepContext("s1", "s1")
-        .stateInternals()
-        .state(StateNamespaces.global(), intBag)
-        .add(1);
-
-    InProcessExecutionContext barContext =
-        context.getExecutionContext(created.getProducingTransformInternal(), "bar");
-    assertThat(barContext, not(equalTo(fooContext)));
-    assertThat(
-        barContext
-            .getOrCreateStepContext("s1", "s1")
-            .stateInternals()
-            .state(StateNamespaces.global(), intBag)
-            .read(),
-        emptyIterable());
-  }
-
-  @Test
-  public void getExecutionContextDifferentStepsIndependentState() {
-    String myKey = "foo";
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(), myKey);
-
-    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    fooContext
-        .getOrCreateStepContext("s1", "s1")
-        .stateInternals()
-        .state(StateNamespaces.global(), intBag)
-        .add(1);
-
-    InProcessExecutionContext barContext =
-        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
-    assertThat(
-        barContext
-            .getOrCreateStepContext("s1", "s1")
-            .stateInternals()
-            .state(StateNamespaces.global(), intBag)
-            .read(),
-        emptyIterable());
-  }
-
-  @Test
-  public void handleResultMergesCounters() {
-    CounterSet counters = context.createCounterSet();
-    Counter<Long> myCounter = Counter.longs("foo", AggregationKind.SUM);
-    counters.addCounter(myCounter);
-
-    myCounter.addValue(4L);
-    InProcessTransformResult result =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal())
-            .withCounters(counters)
-            .build();
-    context.handleResult(null, ImmutableList.<TimerData>of(), result);
-    assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(4L));
-
-    CounterSet againCounters = context.createCounterSet();
-    Counter<Long> myLongCounterAgain = Counter.longs("foo", AggregationKind.SUM);
-    againCounters.add(myLongCounterAgain);
-    myLongCounterAgain.addValue(8L);
-
-    InProcessTransformResult secondResult =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
-            .withCounters(againCounters)
-            .build();
-    context.handleResult(
-        context.createRootBundle(created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        secondResult);
-    assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(12L));
-  }
-
-  @Test
-  public void handleResultStoresState() {
-    String myKey = "foo";
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
-
-    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
-    CopyOnAccessInMemoryStateInternals<Object> state =
-        fooContext.getOrCreateStepContext("s1", "s1").stateInternals();
-    BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
-    bag.add(1);
-    bag.add(2);
-    bag.add(4);
-
-    InProcessTransformResult stateResult =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
-            .withState(state)
-            .build();
-
-    context.handleResult(
-        context.createKeyedBundle(null, myKey, created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        stateResult);
-
-    InProcessExecutionContext afterResultContext =
-        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
-
-    CopyOnAccessInMemoryStateInternals<Object> afterResultState =
-        afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals();
-    assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
-  }
-
-  @Test
-  public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
-    final CountDownLatch callLatch = new CountDownLatch(1);
-    Runnable callback =
-        new Runnable() {
-          @Override
-          public void run() {
-            callLatch.countDown();
-          }
-        };
-
-    // Should call back after the end of the global window
-    context.scheduleAfterOutputWouldBeProduced(
-        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-
-    InProcessTransformResult result =
-        StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
-            .build();
-
-    context.handleResult(null, ImmutableList.<TimerData>of(), result);
-
-    // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
-    // will likely be flaky if this logic is broken
-    assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
-
-    InProcessTransformResult finishedResult =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
-    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
-    // Obtain the value via blocking call
-    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
-  }
-
-  @Test
-  public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
-    InProcessTransformResult finishedResult =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
-    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
-
-    final CountDownLatch callLatch = new CountDownLatch(1);
-    Runnable callback =
-        new Runnable() {
-          @Override
-          public void run() {
-            callLatch.countDown();
-          }
-        };
-    context.scheduleAfterOutputWouldBeProduced(
-        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
-  }
-
-  @Test
-  public void extractFiredTimersExtractsTimers() {
-    InProcessTransformResult holdResult =
-        StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
-            .build();
-    context.handleResult(null, ImmutableList.<TimerData>of(), holdResult);
-
-    String key = "foo";
-    TimerData toFire =
-        TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
-    InProcessTransformResult timerResult =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
-            .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
-            .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
-            .build();
-
-    // haven't added any timers, must be empty
-    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
-    context.handleResult(
-        context.createKeyedBundle(null, key, created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        timerResult);
-
-    // timer hasn't fired
-    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
-
-    InProcessTransformResult advanceResult =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
-    // Should cause the downstream timer to fire
-    context.handleResult(null, ImmutableList.<TimerData>of(), advanceResult);
-
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired = context.extractFiredTimers();
-    assertThat(
-        fired,
-        Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
-    Map<Object, FiredTimers> downstreamFired =
-        fired.get(downstream.getProducingTransformInternal());
-    assertThat(downstreamFired, Matchers.<Object>hasKey(key));
-
-    FiredTimers firedForKey = downstreamFired.get(key);
-    assertThat(firedForKey.getTimers(TimeDomain.PROCESSING_TIME), emptyIterable());
-    assertThat(firedForKey.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), emptyIterable());
-    assertThat(firedForKey.getTimers(TimeDomain.EVENT_TIME), contains(toFire));
-
-    // Don't reextract timers
-    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
-  }
-
-  @Test
-  public void createBundleKeyedResultPropagatesKey() {
-    CommittedBundle<KV<String, Integer>> newBundle =
-        context
-            .createBundle(
-                bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
-                downstream)
-            .commit(Instant.now());
-    assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
-  }
-
-  @Test
-  public void createKeyedBundleKeyed() {
-    CommittedBundle<KV<String, Integer>> keyedBundle =
-        context
-            .createKeyedBundle(
-                bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
-            .commit(Instant.now());
-    assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
-  }
-
-  @Test
-  public void isDoneWithUnboundedPCollectionAndShutdown() {
-    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
-
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(true));
-  }
-
-  @Test
-  public void isDoneWithUnboundedPCollectionAndNotShutdown() {
-    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
-
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
-  }
-
-  @Test
-  public void isDoneWithOnlyBoundedPCollections() {
-    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
-    assertThat(context.isDone(created.getProducingTransformInternal()), is(false));
-
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
-    assertThat(context.isDone(created.getProducingTransformInternal()), is(true));
-  }
-
-  @Test
-  public void isDoneWithPartiallyDone() {
-    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
-    assertThat(context.isDone(), is(false));
-
-    UncommittedBundle<Integer> rootBundle = context.createRootBundle(created);
-    rootBundle.add(WindowedValue.valueInGlobalWindow(1));
-    CommittedResult handleResult =
-        context.handleResult(
-            null,
-            ImmutableList.<TimerData>of(),
-            StepTransformResult.withoutHold(created.getProducingTransformInternal())
-                .addOutput(rootBundle)
-                .build());
-    @SuppressWarnings("unchecked")
-    CommittedBundle<Integer> committedBundle =
-        (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
-    assertThat(context.isDone(), is(false));
-
-    for (AppliedPTransform<?, ?, ?> consumers : valueToConsumers.get(created)) {
-      context.handleResult(
-          committedBundle,
-          ImmutableList.<TimerData>of(),
-          StepTransformResult.withoutHold(consumers).build());
-    }
-    assertThat(context.isDone(), is(true));
-  }
-
-  @Test
-  public void isDoneWithUnboundedAndNotShutdown() {
-    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
-    assertThat(context.isDone(), is(false));
-
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
-    context.handleResult(
-        null,
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
-    context.handleResult(
-        context.createRootBundle(created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build());
-    assertThat(context.isDone(), is(false));
-
-    context.handleResult(
-        context.createRootBundle(created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
-    assertThat(context.isDone(), is(false));
-  }
-
-  private static class TestBoundedWindow extends BoundedWindow {
-    private final Instant ts;
-
-    public TestBoundedWindow(Instant ts) {
-      this.ts = ts;
-    }
-
-    @Override
-    public Instant maxTimestamp() {
-      return ts;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
deleted file mode 100644
index 59a96ed..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRegistrarTest.java
+++ /dev/null
@@ -1,74 +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.runners.inprocess;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
-import org.apache.beam.sdk.runners.inprocess.InProcessRegistrar.InProcessRunner;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ServiceLoader;
-
-/** Tests for {@link InProcessRunner}. */
-@RunWith(JUnit4.class)
-public class InProcessPipelineRegistrarTest {
-  @Test
-  public void testCorrectOptionsAreReturned() {
-    assertEquals(
-        ImmutableList.of(InProcessPipelineOptions.class),
-        new InProcessRegistrar.InProcessOptions().getPipelineOptions());
-  }
-
-  @Test
-  public void testCorrectRunnersAreReturned() {
-    assertEquals(
-        ImmutableList.of(InProcessPipelineRunner.class),
-        new InProcessRegistrar.InProcessRunner().getPipelineRunners());
-  }
-
-  @Test
-  public void testServiceLoaderForOptions() {
-    for (PipelineOptionsRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
-      if (registrar instanceof InProcessRegistrar.InProcessOptions) {
-        return;
-      }
-    }
-    fail("Expected to find " + InProcessRegistrar.InProcessOptions.class);
-  }
-
-  @Test
-  public void testServiceLoaderForRunner() {
-    for (PipelineRunnerRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
-      if (registrar instanceof InProcessRegistrar.InProcessRunner) {
-        return;
-      }
-    }
-    fail("Expected to find " + InProcessRegistrar.InProcessRunner.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
deleted file mode 100644
index e9e9e36..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunnerTest.java
+++ /dev/null
@@ -1,78 +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.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.InProcessPipelineResult;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-
-/**
- * Tests for basic {@link InProcessPipelineRunner} functionality.
- */
-@RunWith(JUnit4.class)
-public class InProcessPipelineRunnerTest implements Serializable {
-  @Test
-  public void wordCountShouldSucceed() throws Throwable {
-    Pipeline p = getPipeline();
-
-    PCollection<KV<String, Long>> counts =
-        p.apply(Create.of("foo", "bar", "foo", "baz", "bar", "foo"))
-            .apply(MapElements.via(new SimpleFunction<String, String>() {
-              @Override
-              public String apply(String input) {
-                return input;
-              }
-            }))
-            .apply(Count.<String>perElement());
-    PCollection<String> countStrs =
-        counts.apply(MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
-          @Override
-          public String apply(KV<String, Long> input) {
-            String str = String.format("%s: %s", input.getKey(), input.getValue());
-            return str;
-          }
-        }));
-
-    PAssert.that(countStrs).containsInAnyOrder("baz: 1", "bar: 2", "foo: 3");
-
-    InProcessPipelineResult result = ((InProcessPipelineResult) p.run());
-    result.awaitCompletion();
-  }
-
-  private Pipeline getPipeline() {
-    PipelineOptions opts = PipelineOptionsFactory.create();
-    opts.setRunner(InProcessPipelineRunner.class);
-
-    Pipeline p = Pipeline.create(opts);
-    return p;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
deleted file mode 100644
index 03443f8..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainerTest.java
+++ /dev/null
@@ -1,496 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasEntry;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.doAnswer;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessEvaluationContext.ReadyCheckingSideInputReader;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Mean;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.WithKeys;
-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.PaneInfo.Timing;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/**
- * Tests for {@link InProcessSideInputContainer}.
- */
-@RunWith(JUnit4.class)
-public class InProcessSideInputContainerTest {
-  private static final BoundedWindow FIRST_WINDOW =
-      new BoundedWindow() {
-        @Override
-        public Instant maxTimestamp() {
-          return new Instant(789541L);
-        }
-
-        @Override
-        public String toString() {
-          return "firstWindow";
-        }
-      };
-
-  private static final BoundedWindow SECOND_WINDOW =
-      new BoundedWindow() {
-        @Override
-        public Instant maxTimestamp() {
-          return new Instant(14564786L);
-        }
-
-        @Override
-        public String toString() {
-          return "secondWindow";
-        }
-      };
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Mock
-  private InProcessEvaluationContext context;
-
-  private TestPipeline pipeline;
-
-  private InProcessSideInputContainer container;
-
-  private PCollectionView<Map<String, Integer>> mapView;
-  private PCollectionView<Double> singletonView;
-
-  // Not present in container.
-  private PCollectionView<Iterable<Integer>> iterableView;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    pipeline = TestPipeline.create();
-
-    PCollection<Integer> create =
-        pipeline.apply("forBaseCollection", Create.<Integer>of(1, 2, 3, 4));
-
-    mapView =
-        create.apply("forKeyTypes", WithKeys.<String, Integer>of("foo"))
-            .apply("asMapView", View.<String, Integer>asMap());
-
-    singletonView = create.apply("forCombinedTypes", Mean.<Integer>globally().asSingletonView());
-    iterableView = create.apply("asIterableView", View.<Integer>asIterable());
-
-    container = InProcessSideInputContainer.create(
-        context, ImmutableList.of(iterableView, mapView, singletonView));
-  }
-
-  @Test
-  public void getAfterWriteReturnsPaneInWindow() throws Exception {
-    WindowedValue<KV<String, Integer>> one =
-        WindowedValue.of(
-            KV.of("one", 1), new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    WindowedValue<KV<String, Integer>> two =
-        WindowedValue.of(
-            KV.of("two", 2), new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
-    Map<String, Integer> viewContents =
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-            .get(mapView, FIRST_WINDOW);
-    assertThat(viewContents, hasEntry("one", 1));
-    assertThat(viewContents, hasEntry("two", 2));
-    assertThat(viewContents.size(), is(2));
-  }
-
-  @Test
-  public void getReturnsLatestPaneInWindow() throws Exception {
-    WindowedValue<KV<String, Integer>> one =
-        WindowedValue.of(
-            KV.of("one", 1),
-            new Instant(1L),
-            SECOND_WINDOW,
-            PaneInfo.createPane(true, false, Timing.EARLY));
-    WindowedValue<KV<String, Integer>> two =
-        WindowedValue.of(
-            KV.of("two", 2),
-            new Instant(20L),
-            SECOND_WINDOW,
-            PaneInfo.createPane(true, false, Timing.EARLY));
-    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
-    Map<String, Integer> viewContents =
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-            .get(mapView, SECOND_WINDOW);
-    assertThat(viewContents, hasEntry("one", 1));
-    assertThat(viewContents, hasEntry("two", 2));
-    assertThat(viewContents.size(), is(2));
-
-    WindowedValue<KV<String, Integer>> three =
-        WindowedValue.of(
-            KV.of("three", 3),
-            new Instant(300L),
-            SECOND_WINDOW,
-            PaneInfo.createPane(false, false, Timing.EARLY, 1, -1));
-    container.write(mapView, ImmutableList.<WindowedValue<?>>of(three));
-
-    Map<String, Integer> overwrittenViewContents =
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-            .get(mapView, SECOND_WINDOW);
-    assertThat(overwrittenViewContents, hasEntry("three", 3));
-    assertThat(overwrittenViewContents.size(), is(1));
-  }
-
-  /**
-   * Demonstrates that calling get() on a window that currently has no data does not return until
-   * there is data in the pane.
-   */
-  @Test
-  public void getBlocksUntilPaneAvailable() throws Exception {
-    BoundedWindow window =
-        new BoundedWindow() {
-          @Override
-          public Instant maxTimestamp() {
-            return new Instant(1024L);
-          }
-        };
-    Future<Double> singletonFuture =
-        getFutureOfView(
-            container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView)),
-            singletonView,
-            window);
-
-    WindowedValue<Double> singletonValue =
-        WindowedValue.of(4.75, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
-    assertThat(singletonFuture.isDone(), is(false));
-    container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
-    assertThat(singletonFuture.get(), equalTo(4.75));
-  }
-
-  @Test
-  public void withPCollectionViewsWithPutInOriginalReturnsContents() throws Exception {
-    BoundedWindow window = new BoundedWindow() {
-      @Override
-      public Instant maxTimestamp() {
-        return new Instant(1024L);
-      }
-    };
-    SideInputReader newReader =
-        container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView));
-    Future<Double> singletonFuture = getFutureOfView(newReader, singletonView, window);
-
-    WindowedValue<Double> singletonValue =
-        WindowedValue.of(24.125, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
-    assertThat(singletonFuture.isDone(), is(false));
-    container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
-    assertThat(singletonFuture.get(), equalTo(24.125));
-  }
-
-  @Test
-  public void withPCollectionViewsErrorsForContainsNotInViews() {
-    PCollectionView<Map<String, Iterable<String>>> newView =
-        PCollectionViews.multimapView(
-            pipeline,
-            WindowingStrategy.globalDefault(),
-            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("with unknown views " + ImmutableList.of(newView).toString());
-
-    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
-  }
-
-  @Test
-  public void withViewsForViewNotInContainerFails() {
-    PCollectionView<Map<String, Iterable<String>>> newView =
-        PCollectionViews.multimapView(
-            pipeline,
-            WindowingStrategy.globalDefault(),
-            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("unknown views");
-    thrown.expectMessage(newView.toString());
-
-    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
-  }
-
-  @Test
-  public void getOnReaderForViewNotInReaderFails() {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("unknown view: " + iterableView.toString());
-
-    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-        .get(iterableView, GlobalWindow.INSTANCE);
-  }
-
-  @Test
-  public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exception {
-    WindowedValue<Double> firstWindowedValue =
-        WindowedValue.of(
-            2.875,
-            FIRST_WINDOW.maxTimestamp().minus(200L),
-            FIRST_WINDOW,
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    WindowedValue<Double> secondWindowedValue =
-        WindowedValue.of(
-            4.125,
-            SECOND_WINDOW.maxTimestamp().minus(2_000_000L),
-            SECOND_WINDOW,
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    container.write(singletonView, ImmutableList.of(firstWindowedValue, secondWindowedValue));
-    assertThat(
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
-            .get(singletonView, FIRST_WINDOW),
-        equalTo(2.875));
-    assertThat(
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
-            .get(singletonView, SECOND_WINDOW),
-        equalTo(4.125));
-  }
-
-  @Test
-  public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Exception {
-    WindowedValue<Integer> firstValue =
-        WindowedValue.of(
-            44,
-            FIRST_WINDOW.maxTimestamp().minus(200L),
-            FIRST_WINDOW,
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    WindowedValue<Integer> secondValue =
-        WindowedValue.of(
-            44,
-            FIRST_WINDOW.maxTimestamp().minus(200L),
-            FIRST_WINDOW,
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-
-    container.write(iterableView, ImmutableList.of(firstValue, secondValue));
-
-    assertThat(
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(iterableView))
-            .get(iterableView, FIRST_WINDOW),
-        contains(44, 44));
-  }
-
-  @Test
-  public void writeForElementInMultipleWindowsSucceeds() throws Exception {
-    WindowedValue<Double> multiWindowedValue =
-        WindowedValue.of(
-            2.875,
-            FIRST_WINDOW.maxTimestamp().minus(200L),
-            ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    container.write(singletonView, ImmutableList.of(multiWindowedValue));
-    assertThat(
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
-            .get(singletonView, FIRST_WINDOW),
-        equalTo(2.875));
-    assertThat(
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
-            .get(singletonView, SECOND_WINDOW),
-        equalTo(2.875));
-  }
-
-  @Test
-  public void finishDoesNotOverwriteWrittenElements() throws Exception {
-    WindowedValue<KV<String, Integer>> one =
-        WindowedValue.of(
-            KV.of("one", 1),
-            new Instant(1L),
-            SECOND_WINDOW,
-            PaneInfo.createPane(true, false, Timing.EARLY));
-    WindowedValue<KV<String, Integer>> two =
-        WindowedValue.of(
-            KV.of("two", 2),
-            new Instant(20L),
-            SECOND_WINDOW,
-            PaneInfo.createPane(true, false, Timing.EARLY));
-    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
-
-    immediatelyInvokeCallback(mapView, SECOND_WINDOW);
-
-    Map<String, Integer> viewContents =
-        container
-            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-            .get(mapView, SECOND_WINDOW);
-
-    assertThat(viewContents, hasEntry("one", 1));
-    assertThat(viewContents, hasEntry("two", 2));
-    assertThat(viewContents.size(), is(2));
-  }
-
-  @Test
-  public void finishOnPendingViewsSetsEmptyElements() throws Exception {
-    immediatelyInvokeCallback(mapView, SECOND_WINDOW);
-    Future<Map<String, Integer>> mapFuture =
-        getFutureOfView(
-            container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView)),
-            mapView,
-            SECOND_WINDOW);
-
-    assertThat(mapFuture.get().isEmpty(), is(true));
-  }
-
-  /**
-   * Demonstrates that calling isReady on an empty container throws an
-   * {@link IllegalArgumentException}.
-   */
-  @Test
-  public void isReadyInEmptyReaderThrows() {
-    ReadyCheckingSideInputReader reader =
-        container.createReaderForViews(ImmutableList.<PCollectionView<?>>of());
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("does not contain");
-    thrown.expectMessage(ImmutableList.of().toString());
-    reader.isReady(mapView, GlobalWindow.INSTANCE);
-  }
-
-  /**
-   * Demonstrates that calling isReady returns false until elements are written to the
-   * {@link PCollectionView}, {@link BoundedWindow} pair, at which point it returns true.
-   */
-  @Test
-  public void isReadyForSomeNotReadyViewsFalseUntilElements() {
-    container.write(
-        mapView,
-        ImmutableList.of(
-            WindowedValue.of(
-                KV.of("one", 1),
-                SECOND_WINDOW.maxTimestamp().minus(100L),
-                SECOND_WINDOW,
-                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-
-    ReadyCheckingSideInputReader reader =
-        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
-    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
-    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
-
-    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(false));
-
-    container.write(
-        mapView,
-        ImmutableList.of(
-            WindowedValue.of(
-                KV.of("too", 2),
-                FIRST_WINDOW.maxTimestamp().minus(100L),
-                FIRST_WINDOW,
-                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
-
-    container.write(
-        singletonView,
-        ImmutableList.of(
-            WindowedValue.of(
-                1.25,
-                SECOND_WINDOW.maxTimestamp().minus(100L),
-                SECOND_WINDOW,
-                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
-    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
-
-    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-  }
-
-  @Test
-  public void isReadyForEmptyWindowTrue() {
-    immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
-
-    ReadyCheckingSideInputReader reader =
-        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
-    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-
-    immediatelyInvokeCallback(singletonView, GlobalWindow.INSTANCE);
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(true));
-  }
-
-  /**
-   * When a callAfterWindowCloses with the specified view's producing transform, window, and
-   * windowing strategy is invoked, immediately execute the callback.
-   */
-  private void immediatelyInvokeCallback(PCollectionView<?> view, BoundedWindow window) {
-    doAnswer(
-            new Answer<Void>() {
-              @Override
-              public Void answer(InvocationOnMock invocation) throws Throwable {
-                Object callback = invocation.getArguments()[3];
-                Runnable callbackRunnable = (Runnable) callback;
-                callbackRunnable.run();
-                return null;
-              }
-            })
-        .when(context)
-        .scheduleAfterOutputWouldBeProduced(
-            Mockito.eq(view),
-            Mockito.eq(window),
-            Mockito.eq(view.getWindowingStrategyInternal()),
-            Mockito.any(Runnable.class));
-  }
-
-  private <ValueT> Future<ValueT> getFutureOfView(final SideInputReader myReader,
-      final PCollectionView<ValueT> view, final BoundedWindow window) {
-    Callable<ValueT> callable = new Callable<ValueT>() {
-      @Override
-      public ValueT call() throws Exception {
-        return myReader.get(view, window);
-      }
-    };
-    return Executors.newSingleThreadExecutor().submit(callable);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
deleted file mode 100644
index b496981..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternalsTest.java
+++ /dev/null
@@ -1,133 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-
-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.MockitoAnnotations;
-
-/**
- * Tests for {@link InProcessTimerInternals}.
- */
-@RunWith(JUnit4.class)
-public class InProcessTimerInternalsTest {
-  private MockClock clock;
-  @Mock private TransformWatermarks watermarks;
-
-  private TimerUpdateBuilder timerUpdateBuilder;
-
-  private InProcessTimerInternals internals;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    clock = MockClock.fromInstant(new Instant(0));
-
-    timerUpdateBuilder = TimerUpdate.builder(1234);
-
-    internals = InProcessTimerInternals.create(clock, watermarks, timerUpdateBuilder);
-  }
-
-  @Test
-  public void setTimerAddsToBuilder() {
-    TimerData eventTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
-    TimerData processingTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTimer =
-        TimerData.of(
-            StateNamespaces.global(),
-            new Instant(98745632189L),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    internals.setTimer(eventTimer);
-    internals.setTimer(processingTimer);
-    internals.setTimer(synchronizedProcessingTimer);
-
-    assertThat(
-        internals.getTimerUpdate().getSetTimers(),
-        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
-  }
-
-  @Test
-  public void deleteTimerDeletesOnBuilder() {
-    TimerData eventTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
-    TimerData processingTimer =
-        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTimer =
-        TimerData.of(
-            StateNamespaces.global(),
-            new Instant(98745632189L),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    internals.deleteTimer(eventTimer);
-    internals.deleteTimer(processingTimer);
-    internals.deleteTimer(synchronizedProcessingTimer);
-
-    assertThat(
-        internals.getTimerUpdate().getDeletedTimers(),
-        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
-  }
-
-  @Test
-  public void getProcessingTimeIsClockNow() {
-    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
-    Instant oldProcessingTime = internals.currentProcessingTime();
-
-    clock.advance(Duration.standardHours(12));
-
-    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
-    assertThat(
-        internals.currentProcessingTime(),
-        equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
-  }
-
-  @Test
-  public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
-    when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
-    assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
-  }
-
-  @Test
-  public void getInputWatermarkTimeUsesWatermarkTime() {
-    when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
-    assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
-  }
-
-  @Test
-  public void getOutputWatermarkTimeUsesWatermarkTime() {
-    when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
-    assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
deleted file mode 100644
index b89340e..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitorTest.java
+++ /dev/null
@@ -1,192 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.Keys;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Collections;
-import java.util.Set;
-
-/**
- * Tests for {@link KeyedPValueTrackingVisitor}.
- */
-@RunWith(JUnit4.class)
-public class KeyedPValueTrackingVisitorTest {
-  @Rule public ExpectedException thrown = ExpectedException.none();
-
-  private KeyedPValueTrackingVisitor visitor;
-  private Pipeline p;
-
-  @Before
-  public void setup() {
-    PipelineOptions options = PipelineOptionsFactory.create();
-
-    p = Pipeline.create(options);
-    @SuppressWarnings("rawtypes")
-    Set<Class<? extends PTransform>> producesKeyed =
-        ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);
-    visitor = KeyedPValueTrackingVisitor.create(producesKeyed);
-  }
-
-  @Test
-  public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(new PrimitiveKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-  @Test
-  public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply("firstKey", new PrimitiveKeyer<Integer>())
-            .apply("secondKey", new PrimitiveKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-  @Test
-  public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(new CompositeKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-  @Test
-  public void compositeProducesKeyedOutputKeyedInputKeyedOutut() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply("firstKey", new CompositeKeyer<Integer>())
-            .apply("secondKey", new CompositeKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-
-  @Test
-  public void noInputUnkeyedOutput() {
-    PCollection<KV<Integer, Iterable<Void>>> unkeyed =
-        p.apply(
-            Create.of(KV.<Integer, Iterable<Void>>of(-1, Collections.<Void>emptyList()))
-                .withCoder(KvCoder.of(VarIntCoder.of(), IterableCoder.of(VoidCoder.of()))));
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
-  }
-
-  @Test
-  public void keyedInputNotProducesKeyedOutputUnkeyedOutput() {
-    PCollection<Integer> onceKeyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply(new PrimitiveKeyer<Integer>())
-            .apply(ParDo.of(new IdentityFn<Integer>()));
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed)));
-  }
-
-  @Test
-  public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() {
-    PCollection<Integer> unkeyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(ParDo.of(new IdentityFn<Integer>()));
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
-  }
-
-  @Test
-  public void traverseMultipleTimesThrows() {
-    p.apply(
-            Create.<KV<Integer, Void>>of(
-                    KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
-                .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of())))
-        .apply(GroupByKey.<Integer, Void>create())
-        .apply(Keys.<Integer>create());
-
-    p.traverseTopologically(visitor);
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("already been finalized");
-    thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName());
-    p.traverseTopologically(visitor);
-  }
-
-  @Test
-  public void getKeyedPValuesBeforeTraverseThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getKeyedPValues");
-    visitor.getKeyedPValues();
-  }
-
-  private static class PrimitiveKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
-    @Override
-    public PCollection<K> apply(PCollection<K> input) {
-      return PCollection.<K>createPrimitiveOutputInternal(
-              input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-          .setCoder(input.getCoder());
-    }
-  }
-
-  private static class CompositeKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
-    @Override
-    public PCollection<K> apply(PCollection<K> input) {
-      return input.apply(new PrimitiveKeyer<K>()).apply(ParDo.of(new IdentityFn<K>()));
-    }
-  }
-
-  private static class IdentityFn<K> extends DoFn<K, K> {
-    @Override
-    public void processElement(DoFn<K, K>.ProcessContext c) throws Exception {
-      c.output(c.element());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
deleted file mode 100644
index 152cac4..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/MockClock.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A clock that returns a constant value for now which can be set with calls to
- * {@link #set(Instant)}.
- *
- * <p>For uses of the {@link Clock} interface in unit tests.
- */
-public class MockClock implements Clock {
-
-  private Instant now;
-
-  public static MockClock fromInstant(Instant initial) {
-    return new MockClock(initial);
-  }
-
-  private MockClock(Instant initialNow) {
-    this.now = initialNow;
-  }
-
-  public void set(Instant newNow) {
-    checkArgument(!newNow.isBefore(now), "Cannot move MockClock backwards in time from %s to %s",
-        now, newNow);
-    this.now = newNow;
-  }
-
-  public void advance(Duration duration) {
-    checkArgument(
-        duration.getMillis() > 0,
-        "Cannot move MockClock backwards in time by duration %s",
-        duration);
-    set(now.plus(duration));
-  }
-
-  @Override
-  public Instant now() {
-    return now;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
deleted file mode 100644
index a048e3a..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,431 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import org.hamcrest.Matchers;
-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 java.io.Serializable;
-
-/**
- * Tests for {@link ParDoMultiEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class ParDoMultiEvaluatorFactoryTest implements Serializable {
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void testParDoMultiInMemoryTransformEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
-    final TupleTag<String> elementTag = new TupleTag<>();
-    final TupleTag<Integer> lengthTag = new TupleTag<>();
-
-    BoundMulti<String, KV<String, Integer>> pardo =
-        ParDo.of(
-                new DoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.output(KV.<String, Integer>of(c.element(), c.element().length()));
-                    c.sideOutput(elementTag, c.element());
-                    c.sideOutput(lengthTag, c.element().length());
-                  }
-                })
-            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag).and(lengthTag));
-    PCollectionTuple outputTuple = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
-    PCollection<String> elementOutput = outputTuple.get(elementTag);
-    PCollection<Integer> lengthOutput = outputTuple.get(lengthTag);
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-    UncommittedBundle<Integer> lengthOutputBundle = bundleFactory.createRootBundle(lengthOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-    when(evaluationContext.createBundle(inputBundle, elementOutput))
-        .thenReturn(elementOutputBundle);
-    when(evaluationContext.createBundle(inputBundle, lengthOutput)).thenReturn(lengthOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoMultiEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getOutputBundles(),
-        Matchers.<UncommittedBundle<?>>containsInAnyOrder(
-            lengthOutputBundle, mainOutputBundle, elementOutputBundle));
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    assertThat(result.getCounters(), equalTo(counters));
-
-    assertThat(
-        mainOutputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
-            WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
-            WindowedValue.valueInGlobalWindow(
-                KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-    assertThat(
-        elementOutputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<String>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow("foo"),
-            WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
-            WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-    assertThat(
-        lengthOutputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<Integer>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow(3),
-            WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
-            WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-  }
-
-  @Test
-  public void testParDoMultiUndeclaredSideOutput() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
-    final TupleTag<String> elementTag = new TupleTag<>();
-    final TupleTag<Integer> lengthTag = new TupleTag<>();
-
-    BoundMulti<String, KV<String, Integer>> pardo =
-        ParDo.of(
-                new DoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.output(KV.<String, Integer>of(c.element(), c.element().length()));
-                    c.sideOutput(elementTag, c.element());
-                    c.sideOutput(lengthTag, c.element().length());
-                  }
-                })
-            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
-    PCollectionTuple outputTuple = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
-    PCollection<String> elementOutput = outputTuple.get(elementTag);
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-    when(evaluationContext.createBundle(inputBundle, elementOutput))
-        .thenReturn(elementOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoMultiEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getOutputBundles(),
-        Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    assertThat(result.getCounters(), equalTo(counters));
-
-    assertThat(
-        mainOutputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
-            WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
-            WindowedValue.valueInGlobalWindow(
-                KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-    assertThat(
-        elementOutputBundle.commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<String>>containsInAnyOrder(
-            WindowedValue.valueInGlobalWindow("foo"),
-            WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
-            WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
-  }
-
-  @Test
-  public void finishBundleWithStatePutsStateInResult() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
-    final TupleTag<String> elementTag = new TupleTag<>();
-
-    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
-        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEndOfWindow());
-    final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
-    final StateNamespace windowNs =
-        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
-    BoundMulti<String, KV<String, Integer>> pardo =
-        ParDo.of(
-                new DoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(StateNamespaces.global(), watermarkTag)
-                        .add(new Instant(20202L + c.element().length()));
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(
-                            StateNamespaces.window(
-                                GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                            bagTag)
-                        .add(c.element());
-                  }
-                })
-            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
-    PCollectionTuple outputTuple = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
-    PCollection<String> elementOutput = outputTuple.get(elementTag);
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-    when(evaluationContext.createBundle(inputBundle, elementOutput))
-        .thenReturn(elementOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, "myKey", null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoMultiEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getOutputBundles(),
-        Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
-    assertThat(result.getWatermarkHold(), equalTo(new Instant(20205L)));
-    assertThat(result.getState(), not(nullValue()));
-    assertThat(
-        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
-        equalTo(new Instant(20205L)));
-    assertThat(
-        result.getState().state(windowNs, bagTag).read(),
-        containsInAnyOrder("foo", "bara", "bazam"));
-  }
-
-  @Test
-  public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
-
-    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
-    final TupleTag<String> elementTag = new TupleTag<>();
-
-    final TimerData addedTimer =
-        TimerData.of(
-            StateNamespaces.window(
-                IntervalWindow.getCoder(),
-                new IntervalWindow(
-                    new Instant(0).plus(Duration.standardMinutes(5)),
-                    new Instant(1)
-                        .plus(Duration.standardMinutes(5))
-                        .plus(Duration.standardHours(1)))),
-            new Instant(54541L),
-            TimeDomain.EVENT_TIME);
-    final TimerData deletedTimer =
-        TimerData.of(
-            StateNamespaces.window(
-                IntervalWindow.getCoder(),
-                new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
-            new Instant(3400000),
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-
-    BoundMulti<String, KV<String, Integer>> pardo =
-        ParDo.of(
-                new DoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals().stateInternals();
-                    c.windowingInternals()
-                        .timerInternals()
-                        .setTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0).plus(Duration.standardMinutes(5)),
-                                        new Instant(1)
-                                            .plus(Duration.standardMinutes(5))
-                                            .plus(Duration.standardHours(1)))),
-                                new Instant(54541L),
-                                TimeDomain.EVENT_TIME));
-                    c.windowingInternals()
-                        .timerInternals()
-                        .deleteTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0),
-                                        new Instant(0).plus(Duration.standardHours(1)))),
-                                new Instant(3400000),
-                                TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
-                  }
-                })
-            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
-    PCollectionTuple outputTuple = input.apply(pardo);
-
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createRootBundle(input).commit(Instant.now());
-
-    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
-    PCollection<String> elementOutput = outputTuple.get(elementTag);
-
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
-        bundleFactory.createRootBundle(mainOutput);
-    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
-
-    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
-    when(evaluationContext.createBundle(inputBundle, elementOutput))
-        .thenReturn(elementOutputBundle);
-
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, "myKey", null, null);
-    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
-        .thenReturn(executionContext);
-    CounterSet counters = new CounterSet();
-    when(evaluationContext.createCounterSet()).thenReturn(counters);
-
-    org.apache.beam.sdk.runners.inprocess.TransformEvaluator<String> evaluator =
-        new ParDoMultiEvaluatorFactory()
-            .forApplication(
-                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
-
-    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
-    evaluator.processElement(
-        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
-    evaluator.processElement(
-        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(
-        result.getTimerUpdate(),
-        equalTo(
-            TimerUpdate.builder("myKey")
-                .setTimer(addedTimer)
-                .setTimer(addedTimer)
-                .setTimer(addedTimer)
-                .deletedTimer(deletedTimer)
-                .deletedTimer(deletedTimer)
-                .deletedTimer(deletedTimer)
-                .build()));
-  }
-}


[07/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
deleted file mode 100644
index d941cc7..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InMemoryWatermarkManager.java
+++ /dev/null
@@ -1,1327 +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.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Ordering;
-import com.google.common.collect.SortedMultiset;
-import com.google.common.collect.TreeMultiset;
-
-import org.joda.time.Instant;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.Objects;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of
- * {@link AppliedPTransform AppliedPTransforms} to provide event-time and completion tracking for
- * in-memory execution. {@link InMemoryWatermarkManager} is designed to update and return a
- * consistent view of watermarks in the presence of concurrent updates.
- *
- * <p>An {@link InMemoryWatermarkManager} is provided with the collection of root
- * {@link AppliedPTransform AppliedPTransforms} and a map of {@link PCollection PCollections} to
- * all the {@link AppliedPTransform AppliedPTransforms} that consume them at construction time.
- *
- * <p>Whenever a root {@link AppliedPTransform transform} produces elements, the
- * {@link InMemoryWatermarkManager} is provided with the produced elements and the output watermark
- * of the producing {@link AppliedPTransform transform}. The
- * {@link InMemoryWatermarkManager watermark manager} is responsible for computing the watermarks
- * of all {@link AppliedPTransform transforms} that consume one or more
- * {@link PCollection PCollections}.
- *
- * <p>Whenever a non-root {@link AppliedPTransform} finishes processing one or more in-flight
- * elements (referred to as the input {@link CommittedBundle bundle}), the following occurs
- * atomically:
- * <ul>
- *  <li>All of the in-flight elements are removed from the collection of pending elements for the
- *      {@link AppliedPTransform}.</li>
- *  <li>All of the elements produced by the {@link AppliedPTransform} are added to the collection
- *      of pending elements for each {@link AppliedPTransform} that consumes them.</li>
- *  <li>The input watermark for the {@link AppliedPTransform} becomes the maximum value of
- *    <ul>
- *      <li>the previous input watermark</li>
- *      <li>the minimum of
- *        <ul>
- *          <li>the timestamps of all currently pending elements</li>
- *          <li>all input {@link PCollection} watermarks</li>
- *        </ul>
- *      </li>
- *    </ul>
- *  </li>
- *  <li>The output watermark for the {@link AppliedPTransform} becomes the maximum of
- *    <ul>
- *      <li>the previous output watermark</li>
- *      <li>the minimum of
- *        <ul>
- *          <li>the current input watermark</li>
- *          <li>the current watermark holds</li>
- *        </ul>
- *      </li>
- *    </ul>
- *  </li>
- *  <li>The watermark of the output {@link PCollection} can be advanced to the output watermark of
- *      the {@link AppliedPTransform}</li>
- *  <li>The watermark of all downstream {@link AppliedPTransform AppliedPTransforms} can be
- *      advanced.</li>
- * </ul>
- *
- * <p>The watermark of a {@link PCollection} is equal to the output watermark of the
- * {@link AppliedPTransform} that produces it.
- *
- * <p>The watermarks for a {@link PTransform} are updated as follows when output is committed:<pre>
- * Watermark_In'  = MAX(Watermark_In, MIN(U(TS_Pending), U(Watermark_InputPCollection)))
- * Watermark_Out' = MAX(Watermark_Out, MIN(Watermark_In', U(StateHold)))
- * Watermark_PCollection = Watermark_Out_ProducingPTransform
- * </pre>
- */
-public class InMemoryWatermarkManager {
-  /**
-   * The watermark of some {@link Pipeline} element, usually a {@link PTransform} or a
-   * {@link PCollection}.
-   *
-   * <p>A watermark is a monotonically increasing value, which represents the point up to which the
-   * system believes it has received all of the data. Data that arrives with a timestamp that is
-   * before the watermark is considered late. {@link BoundedWindow#TIMESTAMP_MAX_VALUE} is a special
-   * timestamp which indicates we have received all of the data and there will be no more on-time or
-   * late data. This value is represented by {@link InMemoryWatermarkManager#THE_END_OF_TIME}.
-   */
-  private static interface Watermark {
-    /**
-     * Returns the current value of this watermark.
-     */
-    Instant get();
-
-    /**
-     * Refreshes the value of this watermark from its input watermarks and watermark holds.
-     *
-     * @return true if the value of the watermark has changed (and thus dependent watermark must
-     *         also be updated
-     */
-    WatermarkUpdate refresh();
-  }
-
-  /**
-   * The result of computing a {@link Watermark}.
-   */
-  private static enum WatermarkUpdate {
-    /** The watermark is later than the value at the previous time it was computed. */
-    ADVANCED(true),
-    /** The watermark is equal to the value at the previous time it was computed. */
-    NO_CHANGE(false);
-
-    private final boolean advanced;
-
-    private WatermarkUpdate(boolean advanced) {
-      this.advanced = advanced;
-    }
-
-    public boolean isAdvanced() {
-      return advanced;
-    }
-
-    /**
-     * Returns the {@link WatermarkUpdate} that is a result of combining the two watermark updates.
-     *
-     * If either of the input {@link WatermarkUpdate WatermarkUpdates} were advanced, the result
-     * {@link WatermarkUpdate} has been advanced.
-     */
-    public WatermarkUpdate union(WatermarkUpdate that) {
-      if (this.advanced) {
-        return this;
-      }
-      return that;
-    }
-
-    /**
-     * Returns the {@link WatermarkUpdate} based on the former and current
-     * {@link Instant timestamps}.
-     */
-    public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTime) {
-      if (currentTime.isAfter(oldTime)) {
-        return ADVANCED;
-      }
-      return NO_CHANGE;
-    }
-  }
-
-  /**
-   * The input {@link Watermark} of an {@link AppliedPTransform}.
-   *
-   * <p>At any point, the value of an {@link AppliedPTransformInputWatermark} is equal to the
-   * minimum watermark across all of its input {@link Watermark Watermarks}, and the minimum
-   * timestamp of all of the pending elements, restricted to be monotonically increasing.
-   *
-   * <p>See {@link #refresh()} for more information.
-   */
-  private static class AppliedPTransformInputWatermark implements Watermark {
-    private final Collection<? extends Watermark> inputWatermarks;
-    private final SortedMultiset<WindowedValue<?>> pendingElements;
-    private final Map<Object, NavigableSet<TimerData>> objectTimers;
-
-    private AtomicReference<Instant> currentWatermark;
-
-    public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
-      this.inputWatermarks = inputWatermarks;
-      this.pendingElements = TreeMultiset.create(new WindowedValueByTimestampComparator());
-      this.objectTimers = new HashMap<>();
-      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    @Override
-    public Instant get() {
-      return currentWatermark.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link AppliedPTransformInputWatermark} becomes
-     * equal to the maximum value of
-     * <ul>
-     *   <li>the previous input watermark</li>
-     *   <li>the minimum of
-     *     <ul>
-     *       <li>the timestamps of all currently pending elements</li>
-     *       <li>all input {@link PCollection} watermarks</li>
-     *     </ul>
-     *   </li>
-     * </ul>
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldWatermark = currentWatermark.get();
-      Instant minInputWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE;
-      for (Watermark inputWatermark : inputWatermarks) {
-        minInputWatermark = INSTANT_ORDERING.min(minInputWatermark, inputWatermark.get());
-      }
-      if (!pendingElements.isEmpty()) {
-        minInputWatermark = INSTANT_ORDERING.min(
-            minInputWatermark, pendingElements.firstEntry().getElement().getTimestamp());
-      }
-      Instant newWatermark = INSTANT_ORDERING.max(oldWatermark, minInputWatermark);
-      currentWatermark.set(newWatermark);
-      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
-    }
-
-    private synchronized void addPendingElements(Iterable<? extends WindowedValue<?>> newPending) {
-      for (WindowedValue<?> pendingElement : newPending) {
-        pendingElements.add(pendingElement);
-      }
-    }
-
-    private synchronized void removePendingElements(
-        Iterable<? extends WindowedValue<?>> finishedElements) {
-      for (WindowedValue<?> finishedElement : finishedElements) {
-        pendingElements.remove(finishedElement);
-      }
-    }
-
-    private synchronized void updateTimers(TimerUpdate update) {
-      NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
-      if (keyTimers == null) {
-        keyTimers = new TreeSet<>();
-        objectTimers.put(update.key, keyTimers);
-      }
-      for (TimerData timer : update.setTimers) {
-        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
-          keyTimers.add(timer);
-        }
-      }
-      for (TimerData timer : update.deletedTimers) {
-        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
-          keyTimers.remove(timer);
-        }
-      }
-      // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
-    }
-
-    private synchronized Map<Object, List<TimerData>> extractFiredEventTimeTimers() {
-      return extractFiredTimers(currentWatermark.get(), objectTimers);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(AppliedPTransformInputWatermark.class)
-          .add("pendingElements", pendingElements)
-          .add("currentWatermark", currentWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * The output {@link Watermark} of an {@link AppliedPTransform}.
-   *
-   * <p>The value of an {@link AppliedPTransformOutputWatermark} is equal to the minimum of the
-   * current watermark hold and the {@link AppliedPTransformInputWatermark} for the same
-   * {@link AppliedPTransform}, restricted to be monotonically increasing. See
-   * {@link #refresh()} for more information.
-   */
-  private static class AppliedPTransformOutputWatermark implements Watermark {
-    private final Watermark inputWatermark;
-    private final PerKeyHolds holds;
-    private AtomicReference<Instant> currentWatermark;
-
-    public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
-      this.inputWatermark = inputWatermark;
-      holds = new PerKeyHolds();
-      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    public synchronized void updateHold(Object key, Instant newHold) {
-      if (newHold == null) {
-        holds.removeHold(key);
-      } else {
-        holds.updateHold(key, newHold);
-      }
-    }
-
-    @Override
-    public Instant get() {
-      return currentWatermark.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link AppliedPTransformOutputWatermark} becomes
-     * equal to the maximum value of:
-     * <ul>
-     *   <li>the previous output watermark</li>
-     *   <li>the minimum of
-     *     <ul>
-     *       <li>the current input watermark</li>
-     *       <li>the current watermark holds</li>
-     *     </ul>
-     *   </li>
-     * </ul>
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldWatermark = currentWatermark.get();
-      Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
-      newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
-      currentWatermark.set(newWatermark);
-      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(AppliedPTransformOutputWatermark.class)
-          .add("holds", holds)
-          .add("currentWatermark", currentWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * The input {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
-   * {@link AppliedPTransform}.
-   *
-   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeInputWatermark} is equal
-   * to the minimum across all pending bundles at the {@link AppliedPTransform} and all upstream
-   * {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the input
-   * synchronized processing time at any step is equal to the maximum of:
-   * <ul>
-   *   <li>The most recently returned synchronized processing input time
-   *   <li>The minimum of
-   *     <ul>
-   *       <li>The current processing time
-   *       <li>The current synchronized processing time input hold
-   *     </ul>
-   * </ul>
-   */
-  private static class SynchronizedProcessingTimeInputWatermark implements Watermark {
-    private final Collection<? extends Watermark> inputWms;
-    private final Collection<CommittedBundle<?>> pendingBundles;
-    private final Map<Object, NavigableSet<TimerData>> processingTimers;
-    private final Map<Object, NavigableSet<TimerData>> synchronizedProcessingTimers;
-
-    private final PriorityQueue<TimerData> pendingTimers;
-
-    private AtomicReference<Instant> earliestHold;
-
-    public SynchronizedProcessingTimeInputWatermark(Collection<? extends Watermark> inputWms) {
-      this.inputWms = inputWms;
-      this.pendingBundles = new HashSet<>();
-      this.processingTimers = new HashMap<>();
-      this.synchronizedProcessingTimers = new HashMap<>();
-      this.pendingTimers = new PriorityQueue<>();
-      Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
-      for (Watermark wm : inputWms) {
-        initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
-      }
-      earliestHold = new AtomicReference<>(initialHold);
-    }
-
-    @Override
-    public Instant get() {
-      return earliestHold.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeInputWatermark}
-     * becomes equal to the minimum value of
-     * <ul>
-     *   <li>the timestamps of all currently pending bundles</li>
-     *   <li>all input {@link PCollection} synchronized processing time watermarks</li>
-     * </ul>
-     *
-     * <p>Note that this value is not monotonic, but the returned value for the synchronized
-     * processing time must be.
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      Instant oldHold = earliestHold.get();
-      Instant minTime = THE_END_OF_TIME.get();
-      for (Watermark input : inputWms) {
-        minTime = INSTANT_ORDERING.min(minTime, input.get());
-      }
-      for (CommittedBundle<?> bundle : pendingBundles) {
-        // TODO: Track elements in the bundle by the processing time they were output instead of
-        // entire bundles. Requried to support arbitrarily splitting and merging bundles between
-        // steps
-        minTime = INSTANT_ORDERING.min(minTime, bundle.getSynchronizedProcessingOutputWatermark());
-      }
-      earliestHold.set(minTime);
-      return WatermarkUpdate.fromTimestamps(oldHold, minTime);
-    }
-
-    public synchronized void addPending(CommittedBundle<?> bundle) {
-      pendingBundles.add(bundle);
-    }
-
-    public synchronized void removePending(CommittedBundle<?> bundle) {
-      pendingBundles.remove(bundle);
-    }
-
-    /**
-     * Return the earliest timestamp of the earliest timer that has not been completed. This is
-     * either the earliest timestamp across timers that have not been completed, or the earliest
-     * timestamp across timers that have been delivered but have not been completed.
-     */
-    public synchronized Instant getEarliestTimerTimestamp() {
-      Instant earliest = THE_END_OF_TIME.get();
-      for (NavigableSet<TimerData> timers : processingTimers.values()) {
-        if (!timers.isEmpty()) {
-          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
-        }
-      }
-      for (NavigableSet<TimerData> timers : synchronizedProcessingTimers.values()) {
-        if (!timers.isEmpty()) {
-          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
-        }
-      }
-      if (!pendingTimers.isEmpty()) {
-        earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
-      }
-      return earliest;
-    }
-
-    private synchronized void updateTimers(TimerUpdate update) {
-      for (TimerData completedTimer : update.completedTimers) {
-        pendingTimers.remove(completedTimer);
-      }
-      Map<TimeDomain, NavigableSet<TimerData>> timerMap = timerMap(update.key);
-      for (TimerData addedTimer : update.setTimers) {
-        NavigableSet<TimerData> timerQueue = timerMap.get(addedTimer.getDomain());
-        if (timerQueue != null) {
-          timerQueue.add(addedTimer);
-        }
-      }
-      for (TimerData deletedTimer : update.deletedTimers) {
-        NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
-        if (timerQueue != null) {
-          timerQueue.remove(deletedTimer);
-        }
-      }
-    }
-
-    private synchronized Map<Object, List<TimerData>> extractFiredDomainTimers(
-        TimeDomain domain, Instant firingTime) {
-      Map<Object, List<TimerData>> firedTimers;
-      switch (domain) {
-        case PROCESSING_TIME:
-          firedTimers = extractFiredTimers(firingTime, processingTimers);
-          break;
-        case SYNCHRONIZED_PROCESSING_TIME:
-          firedTimers =
-              extractFiredTimers(
-                  INSTANT_ORDERING.min(firingTime, earliestHold.get()),
-                  synchronizedProcessingTimers);
-          break;
-        default:
-          throw new IllegalArgumentException(
-              "Called getFiredTimers on a Synchronized Processing Time watermark"
-                  + " and gave a non-processing time domain "
-                  + domain);
-      }
-      for (Map.Entry<Object, ? extends Collection<TimerData>> firedTimer : firedTimers.entrySet()) {
-        pendingTimers.addAll(firedTimer.getValue());
-      }
-      return firedTimers;
-    }
-
-    private Map<TimeDomain, NavigableSet<TimerData>> timerMap(Object key) {
-      NavigableSet<TimerData> processingQueue = processingTimers.get(key);
-      if (processingQueue == null) {
-        processingQueue = new TreeSet<>();
-        processingTimers.put(key, processingQueue);
-      }
-      NavigableSet<TimerData> synchronizedProcessingQueue =
-          synchronizedProcessingTimers.get(key);
-      if (synchronizedProcessingQueue == null) {
-        synchronizedProcessingQueue = new TreeSet<>();
-        synchronizedProcessingTimers.put(key, synchronizedProcessingQueue);
-      }
-      EnumMap<TimeDomain, NavigableSet<TimerData>> result = new EnumMap<>(TimeDomain.class);
-      result.put(TimeDomain.PROCESSING_TIME, processingQueue);
-      result.put(TimeDomain.SYNCHRONIZED_PROCESSING_TIME, synchronizedProcessingQueue);
-      return result;
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(SynchronizedProcessingTimeInputWatermark.class)
-          .add("earliestHold", earliestHold)
-          .toString();
-    }
-  }
-
-  /**
-   * The output {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
-   * {@link AppliedPTransform}.
-   *
-   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeOutputWatermark} is
-   * equal to the minimum across all incomplete timers at the {@link AppliedPTransform} and all
-   * upstream {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the output
-   * synchronized processing time at any step is equal to the maximum of:
-   * <ul>
-   *   <li>The most recently returned synchronized processing output time
-   *   <li>The minimum of
-   *     <ul>
-   *       <li>The current processing time
-   *       <li>The current synchronized processing time output hold
-   *     </ul>
-   * </ul>
-   */
-  private static class SynchronizedProcessingTimeOutputWatermark implements Watermark {
-    private final SynchronizedProcessingTimeInputWatermark inputWm;
-    private AtomicReference<Instant> latestRefresh;
-
-    public SynchronizedProcessingTimeOutputWatermark(
-        SynchronizedProcessingTimeInputWatermark inputWm) {
-      this.inputWm = inputWm;
-      this.latestRefresh = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    @Override
-    public Instant get() {
-      return latestRefresh.get();
-    }
-
-    /**
-     * {@inheritDoc}.
-     *
-     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeOutputWatermark}
-     * becomes equal to the minimum value of:
-     * <ul>
-     *   <li>the current input watermark.
-     *   <li>all {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} timers that are based on the input
-     *       watermark.
-     *   <li>all {@link TimeDomain#PROCESSING_TIME} timers that are based on the input watermark.
-     * </ul>
-     *
-     * <p>Note that this value is not monotonic, but the returned value for the synchronized
-     * processing time must be.
-     */
-    @Override
-    public synchronized WatermarkUpdate refresh() {
-      // Hold the output synchronized processing time to the input watermark, which takes into
-      // account buffered bundles, and the earliest pending timer, which determines what to hold
-      // downstream timers to.
-      Instant oldRefresh = latestRefresh.get();
-      Instant newTimestamp =
-          INSTANT_ORDERING.min(inputWm.get(), inputWm.getEarliestTimerTimestamp());
-      latestRefresh.set(newTimestamp);
-      return WatermarkUpdate.fromTimestamps(oldRefresh, newTimestamp);
-    }
-
-    @Override
-    public synchronized String toString() {
-      return MoreObjects.toStringHelper(SynchronizedProcessingTimeOutputWatermark.class)
-          .add("latestRefresh", latestRefresh)
-          .toString();
-    }
-  }
-
-  /**
-   * The {@code Watermark} that is after the latest time it is possible to represent in the global
-   * window. This is a distinguished value representing a complete {@link PTransform}.
-   */
-  private static final Watermark THE_END_OF_TIME = new Watermark() {
-        @Override
-        public WatermarkUpdate refresh() {
-          // THE_END_OF_TIME is a distinguished value that cannot be advanced.
-          return WatermarkUpdate.NO_CHANGE;
-        }
-
-        @Override
-        public Instant get() {
-          return BoundedWindow.TIMESTAMP_MAX_VALUE;
-        }
-      };
-
-  private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
-
-  /**
-   * A function that takes a WindowedValue and returns the exploded representation of that
-   * {@link WindowedValue}.
-   */
-  private static final Function<WindowedValue<?>, ? extends Iterable<? extends WindowedValue<?>>>
-      EXPLODE_WINDOWS_FN =
-          new Function<WindowedValue<?>, Iterable<? extends WindowedValue<?>>>() {
-            @Override
-            public Iterable<? extends WindowedValue<?>> apply(WindowedValue<?> input) {
-              return input.explodeWindows();
-            }
-          };
-
-  /**
-   * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
-   * latestTime argument and put in in the result with the same key, then remove all of the keys
-   * which have no more pending timers.
-   *
-   * The result collection retains ordering of timers (from earliest to latest).
-   */
-  private static Map<Object, List<TimerData>> extractFiredTimers(
-      Instant latestTime, Map<Object, NavigableSet<TimerData>> objectTimers) {
-    Map<Object, List<TimerData>> result = new HashMap<>();
-    Set<Object> emptyKeys = new HashSet<>();
-    for (Map.Entry<Object, NavigableSet<TimerData>> pendingTimers : objectTimers.entrySet()) {
-      NavigableSet<TimerData> timers = pendingTimers.getValue();
-      if (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
-        ArrayList<TimerData> keyFiredTimers = new ArrayList<>();
-        result.put(pendingTimers.getKey(), keyFiredTimers);
-        while (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
-          keyFiredTimers.add(timers.first());
-          timers.remove(timers.first());
-        }
-      }
-      if (timers.isEmpty()) {
-        emptyKeys.add(pendingTimers.getKey());
-      }
-    }
-    objectTimers.keySet().removeAll(emptyKeys);
-    return result;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * The {@link Clock} providing the current time in the {@link TimeDomain#PROCESSING_TIME} domain.
-   */
-  private final Clock clock;
-
-  /**
-   * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
-   * that consume that {@link PCollection}.
-   */
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
-
-  /**
-   * The input and output watermark of each {@link AppliedPTransform}.
-   */
-  private final Map<AppliedPTransform<?, ?, ?>, TransformWatermarks> transformToWatermarks;
-
-  /**
-   * Creates a new {@link InMemoryWatermarkManager}. All watermarks within the newly created
-   * {@link InMemoryWatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
-   * minimum watermark, with no watermark holds or pending elements.
-   *
-   * @param rootTransforms the root-level transforms of the {@link Pipeline}
-   * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
-   *                  transforms that consume it as a part of their input
-   */
-  public static InMemoryWatermarkManager create(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
-    return new InMemoryWatermarkManager(clock, rootTransforms, consumers);
-  }
-
-  private InMemoryWatermarkManager(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
-    this.clock = clock;
-    this.consumers = consumers;
-
-    transformToWatermarks = new HashMap<>();
-
-    for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
-      getTransformWatermark(rootTransform);
-    }
-    for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
-      for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
-        getTransformWatermark(transform);
-      }
-    }
-  }
-
-  private TransformWatermarks getTransformWatermark(AppliedPTransform<?, ?, ?> transform) {
-    TransformWatermarks wms = transformToWatermarks.get(transform);
-    if (wms == null) {
-      List<Watermark> inputCollectionWatermarks = getInputWatermarks(transform);
-      AppliedPTransformInputWatermark inputWatermark =
-          new AppliedPTransformInputWatermark(inputCollectionWatermarks);
-      AppliedPTransformOutputWatermark outputWatermark =
-          new AppliedPTransformOutputWatermark(inputWatermark);
-
-      SynchronizedProcessingTimeInputWatermark inputProcessingWatermark =
-          new SynchronizedProcessingTimeInputWatermark(getInputProcessingWatermarks(transform));
-      SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark =
-          new SynchronizedProcessingTimeOutputWatermark(inputProcessingWatermark);
-
-      wms =
-          new TransformWatermarks(
-              inputWatermark, outputWatermark, inputProcessingWatermark, outputProcessingWatermark);
-      transformToWatermarks.put(transform, wms);
-    }
-    return wms;
-  }
-
-  private Collection<Watermark> getInputProcessingWatermarks(
-      AppliedPTransform<?, ?, ?> transform) {
-    ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
-    if (inputs.isEmpty()) {
-      inputWmsBuilder.add(THE_END_OF_TIME);
-    }
-    for (PValue pvalue : inputs) {
-      Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal())
-              .synchronizedProcessingOutputWatermark;
-      inputWmsBuilder.add(producerOutputWatermark);
-    }
-    return inputWmsBuilder.build();
-  }
-
-  private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
-    if (inputs.isEmpty()) {
-      inputWatermarksBuilder.add(THE_END_OF_TIME);
-    }
-    for (PValue pvalue : inputs) {
-      Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
-      inputWatermarksBuilder.add(producerOutputWatermark);
-    }
-    List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
-    return inputCollectionWatermarks;
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Gets the input and output watermarks for an {@link AppliedPTransform}. If the
-   * {@link AppliedPTransform PTransform} has not processed any elements, return a watermark of
-   * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
-   *
-   * @return a snapshot of the input watermark and output watermark for the provided transform
-   */
-  public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    return transformToWatermarks.get(transform);
-  }
-
-  /**
-   * Updates the watermarks of a transform with one or more inputs.
-   *
-   * <p>Each transform has two monotonically increasing watermarks: the input watermark, which can,
-   * at any time, be updated to equal:
-   * <pre>
-   * MAX(CurrentInputWatermark, MIN(PendingElements, InputPCollectionWatermarks))
-   * </pre>
-   * and the output watermark, which can, at any time, be updated to equal:
-   * <pre>
-   * MAX(CurrentOutputWatermark, MIN(InputWatermark, WatermarkHolds))
-   * </pre>.
-   *
-   * @param completed the input that has completed
-   * @param transform the transform that has completed processing the input
-   * @param outputs the bundles the transform has output
-   * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
-   *                     is no hold
-   */
-  public void updateWatermarks(
-      @Nullable CommittedBundle<?> completed,
-      AppliedPTransform<?, ?, ?> transform,
-      TimerUpdate timerUpdate,
-      Iterable<? extends CommittedBundle<?>> outputs,
-      @Nullable Instant earliestHold) {
-    updatePending(completed, transform, timerUpdate, outputs);
-    TransformWatermarks transformWms = transformToWatermarks.get(transform);
-    transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
-    refreshWatermarks(transform);
-  }
-
-  private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
-    TransformWatermarks myWatermarks = transformToWatermarks.get(transform);
-    WatermarkUpdate updateResult = myWatermarks.refresh();
-    if (updateResult.isAdvanced()) {
-      for (PValue outputPValue : transform.getOutput().expand()) {
-        Collection<AppliedPTransform<?, ?, ?>> downstreamTransforms = consumers.get(outputPValue);
-        if (downstreamTransforms != null) {
-          for (AppliedPTransform<?, ?, ?> downstreamTransform : downstreamTransforms) {
-            refreshWatermarks(downstreamTransform);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Removes all of the completed Timers from the collection of pending timers, adds all new timers,
-   * and removes all deleted timers. Removes all elements consumed by the input bundle from the
-   * {@link PTransform PTransforms} collection of pending elements, and adds all elements produced
-   * by the {@link PTransform} to the pending queue of each consumer.
-   */
-  private void updatePending(
-      CommittedBundle<?> input,
-      AppliedPTransform<?, ?, ?> transform,
-      TimerUpdate timerUpdate,
-      Iterable<? extends CommittedBundle<?>> outputs) {
-    TransformWatermarks completedTransform = transformToWatermarks.get(transform);
-    completedTransform.updateTimers(timerUpdate);
-    if (input != null) {
-      completedTransform.removePending(input);
-    }
-
-    for (CommittedBundle<?> bundle : outputs) {
-      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
-        TransformWatermarks watermarks = transformToWatermarks.get(consumer);
-        watermarks.addPending(bundle);
-      }
-    }
-  }
-
-  /**
-   * Returns a map of each {@link PTransform} that has pending timers to those timers. All of the
-   * pending timers will be removed from this {@link InMemoryWatermarkManager}.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> allTimers = new HashMap<>();
-    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
-        transformToWatermarks.entrySet()) {
-      Map<Object, FiredTimers> keyFiredTimers = watermarksEntry.getValue().extractFiredTimers();
-      if (!keyFiredTimers.isEmpty()) {
-        allTimers.put(watermarksEntry.getKey(), keyFiredTimers);
-      }
-    }
-    return allTimers;
-  }
-
-  /**
-   * A (key, Instant) pair that holds the watermark. Holds are per-key, but the watermark is global,
-   * and as such the watermark manager must track holds and the release of holds on a per-key basis.
-   *
-   * <p>The {@link #compareTo(KeyedHold)} method of {@link KeyedHold} is not consistent with equals,
-   * as the key is arbitrarily ordered via identity, rather than object equality.
-   */
-  private static final class KeyedHold implements Comparable<KeyedHold> {
-    private static final Ordering<Object> KEY_ORDERING = Ordering.arbitrary().nullsLast();
-
-    private final Object key;
-    private final Instant timestamp;
-
-    /**
-     * Create a new KeyedHold with the specified key and timestamp.
-     */
-    public static KeyedHold of(Object key, Instant timestamp) {
-      return new KeyedHold(key, MoreObjects.firstNonNull(timestamp, THE_END_OF_TIME.get()));
-    }
-
-    private KeyedHold(Object key, Instant timestamp) {
-      this.key = key;
-      this.timestamp = timestamp;
-    }
-
-    @Override
-    public int compareTo(KeyedHold that) {
-      return ComparisonChain.start()
-          .compare(this.timestamp, that.timestamp)
-          .compare(this.key, that.key, KEY_ORDERING)
-          .result();
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(timestamp, key);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == null || !(other instanceof KeyedHold)) {
-        return false;
-      }
-      KeyedHold that = (KeyedHold) other;
-      return Objects.equals(this.timestamp, that.timestamp) && Objects.equals(this.key, that.key);
-    }
-
-    /**
-     * Get the value of this {@link KeyedHold}.
-     */
-    public Instant getTimestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(KeyedHold.class)
-          .add("key", key)
-          .add("hold", timestamp)
-          .toString();
-    }
-  }
-
-  private static class PerKeyHolds {
-    private final Map<Object, KeyedHold> keyedHolds;
-    private final PriorityQueue<KeyedHold> allHolds;
-
-    private PerKeyHolds() {
-      this.keyedHolds = new HashMap<>();
-      this.allHolds = new PriorityQueue<>();
-    }
-
-    /**
-     * Gets the minimum hold across all keys in this {@link PerKeyHolds}, or THE_END_OF_TIME if
-     * there are no holds within this {@link PerKeyHolds}.
-     */
-    public Instant getMinHold() {
-      return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
-    }
-
-    /**
-     * Updates the hold of the provided key to the provided value, removing any other holds for
-     * the same key.
-     */
-    public void updateHold(@Nullable Object key, Instant newHold) {
-      removeHold(key);
-      KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
-      keyedHolds.put(key, newKeyedHold);
-      allHolds.offer(newKeyedHold);
-    }
-
-    /**
-     * Removes the hold of the provided key.
-     */
-    public void removeHold(Object key) {
-      KeyedHold oldHold = keyedHolds.get(key);
-      if (oldHold != null) {
-        allHolds.remove(oldHold);
-      }
-    }
-  }
-
-  /**
-   * A reference to the input and output watermarks of an {@link AppliedPTransform}.
-   */
-  public class TransformWatermarks {
-    private final AppliedPTransformInputWatermark inputWatermark;
-    private final AppliedPTransformOutputWatermark outputWatermark;
-
-    private final SynchronizedProcessingTimeInputWatermark synchronizedProcessingInputWatermark;
-    private final SynchronizedProcessingTimeOutputWatermark synchronizedProcessingOutputWatermark;
-
-    private Instant latestSynchronizedInputWm;
-    private Instant latestSynchronizedOutputWm;
-
-    private TransformWatermarks(
-        AppliedPTransformInputWatermark inputWatermark,
-        AppliedPTransformOutputWatermark outputWatermark,
-        SynchronizedProcessingTimeInputWatermark inputSynchProcessingWatermark,
-        SynchronizedProcessingTimeOutputWatermark outputSynchProcessingWatermark) {
-      this.inputWatermark = inputWatermark;
-      this.outputWatermark = outputWatermark;
-
-      this.synchronizedProcessingInputWatermark = inputSynchProcessingWatermark;
-      this.synchronizedProcessingOutputWatermark = outputSynchProcessingWatermark;
-      this.latestSynchronizedInputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      this.latestSynchronizedOutputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Returns the input watermark of the {@link AppliedPTransform}.
-     */
-    public Instant getInputWatermark() {
-      return Preconditions.checkNotNull(inputWatermark.get());
-    }
-
-    /**
-     * Returns the output watermark of the {@link AppliedPTransform}.
-     */
-    public Instant getOutputWatermark() {
-      return outputWatermark.get();
-    }
-
-    /**
-     * Returns the synchronized processing input time of the {@link AppliedPTransform}.
-     *
-     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
-     * presence of holds, will increase as the system time progresses.
-     */
-    public synchronized Instant getSynchronizedProcessingInputTime() {
-      latestSynchronizedInputWm = INSTANT_ORDERING.max(
-          latestSynchronizedInputWm,
-          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingInputWatermark.get()));
-      return latestSynchronizedInputWm;
-    }
-
-    /**
-     * Returns the synchronized processing output time of the {@link AppliedPTransform}.
-     *
-     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
-     * presence of holds, will increase as the system time progresses.
-     */
-    public synchronized Instant getSynchronizedProcessingOutputTime() {
-      latestSynchronizedOutputWm = INSTANT_ORDERING.max(
-          latestSynchronizedOutputWm,
-          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingOutputWatermark.get()));
-      return latestSynchronizedOutputWm;
-    }
-
-    private WatermarkUpdate refresh() {
-      inputWatermark.refresh();
-      synchronizedProcessingInputWatermark.refresh();
-      WatermarkUpdate eventOutputUpdate = outputWatermark.refresh();
-      WatermarkUpdate syncOutputUpdate = synchronizedProcessingOutputWatermark.refresh();
-      return eventOutputUpdate.union(syncOutputUpdate);
-    }
-
-    private void setEventTimeHold(Object key, Instant newHold) {
-      outputWatermark.updateHold(key, newHold);
-    }
-
-    private void removePending(CommittedBundle<?> bundle) {
-      inputWatermark.removePendingElements(elementsFromBundle(bundle));
-      synchronizedProcessingInputWatermark.removePending(bundle);
-    }
-
-    private void addPending(CommittedBundle<?> bundle) {
-      inputWatermark.addPendingElements(elementsFromBundle(bundle));
-      synchronizedProcessingInputWatermark.addPending(bundle);
-    }
-
-    private Iterable<? extends WindowedValue<?>> elementsFromBundle(CommittedBundle<?> bundle) {
-      return FluentIterable.from(bundle.getElements()).transformAndConcat(EXPLODE_WINDOWS_FN);
-    }
-
-    private Map<Object, FiredTimers> extractFiredTimers() {
-      Map<Object, List<TimerData>> eventTimeTimers = inputWatermark.extractFiredEventTimeTimers();
-      Map<Object, List<TimerData>> processingTimers;
-      Map<Object, List<TimerData>> synchronizedTimers;
-      if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
-        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
-        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
-      } else {
-        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.PROCESSING_TIME, clock.now());
-        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
-            TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime());
-      }
-      Map<Object, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
-      groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
-
-      Map<Object, FiredTimers> keyFiredTimers = new HashMap<>();
-      for (Map.Entry<Object, Map<TimeDomain, List<TimerData>>> firedTimers :
-          groupedTimers.entrySet()) {
-        keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
-      }
-      return keyFiredTimers;
-    }
-
-    @SafeVarargs
-    private final void groupFiredTimers(
-        Map<Object, Map<TimeDomain, List<TimerData>>> groupedToMutate,
-        Map<Object, List<TimerData>>... timersToGroup) {
-      for (Map<Object, List<TimerData>> subGroup : timersToGroup) {
-        for (Map.Entry<Object, List<TimerData>> newTimers : subGroup.entrySet()) {
-          Map<TimeDomain, List<TimerData>> grouped = groupedToMutate.get(newTimers.getKey());
-          if (grouped == null) {
-            grouped = new HashMap<>();
-            groupedToMutate.put(newTimers.getKey(), grouped);
-          }
-          grouped.put(newTimers.getValue().get(0).getDomain(), newTimers.getValue());
-        }
-      }
-    }
-
-    private void updateTimers(TimerUpdate update) {
-      inputWatermark.updateTimers(update);
-      synchronizedProcessingInputWatermark.updateTimers(update);
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(TransformWatermarks.class)
-          .add("inputWatermark", inputWatermark)
-          .add("outputWatermark", outputWatermark)
-          .add("inputProcessingTime", synchronizedProcessingInputWatermark)
-          .add("outputProcessingTime", synchronizedProcessingOutputWatermark)
-          .toString();
-    }
-  }
-
-  /**
-   * A collection of newly set, deleted, and completed timers.
-   *
-   * <p>setTimers and deletedTimers are collections of {@link TimerData} that have been added to the
-   * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as
-   * the input to the executed step.
-   */
-  public static class TimerUpdate {
-    private final Object key;
-    private final Iterable<? extends TimerData> completedTimers;
-
-    private final Iterable<? extends TimerData> setTimers;
-    private final Iterable<? extends TimerData> deletedTimers;
-
-    /**
-     * Returns a TimerUpdate for a null key with no timers.
-     */
-    public static TimerUpdate empty() {
-      return new TimerUpdate(
-          null,
-          Collections.<TimerData>emptyList(),
-          Collections.<TimerData>emptyList(),
-          Collections.<TimerData>emptyList());
-    }
-
-    /**
-     * Creates a new {@link TimerUpdate} builder with the provided completed timers that needs the
-     * set and deleted timers to be added to it.
-     */
-    public static TimerUpdateBuilder builder(Object key) {
-      return new TimerUpdateBuilder(key);
-    }
-
-    /**
-     * A {@link TimerUpdate} builder that needs to be provided with set timers and deleted timers.
-     */
-    public static final class TimerUpdateBuilder {
-      private final Object key;
-      private final Collection<TimerData> completedTimers;
-      private final Collection<TimerData> setTimers;
-      private final Collection<TimerData> deletedTimers;
-
-      private TimerUpdateBuilder(Object key) {
-        this.key = key;
-        this.completedTimers = new HashSet<>();
-        this.setTimers = new HashSet<>();
-        this.deletedTimers = new HashSet<>();
-      }
-
-      /**
-       * Adds all of the provided timers to the collection of completed timers, and returns this
-       * {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder withCompletedTimers(Iterable<TimerData> completedTimers) {
-        Iterables.addAll(this.completedTimers, completedTimers);
-        return this;
-      }
-
-      /**
-       * Adds the provided timer to the collection of set timers, removing it from deleted timers if
-       * it has previously been deleted. Returns this {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder setTimer(TimerData setTimer) {
-        deletedTimers.remove(setTimer);
-        setTimers.add(setTimer);
-        return this;
-      }
-
-      /**
-       * Adds the provided timer to the collection of deleted timers, removing it from set timers if
-       * it has previously been set. Returns this {@link TimerUpdateBuilder}.
-       */
-      public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
-        deletedTimers.add(deletedTimer);
-        setTimers.remove(deletedTimer);
-        return this;
-      }
-
-      /**
-       * Returns a new {@link TimerUpdate} with the most recently set completedTimers, setTimers,
-       * and deletedTimers.
-       */
-      public TimerUpdate build() {
-        return new TimerUpdate(
-            key,
-            ImmutableSet.copyOf(completedTimers),
-            ImmutableSet.copyOf(setTimers),
-            ImmutableSet.copyOf(deletedTimers));
-      }
-    }
-
-    private TimerUpdate(
-        Object key,
-        Iterable<? extends TimerData> completedTimers,
-        Iterable<? extends TimerData> setTimers,
-        Iterable<? extends TimerData> deletedTimers) {
-      this.key = key;
-      this.completedTimers = completedTimers;
-      this.setTimers = setTimers;
-      this.deletedTimers = deletedTimers;
-    }
-
-    @VisibleForTesting
-    Object getKey() {
-      return key;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getCompletedTimers() {
-      return completedTimers;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getSetTimers() {
-      return setTimers;
-    }
-
-    @VisibleForTesting
-    Iterable<? extends TimerData> getDeletedTimers() {
-      return deletedTimers;
-    }
-
-    /**
-     * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
-     */
-    public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
-      return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(key, completedTimers, setTimers, deletedTimers);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == null || !(other instanceof TimerUpdate)) {
-        return false;
-      }
-      TimerUpdate that = (TimerUpdate) other;
-      return Objects.equals(this.key, that.key)
-          && Objects.equals(this.completedTimers, that.completedTimers)
-          && Objects.equals(this.setTimers, that.setTimers)
-          && Objects.equals(this.deletedTimers, that.deletedTimers);
-    }
-  }
-
-  /**
-   * A pair of {@link TimerData} and key which can be delivered to the appropriate
-   * {@link AppliedPTransform}. A timer fires at the transform that set it with a specific key when
-   * the time domain in which it lives progresses past a specified time, as determined by the
-   * {@link InMemoryWatermarkManager}.
-   */
-  public static class FiredTimers {
-    private final Map<TimeDomain, ? extends Collection<TimerData>> timers;
-
-    private FiredTimers(Map<TimeDomain, ? extends Collection<TimerData>> timers) {
-      this.timers = timers;
-    }
-
-    /**
-     * Gets all of the timers that have fired within the provided {@link TimeDomain}. If no timers
-     * fired within the provided domain, return an empty collection.
-     *
-     * <p>Timers within a {@link TimeDomain} are guaranteed to be in order of increasing timestamp.
-     */
-    public Collection<TimerData> getTimers(TimeDomain domain) {
-      Collection<TimerData> domainTimers = timers.get(domain);
-      if (domainTimers == null) {
-        return Collections.emptyList();
-      }
-      return domainTimers;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString();
-    }
-  }
-
-  private static class WindowedValueByTimestampComparator extends Ordering<WindowedValue<?>> {
-    @Override
-    public int compare(WindowedValue<?> o1, WindowedValue<?> o2) {
-      return ComparisonChain.start()
-          .compare(o1.getTimestamp(), o2.getTimestamp())
-          .result();
-    }
-  }
-
-  public Set<AppliedPTransform<?, ?, ?>> getCompletedTransforms() {
-    Set<AppliedPTransform<?, ?, ?>> result = new HashSet<>();
-    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> wms :
-        transformToWatermarks.entrySet()) {
-      if (wms.getValue().getOutputWatermark().equals(THE_END_OF_TIME.get())) {
-        result.add(wms.getKey());
-      }
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
deleted file mode 100644
index 58bc4fb..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleFactory.java
+++ /dev/null
@@ -1,162 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory that produces bundles that perform no additional validation.
- */
-class InProcessBundleFactory implements BundleFactory {
-  public static InProcessBundleFactory create() {
-    return new InProcessBundleFactory();
-  }
-
-  private InProcessBundleFactory() {}
-
-  @Override
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return InProcessBundle.create(output, null);
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return InProcessBundle.create(output, input.getKey());
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, @Nullable Object key, PCollection<T> output) {
-    return InProcessBundle.create(output, key);
-  }
-
-  /**
-   * A {@link UncommittedBundle} that buffers elements in memory.
-   */
-  private static final class InProcessBundle<T> implements UncommittedBundle<T> {
-    private final PCollection<T> pcollection;
-    @Nullable private final Object key;
-    private boolean committed = false;
-    private ImmutableList.Builder<WindowedValue<T>> elements;
-
-    /**
-     * Create a new {@link InProcessBundle} for the specified {@link PCollection}.
-     */
-    public static <T> InProcessBundle<T> create(PCollection<T> pcollection, @Nullable Object key) {
-      return new InProcessBundle<T>(pcollection, key);
-    }
-
-    private InProcessBundle(PCollection<T> pcollection, Object key) {
-      this.pcollection = pcollection;
-      this.key = key;
-      this.elements = ImmutableList.builder();
-    }
-
-    @Override
-    public PCollection<T> getPCollection() {
-      return pcollection;
-    }
-
-    @Override
-    public InProcessBundle<T> add(WindowedValue<T> element) {
-      checkState(
-          !committed,
-          "Can't add element %s to committed bundle in PCollection %s",
-          element,
-          pcollection);
-      elements.add(element);
-      return this;
-    }
-
-    @Override
-    public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
-      checkState(!committed, "Can't commit already committed bundle %s", this);
-      committed = true;
-      final Iterable<WindowedValue<T>> committedElements = elements.build();
-      return new CommittedInProcessBundle<>(
-          pcollection, key, committedElements, synchronizedCompletionTime);
-    }
-  }
-
-  private static class CommittedInProcessBundle<T> implements CommittedBundle<T> {
-    public CommittedInProcessBundle(
-        PCollection<T> pcollection,
-        Object key,
-        Iterable<WindowedValue<T>> committedElements,
-        Instant synchronizedCompletionTime) {
-      this.pcollection = pcollection;
-      this.key = key;
-      this.committedElements = committedElements;
-      this.synchronizedCompletionTime = synchronizedCompletionTime;
-    }
-
-    private final PCollection<T> pcollection;
-    private final Object key;
-    private final Iterable<WindowedValue<T>> committedElements;
-    private final Instant synchronizedCompletionTime;
-
-    @Override
-    @Nullable
-    public Object getKey() {
-      return key;
-    }
-
-    @Override
-    public Iterable<WindowedValue<T>> getElements() {
-      return committedElements;
-    }
-
-    @Override
-    public PCollection<T> getPCollection() {
-      return pcollection;
-    }
-
-    @Override
-    public Instant getSynchronizedProcessingOutputWatermark() {
-      return synchronizedCompletionTime;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .omitNullValues()
-          .add("pcollection", pcollection)
-          .add("key", key)
-          .add("elements", committedElements)
-          .toString();
-    }
-
-    @Override
-    public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
-      return new CommittedInProcessBundle<>(
-          pcollection, key, ImmutableList.copyOf(elements), synchronizedCompletionTime);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
deleted file mode 100644
index 4678a53..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessBundleOutputManager.java
+++ /dev/null
@@ -1,51 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.Map;
-
-/**
- * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessBundleOutputManager implements OutputManager {
-  private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
-
-  public static InProcessBundleOutputManager create(
-      Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
-    return new InProcessBundleOutputManager(outputBundles);
-  }
-
-  public InProcessBundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
-    this.bundles = bundles;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-    @SuppressWarnings("rawtypes")
-    UncommittedBundle bundle = bundles.get(tag);
-    bundle.add(output);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
deleted file mode 100644
index 7c0dcee..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessEvaluationContext.java
+++ /dev/null
@@ -1,425 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.ExecutionContext;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * The evaluation context for a specific pipeline being executed by the
- * {@link InProcessPipelineRunner}. Contains state shared within the execution across all
- * transforms.
- *
- * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
- * {@link InProcessPipelineRunner} that can be used while evaluating a {@link PTransform}. This
- * consists of views into underlying state and watermark implementations, access to read and write
- * {@link PCollectionView PCollectionViews}, and constructing {@link CounterSet CounterSets} and
- * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when
- * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and
- * known to be empty).
- *
- * <p>{@link InProcessEvaluationContext} also handles results by committing finalizing bundles based
- * on the current global state and updating the global state appropriately. This includes updating
- * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that
- * can be executed.
- */
-class InProcessEvaluationContext {
-  /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
-  private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
-
-  /** The options that were used to create this {@link Pipeline}. */
-  private final InProcessPipelineOptions options;
-
-  private final BundleFactory bundleFactory;
-  /** The current processing time and event time watermarks and timers. */
-  private final InMemoryWatermarkManager watermarkManager;
-
-  /** Executes callbacks based on the progression of the watermark. */
-  private final WatermarkCallbackExecutor callbackExecutor;
-
-  /** The stateInternals of the world, by applied PTransform and key. */
-  private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals<?>>
-      applicationStateInternals;
-
-  private final InProcessSideInputContainer sideInputContainer;
-
-  private final CounterSet mergedCounters;
-
-  public static InProcessEvaluationContext create(
-      InProcessPipelineOptions options,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
-    return new InProcessEvaluationContext(
-        options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
-  }
-
-  private InProcessEvaluationContext(
-      InProcessPipelineOptions options,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
-    this.options = checkNotNull(options);
-    this.bundleFactory = checkNotNull(bundleFactory);
-    checkNotNull(rootTransforms);
-    checkNotNull(valueToConsumers);
-    checkNotNull(stepNames);
-    checkNotNull(views);
-    this.stepNames = stepNames;
-
-    this.watermarkManager =
-        InMemoryWatermarkManager.create(
-            NanosOffsetClock.create(), rootTransforms, valueToConsumers);
-    this.sideInputContainer = InProcessSideInputContainer.create(this, views);
-
-    this.applicationStateInternals = new ConcurrentHashMap<>();
-    this.mergedCounters = new CounterSet();
-
-    this.callbackExecutor = WatermarkCallbackExecutor.create();
-  }
-
-  /**
-   * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided
-   * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
-   *
-   * <p>The result is the output of running the transform contained in the
-   * {@link InProcessTransformResult} on the contents of the provided bundle.
-   *
-   * @param completedBundle the bundle that was processed to produce the result. Potentially
-   *                        {@code null} if the transform that produced the result is a root
-   *                        transform
-   * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
-   *                        or an empty iterable if no timers were delivered
-   * @param result the result of evaluating the input bundle
-   * @return the committed bundles contained within the handled {@code result}
-   */
-  public synchronized CommittedResult handleResult(
-      @Nullable CommittedBundle<?> completedBundle,
-      Iterable<TimerData> completedTimers,
-      InProcessTransformResult result) {
-    Iterable<? extends CommittedBundle<?>> committedBundles =
-        commitBundles(result.getOutputBundles());
-    // Update watermarks and timers
-    watermarkManager.updateWatermarks(
-        completedBundle,
-        result.getTransform(),
-        result.getTimerUpdate().withCompletedTimers(completedTimers),
-        committedBundles,
-        result.getWatermarkHold());
-    fireAllAvailableCallbacks();
-    // Update counters
-    if (result.getCounters() != null) {
-      mergedCounters.merge(result.getCounters());
-    }
-    // Update state internals
-    CopyOnAccessInMemoryStateInternals<?> theirState = result.getState();
-    if (theirState != null) {
-      CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit();
-      StepAndKey stepAndKey =
-          StepAndKey.of(
-              result.getTransform(), completedBundle == null ? null : completedBundle.getKey());
-      if (!committedState.isEmpty()) {
-        applicationStateInternals.put(stepAndKey, committedState);
-      } else {
-        applicationStateInternals.remove(stepAndKey);
-      }
-    }
-    return CommittedResult.create(result, committedBundles);
-  }
-
-  private Iterable<? extends CommittedBundle<?>> commitBundles(
-      Iterable<? extends UncommittedBundle<?>> bundles) {
-    ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
-    for (UncommittedBundle<?> inProgress : bundles) {
-      AppliedPTransform<?, ?, ?> producing =
-          inProgress.getPCollection().getProducingTransformInternal();
-      TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
-      CommittedBundle<?> committed =
-          inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
-      // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
-      // filter them out
-      if (!Iterables.isEmpty(committed.getElements())) {
-        completed.add(committed);
-      }
-    }
-    return completed.build();
-  }
-
-  private void fireAllAvailableCallbacks() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      fireAvailableCallbacks(transform);
-    }
-  }
-
-  private void fireAvailableCallbacks(AppliedPTransform<?, ?, ?> producingTransform) {
-    TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
-    callbackExecutor.fireForWatermark(producingTransform, watermarks.getOutputWatermark());
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} for use by a source.
-   */
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return bundleFactory.createRootBundle(output);
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} whose elements belong to the specified {@link
-   * PCollection}.
-   */
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return bundleFactory.createBundle(input, output);
-  }
-
-  /**
-   * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}.
-   */
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output) {
-    return bundleFactory.createKeyedBundle(input, key, output);
-  }
-
-  /**
-   * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided
-   * {@link PCollectionView}.
-   */
-  public <ElemT, ViewT> PCollectionViewWriter<ElemT, ViewT> createPCollectionViewWriter(
-      PCollection<Iterable<ElemT>> input, final PCollectionView<ViewT> output) {
-    return new PCollectionViewWriter<ElemT, ViewT>() {
-      @Override
-      public void add(Iterable<WindowedValue<ElemT>> values) {
-        sideInputContainer.write(output, values);
-      }
-    };
-  }
-
-  /**
-   * Schedule a callback to be executed after output would be produced for the given window
-   * if there had been input.
-   *
-   * <p>Output would be produced when the watermark for a {@link PValue} passes the point at
-   * which the trigger for the specified window (with the specified windowing strategy) must have
-   * fired from the perspective of that {@link PValue}, as specified by the value of
-   * {@link Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the
-   * {@link WindowingStrategy}. When the callback has fired, either values will have been produced
-   * for a key in that window, the window is empty, or all elements in the window are late. The
-   * callback will be executed regardless of whether values have been produced.
-   */
-  public void scheduleAfterOutputWouldBeProduced(
-      PValue value,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    AppliedPTransform<?, ?, ?> producing = getProducing(value);
-    callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
-
-    fireAvailableCallbacks(lookupProducing(value));
-  }
-
-  private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
-    if (value.getProducingTransformInternal() != null) {
-      return value.getProducingTransformInternal();
-    }
-    return lookupProducing(value);
-  }
-
-  private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
-        return transform;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Get the options used by this {@link Pipeline}.
-   */
-  public InProcessPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  /**
-   * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
-   */
-  public InProcessExecutionContext getExecutionContext(
-      AppliedPTransform<?, ?, ?> application, Object key) {
-    StepAndKey stepAndKey = StepAndKey.of(application, key);
-    return new InProcessExecutionContext(
-        options.getClock(),
-        key,
-        (CopyOnAccessInMemoryStateInternals<Object>) applicationStateInternals.get(stepAndKey),
-        watermarkManager.getWatermarks(application));
-  }
-
-  /**
-   * Get all of the steps used in this {@link Pipeline}.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
-    return stepNames.keySet();
-  }
-
-  /**
-   * Get the Step Name for the provided application.
-   */
-  public String getStepName(AppliedPTransform<?, ?, ?> application) {
-    return stepNames.get(application);
-  }
-
-  /**
-   * Returns a {@link ReadyCheckingSideInputReader} capable of reading the provided
-   * {@link PCollectionView PCollectionViews}.
-   *
-   * @param sideInputs the {@link PCollectionView PCollectionViews} the result should be able to
-   * read
-   * @return a {@link SideInputReader} that can read all of the provided {@link PCollectionView
-   * PCollectionViews}
-   */
-  public ReadyCheckingSideInputReader createSideInputReader(
-      final List<PCollectionView<?>> sideInputs) {
-    return sideInputContainer.createReaderForViews(sideInputs);
-  }
-
-  /**
-   * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has
-   * had its contents set in a window.
-   */
-  static interface ReadyCheckingSideInputReader extends SideInputReader {
-    /**
-     * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}.
-     */
-    boolean isReady(PCollectionView<?> view, BoundedWindow window);
-  }
-
-  /**
-   * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent
-   * of all other {@link CounterSet CounterSets} created by this call.
-   *
-   * The {@link InProcessEvaluationContext} is responsible for unifying the counters present in
-   * all created {@link CounterSet CounterSets} when the transforms that call this method
-   * complete.
-   */
-  public CounterSet createCounterSet() {
-    return new CounterSet();
-  }
-
-  /**
-   * Returns all of the counters that have been merged into this context via calls to
-   * {@link CounterSet#merge(CounterSet)}.
-   */
-  public CounterSet getCounters() {
-    return mergedCounters;
-  }
-
-  /**
-   * Extracts all timers that have been fired and have not already been extracted.
-   *
-   * <p>This is a destructive operation. Timers will only appear in the result of this method once
-   * for each time they are set.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired =
-        watermarkManager.extractFiredTimers();
-    return fired;
-  }
-
-  /**
-   * Returns true if the step will not produce additional output.
-   *
-   * <p>If the provided transform produces only {@link IsBounded#BOUNDED}
-   * {@link PCollection PCollections}, returns true if the watermark is at
-   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE positive infinity}.
-   *
-   * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
-   * {@link PCollection PCollections}, returns the value of
-   * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
-   */
-  public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
-    // if the PTransform's watermark isn't at the max value, it isn't done
-    if (watermarkManager
-        .getWatermarks(transform)
-        .getOutputWatermark()
-        .isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
-      return false;
-    }
-    // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
-    // the PTransform may produce additional output. It is not done.
-    for (PValue output : transform.getOutput().expand()) {
-      if (output instanceof PCollection) {
-        IsBounded bounded = ((PCollection<?>) output).isBounded();
-        if (bounded.equals(IsBounded.UNBOUNDED)
-            && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
-          return false;
-        }
-      }
-    }
-    // The PTransform's watermark was at positive infinity and all of its outputs are known to be
-    // done. It is done.
-    return true;
-  }
-
-  /**
-   * Returns true if all steps are done.
-   */
-  public boolean isDone() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      if (!isDone(transform)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.java
deleted file mode 100644
index e6441cf..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutionContext.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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.BaseExecutionContext;
-import org.apache.beam.sdk.util.ExecutionContext;
-import org.apache.beam.sdk.util.TimerInternals;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-/**
- * Execution Context for the {@link InProcessPipelineRunner}.
- *
- * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
- * for each thread that requires it.
- */
-class InProcessExecutionContext
-    extends BaseExecutionContext<InProcessExecutionContext.InProcessStepContext> {
-  private final Clock clock;
-  private final Object key;
-  private final CopyOnAccessInMemoryStateInternals<Object> existingState;
-  private final TransformWatermarks watermarks;
-
-  public InProcessExecutionContext(Clock clock, Object key,
-      CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
-    this.clock = clock;
-    this.key = key;
-    this.existingState = existingState;
-    this.watermarks = watermarks;
-  }
-
-  @Override
-  protected InProcessStepContext createStepContext(String stepName, String transformName) {
-    return new InProcessStepContext(this, stepName, transformName);
-  }
-
-  /**
-   * Step Context for the {@link InProcessPipelineRunner}.
-   */
-  public class InProcessStepContext
-      extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
-    private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
-    private InProcessTimerInternals timerInternals;
-
-    public InProcessStepContext(
-        ExecutionContext executionContext, String stepName, String transformName) {
-      super(executionContext, stepName, transformName);
-    }
-
-    @Override
-    public CopyOnAccessInMemoryStateInternals<Object> stateInternals() {
-      if (stateInternals == null) {
-        stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
-      }
-      return stateInternals;
-    }
-
-    @Override
-    public InProcessTimerInternals timerInternals() {
-      if (timerInternals == null) {
-        timerInternals =
-            InProcessTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
-      }
-      return timerInternals;
-    }
-
-    /**
-     * Commits the state of this step, and returns the committed state. If the step has not
-     * accessed any state, return null.
-     */
-    public CopyOnAccessInMemoryStateInternals<?> commitState() {
-      if (stateInternals != null) {
-        return stateInternals.commit();
-      }
-      return null;
-    }
-
-    /**
-     * Gets the timer update of the {@link TimerInternals} of this {@link InProcessStepContext},
-     * which is empty if the {@link TimerInternals} were never accessed.
-     */
-    public TimerUpdate getTimerUpdate() {
-      if (timerInternals == null) {
-        return TimerUpdate.empty();
-      }
-      return timerInternals.getTimerUpdate();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
deleted file mode 100644
index 059aae6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessExecutor.java
+++ /dev/null
@@ -1,48 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import java.util.Collection;
-
-/**
- * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both
- * source and intermediate {@link PTransform PTransforms}.
- */
-interface InProcessExecutor {
-  /**
-   * Starts this executor. The provided collection is the collection of root transforms to
-   * initially schedule.
-   *
-   * @param rootTransforms
-   */
-  void start(Collection<AppliedPTransform<?, ?, ?>> rootTransforms);
-
-  /**
-   * Blocks until the job being executed enters a terminal state. A job is completed after all
-   * root {@link AppliedPTransform AppliedPTransforms} have completed, and all
-   * {@link CommittedBundle Bundles} have been consumed. Jobs may also terminate abnormally.
-   *
-   * @throws Throwable whenever an executor thread throws anything, transfers the throwable to the
-   *                   waiting thread and rethrows it
-   */
-  void awaitCompletion() throws Throwable;
-}



[10/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
new file mode 100644
index 0000000..59c4d8e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
@@ -0,0 +1,526 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessExecutionContext.InProcessStepContext;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
+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.PaneInfo.Timing;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for {@link InProcessEvaluationContext}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessEvaluationContextTest {
+  private TestPipeline p;
+  private InProcessEvaluationContext context;
+
+  private PCollection<Integer> created;
+  private PCollection<KV<String, Integer>> downstream;
+  private PCollectionView<Iterable<Integer>> view;
+  private PCollection<Long> unbounded;
+  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms;
+  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
+
+  private BundleFactory bundleFactory;
+
+  @Before
+  public void setup() {
+    InProcessPipelineRunner runner =
+        InProcessPipelineRunner.fromOptions(PipelineOptionsFactory.create());
+
+    p = TestPipeline.create();
+
+    created = p.apply(Create.of(1, 2, 3));
+    downstream = created.apply(WithKeys.<String, Integer>of("foo"));
+    view = created.apply(View.<Integer>asIterable());
+    unbounded = p.apply(CountingInput.unbounded());
+
+    ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
+    p.traverseTopologically(cVis);
+    rootTransforms = cVis.getRootTransforms();
+    valueToConsumers = cVis.getValueToConsumers();
+
+    bundleFactory = InProcessBundleFactory.create();
+
+    context =
+        InProcessEvaluationContext.create(
+            runner.getPipelineOptions(),
+            InProcessBundleFactory.create(),
+            rootTransforms,
+            valueToConsumers,
+            cVis.getStepNames(),
+            cVis.getViews());
+  }
+
+  @Test
+  public void writeToViewWriterThenReadReads() {
+    PCollectionViewWriter<Integer, Iterable<Integer>> viewWriter =
+        context.createPCollectionViewWriter(
+            PCollection.<Iterable<Integer>>createPrimitiveOutputInternal(
+                p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED),
+            view);
+    BoundedWindow window = new TestBoundedWindow(new Instant(1024L));
+    BoundedWindow second = new TestBoundedWindow(new Instant(899999L));
+    WindowedValue<Integer> firstValue =
+        WindowedValue.of(1, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    WindowedValue<Integer> secondValue =
+        WindowedValue.of(
+            2, new Instant(8766L), second, PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0));
+    Iterable<WindowedValue<Integer>> values = ImmutableList.of(firstValue, secondValue);
+    viewWriter.add(values);
+
+    SideInputReader reader =
+        context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
+    assertThat(reader.get(view, window), containsInAnyOrder(1));
+    assertThat(reader.get(view, second), containsInAnyOrder(2));
+
+    WindowedValue<Integer> overrittenSecondValue =
+        WindowedValue.of(
+            4444, new Instant(8677L), second, PaneInfo.createPane(false, true, Timing.LATE, 1, 1));
+    viewWriter.add(Collections.singleton(overrittenSecondValue));
+    assertThat(reader.get(view, second), containsInAnyOrder(4444));
+  }
+
+  @Test
+  public void getExecutionContextSameStepSameKeyState() {
+    InProcessExecutionContext fooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    InProcessStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1");
+    stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
+
+    context.handleResult(
+        InProcessBundleFactory.create()
+            .createKeyedBundle(null, "foo", created)
+            .commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(created.getProducingTransformInternal())
+            .withState(stepContext.commitState())
+            .build());
+
+    InProcessExecutionContext secondFooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+    assertThat(
+        secondFooContext
+            .getOrCreateStepContext("s1", "s1")
+            .stateInternals()
+            .state(StateNamespaces.global(), intBag)
+            .read(),
+        contains(1));
+  }
+
+
+  @Test
+  public void getExecutionContextDifferentKeysIndependentState() {
+    InProcessExecutionContext fooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(), "foo");
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    fooContext
+        .getOrCreateStepContext("s1", "s1")
+        .stateInternals()
+        .state(StateNamespaces.global(), intBag)
+        .add(1);
+
+    InProcessExecutionContext barContext =
+        context.getExecutionContext(created.getProducingTransformInternal(), "bar");
+    assertThat(barContext, not(equalTo(fooContext)));
+    assertThat(
+        barContext
+            .getOrCreateStepContext("s1", "s1")
+            .stateInternals()
+            .state(StateNamespaces.global(), intBag)
+            .read(),
+        emptyIterable());
+  }
+
+  @Test
+  public void getExecutionContextDifferentStepsIndependentState() {
+    String myKey = "foo";
+    InProcessExecutionContext fooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(), myKey);
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    fooContext
+        .getOrCreateStepContext("s1", "s1")
+        .stateInternals()
+        .state(StateNamespaces.global(), intBag)
+        .add(1);
+
+    InProcessExecutionContext barContext =
+        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+    assertThat(
+        barContext
+            .getOrCreateStepContext("s1", "s1")
+            .stateInternals()
+            .state(StateNamespaces.global(), intBag)
+            .read(),
+        emptyIterable());
+  }
+
+  @Test
+  public void handleResultMergesCounters() {
+    CounterSet counters = context.createCounterSet();
+    Counter<Long> myCounter = Counter.longs("foo", AggregationKind.SUM);
+    counters.addCounter(myCounter);
+
+    myCounter.addValue(4L);
+    InProcessTransformResult result =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal())
+            .withCounters(counters)
+            .build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), result);
+    assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(4L));
+
+    CounterSet againCounters = context.createCounterSet();
+    Counter<Long> myLongCounterAgain = Counter.longs("foo", AggregationKind.SUM);
+    againCounters.add(myLongCounterAgain);
+    myLongCounterAgain.addValue(8L);
+
+    InProcessTransformResult secondResult =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+            .withCounters(againCounters)
+            .build();
+    context.handleResult(
+        context.createRootBundle(created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        secondResult);
+    assertThat((Long) context.getCounters().getExistingCounter("foo").getAggregate(), equalTo(12L));
+  }
+
+  @Test
+  public void handleResultStoresState() {
+    String myKey = "foo";
+    InProcessExecutionContext fooContext =
+        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    CopyOnAccessInMemoryStateInternals<Object> state =
+        fooContext.getOrCreateStepContext("s1", "s1").stateInternals();
+    BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
+    bag.add(1);
+    bag.add(2);
+    bag.add(4);
+
+    InProcessTransformResult stateResult =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+            .withState(state)
+            .build();
+
+    context.handleResult(
+        context.createKeyedBundle(null, myKey, created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        stateResult);
+
+    InProcessExecutionContext afterResultContext =
+        context.getExecutionContext(downstream.getProducingTransformInternal(), myKey);
+
+    CopyOnAccessInMemoryStateInternals<Object> afterResultState =
+        afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals();
+    assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
+  }
+
+  @Test
+  public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
+    final CountDownLatch callLatch = new CountDownLatch(1);
+    Runnable callback =
+        new Runnable() {
+          @Override
+          public void run() {
+            callLatch.countDown();
+          }
+        };
+
+    // Should call back after the end of the global window
+    context.scheduleAfterOutputWouldBeProduced(
+        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
+
+    InProcessTransformResult result =
+        StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
+            .build();
+
+    context.handleResult(null, ImmutableList.<TimerData>of(), result);
+
+    // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
+    // will likely be flaky if this logic is broken
+    assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
+
+    InProcessTransformResult finishedResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+    // Obtain the value via blocking call
+    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
+  }
+
+  @Test
+  public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
+    InProcessTransformResult finishedResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+
+    final CountDownLatch callLatch = new CountDownLatch(1);
+    Runnable callback =
+        new Runnable() {
+          @Override
+          public void run() {
+            callLatch.countDown();
+          }
+        };
+    context.scheduleAfterOutputWouldBeProduced(
+        downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
+    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
+  }
+
+  @Test
+  public void extractFiredTimersExtractsTimers() {
+    InProcessTransformResult holdResult =
+        StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
+            .build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), holdResult);
+
+    String key = "foo";
+    TimerData toFire =
+        TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
+    InProcessTransformResult timerResult =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+            .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
+            .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
+            .build();
+
+    // haven't added any timers, must be empty
+    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+    context.handleResult(
+        context.createKeyedBundle(null, key, created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        timerResult);
+
+    // timer hasn't fired
+    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+
+    InProcessTransformResult advanceResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    // Should cause the downstream timer to fire
+    context.handleResult(null, ImmutableList.<TimerData>of(), advanceResult);
+
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> fired = context.extractFiredTimers();
+    assertThat(
+        fired,
+        Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
+    Map<Object, FiredTimers> downstreamFired =
+        fired.get(downstream.getProducingTransformInternal());
+    assertThat(downstreamFired, Matchers.<Object>hasKey(key));
+
+    FiredTimers firedForKey = downstreamFired.get(key);
+    assertThat(firedForKey.getTimers(TimeDomain.PROCESSING_TIME), emptyIterable());
+    assertThat(firedForKey.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), emptyIterable());
+    assertThat(firedForKey.getTimers(TimeDomain.EVENT_TIME), contains(toFire));
+
+    // Don't reextract timers
+    assertThat(context.extractFiredTimers().entrySet(), emptyIterable());
+  }
+
+  @Test
+  public void createBundleKeyedResultPropagatesKey() {
+    CommittedBundle<KV<String, Integer>> newBundle =
+        context
+            .createBundle(
+                bundleFactory.createKeyedBundle(null, "foo", created).commit(Instant.now()),
+                downstream)
+            .commit(Instant.now());
+    assertThat(newBundle.getKey(), Matchers.<Object>equalTo("foo"));
+  }
+
+  @Test
+  public void createKeyedBundleKeyed() {
+    CommittedBundle<KV<String, Integer>> keyedBundle =
+        context
+            .createKeyedBundle(
+                bundleFactory.createRootBundle(created).commit(Instant.now()), "foo", downstream)
+            .commit(Instant.now());
+    assertThat(keyedBundle.getKey(), Matchers.<Object>equalTo("foo"));
+  }
+
+  @Test
+  public void isDoneWithUnboundedPCollectionAndShutdown() {
+    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
+    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(true));
+  }
+
+  @Test
+  public void isDoneWithUnboundedPCollectionAndNotShutdown() {
+    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+  }
+
+  @Test
+  public void isDoneWithOnlyBoundedPCollections() {
+    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+    assertThat(context.isDone(created.getProducingTransformInternal()), is(false));
+
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
+    assertThat(context.isDone(created.getProducingTransformInternal()), is(true));
+  }
+
+  @Test
+  public void isDoneWithPartiallyDone() {
+    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true);
+    assertThat(context.isDone(), is(false));
+
+    UncommittedBundle<Integer> rootBundle = context.createRootBundle(created);
+    rootBundle.add(WindowedValue.valueInGlobalWindow(1));
+    CommittedResult handleResult =
+        context.handleResult(
+            null,
+            ImmutableList.<TimerData>of(),
+            StepTransformResult.withoutHold(created.getProducingTransformInternal())
+                .addOutput(rootBundle)
+                .build());
+    @SuppressWarnings("unchecked")
+    CommittedBundle<Integer> committedBundle =
+        (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+    assertThat(context.isDone(), is(false));
+
+    for (AppliedPTransform<?, ?, ?> consumers : valueToConsumers.get(created)) {
+      context.handleResult(
+          committedBundle,
+          ImmutableList.<TimerData>of(),
+          StepTransformResult.withoutHold(consumers).build());
+    }
+    assertThat(context.isDone(), is(true));
+  }
+
+  @Test
+  public void isDoneWithUnboundedAndNotShutdown() {
+    context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
+    assertThat(context.isDone(), is(false));
+
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build());
+    context.handleResult(
+        null,
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
+    context.handleResult(
+        context.createRootBundle(created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build());
+    assertThat(context.isDone(), is(false));
+
+    context.handleResult(
+        context.createRootBundle(created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
+    assertThat(context.isDone(), is(false));
+  }
+
+  private static class TestBoundedWindow extends BoundedWindow {
+    private final Instant ts;
+
+    public TestBoundedWindow(Instant ts) {
+      this.ts = ts;
+    }
+
+    @Override
+    public Instant maxTimestamp() {
+      return ts;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
new file mode 100644
index 0000000..54094c4
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRegistrarTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.runners.direct.InProcessRegistrar.InProcessRunner;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ServiceLoader;
+
+/** Tests for {@link InProcessRunner}. */
+@RunWith(JUnit4.class)
+public class InProcessPipelineRegistrarTest {
+  @Test
+  public void testCorrectOptionsAreReturned() {
+    assertEquals(
+        ImmutableList.of(InProcessPipelineOptions.class),
+        new InProcessRegistrar.InProcessOptions().getPipelineOptions());
+  }
+
+  @Test
+  public void testCorrectRunnersAreReturned() {
+    assertEquals(
+        ImmutableList.of(InProcessPipelineRunner.class),
+        new InProcessRegistrar.InProcessRunner().getPipelineRunners());
+  }
+
+  @Test
+  public void testServiceLoaderForOptions() {
+    for (PipelineOptionsRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
+      if (registrar instanceof InProcessRegistrar.InProcessOptions) {
+        return;
+      }
+    }
+    fail("Expected to find " + InProcessRegistrar.InProcessOptions.class);
+  }
+
+  @Test
+  public void testServiceLoaderForRunner() {
+    for (PipelineRunnerRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
+      if (registrar instanceof InProcessRegistrar.InProcessRunner) {
+        return;
+      }
+    }
+    fail("Expected to find " + InProcessRegistrar.InProcessRunner.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
new file mode 100644
index 0000000..87db39a
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.InProcessPipelineResult;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+
+/**
+ * Tests for basic {@link InProcessPipelineRunner} functionality.
+ */
+@RunWith(JUnit4.class)
+public class InProcessPipelineRunnerTest implements Serializable {
+  @Test
+  public void wordCountShouldSucceed() throws Throwable {
+    Pipeline p = getPipeline();
+
+    PCollection<KV<String, Long>> counts =
+        p.apply(Create.of("foo", "bar", "foo", "baz", "bar", "foo"))
+            .apply(MapElements.via(new SimpleFunction<String, String>() {
+              @Override
+              public String apply(String input) {
+                return input;
+              }
+            }))
+            .apply(Count.<String>perElement());
+    PCollection<String> countStrs =
+        counts.apply(MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
+          @Override
+          public String apply(KV<String, Long> input) {
+            String str = String.format("%s: %s", input.getKey(), input.getValue());
+            return str;
+          }
+        }));
+
+    PAssert.that(countStrs).containsInAnyOrder("baz: 1", "bar: 2", "foo: 3");
+
+    InProcessPipelineResult result = ((InProcessPipelineResult) p.run());
+    result.awaitCompletion();
+  }
+
+  private Pipeline getPipeline() {
+    PipelineOptions opts = PipelineOptionsFactory.create();
+    opts.setRunner(InProcessPipelineRunner.class);
+
+    Pipeline p = Pipeline.create(opts);
+    return p;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
new file mode 100644
index 0000000..d8a78f2
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
@@ -0,0 +1,496 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doAnswer;
+
+import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
+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.PaneInfo.Timing;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * Tests for {@link InProcessSideInputContainer}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessSideInputContainerTest {
+  private static final BoundedWindow FIRST_WINDOW =
+      new BoundedWindow() {
+        @Override
+        public Instant maxTimestamp() {
+          return new Instant(789541L);
+        }
+
+        @Override
+        public String toString() {
+          return "firstWindow";
+        }
+      };
+
+  private static final BoundedWindow SECOND_WINDOW =
+      new BoundedWindow() {
+        @Override
+        public Instant maxTimestamp() {
+          return new Instant(14564786L);
+        }
+
+        @Override
+        public String toString() {
+          return "secondWindow";
+        }
+      };
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Mock
+  private InProcessEvaluationContext context;
+
+  private TestPipeline pipeline;
+
+  private InProcessSideInputContainer container;
+
+  private PCollectionView<Map<String, Integer>> mapView;
+  private PCollectionView<Double> singletonView;
+
+  // Not present in container.
+  private PCollectionView<Iterable<Integer>> iterableView;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    pipeline = TestPipeline.create();
+
+    PCollection<Integer> create =
+        pipeline.apply("forBaseCollection", Create.<Integer>of(1, 2, 3, 4));
+
+    mapView =
+        create.apply("forKeyTypes", WithKeys.<String, Integer>of("foo"))
+            .apply("asMapView", View.<String, Integer>asMap());
+
+    singletonView = create.apply("forCombinedTypes", Mean.<Integer>globally().asSingletonView());
+    iterableView = create.apply("asIterableView", View.<Integer>asIterable());
+
+    container = InProcessSideInputContainer.create(
+        context, ImmutableList.of(iterableView, mapView, singletonView));
+  }
+
+  @Test
+  public void getAfterWriteReturnsPaneInWindow() throws Exception {
+    WindowedValue<KV<String, Integer>> one =
+        WindowedValue.of(
+            KV.of("one", 1), new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    WindowedValue<KV<String, Integer>> two =
+        WindowedValue.of(
+            KV.of("two", 2), new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+    Map<String, Integer> viewContents =
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+            .get(mapView, FIRST_WINDOW);
+    assertThat(viewContents, hasEntry("one", 1));
+    assertThat(viewContents, hasEntry("two", 2));
+    assertThat(viewContents.size(), is(2));
+  }
+
+  @Test
+  public void getReturnsLatestPaneInWindow() throws Exception {
+    WindowedValue<KV<String, Integer>> one =
+        WindowedValue.of(
+            KV.of("one", 1),
+            new Instant(1L),
+            SECOND_WINDOW,
+            PaneInfo.createPane(true, false, Timing.EARLY));
+    WindowedValue<KV<String, Integer>> two =
+        WindowedValue.of(
+            KV.of("two", 2),
+            new Instant(20L),
+            SECOND_WINDOW,
+            PaneInfo.createPane(true, false, Timing.EARLY));
+    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+    Map<String, Integer> viewContents =
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+            .get(mapView, SECOND_WINDOW);
+    assertThat(viewContents, hasEntry("one", 1));
+    assertThat(viewContents, hasEntry("two", 2));
+    assertThat(viewContents.size(), is(2));
+
+    WindowedValue<KV<String, Integer>> three =
+        WindowedValue.of(
+            KV.of("three", 3),
+            new Instant(300L),
+            SECOND_WINDOW,
+            PaneInfo.createPane(false, false, Timing.EARLY, 1, -1));
+    container.write(mapView, ImmutableList.<WindowedValue<?>>of(three));
+
+    Map<String, Integer> overwrittenViewContents =
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+            .get(mapView, SECOND_WINDOW);
+    assertThat(overwrittenViewContents, hasEntry("three", 3));
+    assertThat(overwrittenViewContents.size(), is(1));
+  }
+
+  /**
+   * Demonstrates that calling get() on a window that currently has no data does not return until
+   * there is data in the pane.
+   */
+  @Test
+  public void getBlocksUntilPaneAvailable() throws Exception {
+    BoundedWindow window =
+        new BoundedWindow() {
+          @Override
+          public Instant maxTimestamp() {
+            return new Instant(1024L);
+          }
+        };
+    Future<Double> singletonFuture =
+        getFutureOfView(
+            container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView)),
+            singletonView,
+            window);
+
+    WindowedValue<Double> singletonValue =
+        WindowedValue.of(4.75, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+    assertThat(singletonFuture.isDone(), is(false));
+    container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
+    assertThat(singletonFuture.get(), equalTo(4.75));
+  }
+
+  @Test
+  public void withPCollectionViewsWithPutInOriginalReturnsContents() throws Exception {
+    BoundedWindow window = new BoundedWindow() {
+      @Override
+      public Instant maxTimestamp() {
+        return new Instant(1024L);
+      }
+    };
+    SideInputReader newReader =
+        container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView));
+    Future<Double> singletonFuture = getFutureOfView(newReader, singletonView, window);
+
+    WindowedValue<Double> singletonValue =
+        WindowedValue.of(24.125, new Instant(475L), window, PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+    assertThat(singletonFuture.isDone(), is(false));
+    container.write(singletonView, ImmutableList.<WindowedValue<?>>of(singletonValue));
+    assertThat(singletonFuture.get(), equalTo(24.125));
+  }
+
+  @Test
+  public void withPCollectionViewsErrorsForContainsNotInViews() {
+    PCollectionView<Map<String, Iterable<String>>> newView =
+        PCollectionViews.multimapView(
+            pipeline,
+            WindowingStrategy.globalDefault(),
+            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("with unknown views " + ImmutableList.of(newView).toString());
+
+    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
+  }
+
+  @Test
+  public void withViewsForViewNotInContainerFails() {
+    PCollectionView<Map<String, Iterable<String>>> newView =
+        PCollectionViews.multimapView(
+            pipeline,
+            WindowingStrategy.globalDefault(),
+            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("unknown views");
+    thrown.expectMessage(newView.toString());
+
+    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(newView));
+  }
+
+  @Test
+  public void getOnReaderForViewNotInReaderFails() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("unknown view: " + iterableView.toString());
+
+    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+        .get(iterableView, GlobalWindow.INSTANCE);
+  }
+
+  @Test
+  public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exception {
+    WindowedValue<Double> firstWindowedValue =
+        WindowedValue.of(
+            2.875,
+            FIRST_WINDOW.maxTimestamp().minus(200L),
+            FIRST_WINDOW,
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    WindowedValue<Double> secondWindowedValue =
+        WindowedValue.of(
+            4.125,
+            SECOND_WINDOW.maxTimestamp().minus(2_000_000L),
+            SECOND_WINDOW,
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    container.write(singletonView, ImmutableList.of(firstWindowedValue, secondWindowedValue));
+    assertThat(
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+            .get(singletonView, FIRST_WINDOW),
+        equalTo(2.875));
+    assertThat(
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+            .get(singletonView, SECOND_WINDOW),
+        equalTo(4.125));
+  }
+
+  @Test
+  public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Exception {
+    WindowedValue<Integer> firstValue =
+        WindowedValue.of(
+            44,
+            FIRST_WINDOW.maxTimestamp().minus(200L),
+            FIRST_WINDOW,
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    WindowedValue<Integer> secondValue =
+        WindowedValue.of(
+            44,
+            FIRST_WINDOW.maxTimestamp().minus(200L),
+            FIRST_WINDOW,
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+
+    container.write(iterableView, ImmutableList.of(firstValue, secondValue));
+
+    assertThat(
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(iterableView))
+            .get(iterableView, FIRST_WINDOW),
+        contains(44, 44));
+  }
+
+  @Test
+  public void writeForElementInMultipleWindowsSucceeds() throws Exception {
+    WindowedValue<Double> multiWindowedValue =
+        WindowedValue.of(
+            2.875,
+            FIRST_WINDOW.maxTimestamp().minus(200L),
+            ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    container.write(singletonView, ImmutableList.of(multiWindowedValue));
+    assertThat(
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+            .get(singletonView, FIRST_WINDOW),
+        equalTo(2.875));
+    assertThat(
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(singletonView))
+            .get(singletonView, SECOND_WINDOW),
+        equalTo(2.875));
+  }
+
+  @Test
+  public void finishDoesNotOverwriteWrittenElements() throws Exception {
+    WindowedValue<KV<String, Integer>> one =
+        WindowedValue.of(
+            KV.of("one", 1),
+            new Instant(1L),
+            SECOND_WINDOW,
+            PaneInfo.createPane(true, false, Timing.EARLY));
+    WindowedValue<KV<String, Integer>> two =
+        WindowedValue.of(
+            KV.of("two", 2),
+            new Instant(20L),
+            SECOND_WINDOW,
+            PaneInfo.createPane(true, false, Timing.EARLY));
+    container.write(mapView, ImmutableList.<WindowedValue<?>>of(one, two));
+
+    immediatelyInvokeCallback(mapView, SECOND_WINDOW);
+
+    Map<String, Integer> viewContents =
+        container
+            .createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+            .get(mapView, SECOND_WINDOW);
+
+    assertThat(viewContents, hasEntry("one", 1));
+    assertThat(viewContents, hasEntry("two", 2));
+    assertThat(viewContents.size(), is(2));
+  }
+
+  @Test
+  public void finishOnPendingViewsSetsEmptyElements() throws Exception {
+    immediatelyInvokeCallback(mapView, SECOND_WINDOW);
+    Future<Map<String, Integer>> mapFuture =
+        getFutureOfView(
+            container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView)),
+            mapView,
+            SECOND_WINDOW);
+
+    assertThat(mapFuture.get().isEmpty(), is(true));
+  }
+
+  /**
+   * Demonstrates that calling isReady on an empty container throws an
+   * {@link IllegalArgumentException}.
+   */
+  @Test
+  public void isReadyInEmptyReaderThrows() {
+    ReadyCheckingSideInputReader reader =
+        container.createReaderForViews(ImmutableList.<PCollectionView<?>>of());
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("does not contain");
+    thrown.expectMessage(ImmutableList.of().toString());
+    reader.isReady(mapView, GlobalWindow.INSTANCE);
+  }
+
+  /**
+   * Demonstrates that calling isReady returns false until elements are written to the
+   * {@link PCollectionView}, {@link BoundedWindow} pair, at which point it returns true.
+   */
+  @Test
+  public void isReadyForSomeNotReadyViewsFalseUntilElements() {
+    container.write(
+        mapView,
+        ImmutableList.of(
+            WindowedValue.of(
+                KV.of("one", 1),
+                SECOND_WINDOW.maxTimestamp().minus(100L),
+                SECOND_WINDOW,
+                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+
+    ReadyCheckingSideInputReader reader =
+        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
+    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
+
+    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(false));
+
+    container.write(
+        mapView,
+        ImmutableList.of(
+            WindowedValue.of(
+                KV.of("too", 2),
+                FIRST_WINDOW.maxTimestamp().minus(100L),
+                FIRST_WINDOW,
+                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
+
+    container.write(
+        singletonView,
+        ImmutableList.of(
+            WindowedValue.of(
+                1.25,
+                SECOND_WINDOW.maxTimestamp().minus(100L),
+                SECOND_WINDOW,
+                PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
+    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
+
+    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+  }
+
+  @Test
+  public void isReadyForEmptyWindowTrue() {
+    immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
+
+    ReadyCheckingSideInputReader reader =
+        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+
+    immediatelyInvokeCallback(singletonView, GlobalWindow.INSTANCE);
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(true));
+  }
+
+  /**
+   * When a callAfterWindowCloses with the specified view's producing transform, window, and
+   * windowing strategy is invoked, immediately execute the callback.
+   */
+  private void immediatelyInvokeCallback(PCollectionView<?> view, BoundedWindow window) {
+    doAnswer(
+            new Answer<Void>() {
+              @Override
+              public Void answer(InvocationOnMock invocation) throws Throwable {
+                Object callback = invocation.getArguments()[3];
+                Runnable callbackRunnable = (Runnable) callback;
+                callbackRunnable.run();
+                return null;
+              }
+            })
+        .when(context)
+        .scheduleAfterOutputWouldBeProduced(
+            Mockito.eq(view),
+            Mockito.eq(window),
+            Mockito.eq(view.getWindowingStrategyInternal()),
+            Mockito.any(Runnable.class));
+  }
+
+  private <ValueT> Future<ValueT> getFutureOfView(final SideInputReader myReader,
+      final PCollectionView<ValueT> view, final BoundedWindow window) {
+    Callable<ValueT> callable = new Callable<ValueT>() {
+      @Override
+      public ValueT call() throws Exception {
+        return myReader.get(view, window);
+      }
+    };
+    return Executors.newSingleThreadExecutor().submit(callable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
new file mode 100644
index 0000000..34a8980
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+
+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.MockitoAnnotations;
+
+/**
+ * Tests for {@link InProcessTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class InProcessTimerInternalsTest {
+  private MockClock clock;
+  @Mock private TransformWatermarks watermarks;
+
+  private TimerUpdateBuilder timerUpdateBuilder;
+
+  private InProcessTimerInternals internals;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    clock = MockClock.fromInstant(new Instant(0));
+
+    timerUpdateBuilder = TimerUpdate.builder(1234);
+
+    internals = InProcessTimerInternals.create(clock, watermarks, timerUpdateBuilder);
+  }
+
+  @Test
+  public void setTimerAddsToBuilder() {
+    TimerData eventTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
+    TimerData processingTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
+    TimerData synchronizedProcessingTimer =
+        TimerData.of(
+            StateNamespaces.global(),
+            new Instant(98745632189L),
+            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    internals.setTimer(eventTimer);
+    internals.setTimer(processingTimer);
+    internals.setTimer(synchronizedProcessingTimer);
+
+    assertThat(
+        internals.getTimerUpdate().getSetTimers(),
+        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
+  }
+
+  @Test
+  public void deleteTimerDeletesOnBuilder() {
+    TimerData eventTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
+    TimerData processingTimer =
+        TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
+    TimerData synchronizedProcessingTimer =
+        TimerData.of(
+            StateNamespaces.global(),
+            new Instant(98745632189L),
+            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    internals.deleteTimer(eventTimer);
+    internals.deleteTimer(processingTimer);
+    internals.deleteTimer(synchronizedProcessingTimer);
+
+    assertThat(
+        internals.getTimerUpdate().getDeletedTimers(),
+        containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
+  }
+
+  @Test
+  public void getProcessingTimeIsClockNow() {
+    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
+    Instant oldProcessingTime = internals.currentProcessingTime();
+
+    clock.advance(Duration.standardHours(12));
+
+    assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
+    assertThat(
+        internals.currentProcessingTime(),
+        equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
+  }
+
+  @Test
+  public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
+    when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
+    assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
+  }
+
+  @Test
+  public void getInputWatermarkTimeUsesWatermarkTime() {
+    when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
+    assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
+  }
+
+  @Test
+  public void getOutputWatermarkTimeUsesWatermarkTime() {
+    when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
+    assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
new file mode 100644
index 0000000..24f9715
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Keys;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Tests for {@link KeyedPValueTrackingVisitor}.
+ */
+@RunWith(JUnit4.class)
+public class KeyedPValueTrackingVisitorTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private KeyedPValueTrackingVisitor visitor;
+  private Pipeline p;
+
+  @Before
+  public void setup() {
+    PipelineOptions options = PipelineOptionsFactory.create();
+
+    p = Pipeline.create(options);
+    @SuppressWarnings("rawtypes")
+    Set<Class<? extends PTransform>> producesKeyed =
+        ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);
+    visitor = KeyedPValueTrackingVisitor.create(producesKeyed);
+  }
+
+  @Test
+  public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() {
+    PCollection<Integer> keyed =
+        p.apply(Create.<Integer>of(1, 2, 3)).apply(new PrimitiveKeyer<Integer>());
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+  }
+
+  @Test
+  public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() {
+    PCollection<Integer> keyed =
+        p.apply(Create.<Integer>of(1, 2, 3))
+            .apply("firstKey", new PrimitiveKeyer<Integer>())
+            .apply("secondKey", new PrimitiveKeyer<Integer>());
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+  }
+
+  @Test
+  public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() {
+    PCollection<Integer> keyed =
+        p.apply(Create.<Integer>of(1, 2, 3)).apply(new CompositeKeyer<Integer>());
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+  }
+
+  @Test
+  public void compositeProducesKeyedOutputKeyedInputKeyedOutut() {
+    PCollection<Integer> keyed =
+        p.apply(Create.<Integer>of(1, 2, 3))
+            .apply("firstKey", new CompositeKeyer<Integer>())
+            .apply("secondKey", new CompositeKeyer<Integer>());
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+  }
+
+
+  @Test
+  public void noInputUnkeyedOutput() {
+    PCollection<KV<Integer, Iterable<Void>>> unkeyed =
+        p.apply(
+            Create.of(KV.<Integer, Iterable<Void>>of(-1, Collections.<Void>emptyList()))
+                .withCoder(KvCoder.of(VarIntCoder.of(), IterableCoder.of(VoidCoder.of()))));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
+  }
+
+  @Test
+  public void keyedInputNotProducesKeyedOutputUnkeyedOutput() {
+    PCollection<Integer> onceKeyed =
+        p.apply(Create.<Integer>of(1, 2, 3))
+            .apply(new PrimitiveKeyer<Integer>())
+            .apply(ParDo.of(new IdentityFn<Integer>()));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed)));
+  }
+
+  @Test
+  public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() {
+    PCollection<Integer> unkeyed =
+        p.apply(Create.<Integer>of(1, 2, 3)).apply(ParDo.of(new IdentityFn<Integer>()));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
+  }
+
+  @Test
+  public void traverseMultipleTimesThrows() {
+    p.apply(
+            Create.<KV<Integer, Void>>of(
+                    KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
+                .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of())))
+        .apply(GroupByKey.<Integer, Void>create())
+        .apply(Keys.<Integer>create());
+
+    p.traverseTopologically(visitor);
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("already been finalized");
+    thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName());
+    p.traverseTopologically(visitor);
+  }
+
+  @Test
+  public void getKeyedPValuesBeforeTraverseThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getKeyedPValues");
+    visitor.getKeyedPValues();
+  }
+
+  private static class PrimitiveKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
+    @Override
+    public PCollection<K> apply(PCollection<K> input) {
+      return PCollection.<K>createPrimitiveOutputInternal(
+              input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+          .setCoder(input.getCoder());
+    }
+  }
+
+  private static class CompositeKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
+    @Override
+    public PCollection<K> apply(PCollection<K> input) {
+      return input.apply(new PrimitiveKeyer<K>()).apply(ParDo.of(new IdentityFn<K>()));
+    }
+  }
+
+  private static class IdentityFn<K> extends DoFn<K, K> {
+    @Override
+    public void processElement(DoFn<K, K>.ProcessContext c) throws Exception {
+      c.output(c.element());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
new file mode 100644
index 0000000..11ecbff
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MockClock.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A clock that returns a constant value for now which can be set with calls to
+ * {@link #set(Instant)}.
+ *
+ * <p>For uses of the {@link Clock} interface in unit tests.
+ */
+public class MockClock implements Clock {
+
+  private Instant now;
+
+  public static MockClock fromInstant(Instant initial) {
+    return new MockClock(initial);
+  }
+
+  private MockClock(Instant initialNow) {
+    this.now = initialNow;
+  }
+
+  public void set(Instant newNow) {
+    checkArgument(!newNow.isBefore(now), "Cannot move MockClock backwards in time from %s to %s",
+        now, newNow);
+    this.now = newNow;
+  }
+
+  public void advance(Duration duration) {
+    checkArgument(
+        duration.getMillis() > 0,
+        "Cannot move MockClock backwards in time by duration %s",
+        duration);
+    set(now.plus(duration));
+  }
+
+  @Override
+  public Instant now() {
+    return now;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
new file mode 100644
index 0000000..cecfe01
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.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.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import org.hamcrest.Matchers;
+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 java.io.Serializable;
+
+/**
+ * Tests for {@link ParDoMultiEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ParDoMultiEvaluatorFactoryTest implements Serializable {
+  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void testParDoMultiInMemoryTransformEvaluator() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+    final TupleTag<String> elementTag = new TupleTag<>();
+    final TupleTag<Integer> lengthTag = new TupleTag<>();
+
+    BoundMulti<String, KV<String, Integer>> pardo =
+        ParDo.of(
+                new DoFn<String, KV<String, Integer>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(KV.<String, Integer>of(c.element(), c.element().length()));
+                    c.sideOutput(elementTag, c.element());
+                    c.sideOutput(lengthTag, c.element().length());
+                  }
+                })
+            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag).and(lengthTag));
+    PCollectionTuple outputTuple = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+    PCollection<String> elementOutput = outputTuple.get(elementTag);
+    PCollection<Integer> lengthOutput = outputTuple.get(lengthTag);
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+    UncommittedBundle<Integer> lengthOutputBundle = bundleFactory.createRootBundle(lengthOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+    when(evaluationContext.createBundle(inputBundle, elementOutput))
+        .thenReturn(elementOutputBundle);
+    when(evaluationContext.createBundle(inputBundle, lengthOutput)).thenReturn(lengthOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, null, null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoMultiEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getOutputBundles(),
+        Matchers.<UncommittedBundle<?>>containsInAnyOrder(
+            lengthOutputBundle, mainOutputBundle, elementOutputBundle));
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    assertThat(result.getCounters(), equalTo(counters));
+
+    assertThat(
+        mainOutputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
+            WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
+            WindowedValue.valueInGlobalWindow(
+                KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+    assertThat(
+        elementOutputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<String>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow("foo"),
+            WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
+            WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+    assertThat(
+        lengthOutputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<Integer>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(3),
+            WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
+            WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+  }
+
+  @Test
+  public void testParDoMultiUndeclaredSideOutput() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+    final TupleTag<String> elementTag = new TupleTag<>();
+    final TupleTag<Integer> lengthTag = new TupleTag<>();
+
+    BoundMulti<String, KV<String, Integer>> pardo =
+        ParDo.of(
+                new DoFn<String, KV<String, Integer>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(KV.<String, Integer>of(c.element(), c.element().length()));
+                    c.sideOutput(elementTag, c.element());
+                    c.sideOutput(lengthTag, c.element().length());
+                  }
+                })
+            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+    PCollectionTuple outputTuple = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+    PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+    when(evaluationContext.createBundle(inputBundle, elementOutput))
+        .thenReturn(elementOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, null, null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoMultiEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getOutputBundles(),
+        Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    assertThat(result.getCounters(), equalTo(counters));
+
+    assertThat(
+        mainOutputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<KV<String, Integer>>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(KV.of("foo", 3)),
+            WindowedValue.timestampedValueInGlobalWindow(KV.of("bara", 4), new Instant(1000)),
+            WindowedValue.valueInGlobalWindow(
+                KV.of("bazam", 5), PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+    assertThat(
+        elementOutputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<String>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow("foo"),
+            WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)),
+            WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+  }
+
+  @Test
+  public void finishBundleWithStatePutsStateInResult() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+    final TupleTag<String> elementTag = new TupleTag<>();
+
+    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
+        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEndOfWindow());
+    final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
+    final StateNamespace windowNs =
+        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
+    BoundMulti<String, KV<String, Integer>> pardo =
+        ParDo.of(
+                new DoFn<String, KV<String, Integer>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.windowingInternals()
+                        .stateInternals()
+                        .state(StateNamespaces.global(), watermarkTag)
+                        .add(new Instant(20202L + c.element().length()));
+                    c.windowingInternals()
+                        .stateInternals()
+                        .state(
+                            StateNamespaces.window(
+                                GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
+                            bagTag)
+                        .add(c.element());
+                  }
+                })
+            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+    PCollectionTuple outputTuple = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+    PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+    when(evaluationContext.createBundle(inputBundle, elementOutput))
+        .thenReturn(elementOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, "myKey", null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoMultiEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getOutputBundles(),
+        Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
+    assertThat(result.getWatermarkHold(), equalTo(new Instant(20205L)));
+    assertThat(result.getState(), not(nullValue()));
+    assertThat(
+        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
+        equalTo(new Instant(20205L)));
+    assertThat(
+        result.getState().state(windowNs, bagTag).read(),
+        containsInAnyOrder("foo", "bara", "bazam"));
+  }
+
+  @Test
+  public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<KV<String, Integer>>() {};
+    final TupleTag<String> elementTag = new TupleTag<>();
+
+    final TimerData addedTimer =
+        TimerData.of(
+            StateNamespaces.window(
+                IntervalWindow.getCoder(),
+                new IntervalWindow(
+                    new Instant(0).plus(Duration.standardMinutes(5)),
+                    new Instant(1)
+                        .plus(Duration.standardMinutes(5))
+                        .plus(Duration.standardHours(1)))),
+            new Instant(54541L),
+            TimeDomain.EVENT_TIME);
+    final TimerData deletedTimer =
+        TimerData.of(
+            StateNamespaces.window(
+                IntervalWindow.getCoder(),
+                new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
+            new Instant(3400000),
+            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+    BoundMulti<String, KV<String, Integer>> pardo =
+        ParDo.of(
+                new DoFn<String, KV<String, Integer>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.windowingInternals().stateInternals();
+                    c.windowingInternals()
+                        .timerInternals()
+                        .setTimer(
+                            TimerData.of(
+                                StateNamespaces.window(
+                                    IntervalWindow.getCoder(),
+                                    new IntervalWindow(
+                                        new Instant(0).plus(Duration.standardMinutes(5)),
+                                        new Instant(1)
+                                            .plus(Duration.standardMinutes(5))
+                                            .plus(Duration.standardHours(1)))),
+                                new Instant(54541L),
+                                TimeDomain.EVENT_TIME));
+                    c.windowingInternals()
+                        .timerInternals()
+                        .deleteTimer(
+                            TimerData.of(
+                                StateNamespaces.window(
+                                    IntervalWindow.getCoder(),
+                                    new IntervalWindow(
+                                        new Instant(0),
+                                        new Instant(0).plus(Duration.standardHours(1)))),
+                                new Instant(3400000),
+                                TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+                  }
+                })
+            .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
+    PCollectionTuple outputTuple = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
+    PCollection<String> elementOutput = outputTuple.get(elementTag);
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+    UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+    when(evaluationContext.createBundle(inputBundle, elementOutput))
+        .thenReturn(elementOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, "myKey", null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoMultiEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getTimerUpdate(),
+        equalTo(
+            TimerUpdate.builder("myKey")
+                .setTimer(addedTimer)
+                .setTimer(addedTimer)
+                .setTimer(addedTimer)
+                .deletedTimer(deletedTimer)
+                .deletedTimer(deletedTimer)
+                .deletedTimer(deletedTimer)
+                .build()));
+  }
+}


[16/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
Move InProcessRunner to its own module


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

Branch: refs/heads/master
Commit: e13cacb81c9c9718a45bfb7aefd839dcdfd442f6
Parents: bba4c64
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Apr 27 15:01:48 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Apr 29 14:28:45 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |    1 +
 runners/direct-java/pom.xml                     |  400 ++++++
 .../direct/AbstractModelEnforcement.java        |   38 +
 .../direct/AvroIOShardedWriteFactory.java       |   76 +
 .../direct/BoundedReadEvaluatorFactory.java     |  155 ++
 .../beam/runners/direct/BundleFactory.java      |   49 +
 .../CachedThreadPoolExecutorServiceFactory.java |   44 +
 .../org/apache/beam/runners/direct/Clock.java   |   30 +
 .../beam/runners/direct/CommittedResult.java    |   46 +
 .../beam/runners/direct/CompletionCallback.java |   36 +
 .../direct/ConsumerTrackingPipelineVisitor.java |  173 +++
 .../runners/direct/EmptyTransformEvaluator.java |   50 +
 .../direct/EncodabilityEnforcementFactory.java  |   70 +
 .../beam/runners/direct/EvaluatorKey.java       |   55 +
 .../runners/direct/ExecutorServiceFactory.java  |   33 +
 .../direct/ExecutorServiceParallelExecutor.java |  478 +++++++
 .../runners/direct/FlattenEvaluatorFactory.java |   85 ++
 .../runners/direct/ForwardingPTransform.java    |   62 +
 .../direct/GroupByKeyEvaluatorFactory.java      |  274 ++++
 .../ImmutabilityCheckingBundleFactory.java      |  131 ++
 .../direct/ImmutabilityEnforcementFactory.java  |  103 ++
 .../direct/InMemoryWatermarkManager.java        | 1327 ++++++++++++++++++
 .../runners/direct/InProcessBundleFactory.java  |  162 +++
 .../direct/InProcessBundleOutputManager.java    |   51 +
 .../direct/InProcessEvaluationContext.java      |  425 ++++++
 .../direct/InProcessExecutionContext.java       |  105 ++
 .../beam/runners/direct/InProcessExecutor.java  |   48 +
 .../direct/InProcessPipelineOptions.java        |  101 ++
 .../runners/direct/InProcessPipelineRunner.java |  370 +++++
 .../beam/runners/direct/InProcessRegistrar.java |   55 +
 .../direct/InProcessSideInputContainer.java     |  271 ++++
 .../runners/direct/InProcessTimerInternals.java |   84 ++
 .../direct/InProcessTransformResult.java        |   77 +
 .../direct/KeyedPValueTrackingVisitor.java      |   96 ++
 .../beam/runners/direct/ModelEnforcement.java   |   63 +
 .../runners/direct/ModelEnforcementFactory.java |   30 +
 .../beam/runners/direct/NanosOffsetClock.java   |   59 +
 .../direct/PTransformOverrideFactory.java       |   33 +
 .../runners/direct/ParDoInProcessEvaluator.java |  173 +++
 .../direct/ParDoMultiEvaluatorFactory.java      |   64 +
 .../direct/ParDoSingleEvaluatorFactory.java     |   63 +
 .../direct/PassthroughTransformEvaluator.java   |   49 +
 .../runners/direct/ShardControlledWrite.java    |   81 ++
 .../apache/beam/runners/direct/StepAndKey.java  |   71 +
 .../runners/direct/StepTransformResult.java     |  165 +++
 .../direct/TextIOShardedWriteFactory.java       |   78 +
 .../beam/runners/direct/TransformEvaluator.java |   46 +
 .../direct/TransformEvaluatorFactory.java       |   44 +
 .../direct/TransformEvaluatorRegistry.java      |   77 +
 .../beam/runners/direct/TransformExecutor.java  |  176 +++
 .../direct/TransformExecutorService.java        |   35 +
 .../direct/TransformExecutorServices.java       |  154 ++
 .../direct/UnboundedReadEvaluatorFactory.java   |  177 +++
 .../runners/direct/ViewEvaluatorFactory.java    |  145 ++
 .../direct/WatermarkCallbackExecutor.java       |  146 ++
 .../runners/direct/WindowEvaluatorFactory.java  |  131 ++
 .../direct/AvroIOShardedWriteFactoryTest.java   |  112 ++
 .../direct/BoundedReadEvaluatorFactoryTest.java |  290 ++++
 .../runners/direct/CommittedResultTest.java     |   77 +
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ++++
 .../EncodabilityEnforcementFactoryTest.java     |  257 ++++
 .../direct/FlattenEvaluatorFactoryTest.java     |  141 ++
 .../direct/ForwardingPTransformTest.java        |  112 ++
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  183 +++
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 +++
 .../ImmutabilityEnforcementFactoryTest.java     |  128 ++
 .../direct/InMemoryWatermarkManagerTest.java    | 1168 +++++++++++++++
 .../direct/InProcessBundleFactoryTest.java      |  223 +++
 .../direct/InProcessEvaluationContextTest.java  |  526 +++++++
 .../direct/InProcessPipelineRegistrarTest.java  |   74 +
 .../direct/InProcessPipelineRunnerTest.java     |   78 +
 .../direct/InProcessSideInputContainerTest.java |  496 +++++++
 .../direct/InProcessTimerInternalsTest.java     |  133 ++
 .../direct/KeyedPValueTrackingVisitorTest.java  |  192 +++
 .../apache/beam/runners/direct/MockClock.java   |   62 +
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  431 ++++++
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  324 +++++
 .../direct/TextIOShardedWriteFactoryTest.java   |  112 ++
 .../direct/TransformExecutorServicesTest.java   |  136 ++
 .../runners/direct/TransformExecutorTest.java   |  538 +++++++
 .../UnboundedReadEvaluatorFactoryTest.java      |  334 +++++
 .../direct/ViewEvaluatorFactoryTest.java        |  101 ++
 .../direct/WatermarkCallbackExecutorTest.java   |  128 ++
 .../direct/WindowEvaluatorFactoryTest.java      |  222 +++
 runners/pom.xml                                 |    1 +
 .../inprocess/AbstractModelEnforcement.java     |   38 -
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 -
 .../inprocess/BoundedReadEvaluatorFactory.java  |  155 --
 .../sdk/runners/inprocess/BundleFactory.java    |   49 -
 .../CachedThreadPoolExecutorServiceFactory.java |   44 -
 .../beam/sdk/runners/inprocess/Clock.java       |   30 -
 .../sdk/runners/inprocess/CommittedResult.java  |   46 -
 .../runners/inprocess/CompletionCallback.java   |   36 -
 .../ConsumerTrackingPipelineVisitor.java        |  173 ---
 .../inprocess/EmptyTransformEvaluator.java      |   50 -
 .../EncodabilityEnforcementFactory.java         |   70 -
 .../sdk/runners/inprocess/EvaluatorKey.java     |   55 -
 .../inprocess/ExecutorServiceFactory.java       |   33 -
 .../ExecutorServiceParallelExecutor.java        |  478 -------
 .../inprocess/FlattenEvaluatorFactory.java      |   85 --
 .../runners/inprocess/ForwardingPTransform.java |   62 -
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  274 ----
 .../ImmutabilityCheckingBundleFactory.java      |  131 --
 .../ImmutabilityEnforcementFactory.java         |  103 --
 .../inprocess/InMemoryWatermarkManager.java     | 1327 ------------------
 .../inprocess/InProcessBundleFactory.java       |  162 ---
 .../inprocess/InProcessBundleOutputManager.java |   51 -
 .../inprocess/InProcessEvaluationContext.java   |  425 ------
 .../inprocess/InProcessExecutionContext.java    |  105 --
 .../runners/inprocess/InProcessExecutor.java    |   48 -
 .../inprocess/InProcessPipelineOptions.java     |  101 --
 .../inprocess/InProcessPipelineRunner.java      |  370 -----
 .../runners/inprocess/InProcessRegistrar.java   |   55 -
 .../inprocess/InProcessSideInputContainer.java  |  271 ----
 .../inprocess/InProcessTimerInternals.java      |   84 --
 .../inprocess/InProcessTransformResult.java     |   77 -
 .../inprocess/KeyedPValueTrackingVisitor.java   |   96 --
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 -
 .../inprocess/ModelEnforcementFactory.java      |   30 -
 .../sdk/runners/inprocess/NanosOffsetClock.java |   59 -
 .../inprocess/PTransformOverrideFactory.java    |   33 -
 .../inprocess/ParDoInProcessEvaluator.java      |  173 ---
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   63 -
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   63 -
 .../PassthroughTransformEvaluator.java          |   49 -
 .../runners/inprocess/ShardControlledWrite.java |   81 --
 .../beam/sdk/runners/inprocess/StepAndKey.java  |   71 -
 .../runners/inprocess/StepTransformResult.java  |  165 ---
 .../inprocess/TextIOShardedWriteFactory.java    |   78 -
 .../runners/inprocess/TransformEvaluator.java   |   46 -
 .../inprocess/TransformEvaluatorFactory.java    |   44 -
 .../inprocess/TransformEvaluatorRegistry.java   |   77 -
 .../runners/inprocess/TransformExecutor.java    |  176 ---
 .../inprocess/TransformExecutorService.java     |   35 -
 .../inprocess/TransformExecutorServices.java    |  154 --
 .../UnboundedReadEvaluatorFactory.java          |  177 ---
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 --
 .../inprocess/WatermarkCallbackExecutor.java    |  146 --
 .../inprocess/WindowEvaluatorFactory.java       |  131 --
 .../AvroIOShardedWriteFactoryTest.java          |  112 --
 .../BoundedReadEvaluatorFactoryTest.java        |  290 ----
 .../runners/inprocess/CommittedResultTest.java  |   77 -
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ----
 .../EncodabilityEnforcementFactoryTest.java     |  257 ----
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 --
 .../inprocess/ForwardingPTransformTest.java     |  112 --
 .../GroupByKeyEvaluatorFactoryTest.java         |  183 ---
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 ---
 .../ImmutabilityEnforcementFactoryTest.java     |  128 --
 .../inprocess/InMemoryWatermarkManagerTest.java | 1168 ---------------
 .../inprocess/InProcessBundleFactoryTest.java   |  223 ---
 .../InProcessEvaluationContextTest.java         |  526 -------
 .../InProcessPipelineRegistrarTest.java         |   74 -
 .../inprocess/InProcessPipelineRunnerTest.java  |   78 -
 .../InProcessSideInputContainerTest.java        |  496 -------
 .../inprocess/InProcessTimerInternalsTest.java  |  133 --
 .../KeyedPValueTrackingVisitorTest.java         |  192 ---
 .../beam/sdk/runners/inprocess/MockClock.java   |   62 -
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 ------
 .../ParDoSingleEvaluatorFactoryTest.java        |  324 -----
 .../TextIOShardedWriteFactoryTest.java          |  112 --
 .../TransformExecutorServicesTest.java          |  136 --
 .../inprocess/TransformExecutorTest.java        |  538 -------
 .../UnboundedReadEvaluatorFactoryTest.java      |  334 -----
 .../inprocess/ViewEvaluatorFactoryTest.java     |  101 --
 .../WatermarkCallbackExecutorTest.java          |  128 --
 .../inprocess/WindowEvaluatorFactoryTest.java   |  222 ---
 sdks/java/pom.xml                               |    1 -
 168 files changed, 14692 insertions(+), 14290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index f1d9e3b..8aad36e 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -25,6 +25,7 @@ matrix:
       env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DforkCount=0"
 
 before_install:
+  - echo "MAVEN_OPTS='-Xmx2048m -XX:MaxPermSize=512m'" > ~/.mavenrc
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
   - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
new file mode 100644
index 0000000..12ba329
--- /dev/null
+++ b/runners/direct-java/pom.xml
@@ -0,0 +1,400 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>runners-parent</artifactId>
+    <version>0.1.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>direct-runner</artifactId>
+
+  <name>Apache Beam :: Runners :: Direct</name>
+
+  <packaging>jar</packaging>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals><goal>analyze-only</goal></goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>default-jar</id>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>default-test-jar</id>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Source plugin for generating source and test-source JARs. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>jar-no-fork</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>attach-test-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>test-jar-no-fork</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <executions>
+          <!-- For now, disables integration tests from the SDK as the runner is not ready. -->
+          <execution>
+            <id>runnable-on-service-tests</id>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+        <configuration>
+          <windowtitle>Apache Beam Direct Runner ${project.version}</windowtitle>
+          <doctitle>Apache Beam Direct Runner, version ${project.version}</doctitle>
+
+          <subpackages>org.apache.beam.runners.direct</subpackages>
+          <use>false</use>
+          <quiet>true</quiet>
+          <bottom><![CDATA[<br>]]></bottom>
+
+          <offlineLinks>
+            <offlineLink>
+              <url>https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/</url>
+              <location>${basedir}/../../sdks/java/javadoc/apiclient-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://avro.apache.org/docs/1.7.7/api/java/</url>
+              <location>${basedir}/../../sdks/java/javadoc/avro-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/</url>
+              <location>${basedir}/../../sdks/java/javadoc/bq-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>https://cloud.google.com/datastore/docs/apis/javadoc/</url>
+              <location>${basedir}/../../sdks/java/javadoc/datastore-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/</url>
+              <location>${basedir}/../../sdks/java/javadoc/guava-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://hamcrest.org/JavaHamcrest/javadoc/1.3/</url>
+              <location>${basedir}/../../sdks/java/javadoc/hamcrest-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://fasterxml.github.io/jackson-annotations/javadoc/2.7/</url>
+              <location>${basedir}/../../sdks/java/javadoc/jackson-annotations-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://fasterxml.github.io/jackson-databind/javadoc/2.7/</url>
+              <location>${basedir}/../../sdks/java/javadoc/jackson-databind-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://www.joda.org/joda-time/apidocs</url>
+              <location>${basedir}/../../sdks/java/javadoc/joda-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>http://junit.sourceforge.net/javadoc/</url>
+              <location>${basedir}/../../sdks/java/javadoc/junit-docs</location>
+            </offlineLink>
+            <offlineLink>
+              <url>https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/</url>
+              <location>${basedir}/../../sdks/java/javadoc/oauth-docs</location>
+            </offlineLink>
+          </offlineLinks>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+            <phase>package</phase>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.4.1</version>
+        <executions>
+          <!-- In the first phase, we pick dependencies and relocate them. -->
+          <execution>
+            <id>bundle-and-repackage</id>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <shadeTestJar>true</shadeTestJar>
+              <artifactSet>
+                <includes>
+                  <include>com.google.guava:guava</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+              <relocations>
+                <!-- TODO: Once ready, change the following pattern to 'com'
+                     only, exclude 'org.apache.beam.**', and remove
+                     the second relocation. -->
+                <relocation>
+                  <pattern>com.google.common</pattern>
+                  <shadedPattern>org.apache.beam.runners.direct.repackaged.com.google.common</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+
+          <!-- In the second phase, we pick remaining dependencies and bundle
+               them without repackaging. -->
+          <execution>
+            <id>bundle-rest-without-repackaging</id>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <shadeTestJar>true</shadeTestJar>
+              <finalName>${project.artifactId}-bundled-${project.version}</finalName>
+              <artifactSet>
+                <excludes>
+                  <exclude>com.google.guava:guava</exclude>
+                </excludes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Coverage analysis for unit tests. -->
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>java-sdk-all</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <version>${google-clients.version}</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client-protobuf</artifactId>
+      <version>${google-clients.version}</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <!-- If updating version, please update the javadoc offlineLink -->
+      <version>${guava.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava-testlib</artifactId>
+      <version>${guava.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <version>${joda.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <version>${jsr305.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+
+    <!-- build dependencies -->
+    <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <version>1.0-rc2</version>
+      <optional>true</optional>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <version>1.1</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <version>${hamcrest.version}</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>java-sdk-all</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
new file mode 100644
index 0000000..948beb6
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.
+ */
+abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
+  @Override
+  public void beforeElement(WindowedValue<T> element) {}
+
+  @Override
+  public void afterElement(WindowedValue<T> element) {}
+
+  @Override
+  public void afterFinish(
+      CommittedBundle<T> input,
+      InProcessTransformResult result,
+      Iterable<? extends CommittedBundle<?>> outputs) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
new file mode 100644
index 0000000..7422f27
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+class AvroIOShardedWriteFactory implements PTransformOverrideFactory {
+  @Override
+  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+      PTransform<InputT, OutputT> transform) {
+    if (transform instanceof AvroIO.Write.Bound) {
+      @SuppressWarnings("unchecked")
+      AvroIO.Write.Bound<InputT> originalWrite = (AvroIO.Write.Bound<InputT>) transform;
+      if (originalWrite.getNumShards() > 1
+          || (originalWrite.getNumShards() == 1
+              && !"".equals(originalWrite.getShardNameTemplate()))) {
+        @SuppressWarnings("unchecked")
+        PTransform<InputT, OutputT> override =
+            (PTransform<InputT, OutputT>) new AvroIOShardedWrite<InputT>(originalWrite);
+        return override;
+      }
+    }
+    return transform;
+  }
+
+  private class AvroIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
+    private final AvroIO.Write.Bound<InputT> initial;
+
+    private AvroIOShardedWrite(AvroIO.Write.Bound<InputT> initial) {
+      this.initial = initial;
+    }
+
+    @Override
+    int getNumShards() {
+      return initial.getNumShards();
+    }
+
+    @Override
+    PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
+      String shardName =
+          IOChannelUtils.constructName(
+              initial.getFilenamePrefix(),
+              initial.getShardNameTemplate(),
+              initial.getFilenameSuffix(),
+              shardNum,
+              getNumShards());
+      return initial.withoutSharding().to(shardName).withSuffix("");
+    }
+
+    @Override
+    protected PTransform<PCollection<InputT>, PDone> delegate() {
+      return initial;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
new file mode 100644
index 0000000..3822d3b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.io.IOException;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
+ * for the {@link Bounded Read.Bounded} primitive {@link PTransform}.
+ */
+final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
+  /*
+   * An evaluator for a Source is stateful, to ensure data is not read multiple times.
+   * Evaluators are cached here to ensure that the reader is not restarted if the evaluator is
+   * retriggered.
+   */
+  private final ConcurrentMap<EvaluatorKey, Queue<? extends BoundedReadEvaluator<?>>>
+      sourceEvaluators = new ConcurrentHashMap<>();
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      @Nullable CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext)
+      throws IOException {
+    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
+  }
+
+  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
+      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+      final InProcessEvaluationContext evaluationContext) {
+    BoundedReadEvaluator<?> evaluator =
+        getTransformEvaluatorQueue(transform, evaluationContext).poll();
+    if (evaluator == null) {
+      return EmptyTransformEvaluator.create(transform);
+    }
+    return evaluator;
+  }
+
+  /**
+   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
+   * provided application of {@link Bounded Read.Bounded}, initializing it if required.
+   *
+   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
+   * already done so.
+   */
+  @SuppressWarnings("unchecked")
+  private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
+      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+      final InProcessEvaluationContext evaluationContext) {
+    // Key by the application and the context the evaluation is occurring in (which call to
+    // Pipeline#run).
+    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
+    Queue<BoundedReadEvaluator<OutputT>> evaluatorQueue =
+        (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+    if (evaluatorQueue == null) {
+      evaluatorQueue = new ConcurrentLinkedQueue<>();
+      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
+        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
+        // factory for this transform
+        BoundedSource<OutputT> source = transform.getTransform().getSource();
+        BoundedReadEvaluator<OutputT> evaluator =
+            new BoundedReadEvaluator<OutputT>(transform, evaluationContext, source);
+        evaluatorQueue.offer(evaluator);
+      } else {
+        // otherwise return the existing Queue that arrived before us
+        evaluatorQueue = (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
+      }
+    }
+    return evaluatorQueue;
+  }
+
+  /**
+   * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource},
+   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
+   * creates the {@link BoundedReader} and consumes all available input.
+   *
+   * <p>A {@link BoundedReadEvaluator} should only be created once per {@link BoundedSource}, and
+   * each evaluator should only be called once per evaluation of the pipeline. Otherwise, the source
+   * may produce duplicate elements.
+   */
+  private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
+    private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
+    private final InProcessEvaluationContext evaluationContext;
+    /**
+     * The source being read from by this {@link BoundedReadEvaluator}. This may not be the same
+     * as the source derived from {@link #transform} due to splitting.
+     */
+    private BoundedSource<OutputT> source;
+
+    public BoundedReadEvaluator(
+        AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
+        InProcessEvaluationContext evaluationContext,
+        BoundedSource<OutputT> source) {
+      this.transform = transform;
+      this.evaluationContext = evaluationContext;
+      this.source = source;
+    }
+
+    @Override
+    public void processElement(WindowedValue<Object> element) {}
+
+    @Override
+    public InProcessTransformResult finishBundle() throws IOException {
+      try (final BoundedReader<OutputT> reader =
+              source.createReader(evaluationContext.getPipelineOptions());) {
+        boolean contentsRemaining = reader.start();
+        UncommittedBundle<OutputT> output =
+            evaluationContext.createRootBundle(transform.getOutput());
+        while (contentsRemaining) {
+          output.add(
+              WindowedValue.timestampedValueInGlobalWindow(
+                  reader.getCurrent(), reader.getCurrentTimestamp()));
+          contentsRemaining = reader.advance();
+        }
+        return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MAX_VALUE)
+            .addOutput(output)
+            .build();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
new file mode 100644
index 0000000..34529e7
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BundleFactory.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * A factory that creates {@link UncommittedBundle UncommittedBundles}.
+ */
+public interface BundleFactory {
+  /**
+   * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle belong to
+   * the {@code output} {@link PCollection}.
+   */
+  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output);
+
+  /**
+   * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
+   * belong to the {@code output} {@link PCollection}.
+   */
+  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output);
+
+  /**
+   * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
+   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
+   * belong to the {@code output} {@link PCollection}.
+   */
+  public <T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, Object key, PCollection<T> output);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.java
new file mode 100644
index 0000000..5b8e5fc
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CachedThreadPoolExecutorServiceFactory.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.direct;
+
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link ExecutorServiceFactory} that produces cached thread pools via
+ * {@link Executors#newCachedThreadPool()}.
+ */
+class CachedThreadPoolExecutorServiceFactory
+    implements DefaultValueFactory<ExecutorServiceFactory>, ExecutorServiceFactory {
+  private static final CachedThreadPoolExecutorServiceFactory INSTANCE =
+      new CachedThreadPoolExecutorServiceFactory();
+
+  @Override
+  public ExecutorServiceFactory create(PipelineOptions options) {
+    return INSTANCE;
+  }
+
+  @Override
+  public ExecutorService create() {
+    return Executors.newCachedThreadPool();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
new file mode 100644
index 0000000..88f8aab
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/Clock.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.joda.time.Instant;
+
+/**
+ * Access to the current time.
+ */
+public interface Clock {
+  /**
+   * Returns the current time as an {@link Instant}.
+   */
+  Instant now();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
new file mode 100644
index 0000000..d15e012
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import com.google.auto.value.AutoValue;
+
+/**
+ * A {@link InProcessTransformResult} that has been committed.
+ */
+@AutoValue
+abstract class CommittedResult {
+  /**
+   * Returns the {@link AppliedPTransform} that produced this result.
+   */
+  public abstract AppliedPTransform<?, ?, ?> getTransform();
+
+  /**
+   * Returns the outputs produced by the transform.
+   */
+  public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
+
+  public static CommittedResult create(
+      InProcessTransformResult original, Iterable<? extends CommittedBundle<?>> outputs) {
+    return new AutoValue_CommittedResult(original.getTransform(),
+        outputs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
new file mode 100644
index 0000000..7c2c068
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+
+/**
+ * A callback for completing a bundle of input.
+ */
+interface CompletionCallback {
+  /**
+   * Handle a successful result, returning the committed outputs of the result.
+   */
+  CommittedResult handleResult(
+      CommittedBundle<?> inputBundle, InProcessTransformResult result);
+
+  /**
+   * Handle a result that terminated abnormally due to the provided {@link Throwable}.
+   */
+  void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
new file mode 100644
index 0000000..c790463
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
+ * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
+ * input after the upstream transform has produced and committed output.
+ */
+public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
+  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
+  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
+  private Collection<PCollectionView<?>> views = new ArrayList<>();
+  private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
+  private Set<PValue> toFinalize = new HashSet<>();
+  private int numTransforms = 0;
+  private boolean finalized = false;
+
+  @Override
+  public void enterCompositeTransform(TransformTreeNode node) {
+    checkState(
+        !finalized,
+        "Attempting to traverse a pipeline (node %s) with a %s "
+            + "which has already visited a Pipeline and is finalized",
+        node.getFullName(),
+        ConsumerTrackingPipelineVisitor.class.getSimpleName());
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformTreeNode node) {
+    checkState(
+        !finalized,
+        "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
+        node.getFullName(),
+        ConsumerTrackingPipelineVisitor.class.getSimpleName());
+    if (node.isRootNode()) {
+      finalized = true;
+    }
+  }
+
+  @Override
+  public void visitTransform(TransformTreeNode node) {
+    toFinalize.removeAll(node.getInput().expand());
+    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
+    stepNames.put(appliedTransform, genStepName());
+    if (node.getInput().expand().isEmpty()) {
+      rootTransforms.add(appliedTransform);
+    } else {
+      for (PValue value : node.getInput().expand()) {
+        valueToConsumers.get(value).add(appliedTransform);
+      }
+    }
+  }
+
+  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
+        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
+    return application;
+  }
+
+  @Override
+  public void visitValue(PValue value, TransformTreeNode producer) {
+    toFinalize.add(value);
+    for (PValue expandedValue : value.expand()) {
+      valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
+      if (expandedValue instanceof PCollectionView) {
+        views.add((PCollectionView<?>) expandedValue);
+      }
+      expandedValue.recordAsOutput(getAppliedTransform(producer));
+    }
+    value.recordAsOutput(getAppliedTransform(producer));
+  }
+
+  private String genStepName() {
+    return String.format("s%s", numTransforms++);
+  }
+
+
+  /**
+   * Returns a mapping of each fully-expanded {@link PValue} to each
+   * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
+   * returned from {@code getValueToCustomers().get(PValue)},
+   * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
+   */
+  public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
+    checkState(
+        finalized,
+        "Can't call getValueToConsumers before the Pipeline has been completely traversed");
+
+    return valueToConsumers;
+  }
+
+  /**
+   * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
+   * name.
+   */
+  public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
+    checkState(
+        finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
+
+    return stepNames;
+  }
+
+  /**
+   * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
+   * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
+   */
+  public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
+    checkState(
+        finalized,
+        "Can't call getRootTransforms before the Pipeline has been completely traversed");
+
+    return rootTransforms;
+  }
+
+  /**
+   * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
+   * {@link Pipeline}.
+   */
+  public Collection<PCollectionView<?>> getViews() {
+    checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
+
+    return views;
+  }
+
+  /**
+   * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
+   * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
+   * {@link Pipeline} is executed.
+   */
+  public Set<PValue> getUnfinalizedPValues() {
+    checkState(
+        finalized,
+        "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
+
+    return toFinalize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java
new file mode 100644
index 0000000..5379038
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.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.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * A {@link TransformEvaluator} that ignores all input and produces no output. The result of
+ * invoking {@link #finishBundle()} on this evaluator is to return an
+ * {@link InProcessTransformResult} with no elements and a timestamp hold equal to
+ * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}. Because the result contains no elements, this hold
+ * will not affect the watermark.
+ */
+final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
+  public static <T> TransformEvaluator<T> create(AppliedPTransform<?, ?, ?> transform) {
+    return new EmptyTransformEvaluator<T>(transform);
+  }
+
+  private final AppliedPTransform<?, ?, ?> transform;
+
+  private EmptyTransformEvaluator(AppliedPTransform<?, ?, ?> transform) {
+    this.transform = transform;
+  }
+
+  @Override
+  public void processElement(WindowedValue<T> element) throws Exception {}
+
+  @Override
+  public InProcessTransformResult finishBundle() throws Exception {
+    return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.java
new file mode 100644
index 0000000..ccf4c2b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactory.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.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Enforces that all elements in a {@link PCollection} can be encoded using that
+ * {@link PCollection PCollection's} {@link Coder}.
+ */
+class EncodabilityEnforcementFactory implements ModelEnforcementFactory {
+  public static EncodabilityEnforcementFactory create() {
+    return new EncodabilityEnforcementFactory();
+  }
+
+  @Override
+  public <T> ModelEnforcement<T> forBundle(
+      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+    return new EncodabilityEnforcement<>(input);
+  }
+
+  private static class EncodabilityEnforcement<T> extends AbstractModelEnforcement<T> {
+    private Coder<T> coder;
+
+    public EncodabilityEnforcement(CommittedBundle<T> input) {
+      coder = input.getPCollection().getCoder();
+    }
+
+    @Override
+    public void beforeElement(WindowedValue<T> element) {
+      try {
+        T clone = CoderUtils.clone(coder, element.getValue());
+        if (coder.consistentWithEquals()) {
+          checkArgument(
+              coder.structuralValue(element.getValue()).equals(coder.structuralValue(clone)),
+              "Coder %s of class %s does not maintain structural value equality"
+                  + " on input element %s",
+              coder,
+              coder.getClass().getSimpleName(),
+              element.getValue());
+        }
+      } catch (Exception e) {
+        throw UserCodeException.wrap(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.java
new file mode 100644
index 0000000..1c36751
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluatorKey.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.direct;
+
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+
+import java.util.Objects;
+
+/**
+ * A (Transform, Pipeline Execution) key for stateful evaluators.
+ *
+ * Source evaluators are stateful to ensure data is not read multiple times. Evaluators are cached
+ * to ensure that the reader is not restarted if the evaluator is retriggered. An
+ * {@link EvaluatorKey} is used to ensure that multiple Pipelines can be executed without sharing
+ * the same evaluators.
+ */
+final class EvaluatorKey {
+  private final AppliedPTransform<?, ?, ?> transform;
+  private final InProcessEvaluationContext context;
+
+  public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
+    this.transform = transform;
+    this.context = context;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(transform, context);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null || !(other instanceof EvaluatorKey)) {
+      return false;
+    }
+    EvaluatorKey that = (EvaluatorKey) other;
+    return Objects.equals(this.transform, that.transform)
+        && Objects.equals(this.context, that.context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
new file mode 100644
index 0000000..91dc258
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory that creates {@link ExecutorService ExecutorServices}.
+ * {@link ExecutorService ExecutorServices} created by this factory should be independent of one
+ * another (e.g., if any executor is shut down the remaining executors should continue to process
+ * work).
+ */
+public interface ExecutorServiceFactory {
+  /**
+   * Create a new {@link ExecutorService}.
+   */
+  ExecutorService create();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
new file mode 100644
index 0000000..18af363
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -0,0 +1,478 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Optional;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+
+import javax.annotation.Nullable;
+
+/**
+ * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
+ * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
+ */
+final class ExecutorServiceParallelExecutor implements InProcessExecutor {
+  private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
+
+  private final ExecutorService executorService;
+
+  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
+  private final Set<PValue> keyedPValues;
+  private final TransformEvaluatorRegistry registry;
+  @SuppressWarnings("rawtypes")
+  private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+      transformEnforcements;
+
+  private final InProcessEvaluationContext evaluationContext;
+
+  private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
+  private final ConcurrentMap<TransformExecutor<?>, Boolean> scheduledExecutors;
+
+  private final Queue<ExecutorUpdate> allUpdates;
+  private final BlockingQueue<VisibleExecutorUpdate> visibleUpdates;
+
+  private final TransformExecutorService parallelExecutorService;
+  private final CompletionCallback defaultCompletionCallback;
+
+  private Collection<AppliedPTransform<?, ?, ?>> rootNodes;
+
+  public static ExecutorServiceParallelExecutor create(
+      ExecutorService executorService,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      Set<PValue> keyedPValues,
+      TransformEvaluatorRegistry registry,
+      @SuppressWarnings("rawtypes")
+      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
+      InProcessEvaluationContext context) {
+    return new ExecutorServiceParallelExecutor(
+        executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
+  }
+
+  private ExecutorServiceParallelExecutor(
+      ExecutorService executorService,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      Set<PValue> keyedPValues,
+      TransformEvaluatorRegistry registry,
+      @SuppressWarnings("rawtypes")
+      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
+      InProcessEvaluationContext context) {
+    this.executorService = executorService;
+    this.valueToConsumers = valueToConsumers;
+    this.keyedPValues = keyedPValues;
+    this.registry = registry;
+    this.transformEnforcements = transformEnforcements;
+    this.evaluationContext = context;
+
+    scheduledExecutors = new ConcurrentHashMap<>();
+    // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
+    // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
+    // which stops the TransformExecutorServices from being prematurely garbage collected
+    executorServices =
+        CacheBuilder.newBuilder().weakValues().build(serialTransformExecutorServiceCacheLoader());
+
+    this.allUpdates = new ConcurrentLinkedQueue<>();
+    this.visibleUpdates = new ArrayBlockingQueue<>(20);
+
+    parallelExecutorService =
+        TransformExecutorServices.parallel(executorService, scheduledExecutors);
+    defaultCompletionCallback = new DefaultCompletionCallback();
+  }
+
+  private CacheLoader<StepAndKey, TransformExecutorService>
+      serialTransformExecutorServiceCacheLoader() {
+    return new CacheLoader<StepAndKey, TransformExecutorService>() {
+      @Override
+      public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
+        return TransformExecutorServices.serial(executorService, scheduledExecutors);
+      }
+    };
+  }
+
+  @Override
+  public void start(Collection<AppliedPTransform<?, ?, ?>> roots) {
+    rootNodes = ImmutableList.copyOf(roots);
+    Runnable monitorRunnable = new MonitorRunnable();
+    executorService.submit(monitorRunnable);
+  }
+
+  @SuppressWarnings("unchecked")
+  public void scheduleConsumption(
+      AppliedPTransform<?, ?, ?> consumer,
+      @Nullable CommittedBundle<?> bundle,
+      CompletionCallback onComplete) {
+    evaluateBundle(consumer, bundle, onComplete);
+  }
+
+  private <T> void evaluateBundle(
+      final AppliedPTransform<?, ?, ?> transform,
+      @Nullable final CommittedBundle<T> bundle,
+      final CompletionCallback onComplete) {
+    TransformExecutorService transformExecutor;
+
+    if (bundle != null && isKeyed(bundle.getPCollection())) {
+      final StepAndKey stepAndKey =
+          StepAndKey.of(transform, bundle == null ? null : bundle.getKey());
+      // This executor will remain reachable until it has executed all scheduled transforms.
+      // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
+      // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
+      // (scheduled due to the completion of another TransformExecutor) are provided to the
+      // ExecutorService before the Earlier TransformExecutor callable completes.
+      transformExecutor = executorServices.getUnchecked(stepAndKey);
+    } else {
+      transformExecutor = parallelExecutorService;
+    }
+
+    Collection<ModelEnforcementFactory> enforcements =
+        MoreObjects.firstNonNull(
+            transformEnforcements.get(transform.getTransform().getClass()),
+            Collections.<ModelEnforcementFactory>emptyList());
+
+    TransformExecutor<T> callable =
+        TransformExecutor.create(
+            registry,
+            enforcements,
+            evaluationContext,
+            bundle,
+            transform,
+            onComplete,
+            transformExecutor);
+    transformExecutor.schedule(callable);
+  }
+
+  private boolean isKeyed(PValue pvalue) {
+    return keyedPValues.contains(pvalue);
+  }
+
+  private void scheduleConsumers(CommittedBundle<?> bundle) {
+    for (AppliedPTransform<?, ?, ?> consumer : valueToConsumers.get(bundle.getPCollection())) {
+      scheduleConsumption(consumer, bundle, defaultCompletionCallback);
+    }
+  }
+
+  @Override
+  public void awaitCompletion() throws Throwable {
+    VisibleExecutorUpdate update;
+    do {
+      update = visibleUpdates.take();
+      if (update.throwable.isPresent()) {
+        throw update.throwable.get();
+      }
+    } while (!update.isDone());
+    executorService.shutdown();
+  }
+
+  /**
+   * The default {@link CompletionCallback}. The default completion callback is used to complete
+   * transform evaluations that are triggered due to the arrival of elements from an upstream
+   * transform, or for a source transform.
+   */
+  private class DefaultCompletionCallback implements CompletionCallback {
+    @Override
+    public CommittedResult handleResult(
+        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+      CommittedResult committedResult =
+          evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
+      for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
+        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
+      }
+      return committedResult;
+    }
+
+    @Override
+    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
+    }
+  }
+
+  /**
+   * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
+   * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
+   * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
+   * as part of the result.
+   */
+  private class TimerCompletionCallback implements CompletionCallback {
+    private final Iterable<TimerData> timers;
+
+    private TimerCompletionCallback(Iterable<TimerData> timers) {
+      this.timers = timers;
+    }
+
+    @Override
+    public CommittedResult handleResult(
+        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+      CommittedResult committedResult =
+          evaluationContext.handleResult(inputBundle, timers, result);
+      for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
+        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
+      }
+      return committedResult;
+    }
+
+    @Override
+    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
+    }
+  }
+
+  /**
+   * An internal status update on the state of the executor.
+   *
+   * Used to signal when the executor should be shut down (due to an exception).
+   */
+  private static class ExecutorUpdate {
+    private final Optional<? extends CommittedBundle<?>> bundle;
+    private final Optional<? extends Throwable> throwable;
+
+    public static ExecutorUpdate fromBundle(CommittedBundle<?> bundle) {
+      return new ExecutorUpdate(bundle, null);
+    }
+
+    public static ExecutorUpdate fromThrowable(Throwable t) {
+      return new ExecutorUpdate(null, t);
+    }
+
+    private ExecutorUpdate(CommittedBundle<?> producedBundle, Throwable throwable) {
+      this.bundle = Optional.fromNullable(producedBundle);
+      this.throwable = Optional.fromNullable(throwable);
+    }
+
+    public Optional<? extends CommittedBundle<?>> getBundle() {
+      return bundle;
+    }
+
+    public Optional<? extends Throwable> getException() {
+      return throwable;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(ExecutorUpdate.class)
+          .add("bundle", bundle)
+          .add("exception", throwable)
+          .toString();
+    }
+  }
+
+  /**
+   * An update of interest to the user. Used in {@link #awaitCompletion} to decide whether to
+   * return normally or throw an exception.
+   */
+  private static class VisibleExecutorUpdate {
+    private final Optional<? extends Throwable> throwable;
+    private final boolean done;
+
+    public static VisibleExecutorUpdate fromThrowable(Throwable e) {
+      return new VisibleExecutorUpdate(false, e);
+    }
+
+    public static VisibleExecutorUpdate finished() {
+      return new VisibleExecutorUpdate(true, null);
+    }
+
+    private VisibleExecutorUpdate(boolean done, @Nullable Throwable exception) {
+      this.throwable = Optional.fromNullable(exception);
+      this.done = done;
+    }
+
+    public boolean isDone() {
+      return done;
+    }
+  }
+
+  private class MonitorRunnable implements Runnable {
+    private final String runnableName =
+        String.format(
+            "%s$%s-monitor",
+            evaluationContext.getPipelineOptions().getAppName(),
+            ExecutorServiceParallelExecutor.class.getSimpleName());
+
+    @Override
+    public void run() {
+      String oldName = Thread.currentThread().getName();
+      Thread.currentThread().setName(runnableName);
+      try {
+        ExecutorUpdate update = allUpdates.poll();
+        // pull all of the pending work off of the queue
+        while (update != null) {
+          LOG.debug("Executor Update: {}", update);
+          if (update.getBundle().isPresent()) {
+            scheduleConsumers(update.getBundle().get());
+          } else if (update.getException().isPresent()) {
+            visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(update.getException().get()));
+          }
+          update = allUpdates.poll();
+        }
+        boolean timersFired = fireTimers();
+        addWorkIfNecessary(timersFired);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.error("Monitor died due to being interrupted");
+        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(e))) {
+          visibleUpdates.poll();
+        }
+      } catch (Throwable t) {
+        LOG.error("Monitor thread died due to throwable", t);
+        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(t))) {
+          visibleUpdates.poll();
+        }
+      } finally {
+        if (!shouldShutdown()) {
+          // The monitor thread should always be scheduled; but we only need to be scheduled once
+          executorService.submit(this);
+        }
+        Thread.currentThread().setName(oldName);
+      }
+    }
+
+    /**
+     * Fires any available timers. Returns true if at least one timer was fired.
+     */
+    private boolean fireTimers() throws Exception {
+      try {
+        boolean firedTimers = false;
+        for (Map.Entry<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> transformTimers :
+            evaluationContext.extractFiredTimers().entrySet()) {
+          AppliedPTransform<?, ?, ?> transform = transformTimers.getKey();
+          for (Map.Entry<Object, FiredTimers> keyTimers : transformTimers.getValue().entrySet()) {
+            for (TimeDomain domain : TimeDomain.values()) {
+              Collection<TimerData> delivery = keyTimers.getValue().getTimers(domain);
+              if (delivery.isEmpty()) {
+                continue;
+              }
+              KeyedWorkItem<Object, Object> work =
+                  KeyedWorkItems.timersWorkItem(keyTimers.getKey(), delivery);
+              @SuppressWarnings({"unchecked", "rawtypes"})
+              CommittedBundle<?> bundle =
+                  evaluationContext
+                      .createKeyedBundle(
+                          null, keyTimers.getKey(), (PCollection) transform.getInput())
+                      .add(WindowedValue.valueInEmptyWindows(work))
+                      .commit(Instant.now());
+              scheduleConsumption(transform, bundle, new TimerCompletionCallback(delivery));
+              firedTimers = true;
+            }
+          }
+        }
+        return firedTimers;
+      } catch (Exception e) {
+        LOG.error("Internal Error while delivering timers", e);
+        throw e;
+      }
+    }
+
+    private boolean shouldShutdown() {
+      if (evaluationContext.isDone()) {
+        LOG.debug("Pipeline is finished. Shutting down. {}");
+        while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) {
+          visibleUpdates.poll();
+        }
+        executorService.shutdown();
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * If all active {@link TransformExecutor TransformExecutors} are in a blocked state,
+     * add more work from root nodes that may have additional work. This ensures that if a pipeline
+     * has elements available from the root nodes it will add those elements when necessary.
+     */
+    private void addWorkIfNecessary(boolean firedTimers) {
+      // If any timers have fired, they will add more work; We don't need to add more
+      if (firedTimers) {
+        return;
+      }
+      for (TransformExecutor<?> executor : scheduledExecutors.keySet()) {
+        if (!isExecutorBlocked(executor)) {
+          // We have at least one executor that can proceed without adding additional work
+          return;
+        }
+      }
+      // All current TransformExecutors are blocked; add more work from the roots.
+      for (AppliedPTransform<?, ?, ?> root : rootNodes) {
+        if (!evaluationContext.isDone(root)) {
+          scheduleConsumption(root, null, defaultCompletionCallback);
+        }
+      }
+    }
+
+    /**
+     * Return true if the provided executor might make more progress if no action is taken.
+     *
+     * <p>May return false even if all executor threads are currently blocked or cleaning up, as
+     * these can cause more work to be scheduled. If this does not occur, after these calls
+     * terminate, future calls will return true if all executors are waiting.
+     */
+    private boolean isExecutorBlocked(TransformExecutor<?> executor) {
+      Thread thread = executor.getThread();
+      if (thread == null) {
+        return false;
+      }
+      switch (thread.getState()) {
+        case TERMINATED:
+          throw new IllegalStateException(String.format(
+              "Unexpectedly encountered a Terminated TransformExecutor %s", executor));
+        case WAITING:
+        case TIMED_WAITING:
+          // The thread is waiting for some external input. Adding more work may cause the thread
+          // to stop waiting (e.g. the thread is waiting on an unbounded side input)
+          return true;
+        case BLOCKED:
+          // The executor is blocked on acquisition of a java monitor. This usually means it is
+          // making a call to the EvaluationContext, but not a model-blocking call - and will
+          // eventually complete, at which point we may reevaluate.
+        default:
+          // NEW and RUNNABLE threads can make progress
+          return false;
+      }
+    }
+  }
+}


[12/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
new file mode 100644
index 0000000..491363a
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.Read.Bounded;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+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.MockitoAnnotations;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * Tests for {@link BoundedReadEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class BoundedReadEvaluatorFactoryTest {
+  private BoundedSource<Long> source;
+  private PCollection<Long> longs;
+  private TransformEvaluatorFactory factory;
+  @Mock private InProcessEvaluationContext context;
+  private BundleFactory bundleFactory;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    source = CountingSource.upTo(10L);
+    TestPipeline p = TestPipeline.create();
+    longs = p.apply(Read.from(source));
+
+    factory = new BoundedReadEvaluatorFactory();
+    bundleFactory = InProcessBundleFactory.create();
+  }
+
+  @Test
+  public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    assertThat(
+        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(),
+        containsInAnyOrder(
+            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+  }
+
+  /**
+   * Demonstrate that acquiring multiple {@link TransformEvaluator TransformEvaluators} for the same
+   * {@link Bounded Read.Bounded} application with the same evaluation context only produces the
+   * elements once.
+   */
+  @Test
+  public void boundedSourceInMemoryTransformEvaluatorAfterFinishIsEmpty() throws Exception {
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    Iterable<? extends WindowedValue<Long>> outputElements =
+        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
+    assertThat(
+        outputElements,
+        containsInAnyOrder(
+            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+
+    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(secondOutput);
+    TransformEvaluator<?> secondEvaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+    assertThat(secondResult.getOutputBundles(), emptyIterable());
+    assertThat(
+        secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
+    assertThat(
+        outputElements,
+        containsInAnyOrder(
+            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+  }
+
+  /**
+   * Demonstrates that acquiring multiple evaluators from the factory are independent, but
+   * the elements in the source are only produced once.
+   */
+  @Test
+  public void boundedSourceEvaluatorSimultaneousEvaluations() throws Exception {
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
+    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(output).thenReturn(secondOutput);
+
+    // create both evaluators before finishing either.
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+    TransformEvaluator<?> secondEvaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    Iterable<? extends WindowedValue<Long>> outputElements =
+        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
+
+    assertThat(
+        outputElements,
+        containsInAnyOrder(
+            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+    assertThat(secondResult.getOutputBundles(), emptyIterable());
+    assertThat(
+        secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
+    assertThat(
+        outputElements,
+        containsInAnyOrder(
+            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
+  }
+
+  @Test
+  public void boundedSourceEvaluatorClosesReader() throws Exception {
+    TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<Long> pcollection = p.apply(Read.from(source));
+    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+    when(context.createRootBundle(pcollection)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+    evaluator.finishBundle();
+    CommittedBundle<Long> committed = output.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L)));
+    assertThat(TestSource.readerClosed, is(true));
+  }
+
+  @Test
+  public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
+    TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of());
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<Long> pcollection = p.apply(Read.from(source));
+    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+    when(context.createRootBundle(pcollection)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+    evaluator.finishBundle();
+    CommittedBundle<Long> committed = output.commit(Instant.now());
+    assertThat(committed.getElements(), emptyIterable());
+    assertThat(TestSource.readerClosed, is(true));
+  }
+
+  private static class TestSource<T> extends BoundedSource<T> {
+    private static boolean readerClosed;
+    private final Coder<T> coder;
+    private final T[] elems;
+
+    public TestSource(Coder<T> coder, T... elems) {
+      this.elems = elems;
+      this.coder = coder;
+      readerClosed = false;
+    }
+
+    @Override
+    public List<? extends BoundedSource<T>> splitIntoBundles(
+        long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+      return ImmutableList.of(this);
+    }
+
+    @Override
+    public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+      return 0;
+    }
+
+    @Override
+    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+      return false;
+    }
+
+    @Override
+    public BoundedSource.BoundedReader<T> createReader(PipelineOptions options) throws IOException {
+      return new TestReader<>(this, elems);
+    }
+
+    @Override
+    public void validate() {
+    }
+
+    @Override
+    public Coder<T> getDefaultOutputCoder() {
+      return coder;
+    }
+  }
+
+  private static class TestReader<T> extends BoundedReader<T> {
+    private final BoundedSource<T> source;
+    private final List<T> elems;
+    private int index;
+
+    public TestReader(BoundedSource<T> source, T... elems) {
+      this.source = source;
+      this.elems = Arrays.asList(elems);
+      this.index = -1;
+    }
+
+    @Override
+    public BoundedSource<T> getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      return advance();
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      if (elems.size() > index + 1) {
+        index++;
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public T getCurrent() throws NoSuchElementException {
+      return elems.get(index);
+    }
+
+    @Override
+    public void close() throws IOException {
+      TestSource.readerClosed = true;
+    }
+  }
+
+  private static WindowedValue<Long> gw(Long elem) {
+    return WindowedValue.valueInGlobalWindow(elem);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
new file mode 100644
index 0000000..b30e005
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Tests for {@link CommittedResult}.
+ */
+@RunWith(JUnit4.class)
+public class CommittedResultTest implements Serializable {
+  private transient TestPipeline p = TestPipeline.create();
+  private transient AppliedPTransform<?, ?, ?> transform =
+      AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
+      });
+  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void getTransformExtractsFromResult() {
+    CommittedResult result =
+        CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
+            Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
+
+    assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
+  }
+
+  @Test
+  public void getOutputsEqualInput() {
+    List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
+        ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
+            WindowingStrategy.globalDefault(),
+            PCollection.IsBounded.BOUNDED)).commit(Instant.now()),
+            bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
+                WindowingStrategy.globalDefault(),
+                PCollection.IsBounded.UNBOUNDED)).commit(Instant.now()));
+    CommittedResult result =
+        CommittedResult.create(StepTransformResult.withoutHold(transform).build(), outputs);
+
+    assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
new file mode 100644
index 0000000..353eef6
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Tests for {@link ConsumerTrackingPipelineVisitor}.
+ */
+@RunWith(JUnit4.class)
+public class ConsumerTrackingPipelineVisitorTest implements Serializable {
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+  private transient TestPipeline p = TestPipeline.create();
+  private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+
+  @Test
+  public void getViewsReturnsViews() {
+    PCollectionView<List<String>> listView =
+        p.apply("listCreate", Create.of("foo", "bar"))
+            .apply(
+                ParDo.of(
+                    new DoFn<String, String>() {
+                      @Override
+                      public void processElement(DoFn<String, String>.ProcessContext c)
+                          throws Exception {
+                        c.output(Integer.toString(c.element().length()));
+                      }
+                    }))
+            .apply(View.<String>asList());
+    PCollectionView<Object> singletonView =
+        p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getViews(),
+        Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
+  }
+
+  @Test
+  public void getRootTransformsContainsPBegins() {
+    PCollection<String> created = p.apply(Create.of("foo", "bar"));
+    PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
+    PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getRootTransforms(),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            created.getProducingTransformInternal(),
+            counted.getProducingTransformInternal(),
+            unCounted.getProducingTransformInternal()));
+  }
+
+  @Test
+  public void getRootTransformsContainsEmptyFlatten() {
+    PCollection<String> empty =
+        PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getRootTransforms(),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            empty.getProducingTransformInternal()));
+  }
+
+  @Test
+  public void getValueToConsumersSucceeds() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @Override
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+
+    PCollection<String> flattened =
+        PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
+
+    p.traverseTopologically(visitor);
+
+    assertThat(
+        visitor.getValueToConsumers().get(created),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            transformed.getProducingTransformInternal(),
+            flattened.getProducingTransformInternal()));
+    assertThat(
+        visitor.getValueToConsumers().get(transformed),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            flattened.getProducingTransformInternal()));
+    assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
+  }
+
+  @Test
+  public void getUnfinalizedPValuesContainsDanglingOutputs() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @Override
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getUnfinalizedPValues(), Matchers.<PValue>contains(transformed));
+  }
+
+  @Test
+  public void getUnfinalizedPValuesEmpty() {
+    p.apply(Create.of("1", "2", "3"))
+        .apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @Override
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }))
+        .apply(
+            new PTransform<PInput, PDone>() {
+              @Override
+              public PDone apply(PInput input) {
+                return PDone.in(input.getPipeline());
+              }
+            });
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getUnfinalizedPValues(), emptyIterable());
+  }
+
+  @Test
+  public void getStepNamesContainsAllTransforms() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @Override
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+    PDone finished =
+        transformed.apply(
+            new PTransform<PInput, PDone>() {
+              @Override
+              public PDone apply(PInput input) {
+                return PDone.in(input.getPipeline());
+              }
+            });
+
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getStepNames(),
+        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+            created.getProducingTransformInternal(), "s0"));
+    assertThat(
+        visitor.getStepNames(),
+        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+            transformed.getProducingTransformInternal(), "s1"));
+    assertThat(
+        visitor.getStepNames(),
+        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
+            finished.getProducingTransformInternal(), "s2"));
+  }
+
+  @Test
+  public void traverseMultipleTimesThrows() {
+    p.apply(Create.of(1, 2, 3));
+
+    p.traverseTopologically(visitor);
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName());
+    thrown.expectMessage("is finalized");
+    p.traverseTopologically(visitor);
+  }
+
+  @Test
+  public void traverseIndependentPathsSucceeds() {
+    p.apply("left", Create.of(1, 2, 3));
+    p.apply("right", Create.of("foo", "bar", "baz"));
+
+    p.traverseTopologically(visitor);
+  }
+
+  @Test
+  public void getRootTransformsWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getRootTransforms");
+    visitor.getRootTransforms();
+  }
+  @Test
+  public void getStepNamesWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getStepNames");
+    visitor.getStepNames();
+  }
+  @Test
+  public void getUnfinalizedPValuesWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getUnfinalizedPValues");
+    visitor.getUnfinalizedPValues();
+  }
+
+  @Test
+  public void getValueToConsumersWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getValueToConsumers");
+    visitor.getValueToConsumers();
+  }
+
+  @Test
+  public void getViewsWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("getViews");
+    visitor.getViews();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
new file mode 100644
index 0000000..9a358dd
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.isA;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+
+/**
+ * Tests for {@link EncodabilityEnforcementFactory}.
+ */
+@RunWith(JUnit4.class)
+public class EncodabilityEnforcementFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private EncodabilityEnforcementFactory factory = EncodabilityEnforcementFactory.create();
+  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void encodeFailsThrows() {
+    WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
+
+    ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoEncodeCoder(), record);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Encode not allowed");
+    enforcement.beforeElement(record);
+  }
+
+  @Test
+  public void decodeFailsThrows() {
+    WindowedValue<Record> record = WindowedValue.valueInGlobalWindow(new Record());
+
+    ModelEnforcement<Record> enforcement = createEnforcement(new RecordNoDecodeCoder(), record);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Decode not allowed");
+    enforcement.beforeElement(record);
+  }
+
+  @Test
+  public void consistentWithEqualsStructuralValueNotEqualThrows() {
+    WindowedValue<Record> record =
+        WindowedValue.<Record>valueInGlobalWindow(
+            new Record() {
+              @Override
+              public String toString() {
+                return "OriginalRecord";
+              }
+            });
+
+    ModelEnforcement<Record> enforcement =
+        createEnforcement(new RecordStructuralValueCoder(), record);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(IllegalArgumentException.class));
+    thrown.expectMessage("does not maintain structural value equality");
+    thrown.expectMessage(RecordStructuralValueCoder.class.getSimpleName());
+    thrown.expectMessage("OriginalRecord");
+    enforcement.beforeElement(record);
+  }
+
+  @Test
+  public void notConsistentWithEqualsStructuralValueNotEqualSucceeds() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> unencodable =
+        p.apply(
+            Create.of(new Record())
+                .withCoder(new RecordNotConsistentWithEqualsStructuralValueCoder()));
+    AppliedPTransform<?, ?, ?> consumer =
+        unencodable.apply(Count.<Record>globally()).getProducingTransformInternal();
+
+    WindowedValue<Record> record = WindowedValue.<Record>valueInGlobalWindow(new Record());
+
+    CommittedBundle<Record> input =
+        bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
+    ModelEnforcement<Record> enforcement = factory.forBundle(input, consumer);
+
+    enforcement.beforeElement(record);
+    enforcement.afterElement(record);
+    enforcement.afterFinish(
+        input,
+        StepTransformResult.withoutHold(consumer).build(),
+        Collections.<CommittedBundle<?>>emptyList());
+  }
+
+  private <T> ModelEnforcement<T> createEnforcement(Coder<T> coder, WindowedValue<T> record) {
+    TestPipeline p = TestPipeline.create();
+    PCollection<T> unencodable = p.apply(Create.<T>of().withCoder(coder));
+    AppliedPTransform<?, ?, ?> consumer =
+        unencodable.apply(Count.<T>globally()).getProducingTransformInternal();
+    CommittedBundle<T> input =
+        bundleFactory.createRootBundle(unencodable).add(record).commit(Instant.now());
+    ModelEnforcement<T> enforcement = factory.forBundle(input, consumer);
+    return enforcement;
+  }
+
+  @Test
+  public void structurallyEqualResultsSucceeds() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> unencodable = p.apply(Create.of(1).withCoder(VarIntCoder.of()));
+    AppliedPTransform<?, ?, ?> consumer =
+        unencodable.apply(Count.<Integer>globally()).getProducingTransformInternal();
+
+    WindowedValue<Integer> value = WindowedValue.valueInGlobalWindow(1);
+
+    CommittedBundle<Integer> input =
+        bundleFactory.createRootBundle(unencodable).add(value).commit(Instant.now());
+    ModelEnforcement<Integer> enforcement = factory.forBundle(input, consumer);
+
+    enforcement.beforeElement(value);
+    enforcement.afterElement(value);
+    enforcement.afterFinish(
+        input,
+        StepTransformResult.withoutHold(consumer).build(),
+        Collections.<CommittedBundle<?>>emptyList());
+  }
+
+  private static class Record {}
+  private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
+
+    @Override
+    public void encode(
+        Record value,
+        OutputStream outStream,
+        org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {
+      throw new CoderException("Encode not allowed");
+    }
+
+    @Override
+    public Record decode(
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {
+      return null;
+    }
+  }
+
+  private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
+    @Override
+    public void encode(
+        Record value,
+        OutputStream outStream,
+        org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {}
+
+    @Override
+    public Record decode(
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {
+      throw new CoderException("Decode not allowed");
+    }
+  }
+
+  private static class RecordStructuralValueCoder extends AtomicCoder<Record> {
+    @Override
+    public void encode(
+        Record value,
+        OutputStream outStream,
+        org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {}
+
+    @Override
+    public Record decode(
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {
+      return new Record() {
+        @Override
+        public String toString() {
+          return "DecodedRecord";
+        }
+      };
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+      return true;
+    }
+
+    @Override
+    public Object structuralValue(Record value) {
+      return value;
+    }
+  }
+
+  private static class RecordNotConsistentWithEqualsStructuralValueCoder
+      extends AtomicCoder<Record> {
+    @Override
+    public void encode(
+        Record value,
+        OutputStream outStream,
+        org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {}
+
+    @Override
+    public Record decode(
+        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+        throws CoderException, IOException {
+      return new Record() {
+        @Override
+        public String toString() {
+          return "DecodedRecord";
+        }
+      };
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+      return false;
+    }
+
+    @Override
+    public Object structuralValue(Record value) {
+      return value;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
new file mode 100644
index 0000000..66a5106
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.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.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FlattenEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class FlattenEvaluatorFactoryTest {
+  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  @Test
+  public void testFlattenInMemoryEvaluator() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> left = p.apply("left", Create.of(1, 2, 4));
+    PCollection<Integer> right = p.apply("right", Create.of(-1, 2, -4));
+    PCollectionList<Integer> list = PCollectionList.of(left).and(right);
+
+    PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
+
+    CommittedBundle<Integer> leftBundle =
+        bundleFactory.createRootBundle(left).commit(Instant.now());
+    CommittedBundle<Integer> rightBundle =
+        bundleFactory.createRootBundle(right).commit(Instant.now());
+
+    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+
+    UncommittedBundle<Integer> flattenedLeftBundle = bundleFactory.createRootBundle(flattened);
+    UncommittedBundle<Integer> flattenedRightBundle = bundleFactory.createRootBundle(flattened);
+
+    when(context.createBundle(leftBundle, flattened)).thenReturn(flattenedLeftBundle);
+    when(context.createBundle(rightBundle, flattened)).thenReturn(flattenedRightBundle);
+
+    FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
+    TransformEvaluator<Integer> leftSideEvaluator =
+        factory.forApplication(flattened.getProducingTransformInternal(), leftBundle, context);
+    TransformEvaluator<Integer> rightSideEvaluator =
+        factory.forApplication(
+            flattened.getProducingTransformInternal(),
+            rightBundle,
+            context);
+
+    leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
+    rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
+    leftSideEvaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
+    leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING));
+    rightSideEvaluator.processElement(
+        WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
+    rightSideEvaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
+
+    InProcessTransformResult rightSideResult = rightSideEvaluator.finishBundle();
+    InProcessTransformResult leftSideResult = leftSideEvaluator.finishBundle();
+
+    assertThat(
+        rightSideResult.getOutputBundles(),
+        Matchers.<UncommittedBundle<?>>contains(flattenedRightBundle));
+    assertThat(
+        rightSideResult.getTransform(),
+        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+    assertThat(
+        leftSideResult.getOutputBundles(),
+        Matchers.<UncommittedBundle<?>>contains(flattenedLeftBundle));
+    assertThat(
+        leftSideResult.getTransform(),
+        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+
+    assertThat(
+        flattenedLeftBundle.commit(Instant.now()).getElements(),
+        containsInAnyOrder(
+            WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
+            WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING),
+            WindowedValue.valueInGlobalWindow(1)));
+    assertThat(
+        flattenedRightBundle.commit(Instant.now()).getElements(),
+        containsInAnyOrder(
+            WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
+            WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
+            WindowedValue.valueInGlobalWindow(-1)));
+  }
+
+  @Test
+  public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    PCollectionList<Integer> list = PCollectionList.empty(p);
+
+    PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
+
+    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+
+    FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
+    TransformEvaluator<Integer> emptyEvaluator =
+        factory.forApplication(flattened.getProducingTransformInternal(), null, context);
+
+    InProcessTransformResult leftSideResult = emptyEvaluator.finishBundle();
+
+    assertThat(leftSideResult.getOutputBundles(), emptyIterable());
+    assertThat(
+        leftSideResult.getTransform(),
+        Matchers.<AppliedPTransform<?, ?, ?>>equalTo(flattened.getProducingTransformInternal()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
new file mode 100644
index 0000000..9ea71d7
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/**
+ * Tests for {@link ForwardingPTransform}.
+ */
+@RunWith(JUnit4.class)
+public class ForwardingPTransformTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Mock private PTransform<PCollection<Integer>, PCollection<String>> delegate;
+
+  private ForwardingPTransform<PCollection<Integer>, PCollection<String>> forwarding;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    forwarding =
+        new ForwardingPTransform<PCollection<Integer>, PCollection<String>>() {
+          @Override
+          protected PTransform<PCollection<Integer>, PCollection<String>> delegate() {
+            return delegate;
+          }
+        };
+  }
+
+  @Test
+  public void applyDelegates() {
+    @SuppressWarnings("unchecked")
+    PCollection<Integer> collection = mock(PCollection.class);
+    @SuppressWarnings("unchecked")
+    PCollection<String> output = mock(PCollection.class);
+    when(delegate.apply(collection)).thenReturn(output);
+    PCollection<String> result = forwarding.apply(collection);
+    assertThat(result, equalTo(output));
+  }
+
+  @Test
+  public void getNameDelegates() {
+    String name = "My_forwardingptransform-name;for!thisTest";
+    when(delegate.getName()).thenReturn(name);
+    assertThat(forwarding.getName(), equalTo(name));
+  }
+
+  @Test
+  public void validateDelegates() {
+    @SuppressWarnings("unchecked")
+    PCollection<Integer> input = mock(PCollection.class);
+    doThrow(RuntimeException.class).when(delegate).validate(input);
+
+    thrown.expect(RuntimeException.class);
+    forwarding.validate(input);
+  }
+
+  @Test
+  public void getDefaultOutputCoderDelegates() throws Exception {
+    @SuppressWarnings("unchecked")
+    PCollection<Integer> input = mock(PCollection.class);
+    @SuppressWarnings("unchecked")
+    PCollection<String> output = mock(PCollection.class);
+    @SuppressWarnings("unchecked")
+    Coder<String> outputCoder = mock(Coder.class);
+
+    when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder);
+    assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder));
+  }
+
+  @Test
+  public void populateDisplayDataDelegates() {
+    DisplayData.Builder builder = mock(DisplayData.Builder.class);
+    doThrow(RuntimeException.class).when(delegate).populateDisplayData(builder);
+
+    thrown.expect(RuntimeException.class);
+    forwarding.populateDisplayData(builder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
new file mode 100644
index 0000000..267266d
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Multiset;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link GroupByKeyEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class GroupByKeyEvaluatorFactoryTest {
+  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void testInMemoryEvaluator() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    KV<String, Integer> firstFoo = KV.of("foo", -1);
+    KV<String, Integer> secondFoo = KV.of("foo", 1);
+    KV<String, Integer> thirdFoo = KV.of("foo", 3);
+    KV<String, Integer> firstBar = KV.of("bar", 22);
+    KV<String, Integer> secondBar = KV.of("bar", 12);
+    KV<String, Integer> firstBaz = KV.of("baz", Integer.MAX_VALUE);
+    PCollection<KV<String, Integer>> values =
+        p.apply(Create.of(firstFoo, firstBar, secondFoo, firstBaz, secondBar, thirdFoo));
+    PCollection<KV<String, WindowedValue<Integer>>> kvs =
+        values.apply(new ReifyTimestampsAndWindows<String, Integer>());
+    PCollection<KeyedWorkItem<String, Integer>> groupedKvs =
+        kvs.apply(new GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly<String, Integer>());
+
+    CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
+        bundleFactory.createRootBundle(kvs).commit(Instant.now());
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+
+    UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle =
+        bundleFactory.createKeyedBundle(null, "foo", groupedKvs);
+    UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle =
+        bundleFactory.createKeyedBundle(null, "bar", groupedKvs);
+    UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle =
+        bundleFactory.createKeyedBundle(null, "baz", groupedKvs);
+
+    when(evaluationContext.createKeyedBundle(inputBundle, "foo", groupedKvs)).thenReturn(fooBundle);
+    when(evaluationContext.createKeyedBundle(inputBundle, "bar", groupedKvs)).thenReturn(barBundle);
+    when(evaluationContext.createKeyedBundle(inputBundle, "baz", groupedKvs)).thenReturn(bazBundle);
+
+    // The input to a GroupByKey is assumed to be a KvCoder
+    @SuppressWarnings("unchecked")
+    Coder<String> keyCoder =
+        ((KvCoder<String, WindowedValue<Integer>>) kvs.getCoder()).getKeyCoder();
+    TransformEvaluator<KV<String, WindowedValue<Integer>>> evaluator =
+        new GroupByKeyEvaluatorFactory()
+            .forApplication(
+                groupedKvs.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstFoo)));
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondFoo)));
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(thirdFoo)));
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBar)));
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(secondBar)));
+    evaluator.processElement(WindowedValue.valueInEmptyWindows(gwValue(firstBaz)));
+
+    evaluator.finishBundle();
+
+    assertThat(
+        fooBundle.commit(Instant.now()).getElements(),
+        contains(
+            new KeyedWorkItemMatcher<String, Integer>(
+                KeyedWorkItems.elementsWorkItem(
+                    "foo",
+                    ImmutableSet.of(
+                        WindowedValue.valueInGlobalWindow(-1),
+                        WindowedValue.valueInGlobalWindow(1),
+                        WindowedValue.valueInGlobalWindow(3))),
+                keyCoder)));
+    assertThat(
+        barBundle.commit(Instant.now()).getElements(),
+        contains(
+            new KeyedWorkItemMatcher<String, Integer>(
+                KeyedWorkItems.elementsWorkItem(
+                    "bar",
+                    ImmutableSet.of(
+                        WindowedValue.valueInGlobalWindow(12),
+                        WindowedValue.valueInGlobalWindow(22))),
+                keyCoder)));
+    assertThat(
+        bazBundle.commit(Instant.now()).getElements(),
+        contains(
+            new KeyedWorkItemMatcher<String, Integer>(
+                KeyedWorkItems.elementsWorkItem(
+                    "baz",
+                    ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))),
+                keyCoder)));
+  }
+
+  private <K, V> KV<K, WindowedValue<V>> gwValue(KV<K, V> kv) {
+    return KV.of(kv.getKey(), WindowedValue.valueInGlobalWindow(kv.getValue()));
+  }
+
+  private static class KeyedWorkItemMatcher<K, V>
+      extends BaseMatcher<WindowedValue<KeyedWorkItem<K, V>>> {
+    private final KeyedWorkItem<K, V> myWorkItem;
+    private final Coder<K> keyCoder;
+
+    public KeyedWorkItemMatcher(KeyedWorkItem<K, V> myWorkItem, Coder<K> keyCoder) {
+      this.myWorkItem = myWorkItem;
+      this.keyCoder = keyCoder;
+    }
+
+    @Override
+    public boolean matches(Object item) {
+      if (item == null || !(item instanceof WindowedValue)) {
+        return false;
+      }
+      WindowedValue<KeyedWorkItem<K, V>> that = (WindowedValue<KeyedWorkItem<K, V>>) item;
+      Multiset<WindowedValue<V>> myValues = HashMultiset.create();
+      Multiset<WindowedValue<V>> thatValues = HashMultiset.create();
+      for (WindowedValue<V> value : myWorkItem.elementsIterable()) {
+        myValues.add(value);
+      }
+      for (WindowedValue<V> value : that.getValue().elementsIterable()) {
+        thatValues.add(value);
+      }
+      try {
+        return myValues.equals(thatValues)
+            && keyCoder
+                .structuralValue(myWorkItem.key())
+                .equals(keyCoder.structuralValue(that.getValue().key()));
+      } catch (Exception e) {
+        return false;
+      }
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description
+          .appendText("KeyedWorkItem<K, V> containing key ")
+          .appendValue(myWorkItem.key())
+          .appendText(" and values ")
+          .appendValueList("[", ", ", "]", myWorkItem.elementsIterable());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
new file mode 100644
index 0000000..557ebff
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ImmutabilityCheckingBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ImmutabilityCheckingBundleFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private ImmutabilityCheckingBundleFactory factory;
+  private PCollection<byte[]> created;
+  private PCollection<byte[]> transformed;
+
+  @Before
+  public void setup() {
+    TestPipeline p = TestPipeline.create();
+    created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
+    transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
+    factory = ImmutabilityCheckingBundleFactory.create(InProcessBundleFactory.create());
+  }
+
+  @Test
+  public void noMutationRootBundleSucceeds() {
+    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+    byte[] array = new byte[] {0, 1, 2};
+    root.add(WindowedValue.valueInGlobalWindow(array));
+    CommittedBundle<byte[]> committed = root.commit(Instant.now());
+
+    assertThat(
+        committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
+  }
+
+  @Test
+  public void noMutationKeyedBundleSucceeds() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            new byte[] {4, 8, 12},
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    keyed.add(windowedArray);
+
+    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+  }
+
+  @Test
+  public void noMutationCreateBundleSucceeds() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            new byte[] {4, 8, 12},
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    intermediate.add(windowedArray);
+
+    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+  }
+
+  @Test
+  public void mutationBeforeAddRootBundleSucceeds() {
+    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+    byte[] array = new byte[] {0, 1, 2};
+    array[1] = 2;
+    root.add(WindowedValue.valueInGlobalWindow(array));
+    CommittedBundle<byte[]> committed = root.commit(Instant.now());
+
+    assertThat(
+        committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array)));
+  }
+
+  @Test
+  public void mutationBeforeAddKeyedBundleSucceeds() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+    byte[] array = new byte[] {4, 8, 12};
+    array[0] = Byte.MAX_VALUE;
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            array,
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    keyed.add(windowedArray);
+
+    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+  }
+
+  @Test
+  public void mutationBeforeAddCreateBundleSucceeds() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+    byte[] array = new byte[] {4, 8, 12};
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            array,
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    array[2] = -3;
+    intermediate.add(windowedArray);
+
+    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(windowedArray));
+  }
+
+  @Test
+  public void mutationAfterAddRootBundleThrows() {
+    UncommittedBundle<byte[]> root = factory.createRootBundle(created);
+    byte[] array = new byte[] {0, 1, 2};
+    root.add(WindowedValue.valueInGlobalWindow(array));
+
+    array[1] = 2;
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(IllegalMutationException.class));
+    thrown.expectMessage("Values must not be mutated in any way after being output");
+    CommittedBundle<byte[]> committed = root.commit(Instant.now());
+  }
+
+  @Test
+  public void mutationAfterAddKeyedBundleThrows() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> keyed = factory.createKeyedBundle(root, "mykey", transformed);
+
+    byte[] array = new byte[] {4, 8, 12};
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            array,
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    keyed.add(windowedArray);
+
+    array[0] = Byte.MAX_VALUE;
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(IllegalMutationException.class));
+    thrown.expectMessage("Values must not be mutated in any way after being output");
+    CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
+  }
+
+  @Test
+  public void mutationAfterAddCreateBundleThrows() {
+    CommittedBundle<byte[]> root = factory.createRootBundle(created).commit(Instant.now());
+    UncommittedBundle<byte[]> intermediate = factory.createBundle(root, transformed);
+
+    byte[] array = new byte[] {4, 8, 12};
+    WindowedValue<byte[]> windowedArray =
+        WindowedValue.of(
+            array,
+            new Instant(891L),
+            new IntervalWindow(new Instant(0), new Instant(1000)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    intermediate.add(windowedArray);
+
+    array[2] = -3;
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(IllegalMutationException.class));
+    thrown.expectMessage("Values must not be mutated in any way after being output");
+    CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
+  }
+
+  private static class IdentityDoFn<T> extends DoFn<T, T> {
+    @Override
+    public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
+      c.output(c.element());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
new file mode 100644
index 0000000..6cef60d
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Collections;
+
+/**
+ * Tests for {@link ImmutabilityEnforcementFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ImmutabilityEnforcementFactoryTest implements Serializable {
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+  private transient ImmutabilityEnforcementFactory factory;
+  private transient BundleFactory bundleFactory;
+  private transient PCollection<byte[]> pcollection;
+  private transient AppliedPTransform<?, ?, ?> consumer;
+
+  @Before
+  public void setup() {
+    factory = new ImmutabilityEnforcementFactory();
+    bundleFactory = InProcessBundleFactory.create();
+    TestPipeline p = TestPipeline.create();
+    pcollection =
+        p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
+            .apply(
+                ParDo.of(
+                    new DoFn<byte[], byte[]>() {
+                      @Override
+                      public void processElement(DoFn<byte[], byte[]>.ProcessContext c)
+                          throws Exception {
+                        c.element()[0] = 'b';
+                      }
+                    }));
+    consumer = pcollection.apply(Count.<byte[]>globally()).getProducingTransformInternal();
+  }
+
+  @Test
+  public void unchangedSucceeds() {
+    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+    CommittedBundle<byte[]> elements =
+        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+    enforcement.beforeElement(element);
+    enforcement.afterElement(element);
+    enforcement.afterFinish(
+        elements,
+        StepTransformResult.withoutHold(consumer).build(),
+        Collections.<CommittedBundle<?>>emptyList());
+  }
+
+  @Test
+  public void mutatedDuringProcessElementThrows() {
+    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+    CommittedBundle<byte[]> elements =
+        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+    enforcement.beforeElement(element);
+    element.getValue()[0] = 'f';
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage(consumer.getFullName());
+    thrown.expectMessage("illegaly mutated");
+    thrown.expectMessage("Input values must not be mutated");
+    enforcement.afterElement(element);
+    enforcement.afterFinish(
+        elements,
+        StepTransformResult.withoutHold(consumer).build(),
+        Collections.<CommittedBundle<?>>emptyList());
+  }
+
+  @Test
+  public void mutatedAfterProcessElementFails() {
+
+    WindowedValue<byte[]> element = WindowedValue.valueInGlobalWindow("bar".getBytes());
+    CommittedBundle<byte[]> elements =
+        bundleFactory.createRootBundle(pcollection).add(element).commit(Instant.now());
+
+    ModelEnforcement<byte[]> enforcement = factory.forBundle(elements, consumer);
+    enforcement.beforeElement(element);
+    enforcement.afterElement(element);
+
+    element.getValue()[0] = 'f';
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage(consumer.getFullName());
+    thrown.expectMessage("illegaly mutated");
+    thrown.expectMessage("Input values must not be mutated");
+    enforcement.afterFinish(
+        elements,
+        StepTransformResult.withoutHold(consumer).build(),
+        Collections.<CommittedBundle<?>>emptyList());
+  }
+}


[08/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
deleted file mode 100644
index a394090..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,155 +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.runners.inprocess;
-
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
-import org.apache.beam.sdk.io.Read.Bounded;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Bounded Read.Bounded} primitive {@link PTransform}.
- */
-final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
-  /*
-   * An evaluator for a Source is stateful, to ensure data is not read multiple times.
-   * Evaluators are cached here to ensure that the reader is not restarted if the evaluator is
-   * retriggered.
-   */
-  private final ConcurrentMap<EvaluatorKey, Queue<? extends BoundedReadEvaluator<?>>>
-      sourceEvaluators = new ConcurrentHashMap<>();
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws IOException {
-    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
-  }
-
-  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
-      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    BoundedReadEvaluator<?> evaluator =
-        getTransformEvaluatorQueue(transform, evaluationContext).poll();
-    if (evaluator == null) {
-      return EmptyTransformEvaluator.create(transform);
-    }
-    return evaluator;
-  }
-
-  /**
-   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
-   * provided application of {@link Bounded Read.Bounded}, initializing it if required.
-   *
-   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
-   * already done so.
-   */
-  @SuppressWarnings("unchecked")
-  private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
-      final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    // Key by the application and the context the evaluation is occurring in (which call to
-    // Pipeline#run).
-    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
-    Queue<BoundedReadEvaluator<OutputT>> evaluatorQueue =
-        (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-    if (evaluatorQueue == null) {
-      evaluatorQueue = new ConcurrentLinkedQueue<>();
-      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
-        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
-        // factory for this transform
-        BoundedSource<OutputT> source = transform.getTransform().getSource();
-        BoundedReadEvaluator<OutputT> evaluator =
-            new BoundedReadEvaluator<OutputT>(transform, evaluationContext, source);
-        evaluatorQueue.offer(evaluator);
-      } else {
-        // otherwise return the existing Queue that arrived before us
-        evaluatorQueue = (Queue<BoundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-      }
-    }
-    return evaluatorQueue;
-  }
-
-  /**
-   * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource},
-   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
-   * creates the {@link BoundedReader} and consumes all available input.
-   *
-   * <p>A {@link BoundedReadEvaluator} should only be created once per {@link BoundedSource}, and
-   * each evaluator should only be called once per evaluation of the pipeline. Otherwise, the source
-   * may produce duplicate elements.
-   */
-  private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
-    private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
-    /**
-     * The source being read from by this {@link BoundedReadEvaluator}. This may not be the same
-     * as the source derived from {@link #transform} due to splitting.
-     */
-    private BoundedSource<OutputT> source;
-
-    public BoundedReadEvaluator(
-        AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
-        BoundedSource<OutputT> source) {
-      this.transform = transform;
-      this.evaluationContext = evaluationContext;
-      this.source = source;
-    }
-
-    @Override
-    public void processElement(WindowedValue<Object> element) {}
-
-    @Override
-    public InProcessTransformResult finishBundle() throws IOException {
-      try (final BoundedReader<OutputT> reader =
-              source.createReader(evaluationContext.getPipelineOptions());) {
-        boolean contentsRemaining = reader.start();
-        UncommittedBundle<OutputT> output =
-            evaluationContext.createRootBundle(transform.getOutput());
-        while (contentsRemaining) {
-          output.add(
-              WindowedValue.timestampedValueInGlobalWindow(
-                  reader.getCurrent(), reader.getCurrentTimestamp()));
-          contentsRemaining = reader.advance();
-        }
-        return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MAX_VALUE)
-            .addOutput(output)
-            .build();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
deleted file mode 100644
index 5479b00..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/BundleFactory.java
+++ /dev/null
@@ -1,49 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * A factory that creates {@link UncommittedBundle UncommittedBundles}.
- */
-public interface BundleFactory {
-  /**
-   * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle belong to
-   * the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output);
-
-  /**
-   * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output);
-
-  /**
-   * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
-   * {@link InProcessGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
-   * belong to the {@code output} {@link PCollection}.
-   */
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.java
deleted file mode 100644
index 12427d9..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CachedThreadPoolExecutorServiceFactory.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.runners.inprocess;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * A {@link ExecutorServiceFactory} that produces cached thread pools via
- * {@link Executors#newCachedThreadPool()}.
- */
-class CachedThreadPoolExecutorServiceFactory
-    implements DefaultValueFactory<ExecutorServiceFactory>, ExecutorServiceFactory {
-  private static final CachedThreadPoolExecutorServiceFactory INSTANCE =
-      new CachedThreadPoolExecutorServiceFactory();
-
-  @Override
-  public ExecutorServiceFactory create(PipelineOptions options) {
-    return INSTANCE;
-  }
-
-  @Override
-  public ExecutorService create() {
-    return Executors.newCachedThreadPool();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
deleted file mode 100644
index 7a51251..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/Clock.java
+++ /dev/null
@@ -1,30 +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.runners.inprocess;
-
-import org.joda.time.Instant;
-
-/**
- * Access to the current time.
- */
-public interface Clock {
-  /**
-   * Returns the current time as an {@link Instant}.
-   */
-  Instant now();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java
deleted file mode 100644
index 10e9697..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CommittedResult.java
+++ /dev/null
@@ -1,46 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import com.google.auto.value.AutoValue;
-
-/**
- * A {@link InProcessTransformResult} that has been committed.
- */
-@AutoValue
-abstract class CommittedResult {
-  /**
-   * Returns the {@link AppliedPTransform} that produced this result.
-   */
-  public abstract AppliedPTransform<?, ?, ?> getTransform();
-
-  /**
-   * Returns the outputs produced by the transform.
-   */
-  public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
-
-  public static CommittedResult create(
-      InProcessTransformResult original, Iterable<? extends CommittedBundle<?>> outputs) {
-    return new AutoValue_CommittedResult(original.getTransform(),
-        outputs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
deleted file mode 100644
index 30a2b92..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/CompletionCallback.java
+++ /dev/null
@@ -1,36 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-
-/**
- * A callback for completing a bundle of input.
- */
-interface CompletionCallback {
-  /**
-   * Handle a successful result, returning the committed outputs of the result.
-   */
-  CommittedResult handleResult(
-      CommittedBundle<?> inputBundle, InProcessTransformResult result);
-
-  /**
-   * Handle a result that terminated abnormally due to the provided {@link Throwable}.
-   */
-  void handleThrowable(CommittedBundle<?> inputBundle, Throwable t);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
deleted file mode 100644
index f6ea4af..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitor.java
+++ /dev/null
@@ -1,173 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
- * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
- * input after the upstream transform has produced and committed output.
- */
-public class ConsumerTrackingPipelineVisitor implements PipelineVisitor {
-  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
-  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
-  private Collection<PCollectionView<?>> views = new ArrayList<>();
-  private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
-  private Set<PValue> toFinalize = new HashSet<>();
-  private int numTransforms = 0;
-  private boolean finalized = false;
-
-  @Override
-  public void enterCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s "
-            + "which has already visited a Pipeline and is finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    if (node.isRootNode()) {
-      finalized = true;
-    }
-  }
-
-  @Override
-  public void visitTransform(TransformTreeNode node) {
-    toFinalize.removeAll(node.getInput().expand());
-    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
-    stepNames.put(appliedTransform, genStepName());
-    if (node.getInput().expand().isEmpty()) {
-      rootTransforms.add(appliedTransform);
-    } else {
-      for (PValue value : node.getInput().expand()) {
-        valueToConsumers.get(value).add(appliedTransform);
-      }
-    }
-  }
-
-  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
-        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
-    return application;
-  }
-
-  @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
-    toFinalize.add(value);
-    for (PValue expandedValue : value.expand()) {
-      valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
-      if (expandedValue instanceof PCollectionView) {
-        views.add((PCollectionView<?>) expandedValue);
-      }
-      expandedValue.recordAsOutput(getAppliedTransform(producer));
-    }
-    value.recordAsOutput(getAppliedTransform(producer));
-  }
-
-  private String genStepName() {
-    return String.format("s%s", numTransforms++);
-  }
-
-
-  /**
-   * Returns a mapping of each fully-expanded {@link PValue} to each
-   * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
-   * returned from {@code getValueToCustomers().get(PValue)},
-   * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
-   */
-  public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
-    checkState(
-        finalized,
-        "Can't call getValueToConsumers before the Pipeline has been completely traversed");
-
-    return valueToConsumers;
-  }
-
-  /**
-   * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
-   * name.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
-    checkState(
-        finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
-
-    return stepNames;
-  }
-
-  /**
-   * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
-   * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
-    checkState(
-        finalized,
-        "Can't call getRootTransforms before the Pipeline has been completely traversed");
-
-    return rootTransforms;
-  }
-
-  /**
-   * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
-   * {@link Pipeline}.
-   */
-  public Collection<PCollectionView<?>> getViews() {
-    checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
-
-    return views;
-  }
-
-  /**
-   * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
-   * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
-   * {@link Pipeline} is executed.
-   */
-  public Set<PValue> getUnfinalizedPValues() {
-    checkState(
-        finalized,
-        "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
-
-    return toFinalize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
deleted file mode 100644
index d198903..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EmptyTransformEvaluator.java
+++ /dev/null
@@ -1,50 +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.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * A {@link TransformEvaluator} that ignores all input and produces no output. The result of
- * invoking {@link #finishBundle()} on this evaluator is to return an
- * {@link InProcessTransformResult} with no elements and a timestamp hold equal to
- * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}. Because the result contains no elements, this hold
- * will not affect the watermark.
- */
-final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
-  public static <T> TransformEvaluator<T> create(AppliedPTransform<?, ?, ?> transform) {
-    return new EmptyTransformEvaluator<T>(transform);
-  }
-
-  private final AppliedPTransform<?, ?, ?> transform;
-
-  private EmptyTransformEvaluator(AppliedPTransform<?, ?, ?> transform) {
-    this.transform = transform;
-  }
-
-  @Override
-  public void processElement(WindowedValue<T> element) throws Exception {}
-
-  @Override
-  public InProcessTransformResult finishBundle() throws Exception {
-    return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
-        .build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.java
deleted file mode 100644
index d234d4f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EncodabilityEnforcementFactory.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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Enforces that all elements in a {@link PCollection} can be encoded using that
- * {@link PCollection PCollection's} {@link Coder}.
- */
-class EncodabilityEnforcementFactory implements ModelEnforcementFactory {
-  public static EncodabilityEnforcementFactory create() {
-    return new EncodabilityEnforcementFactory();
-  }
-
-  @Override
-  public <T> ModelEnforcement<T> forBundle(
-      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
-    return new EncodabilityEnforcement<>(input);
-  }
-
-  private static class EncodabilityEnforcement<T> extends AbstractModelEnforcement<T> {
-    private Coder<T> coder;
-
-    public EncodabilityEnforcement(CommittedBundle<T> input) {
-      coder = input.getPCollection().getCoder();
-    }
-
-    @Override
-    public void beforeElement(WindowedValue<T> element) {
-      try {
-        T clone = CoderUtils.clone(coder, element.getValue());
-        if (coder.consistentWithEquals()) {
-          checkArgument(
-              coder.structuralValue(element.getValue()).equals(coder.structuralValue(clone)),
-              "Coder %s of class %s does not maintain structural value equality"
-                  + " on input element %s",
-              coder,
-              coder.getClass().getSimpleName(),
-              element.getValue());
-        }
-      } catch (Exception e) {
-        throw UserCodeException.wrap(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.java
deleted file mode 100644
index 9d8fc43..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/EvaluatorKey.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.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import java.util.Objects;
-
-/**
- * A (Transform, Pipeline Execution) key for stateful evaluators.
- *
- * Source evaluators are stateful to ensure data is not read multiple times. Evaluators are cached
- * to ensure that the reader is not restarted if the evaluator is retriggered. An
- * {@link EvaluatorKey} is used to ensure that multiple Pipelines can be executed without sharing
- * the same evaluators.
- */
-final class EvaluatorKey {
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final InProcessEvaluationContext context;
-
-  public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
-    this.transform = transform;
-    this.context = context;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(transform, context);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null || !(other instanceof EvaluatorKey)) {
-      return false;
-    }
-    EvaluatorKey that = (EvaluatorKey) other;
-    return Objects.equals(this.transform, that.transform)
-        && Objects.equals(this.context, that.context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
deleted file mode 100644
index cfbf7b4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceFactory.java
+++ /dev/null
@@ -1,33 +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.runners.inprocess;
-
-import java.util.concurrent.ExecutorService;
-
-/**
- * A factory that creates {@link ExecutorService ExecutorServices}.
- * {@link ExecutorService ExecutorServices} created by this factory should be independent of one
- * another (e.g., if any executor is shut down the remaining executors should continue to process
- * work).
- */
-public interface ExecutorServiceFactory {
-  /**
-   * Create a new {@link ExecutorService}.
-   */
-  ExecutorService create();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
deleted file mode 100644
index 19bf35d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ExecutorServiceParallelExecutor.java
+++ /dev/null
@@ -1,478 +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.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Optional;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
- * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
- */
-final class ExecutorServiceParallelExecutor implements InProcessExecutor {
-  private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
-
-  private final ExecutorService executorService;
-
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
-  private final Set<PValue> keyedPValues;
-  private final TransformEvaluatorRegistry registry;
-  @SuppressWarnings("rawtypes")
-  private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-      transformEnforcements;
-
-  private final InProcessEvaluationContext evaluationContext;
-
-  private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
-  private final ConcurrentMap<TransformExecutor<?>, Boolean> scheduledExecutors;
-
-  private final Queue<ExecutorUpdate> allUpdates;
-  private final BlockingQueue<VisibleExecutorUpdate> visibleUpdates;
-
-  private final TransformExecutorService parallelExecutorService;
-  private final CompletionCallback defaultCompletionCallback;
-
-  private Collection<AppliedPTransform<?, ?, ?>> rootNodes;
-
-  public static ExecutorServiceParallelExecutor create(
-      ExecutorService executorService,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Set<PValue> keyedPValues,
-      TransformEvaluatorRegistry registry,
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
-    return new ExecutorServiceParallelExecutor(
-        executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
-  }
-
-  private ExecutorServiceParallelExecutor(
-      ExecutorService executorService,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Set<PValue> keyedPValues,
-      TransformEvaluatorRegistry registry,
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
-    this.executorService = executorService;
-    this.valueToConsumers = valueToConsumers;
-    this.keyedPValues = keyedPValues;
-    this.registry = registry;
-    this.transformEnforcements = transformEnforcements;
-    this.evaluationContext = context;
-
-    scheduledExecutors = new ConcurrentHashMap<>();
-    // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
-    // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
-    // which stops the TransformExecutorServices from being prematurely garbage collected
-    executorServices =
-        CacheBuilder.newBuilder().weakValues().build(serialTransformExecutorServiceCacheLoader());
-
-    this.allUpdates = new ConcurrentLinkedQueue<>();
-    this.visibleUpdates = new ArrayBlockingQueue<>(20);
-
-    parallelExecutorService =
-        TransformExecutorServices.parallel(executorService, scheduledExecutors);
-    defaultCompletionCallback = new DefaultCompletionCallback();
-  }
-
-  private CacheLoader<StepAndKey, TransformExecutorService>
-      serialTransformExecutorServiceCacheLoader() {
-    return new CacheLoader<StepAndKey, TransformExecutorService>() {
-      @Override
-      public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
-        return TransformExecutorServices.serial(executorService, scheduledExecutors);
-      }
-    };
-  }
-
-  @Override
-  public void start(Collection<AppliedPTransform<?, ?, ?>> roots) {
-    rootNodes = ImmutableList.copyOf(roots);
-    Runnable monitorRunnable = new MonitorRunnable();
-    executorService.submit(monitorRunnable);
-  }
-
-  @SuppressWarnings("unchecked")
-  public void scheduleConsumption(
-      AppliedPTransform<?, ?, ?> consumer,
-      @Nullable CommittedBundle<?> bundle,
-      CompletionCallback onComplete) {
-    evaluateBundle(consumer, bundle, onComplete);
-  }
-
-  private <T> void evaluateBundle(
-      final AppliedPTransform<?, ?, ?> transform,
-      @Nullable final CommittedBundle<T> bundle,
-      final CompletionCallback onComplete) {
-    TransformExecutorService transformExecutor;
-
-    if (bundle != null && isKeyed(bundle.getPCollection())) {
-      final StepAndKey stepAndKey =
-          StepAndKey.of(transform, bundle == null ? null : bundle.getKey());
-      // This executor will remain reachable until it has executed all scheduled transforms.
-      // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
-      // a reference to the scheduled TransformExecutor callable. Follow-up TransformExecutors
-      // (scheduled due to the completion of another TransformExecutor) are provided to the
-      // ExecutorService before the Earlier TransformExecutor callable completes.
-      transformExecutor = executorServices.getUnchecked(stepAndKey);
-    } else {
-      transformExecutor = parallelExecutorService;
-    }
-
-    Collection<ModelEnforcementFactory> enforcements =
-        MoreObjects.firstNonNull(
-            transformEnforcements.get(transform.getTransform().getClass()),
-            Collections.<ModelEnforcementFactory>emptyList());
-
-    TransformExecutor<T> callable =
-        TransformExecutor.create(
-            registry,
-            enforcements,
-            evaluationContext,
-            bundle,
-            transform,
-            onComplete,
-            transformExecutor);
-    transformExecutor.schedule(callable);
-  }
-
-  private boolean isKeyed(PValue pvalue) {
-    return keyedPValues.contains(pvalue);
-  }
-
-  private void scheduleConsumers(CommittedBundle<?> bundle) {
-    for (AppliedPTransform<?, ?, ?> consumer : valueToConsumers.get(bundle.getPCollection())) {
-      scheduleConsumption(consumer, bundle, defaultCompletionCallback);
-    }
-  }
-
-  @Override
-  public void awaitCompletion() throws Throwable {
-    VisibleExecutorUpdate update;
-    do {
-      update = visibleUpdates.take();
-      if (update.throwable.isPresent()) {
-        throw update.throwable.get();
-      }
-    } while (!update.isDone());
-    executorService.shutdown();
-  }
-
-  /**
-   * The default {@link CompletionCallback}. The default completion callback is used to complete
-   * transform evaluations that are triggered due to the arrival of elements from an upstream
-   * transform, or for a source transform.
-   */
-  private class DefaultCompletionCallback implements CompletionCallback {
-    @Override
-    public CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
-      CommittedResult committedResult =
-          evaluationContext.handleResult(inputBundle, Collections.<TimerData>emptyList(), result);
-      for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
-      }
-      return committedResult;
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
-      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
-    }
-  }
-
-  /**
-   * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
-   * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
-   * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
-   * as part of the result.
-   */
-  private class TimerCompletionCallback implements CompletionCallback {
-    private final Iterable<TimerData> timers;
-
-    private TimerCompletionCallback(Iterable<TimerData> timers) {
-      this.timers = timers;
-    }
-
-    @Override
-    public CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
-      CommittedResult committedResult =
-          evaluationContext.handleResult(inputBundle, timers, result);
-      for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
-      }
-      return committedResult;
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
-      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
-    }
-  }
-
-  /**
-   * An internal status update on the state of the executor.
-   *
-   * Used to signal when the executor should be shut down (due to an exception).
-   */
-  private static class ExecutorUpdate {
-    private final Optional<? extends CommittedBundle<?>> bundle;
-    private final Optional<? extends Throwable> throwable;
-
-    public static ExecutorUpdate fromBundle(CommittedBundle<?> bundle) {
-      return new ExecutorUpdate(bundle, null);
-    }
-
-    public static ExecutorUpdate fromThrowable(Throwable t) {
-      return new ExecutorUpdate(null, t);
-    }
-
-    private ExecutorUpdate(CommittedBundle<?> producedBundle, Throwable throwable) {
-      this.bundle = Optional.fromNullable(producedBundle);
-      this.throwable = Optional.fromNullable(throwable);
-    }
-
-    public Optional<? extends CommittedBundle<?>> getBundle() {
-      return bundle;
-    }
-
-    public Optional<? extends Throwable> getException() {
-      return throwable;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(ExecutorUpdate.class)
-          .add("bundle", bundle)
-          .add("exception", throwable)
-          .toString();
-    }
-  }
-
-  /**
-   * An update of interest to the user. Used in {@link #awaitCompletion} to decide whether to
-   * return normally or throw an exception.
-   */
-  private static class VisibleExecutorUpdate {
-    private final Optional<? extends Throwable> throwable;
-    private final boolean done;
-
-    public static VisibleExecutorUpdate fromThrowable(Throwable e) {
-      return new VisibleExecutorUpdate(false, e);
-    }
-
-    public static VisibleExecutorUpdate finished() {
-      return new VisibleExecutorUpdate(true, null);
-    }
-
-    private VisibleExecutorUpdate(boolean done, @Nullable Throwable exception) {
-      this.throwable = Optional.fromNullable(exception);
-      this.done = done;
-    }
-
-    public boolean isDone() {
-      return done;
-    }
-  }
-
-  private class MonitorRunnable implements Runnable {
-    private final String runnableName =
-        String.format(
-            "%s$%s-monitor",
-            evaluationContext.getPipelineOptions().getAppName(),
-            ExecutorServiceParallelExecutor.class.getSimpleName());
-
-    @Override
-    public void run() {
-      String oldName = Thread.currentThread().getName();
-      Thread.currentThread().setName(runnableName);
-      try {
-        ExecutorUpdate update = allUpdates.poll();
-        // pull all of the pending work off of the queue
-        while (update != null) {
-          LOG.debug("Executor Update: {}", update);
-          if (update.getBundle().isPresent()) {
-            scheduleConsumers(update.getBundle().get());
-          } else if (update.getException().isPresent()) {
-            visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(update.getException().get()));
-          }
-          update = allUpdates.poll();
-        }
-        boolean timersFired = fireTimers();
-        addWorkIfNecessary(timersFired);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        LOG.error("Monitor died due to being interrupted");
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(e))) {
-          visibleUpdates.poll();
-        }
-      } catch (Throwable t) {
-        LOG.error("Monitor thread died due to throwable", t);
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.fromThrowable(t))) {
-          visibleUpdates.poll();
-        }
-      } finally {
-        if (!shouldShutdown()) {
-          // The monitor thread should always be scheduled; but we only need to be scheduled once
-          executorService.submit(this);
-        }
-        Thread.currentThread().setName(oldName);
-      }
-    }
-
-    /**
-     * Fires any available timers. Returns true if at least one timer was fired.
-     */
-    private boolean fireTimers() throws Exception {
-      try {
-        boolean firedTimers = false;
-        for (Map.Entry<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> transformTimers :
-            evaluationContext.extractFiredTimers().entrySet()) {
-          AppliedPTransform<?, ?, ?> transform = transformTimers.getKey();
-          for (Map.Entry<Object, FiredTimers> keyTimers : transformTimers.getValue().entrySet()) {
-            for (TimeDomain domain : TimeDomain.values()) {
-              Collection<TimerData> delivery = keyTimers.getValue().getTimers(domain);
-              if (delivery.isEmpty()) {
-                continue;
-              }
-              KeyedWorkItem<Object, Object> work =
-                  KeyedWorkItems.timersWorkItem(keyTimers.getKey(), delivery);
-              @SuppressWarnings({"unchecked", "rawtypes"})
-              CommittedBundle<?> bundle =
-                  evaluationContext
-                      .createKeyedBundle(
-                          null, keyTimers.getKey(), (PCollection) transform.getInput())
-                      .add(WindowedValue.valueInEmptyWindows(work))
-                      .commit(Instant.now());
-              scheduleConsumption(transform, bundle, new TimerCompletionCallback(delivery));
-              firedTimers = true;
-            }
-          }
-        }
-        return firedTimers;
-      } catch (Exception e) {
-        LOG.error("Internal Error while delivering timers", e);
-        throw e;
-      }
-    }
-
-    private boolean shouldShutdown() {
-      if (evaluationContext.isDone()) {
-        LOG.debug("Pipeline is finished. Shutting down. {}");
-        while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) {
-          visibleUpdates.poll();
-        }
-        executorService.shutdown();
-        return true;
-      }
-      return false;
-    }
-
-    /**
-     * If all active {@link TransformExecutor TransformExecutors} are in a blocked state,
-     * add more work from root nodes that may have additional work. This ensures that if a pipeline
-     * has elements available from the root nodes it will add those elements when necessary.
-     */
-    private void addWorkIfNecessary(boolean firedTimers) {
-      // If any timers have fired, they will add more work; We don't need to add more
-      if (firedTimers) {
-        return;
-      }
-      for (TransformExecutor<?> executor : scheduledExecutors.keySet()) {
-        if (!isExecutorBlocked(executor)) {
-          // We have at least one executor that can proceed without adding additional work
-          return;
-        }
-      }
-      // All current TransformExecutors are blocked; add more work from the roots.
-      for (AppliedPTransform<?, ?, ?> root : rootNodes) {
-        if (!evaluationContext.isDone(root)) {
-          scheduleConsumption(root, null, defaultCompletionCallback);
-        }
-      }
-    }
-
-    /**
-     * Return true if the provided executor might make more progress if no action is taken.
-     *
-     * <p>May return false even if all executor threads are currently blocked or cleaning up, as
-     * these can cause more work to be scheduled. If this does not occur, after these calls
-     * terminate, future calls will return true if all executors are waiting.
-     */
-    private boolean isExecutorBlocked(TransformExecutor<?> executor) {
-      Thread thread = executor.getThread();
-      if (thread == null) {
-        return false;
-      }
-      switch (thread.getState()) {
-        case TERMINATED:
-          throw new IllegalStateException(String.format(
-              "Unexpectedly encountered a Terminated TransformExecutor %s", executor));
-        case WAITING:
-        case TIMED_WAITING:
-          // The thread is waiting for some external input. Adding more work may cause the thread
-          // to stop waiting (e.g. the thread is waiting on an unbounded side input)
-          return true;
-        case BLOCKED:
-          // The executor is blocked on acquisition of a java monitor. This usually means it is
-          // making a call to the EvaluationContext, but not a model-blocking call - and will
-          // eventually complete, at which point we may reevaluate.
-        default:
-          // NEW and RUNNABLE threads can make progress
-          return false;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
deleted file mode 100644
index 4e23dde..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/FlattenEvaluatorFactory.java
+++ /dev/null
@@ -1,85 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
- * {@link PTransform}.
- */
-class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
-            (AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
-      final AppliedPTransform<
-              PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
-          application,
-      final CommittedBundle<InputT> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
-    if (inputBundle == null) {
-      // it is impossible to call processElement on a flatten with no input bundle. A Flatten with
-      // no input bundle occurs as an output of Flatten.pcollections(PCollectionList.empty())
-      return new FlattenEvaluator<>(
-          null, StepTransformResult.withoutHold(application).build());
-    }
-    final UncommittedBundle<InputT> outputBundle =
-        evaluationContext.createBundle(inputBundle, application.getOutput());
-    final InProcessTransformResult result =
-        StepTransformResult.withoutHold(application).addOutput(outputBundle).build();
-    return new FlattenEvaluator<>(outputBundle, result);
-  }
-
-  private static class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final UncommittedBundle<InputT> outputBundle;
-    private final InProcessTransformResult result;
-
-    public FlattenEvaluator(
-        UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
-      this.outputBundle = outputBundle;
-      this.result = result;
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> element) {
-      outputBundle.add(element);
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() {
-      return result;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
deleted file mode 100644
index 85aa1c4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ForwardingPTransform.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.runners.inprocess;
-
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.TypedPValue;
-
-/**
- * A base class for implementing {@link PTransform} overrides, which behave identically to the
- * delegate transform but with overridden methods. Implementors are required to implement
- * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}.
- */
-public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
-    extends PTransform<InputT, OutputT> {
-  protected abstract PTransform<InputT, OutputT> delegate();
-
-  @Override
-  public OutputT apply(InputT input) {
-    return delegate().apply(input);
-  }
-
-  @Override
-  public void validate(InputT input) {
-    delegate().validate(input);
-  }
-
-  @Override
-  public String getName() {
-    return delegate().getName();
-  }
-
-  @Override
-  public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unused")
-      TypedPValue<T> output) throws CannotProvideCoderException {
-    return delegate().getDefaultOutputCoder(input, output);
-  }
-
-  @Override
-  public void populateDisplayData(DisplayData.Builder builder) {
-    delegate().populateDisplayData(builder);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
deleted file mode 100644
index 4cec841..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/GroupByKeyEvaluatorFactory.java
+++ /dev/null
@@ -1,274 +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.runners.inprocess;
-
-import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.StepTransformResult.Builder;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItemCoder;
-import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.SystemReduceFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKey}
- * {@link PTransform}.
- */
-class GroupByKeyEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<InputT> evaluator = createEvaluator(
-            (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private <K, V> TransformEvaluator<KV<K, WindowedValue<V>>> createEvaluator(
-      final AppliedPTransform<
-              PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-              InProcessGroupByKeyOnly<K, V>>
-          application,
-      final CommittedBundle<KV<K, V>> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
-    return new GroupByKeyEvaluator<K, V>(evaluationContext, inputBundle, application);
-  }
-
-  private static class GroupByKeyEvaluator<K, V>
-      implements TransformEvaluator<KV<K, WindowedValue<V>>> {
-    private final InProcessEvaluationContext evaluationContext;
-
-    private final CommittedBundle<KV<K, V>> inputBundle;
-    private final AppliedPTransform<
-            PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-            InProcessGroupByKeyOnly<K, V>>
-        application;
-    private final Coder<K> keyCoder;
-    private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
-
-    public GroupByKeyEvaluator(
-        InProcessEvaluationContext evaluationContext,
-        CommittedBundle<KV<K, V>> inputBundle,
-        AppliedPTransform<
-                PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
-                InProcessGroupByKeyOnly<K, V>>
-            application) {
-      this.evaluationContext = evaluationContext;
-      this.inputBundle = inputBundle;
-      this.application = application;
-
-      PCollection<KV<K, WindowedValue<V>>> input = application.getInput();
-      keyCoder = getKeyCoder(input.getCoder());
-      groupingMap = new HashMap<>();
-    }
-
-    private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
-      if (!(coder instanceof KvCoder)) {
-        throw new IllegalStateException();
-      }
-      @SuppressWarnings("unchecked")
-      Coder<K> keyCoder = ((KvCoder<K, WindowedValue<V>>) coder).getKeyCoder();
-      return keyCoder;
-    }
-
-    @Override
-    public void processElement(WindowedValue<KV<K, WindowedValue<V>>> element) {
-      KV<K, WindowedValue<V>> kv = element.getValue();
-      K key = kv.getKey();
-      byte[] encodedKey;
-      try {
-        encodedKey = encodeToByteArray(keyCoder, key);
-      } catch (CoderException exn) {
-        // TODO: Put in better element printing:
-        // truncate if too long.
-        throw new IllegalArgumentException(
-            String.format("unable to encode key %s of input to %s using %s", key, this, keyCoder),
-            exn);
-      }
-      GroupingKey<K> groupingKey = new GroupingKey<>(key, encodedKey);
-      List<WindowedValue<V>> values = groupingMap.get(groupingKey);
-      if (values == null) {
-        values = new ArrayList<WindowedValue<V>>();
-        groupingMap.put(groupingKey, values);
-      }
-      values.add(kv.getValue());
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() {
-      Builder resultBuilder = StepTransformResult.withoutHold(application);
-      for (Map.Entry<GroupingKey<K>, List<WindowedValue<V>>> groupedEntry :
-          groupingMap.entrySet()) {
-        K key = groupedEntry.getKey().key;
-        KeyedWorkItem<K, V> groupedKv =
-            KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
-        UncommittedBundle<KeyedWorkItem<K, V>> bundle =
-            evaluationContext.createKeyedBundle(inputBundle, key, application.getOutput());
-        bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
-        resultBuilder.addOutput(bundle);
-      }
-      return resultBuilder.build();
-    }
-
-    private static class GroupingKey<K> {
-      private K key;
-      private byte[] encodedKey;
-
-      public GroupingKey(K key, byte[] encodedKey) {
-        this.key = key;
-        this.encodedKey = encodedKey;
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (o instanceof GroupingKey) {
-          GroupingKey<?> that = (GroupingKey<?>) o;
-          return Arrays.equals(this.encodedKey, that.encodedKey);
-        } else {
-          return false;
-        }
-      }
-
-      @Override
-      public int hashCode() {
-        return Arrays.hashCode(encodedKey);
-      }
-    }
-  }
-
-  /**
-   * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
-   */
-  public static final class InProcessGroupByKeyOverrideFactory
-      implements PTransformOverrideFactory {
-    @Override
-    public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-        PTransform<InputT, OutputT> transform) {
-      if (transform instanceof GroupByKey) {
-        @SuppressWarnings({"rawtypes", "unchecked"})
-        PTransform<InputT, OutputT> override = new InProcessGroupByKey((GroupByKey) transform);
-        return override;
-      }
-      return transform;
-    }
-  }
-
-  /**
-   * An in-memory implementation of the {@link GroupByKey} primitive as a composite
-   * {@link PTransform}.
-   */
-  private static final class InProcessGroupByKey<K, V>
-      extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-    private final GroupByKey<K, V> original;
-
-    private InProcessGroupByKey(GroupByKey<K, V> from) {
-      this.original = from;
-    }
-
-    @Override
-    public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
-      return original;
-    }
-
-    @Override
-    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-      KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
-      // This operation groups by the combination of key and window,
-      // merging windows as needed, using the windows assigned to the
-      // key/value input elements and the window merge operation of the
-      // window function associated with the input PCollection.
-      WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
-      // Use the default GroupAlsoByWindow implementation
-      DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow =
-          groupAlsoByWindow(windowingStrategy, inputCoder.getValueCoder());
-
-      // By default, implement GroupByKey via a series of lower-level operations.
-      return input
-          // Make each input element's timestamp and assigned windows
-          // explicit, in the value part.
-          .apply(new ReifyTimestampsAndWindows<K, V>())
-
-          .apply(new InProcessGroupByKeyOnly<K, V>())
-          .setCoder(KeyedWorkItemCoder.of(inputCoder.getKeyCoder(),
-              inputCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
-
-          // Group each key's values by window, merging windows as needed.
-          .apply("GroupAlsoByWindow", ParDo.of(groupAlsoByWindow))
-
-          // And update the windowing strategy as appropriate.
-          .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
-          .setCoder(
-              KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
-    }
-
-    private <W extends BoundedWindow>
-        DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow(
-            final WindowingStrategy<?, W> windowingStrategy, final Coder<V> inputCoder) {
-      return GroupAlsoByWindowViaWindowSetDoFn.create(
-          windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
-    }
-  }
-
-  /**
-   * An implementation primitive to use in the evaluation of a {@link GroupByKey}
-   * {@link PTransform}.
-   */
-  public static final class InProcessGroupByKeyOnly<K, V>
-      extends PTransform<PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>> {
-    @Override
-    public PCollection<KeyedWorkItem<K, V>> apply(PCollection<KV<K, WindowedValue<V>>> input) {
-      return PCollection.<KeyedWorkItem<K, V>>createPrimitiveOutputInternal(
-          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
-    }
-
-    @VisibleForTesting
-    InProcessGroupByKeyOnly() {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
deleted file mode 100644
index 04ece1c..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityCheckingBundleFactory.java
+++ /dev/null
@@ -1,131 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.MutationDetector;
-import org.apache.beam.sdk.util.MutationDetectors;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.util.Throwables;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.SetMultimap;
-
-import org.joda.time.Instant;
-
-/**
- * A {@link BundleFactory} that ensures that elements added to it are not mutated after being
- * output. Immutability checks are enforced at the time {@link UncommittedBundle#commit(Instant)} is
- * called, checking the value at that time against the value at the time the element was added. All
- * elements added to the bundle will be encoded by the {@link Coder} of the underlying
- * {@link PCollection}.
- *
- * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
- * after it is added to an output {@link PCollection}.
- */
-class ImmutabilityCheckingBundleFactory implements BundleFactory {
-  /**
-   * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
-   * {@link BundleFactory} to create the output bundle.
-   */
-  public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
-    return new ImmutabilityCheckingBundleFactory(underlying);
-  }
-
-  private final BundleFactory underlying;
-
-  private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
-    this.underlying = checkNotNull(underlying);
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createRootBundle(output));
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createBundle(input, output));
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, Object key, PCollection<T> output) {
-    return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(input, key, output));
-  }
-
-  private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
-    private final UncommittedBundle<T> underlying;
-    private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
-    private Coder<T> coder;
-
-    public ImmutabilityEnforcingBundle(UncommittedBundle<T> underlying) {
-      this.underlying = underlying;
-      mutationDetectors = HashMultimap.create();
-      coder = getPCollection().getCoder();
-    }
-
-    @Override
-    public PCollection<T> getPCollection() {
-      return underlying.getPCollection();
-    }
-
-    @Override
-    public UncommittedBundle<T> add(WindowedValue<T> element) {
-      try {
-        mutationDetectors.put(
-            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
-      } catch (CoderException e) {
-        throw Throwables.propagate(e);
-      }
-      underlying.add(element);
-      return this;
-    }
-
-    @Override
-    public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
-      for (MutationDetector detector : mutationDetectors.values()) {
-        try {
-          detector.verifyUnmodified();
-        } catch (IllegalMutationException exn) {
-          throw UserCodeException.wrap(
-              new IllegalMutationException(
-                  String.format(
-                      "PTransform %s mutated value %s after it was output (new value was %s)."
-                          + " Values must not be mutated in any way after being output.",
-                      underlying.getPCollection().getProducingTransformInternal().getFullName(),
-                      exn.getSavedValue(),
-                      exn.getNewValue()),
-                  exn.getSavedValue(),
-                  exn.getNewValue(),
-                  exn));
-        }
-      }
-      return underlying.commit(synchronizedProcessingTime);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
deleted file mode 100644
index 2f21032..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ImmutabilityEnforcementFactory.java
+++ /dev/null
@@ -1,103 +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.runners.inprocess;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.IllegalMutationException;
-import org.apache.beam.sdk.util.MutationDetector;
-import org.apache.beam.sdk.util.MutationDetectors;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-
-import java.util.IdentityHashMap;
-import java.util.Map;
-
-/**
- * {@link ModelEnforcement} that enforces elements are not modified over the course of processing
- * an element.
- *
- * <p>Implies {@link EncodabilityEnforcment}.
- */
-class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
-  public static ModelEnforcementFactory create() {
-    return new ImmutabilityEnforcementFactory();
-  }
-
-  @Override
-  public <T> ModelEnforcement<T> forBundle(
-      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
-    return new ImmutabilityCheckingEnforcement<T>(input, consumer);
-  }
-
-  private static class ImmutabilityCheckingEnforcement<T> extends AbstractModelEnforcement<T> {
-    private final AppliedPTransform<?, ?, ?> transform;
-    private final Map<WindowedValue<T>, MutationDetector> mutationElements;
-    private final Coder<T> coder;
-
-    private ImmutabilityCheckingEnforcement(
-        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> transform) {
-      this.transform = transform;
-      coder = input.getPCollection().getCoder();
-      mutationElements = new IdentityHashMap<>();
-    }
-
-    @Override
-    public void beforeElement(WindowedValue<T> element) {
-      try {
-        mutationElements.put(
-            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
-      } catch (CoderException e) {
-        throw UserCodeException.wrap(e);
-      }
-    }
-
-    @Override
-    public void afterElement(WindowedValue<T> element) {
-      verifyUnmodified(mutationElements.get(element));
-    }
-
-    @Override
-    public void afterFinish(
-        CommittedBundle<T> input,
-        InProcessTransformResult result,
-        Iterable<? extends CommittedBundle<?>> outputs) {
-      for (MutationDetector detector : mutationElements.values()) {
-        verifyUnmodified(detector);
-      }
-    }
-
-    private void verifyUnmodified(MutationDetector detector) {
-      try {
-        detector.verifyUnmodified();
-      } catch (IllegalMutationException e) {
-        throw new IllegalMutationException(
-            String.format(
-                "PTransform %s illegaly mutated value %s of class %s."
-                    + " Input values must not be mutated in any way.",
-                transform.getFullName(),
-                e.getSavedValue(),
-                e.getSavedValue().getClass()),
-            e.getSavedValue(),
-            e.getNewValue());
-      }
-    }
-  }
-}


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

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


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

Branch: refs/heads/master
Commit: b9116ac426f989af882e6df5dafc5da6c9f203d8
Parents: bba4c64 e13cacb
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Apr 29 14:49:11 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Apr 29 14:49:11 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |    1 +
 runners/direct-java/pom.xml                     |  400 ++++++
 .../direct/AbstractModelEnforcement.java        |   38 +
 .../direct/AvroIOShardedWriteFactory.java       |   76 +
 .../direct/BoundedReadEvaluatorFactory.java     |  155 ++
 .../beam/runners/direct/BundleFactory.java      |   49 +
 .../CachedThreadPoolExecutorServiceFactory.java |   44 +
 .../org/apache/beam/runners/direct/Clock.java   |   30 +
 .../beam/runners/direct/CommittedResult.java    |   46 +
 .../beam/runners/direct/CompletionCallback.java |   36 +
 .../direct/ConsumerTrackingPipelineVisitor.java |  173 +++
 .../runners/direct/EmptyTransformEvaluator.java |   50 +
 .../direct/EncodabilityEnforcementFactory.java  |   70 +
 .../beam/runners/direct/EvaluatorKey.java       |   55 +
 .../runners/direct/ExecutorServiceFactory.java  |   33 +
 .../direct/ExecutorServiceParallelExecutor.java |  478 +++++++
 .../runners/direct/FlattenEvaluatorFactory.java |   85 ++
 .../runners/direct/ForwardingPTransform.java    |   62 +
 .../direct/GroupByKeyEvaluatorFactory.java      |  274 ++++
 .../ImmutabilityCheckingBundleFactory.java      |  131 ++
 .../direct/ImmutabilityEnforcementFactory.java  |  103 ++
 .../direct/InMemoryWatermarkManager.java        | 1327 ++++++++++++++++++
 .../runners/direct/InProcessBundleFactory.java  |  162 +++
 .../direct/InProcessBundleOutputManager.java    |   51 +
 .../direct/InProcessEvaluationContext.java      |  425 ++++++
 .../direct/InProcessExecutionContext.java       |  105 ++
 .../beam/runners/direct/InProcessExecutor.java  |   48 +
 .../direct/InProcessPipelineOptions.java        |  101 ++
 .../runners/direct/InProcessPipelineRunner.java |  370 +++++
 .../beam/runners/direct/InProcessRegistrar.java |   55 +
 .../direct/InProcessSideInputContainer.java     |  271 ++++
 .../runners/direct/InProcessTimerInternals.java |   84 ++
 .../direct/InProcessTransformResult.java        |   77 +
 .../direct/KeyedPValueTrackingVisitor.java      |   96 ++
 .../beam/runners/direct/ModelEnforcement.java   |   63 +
 .../runners/direct/ModelEnforcementFactory.java |   30 +
 .../beam/runners/direct/NanosOffsetClock.java   |   59 +
 .../direct/PTransformOverrideFactory.java       |   33 +
 .../runners/direct/ParDoInProcessEvaluator.java |  173 +++
 .../direct/ParDoMultiEvaluatorFactory.java      |   64 +
 .../direct/ParDoSingleEvaluatorFactory.java     |   63 +
 .../direct/PassthroughTransformEvaluator.java   |   49 +
 .../runners/direct/ShardControlledWrite.java    |   81 ++
 .../apache/beam/runners/direct/StepAndKey.java  |   71 +
 .../runners/direct/StepTransformResult.java     |  165 +++
 .../direct/TextIOShardedWriteFactory.java       |   78 +
 .../beam/runners/direct/TransformEvaluator.java |   46 +
 .../direct/TransformEvaluatorFactory.java       |   44 +
 .../direct/TransformEvaluatorRegistry.java      |   77 +
 .../beam/runners/direct/TransformExecutor.java  |  176 +++
 .../direct/TransformExecutorService.java        |   35 +
 .../direct/TransformExecutorServices.java       |  154 ++
 .../direct/UnboundedReadEvaluatorFactory.java   |  177 +++
 .../runners/direct/ViewEvaluatorFactory.java    |  145 ++
 .../direct/WatermarkCallbackExecutor.java       |  146 ++
 .../runners/direct/WindowEvaluatorFactory.java  |  131 ++
 .../direct/AvroIOShardedWriteFactoryTest.java   |  112 ++
 .../direct/BoundedReadEvaluatorFactoryTest.java |  290 ++++
 .../runners/direct/CommittedResultTest.java     |   77 +
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ++++
 .../EncodabilityEnforcementFactoryTest.java     |  257 ++++
 .../direct/FlattenEvaluatorFactoryTest.java     |  141 ++
 .../direct/ForwardingPTransformTest.java        |  112 ++
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  183 +++
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 +++
 .../ImmutabilityEnforcementFactoryTest.java     |  128 ++
 .../direct/InMemoryWatermarkManagerTest.java    | 1168 +++++++++++++++
 .../direct/InProcessBundleFactoryTest.java      |  223 +++
 .../direct/InProcessEvaluationContextTest.java  |  526 +++++++
 .../direct/InProcessPipelineRegistrarTest.java  |   74 +
 .../direct/InProcessPipelineRunnerTest.java     |   78 +
 .../direct/InProcessSideInputContainerTest.java |  496 +++++++
 .../direct/InProcessTimerInternalsTest.java     |  133 ++
 .../direct/KeyedPValueTrackingVisitorTest.java  |  192 +++
 .../apache/beam/runners/direct/MockClock.java   |   62 +
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  431 ++++++
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  324 +++++
 .../direct/TextIOShardedWriteFactoryTest.java   |  112 ++
 .../direct/TransformExecutorServicesTest.java   |  136 ++
 .../runners/direct/TransformExecutorTest.java   |  538 +++++++
 .../UnboundedReadEvaluatorFactoryTest.java      |  334 +++++
 .../direct/ViewEvaluatorFactoryTest.java        |  101 ++
 .../direct/WatermarkCallbackExecutorTest.java   |  128 ++
 .../direct/WindowEvaluatorFactoryTest.java      |  222 +++
 runners/pom.xml                                 |    1 +
 .../inprocess/AbstractModelEnforcement.java     |   38 -
 .../inprocess/AvroIOShardedWriteFactory.java    |   76 -
 .../inprocess/BoundedReadEvaluatorFactory.java  |  155 --
 .../sdk/runners/inprocess/BundleFactory.java    |   49 -
 .../CachedThreadPoolExecutorServiceFactory.java |   44 -
 .../beam/sdk/runners/inprocess/Clock.java       |   30 -
 .../sdk/runners/inprocess/CommittedResult.java  |   46 -
 .../runners/inprocess/CompletionCallback.java   |   36 -
 .../ConsumerTrackingPipelineVisitor.java        |  173 ---
 .../inprocess/EmptyTransformEvaluator.java      |   50 -
 .../EncodabilityEnforcementFactory.java         |   70 -
 .../sdk/runners/inprocess/EvaluatorKey.java     |   55 -
 .../inprocess/ExecutorServiceFactory.java       |   33 -
 .../ExecutorServiceParallelExecutor.java        |  478 -------
 .../inprocess/FlattenEvaluatorFactory.java      |   85 --
 .../runners/inprocess/ForwardingPTransform.java |   62 -
 .../inprocess/GroupByKeyEvaluatorFactory.java   |  274 ----
 .../ImmutabilityCheckingBundleFactory.java      |  131 --
 .../ImmutabilityEnforcementFactory.java         |  103 --
 .../inprocess/InMemoryWatermarkManager.java     | 1327 ------------------
 .../inprocess/InProcessBundleFactory.java       |  162 ---
 .../inprocess/InProcessBundleOutputManager.java |   51 -
 .../inprocess/InProcessEvaluationContext.java   |  425 ------
 .../inprocess/InProcessExecutionContext.java    |  105 --
 .../runners/inprocess/InProcessExecutor.java    |   48 -
 .../inprocess/InProcessPipelineOptions.java     |  101 --
 .../inprocess/InProcessPipelineRunner.java      |  370 -----
 .../runners/inprocess/InProcessRegistrar.java   |   55 -
 .../inprocess/InProcessSideInputContainer.java  |  271 ----
 .../inprocess/InProcessTimerInternals.java      |   84 --
 .../inprocess/InProcessTransformResult.java     |   77 -
 .../inprocess/KeyedPValueTrackingVisitor.java   |   96 --
 .../sdk/runners/inprocess/ModelEnforcement.java |   63 -
 .../inprocess/ModelEnforcementFactory.java      |   30 -
 .../sdk/runners/inprocess/NanosOffsetClock.java |   59 -
 .../inprocess/PTransformOverrideFactory.java    |   33 -
 .../inprocess/ParDoInProcessEvaluator.java      |  173 ---
 .../inprocess/ParDoMultiEvaluatorFactory.java   |   63 -
 .../inprocess/ParDoSingleEvaluatorFactory.java  |   63 -
 .../PassthroughTransformEvaluator.java          |   49 -
 .../runners/inprocess/ShardControlledWrite.java |   81 --
 .../beam/sdk/runners/inprocess/StepAndKey.java  |   71 -
 .../runners/inprocess/StepTransformResult.java  |  165 ---
 .../inprocess/TextIOShardedWriteFactory.java    |   78 -
 .../runners/inprocess/TransformEvaluator.java   |   46 -
 .../inprocess/TransformEvaluatorFactory.java    |   44 -
 .../inprocess/TransformEvaluatorRegistry.java   |   77 -
 .../runners/inprocess/TransformExecutor.java    |  176 ---
 .../inprocess/TransformExecutorService.java     |   35 -
 .../inprocess/TransformExecutorServices.java    |  154 --
 .../UnboundedReadEvaluatorFactory.java          |  177 ---
 .../runners/inprocess/ViewEvaluatorFactory.java |  145 --
 .../inprocess/WatermarkCallbackExecutor.java    |  146 --
 .../inprocess/WindowEvaluatorFactory.java       |  131 --
 .../AvroIOShardedWriteFactoryTest.java          |  112 --
 .../BoundedReadEvaluatorFactoryTest.java        |  290 ----
 .../runners/inprocess/CommittedResultTest.java  |   77 -
 .../ConsumerTrackingPipelineVisitorTest.java    |  272 ----
 .../EncodabilityEnforcementFactoryTest.java     |  257 ----
 .../inprocess/FlattenEvaluatorFactoryTest.java  |  141 --
 .../inprocess/ForwardingPTransformTest.java     |  112 --
 .../GroupByKeyEvaluatorFactoryTest.java         |  183 ---
 .../ImmutabilityCheckingBundleFactoryTest.java  |  220 ---
 .../ImmutabilityEnforcementFactoryTest.java     |  128 --
 .../inprocess/InMemoryWatermarkManagerTest.java | 1168 ---------------
 .../inprocess/InProcessBundleFactoryTest.java   |  223 ---
 .../InProcessEvaluationContextTest.java         |  526 -------
 .../InProcessPipelineRegistrarTest.java         |   74 -
 .../inprocess/InProcessPipelineRunnerTest.java  |   78 -
 .../InProcessSideInputContainerTest.java        |  496 -------
 .../inprocess/InProcessTimerInternalsTest.java  |  133 --
 .../KeyedPValueTrackingVisitorTest.java         |  192 ---
 .../beam/sdk/runners/inprocess/MockClock.java   |   62 -
 .../ParDoMultiEvaluatorFactoryTest.java         |  431 ------
 .../ParDoSingleEvaluatorFactoryTest.java        |  324 -----
 .../TextIOShardedWriteFactoryTest.java          |  112 --
 .../TransformExecutorServicesTest.java          |  136 --
 .../inprocess/TransformExecutorTest.java        |  538 -------
 .../UnboundedReadEvaluatorFactoryTest.java      |  334 -----
 .../inprocess/ViewEvaluatorFactoryTest.java     |  101 --
 .../WatermarkCallbackExecutorTest.java          |  128 --
 .../inprocess/WindowEvaluatorFactoryTest.java   |  222 ---
 sdks/java/pom.xml                               |    1 -
 168 files changed, 14692 insertions(+), 14290 deletions(-)
----------------------------------------------------------------------



[06/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.java
deleted file mode 100644
index bdc525a..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineOptions.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.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.Hidden;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.Validation.Required;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Options that can be used to configure the {@link InProcessPipelineRunner}.
- */
-public interface InProcessPipelineOptions extends PipelineOptions, ApplicationNameOptions {
-  /**
-   * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService}
-   * to execute {@link PTransform PTransforms}.
-   *
-   * <p>Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that
-   * it cannot enter a state in which it will not schedule additional pending work unless currently
-   * scheduled work completes, as this may cause the {@link Pipeline} to cease processing.
-   *
-   * <p>Defaults to a {@link CachedThreadPoolExecutorServiceFactory}, which produces instances of
-   * {@link Executors#newCachedThreadPool()}.
-   */
-  @JsonIgnore
-  @Required
-  @Hidden
-  @Default.InstanceFactory(CachedThreadPoolExecutorServiceFactory.class)
-  ExecutorServiceFactory getExecutorServiceFactory();
-
-  void setExecutorServiceFactory(ExecutorServiceFactory executorService);
-
-  /**
-   * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the
-   * system time when time values are required by the evaluator.
-   */
-  @Default.InstanceFactory(NanosOffsetClock.Factory.class)
-  @JsonIgnore
-  @Required
-  @Hidden
-  @Description(
-      "The processing time source used by the pipeline. When the current time is "
-          + "needed by the evaluator, the result of clock#now() is used.")
-  Clock getClock();
-
-  void setClock(Clock clock);
-
-  @Default.Boolean(false)
-  @Description(
-      "If the pipeline should shut down producers which have reached the maximum "
-          + "representable watermark. If this is set to true, a pipeline in which all PTransforms "
-          + "have reached the maximum watermark will be shut down, even if there are unbounded "
-          + "sources that could produce additional (late) data. By default, if the pipeline "
-          + "contains any unbounded PCollections, it will run until explicitly shut down.")
-  boolean isShutdownUnboundedProducersWithMaxWatermark();
-
-  void setShutdownUnboundedProducersWithMaxWatermark(boolean shutdown);
-
-  @Default.Boolean(true)
-  @Description(
-      "If the pipeline should block awaiting completion of the pipeline. If set to true, "
-          + "a call to Pipeline#run() will block until all PTransforms are complete. Otherwise, "
-          + "the Pipeline will execute asynchronously. If set to false, the completion of the "
-          + "pipeline can be awaited on by use of InProcessPipelineResult#awaitCompletion().")
-  boolean isBlockOnRun();
-
-  void setBlockOnRun(boolean b);
-
-  @Default.Boolean(true)
-  @Description(
-      "Controls whether the runner should ensure that all of the elements of every "
-          + "PCollection are not mutated. PTransforms are not permitted to mutate input elements "
-          + "at any point, or output elements after they are output.")
-  boolean isTestImmutability();
-
-  void setTestImmutability(boolean test);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
deleted file mode 100644
index 7897f2e..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessPipelineRunner.java
+++ /dev/null
@@ -1,370 +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.runners.inprocess;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
-import org.apache.beam.sdk.runners.AggregatorRetrievalException;
-import org.apache.beam.sdk.runners.AggregatorValues;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly;
-import org.apache.beam.sdk.runners.inprocess.GroupByKeyEvaluatorFactory.InProcessGroupByKeyOverrideFactory;
-import org.apache.beam.sdk.runners.inprocess.ViewEvaluatorFactory.InProcessViewOverrideFactory;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.util.MapAggregatorValues;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.Counter;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-
-import javax.annotation.Nullable;
-
-/**
- * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
- * {@link PCollection PCollections}.
- */
-@Experimental
-public class InProcessPipelineRunner
-    extends PipelineRunner<InProcessPipelineRunner.InProcessPipelineResult> {
-  /**
-   * The default set of transform overrides to use in the {@link InProcessPipelineRunner}.
-   *
-   * <p>A transform override must have a single-argument constructor that takes an instance of the
-   * type of transform it is overriding.
-   */
-  @SuppressWarnings("rawtypes")
-  private static Map<Class<? extends PTransform>, PTransformOverrideFactory>
-      defaultTransformOverrides =
-          ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
-              .put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
-              .put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
-              .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
-              .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
-              .build();
-
-  /**
-   * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
-   * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is
-   * a part of at a later point. This is an uncommitted bundle and can have elements added to it.
-   *
-   * @param <T> the type of elements that can be added to this bundle
-   */
-  public static interface UncommittedBundle<T> {
-    /**
-     * Returns the PCollection that the elements of this {@link UncommittedBundle} belong to.
-     */
-    PCollection<T> getPCollection();
-
-    /**
-     * Outputs an element to this bundle.
-     *
-     * @param element the element to add to this bundle
-     * @return this bundle
-     */
-    UncommittedBundle<T> add(WindowedValue<T> element);
-
-    /**
-     * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
-     * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
-     * will throw an {@link IllegalStateException} if called after a call to commit.
-     * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
-     *                                   committed
-     */
-    CommittedBundle<T> commit(Instant synchronizedProcessingTime);
-  }
-
-  /**
-   * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
-   * eventually committed. Committed elements are executed by the {@link PTransform PTransforms}
-   * that consume the {@link PCollection} this bundle is
-   * a part of at a later point.
-   * @param <T> the type of elements contained within this bundle
-   */
-  public static interface CommittedBundle<T> {
-    /**
-     * Returns the PCollection that the elements of this bundle belong to.
-     */
-    PCollection<T> getPCollection();
-
-    /**
-     * Returns the (possibly null) key that was output in the most recent {@link GroupByKey} in the
-     * execution of this bundle.
-     */
-    @Nullable
-    Object getKey();
-
-    /**
-     * Returns an {@link Iterable} containing all of the elements that have been added to this
-     * {@link CommittedBundle}.
-     */
-    Iterable<WindowedValue<T>> getElements();
-
-    /**
-     * Returns the processing time output watermark at the time the producing {@link PTransform}
-     * committed this bundle. Downstream synchronized processing time watermarks cannot progress
-     * past this point before consuming this bundle.
-     *
-     * <p>This value is no greater than the earliest incomplete processing time or synchronized
-     * processing time {@link TimerData timer} at the time this bundle was committed, including any
-     * timers that fired to produce this bundle.
-     */
-    Instant getSynchronizedProcessingOutputWatermark();
-
-    /**
-     * Return a new {@link CommittedBundle} that is like this one, except calls to
-     * {@link #getElements()} will return the provided elements. This bundle is unchanged.
-     *
-     * <p>
-     * The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
-     * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
-     * the current bundle. This is used to ensure a {@link PTransform} that could not complete
-     * processing on input elements properly holds the synchronized processing time to the
-     * appropriate value.
-     */
-    CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
-  }
-
-  /**
-   * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to
-   * a storage mechanism that can be read from while constructing a {@link PCollectionView}.
-   * @param <ElemT> the type of elements the input {@link PCollection} contains.
-   * @param <ViewT> the type of the PCollectionView this writer writes to.
-   */
-  public static interface PCollectionViewWriter<ElemT, ViewT> {
-    void add(Iterable<WindowedValue<ElemT>> values);
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-  private final InProcessPipelineOptions options;
-
-  public static InProcessPipelineRunner fromOptions(PipelineOptions options) {
-    return new InProcessPipelineRunner(options.as(InProcessPipelineOptions.class));
-  }
-
-  private InProcessPipelineRunner(InProcessPipelineOptions options) {
-    this.options = options;
-  }
-
-  /**
-   * Returns the {@link PipelineOptions} used to create this {@link InProcessPipelineRunner}.
-   */
-  public InProcessPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass());
-    if (overrideFactory != null) {
-      PTransform<InputT, OutputT> customTransform = overrideFactory.override(transform);
-
-      return super.apply(customTransform, input);
-    }
-    // If there is no override, or we should not apply the override, apply the original transform
-    return super.apply(transform, input);
-  }
-
-  @Override
-  public InProcessPipelineResult run(Pipeline pipeline) {
-    ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
-    pipeline.traverseTopologically(consumerTrackingVisitor);
-    for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
-      unfinalized.finishSpecifying();
-    }
-    @SuppressWarnings("rawtypes")
-    KeyedPValueTrackingVisitor keyedPValueVisitor =
-        KeyedPValueTrackingVisitor.create(
-            ImmutableSet.<Class<? extends PTransform>>of(
-                GroupByKey.class, InProcessGroupByKeyOnly.class));
-    pipeline.traverseTopologically(keyedPValueVisitor);
-
-    InProcessEvaluationContext context =
-        InProcessEvaluationContext.create(
-            getPipelineOptions(),
-            createBundleFactory(getPipelineOptions()),
-            consumerTrackingVisitor.getRootTransforms(),
-            consumerTrackingVisitor.getValueToConsumers(),
-            consumerTrackingVisitor.getStepNames(),
-            consumerTrackingVisitor.getViews());
-
-    // independent executor service for each run
-    ExecutorService executorService =
-        context.getPipelineOptions().getExecutorServiceFactory().create();
-    InProcessExecutor executor =
-        ExecutorServiceParallelExecutor.create(
-            executorService,
-            consumerTrackingVisitor.getValueToConsumers(),
-            keyedPValueVisitor.getKeyedPValues(),
-            TransformEvaluatorRegistry.defaultRegistry(),
-            defaultModelEnforcements(options),
-            context);
-    executor.start(consumerTrackingVisitor.getRootTransforms());
-
-    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
-        new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
-    InProcessPipelineResult result =
-        new InProcessPipelineResult(executor, context, aggregatorSteps);
-    if (options.isBlockOnRun()) {
-      try {
-        result.awaitCompletion();
-      } catch (UserCodeException userException) {
-        throw new PipelineExecutionException(userException.getCause());
-      } catch (Throwable t) {
-        Throwables.propagate(t);
-      }
-    }
-    return result;
-  }
-
-  private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-      defaultModelEnforcements(InProcessPipelineOptions options) {
-    ImmutableMap.Builder<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
-        enforcements = ImmutableMap.builder();
-    Collection<ModelEnforcementFactory> parDoEnforcements = createParDoEnforcements(options);
-    enforcements.put(ParDo.Bound.class, parDoEnforcements);
-    enforcements.put(ParDo.BoundMulti.class, parDoEnforcements);
-    return enforcements.build();
-  }
-
-  private Collection<ModelEnforcementFactory> createParDoEnforcements(
-      InProcessPipelineOptions options) {
-    ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
-    if (options.isTestImmutability()) {
-      enforcements.add(ImmutabilityEnforcementFactory.create());
-    }
-    return enforcements.build();
-  }
-
-  private BundleFactory createBundleFactory(InProcessPipelineOptions pipelineOptions) {
-    BundleFactory bundleFactory = InProcessBundleFactory.create();
-    if (pipelineOptions.isTestImmutability()) {
-      bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
-    }
-    return bundleFactory;
-  }
-
-  /**
-   * The result of running a {@link Pipeline} with the {@link InProcessPipelineRunner}.
-   *
-   * Throws {@link UnsupportedOperationException} for all methods.
-   */
-  public static class InProcessPipelineResult implements PipelineResult {
-    private final InProcessExecutor executor;
-    private final InProcessEvaluationContext evaluationContext;
-    private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
-    private State state;
-
-    private InProcessPipelineResult(
-        InProcessExecutor executor,
-        InProcessEvaluationContext evaluationContext,
-        Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
-      this.executor = executor;
-      this.evaluationContext = evaluationContext;
-      this.aggregatorSteps = aggregatorSteps;
-      // Only ever constructed after the executor has started.
-      this.state = State.RUNNING;
-    }
-
-    @Override
-    public State getState() {
-      return state;
-    }
-
-    @Override
-    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-        throws AggregatorRetrievalException {
-      CounterSet counters = evaluationContext.getCounters();
-      Collection<PTransform<?, ?>> steps = aggregatorSteps.get(aggregator);
-      Map<String, T> stepValues = new HashMap<>();
-      for (AppliedPTransform<?, ?, ?> transform : evaluationContext.getSteps()) {
-        if (steps.contains(transform.getTransform())) {
-          String stepName =
-              String.format(
-                  "user-%s-%s", evaluationContext.getStepName(transform), aggregator.getName());
-          Counter<T> counter = (Counter<T>) counters.getExistingCounter(stepName);
-          if (counter != null) {
-            stepValues.put(transform.getFullName(), counter.getAggregate());
-          }
-        }
-      }
-      return new MapAggregatorValues<>(stepValues);
-    }
-
-    /**
-     * Blocks until the {@link Pipeline} execution represented by this
-     * {@link InProcessPipelineResult} is complete, returning the terminal state.
-     *
-     * <p>If the pipeline terminates abnormally by throwing an exception, this will rethrow the
-     * exception. Future calls to {@link #getState()} will return
-     * {@link org.apache.beam.sdk.PipelineResult.State#FAILED}.
-     *
-     * <p>NOTE: if the {@link Pipeline} contains an {@link IsBounded#UNBOUNDED unbounded}
-     * {@link PCollection}, and the {@link PipelineRunner} was created with
-     * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
-     * this method will never return.
-     *
-     * See also {@link InProcessExecutor#awaitCompletion()}.
-     */
-    public State awaitCompletion() throws Throwable {
-      if (!state.isTerminal()) {
-        try {
-          executor.awaitCompletion();
-          state = State.DONE;
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw e;
-        } catch (Throwable t) {
-          state = State.FAILED;
-          throw t;
-        }
-      }
-      return state;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.java
deleted file mode 100644
index 8d29d01..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessRegistrar.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.runners.inprocess;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
- * {@link InProcessPipelineRunner}.
- */
-public class InProcessRegistrar {
-  private InProcessRegistrar() {}
-  /**
-   * Registers the {@link InProcessPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class InProcessRunner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(InProcessPipelineRunner.class);
-    }
-  }
-
-  /**
-   * Registers the {@link InProcessPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class InProcessOptions implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(InProcessPipelineOptions.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
deleted file mode 100644
index fda78fc..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessSideInputContainer.java
+++ /dev/null
@@ -1,271 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessEvaluationContext.ReadyCheckingSideInputReader;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.PCollectionViewWindow;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Throwables;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.SettableFuture;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import javax.annotation.Nullable;
-
-/**
- * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for
- * constructing {@link SideInputReader SideInputReaders} which block until a side input is
- * available and writing to a {@link PCollectionView}.
- */
-class InProcessSideInputContainer {
-  private final InProcessEvaluationContext evaluationContext;
-  private final Collection<PCollectionView<?>> containedViews;
-  private final LoadingCache<PCollectionViewWindow<?>,
-      SettableFuture<Iterable<? extends WindowedValue<?>>>> viewByWindows;
-
-  /**
-   * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
-   * context.
-   */
-  public static InProcessSideInputContainer create(
-      InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
-    CacheLoader<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-        loader = new CacheLoader<PCollectionViewWindow<?>,
-            SettableFuture<Iterable<? extends WindowedValue<?>>>>() {
-          @Override
-          public SettableFuture<Iterable<? extends WindowedValue<?>>> load(
-              PCollectionViewWindow<?> view) {
-            return SettableFuture.create();
-          }
-        };
-    LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-        viewByWindows = CacheBuilder.newBuilder().build(loader);
-    return new InProcessSideInputContainer(context, containedViews, viewByWindows);
-  }
-
-  private InProcessSideInputContainer(InProcessEvaluationContext context,
-      Collection<PCollectionView<?>> containedViews,
-      LoadingCache<PCollectionViewWindow<?>, SettableFuture<Iterable<? extends WindowedValue<?>>>>
-      viewByWindows) {
-    this.evaluationContext = context;
-    this.containedViews = ImmutableSet.copyOf(containedViews);
-    this.viewByWindows = viewByWindows;
-  }
-
-  /**
-   * Return a view of this {@link InProcessSideInputContainer} that contains only the views in the
-   * provided argument. The returned {@link InProcessSideInputContainer} is unmodifiable without
-   * casting, but will change as this {@link InProcessSideInputContainer} is modified.
-   */
-  public ReadyCheckingSideInputReader createReaderForViews(
-      Collection<PCollectionView<?>> newContainedViews) {
-    if (!containedViews.containsAll(newContainedViews)) {
-      Set<PCollectionView<?>> currentlyContained = ImmutableSet.copyOf(containedViews);
-      Set<PCollectionView<?>> newRequested = ImmutableSet.copyOf(newContainedViews);
-      throw new IllegalArgumentException("Can't create a SideInputReader with unknown views "
-          + Sets.difference(newRequested, currentlyContained));
-    }
-    return new SideInputContainerSideInputReader(newContainedViews);
-  }
-
-  /**
-   * Write the provided values to the provided view.
-   *
-   * <p>The windowed values are first exploded, then for each window the pane is determined. For
-   * each window, if the pane is later than the current pane stored within this container, write
-   * all of the values to the container as the new values of the {@link PCollectionView}.
-   *
-   * <p>The provided iterable is expected to contain only a single window and pane.
-   */
-  public void write(PCollectionView<?> view, Iterable<? extends WindowedValue<?>> values) {
-    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow =
-        indexValuesByWindow(values);
-    for (Map.Entry<BoundedWindow, Collection<WindowedValue<?>>> windowValues :
-        valuesPerWindow.entrySet()) {
-      updatePCollectionViewWindowValues(view, windowValues.getKey(), windowValues.getValue());
-    }
-  }
-
-  /**
-   * Index the provided values by all {@link BoundedWindow windows} in which they appear.
-   */
-  private Map<BoundedWindow, Collection<WindowedValue<?>>> indexValuesByWindow(
-      Iterable<? extends WindowedValue<?>> values) {
-    Map<BoundedWindow, Collection<WindowedValue<?>>> valuesPerWindow = new HashMap<>();
-    for (WindowedValue<?> value : values) {
-      for (BoundedWindow window : value.getWindows()) {
-        Collection<WindowedValue<?>> windowValues = valuesPerWindow.get(window);
-        if (windowValues == null) {
-          windowValues = new ArrayList<>();
-          valuesPerWindow.put(window, windowValues);
-        }
-        windowValues.add(value);
-      }
-    }
-    return valuesPerWindow;
-  }
-
-  /**
-   * Set the value of the {@link PCollectionView} in the {@link BoundedWindow} to be based on the
-   * specified values, if the values are part of a later pane than currently exist within the
-   * {@link PCollectionViewWindow}.
-   */
-  private void updatePCollectionViewWindowValues(
-      PCollectionView<?> view, BoundedWindow window, Collection<WindowedValue<?>> windowValues) {
-    PCollectionViewWindow<?> windowedView = PCollectionViewWindow.of(view, window);
-    SettableFuture<Iterable<? extends WindowedValue<?>>> future = null;
-    try {
-      future = viewByWindows.get(windowedView);
-      if (future.isDone()) {
-        Iterator<? extends WindowedValue<?>> existingValues = future.get().iterator();
-        PaneInfo newPane = windowValues.iterator().next().getPane();
-        // The current value may have no elements, if no elements were produced for the window,
-        // but we are recieving late data.
-        if (!existingValues.hasNext()
-            || newPane.getIndex() > existingValues.next().getPane().getIndex()) {
-          viewByWindows.invalidate(windowedView);
-          viewByWindows.get(windowedView).set(windowValues);
-        }
-      } else {
-        future.set(windowValues);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      if (future != null && !future.isDone()) {
-        future.set(Collections.<WindowedValue<?>>emptyList());
-      }
-    } catch (ExecutionException e) {
-      Throwables.propagate(e.getCause());
-    }
-  }
-
-  private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
-    private final Collection<PCollectionView<?>> readerViews;
-
-    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
-      this.readerViews = ImmutableSet.copyOf(readerViews);
-    }
-
-    @Override
-    public boolean isReady(final PCollectionView<?> view, final BoundedWindow window) {
-      checkArgument(
-          readerViews.contains(view),
-          "Tried to check if view %s was ready in a SideInputReader that does not contain it. "
-              + "Contained views; %s",
-          view,
-          readerViews);
-      return getViewFuture(view, window).isDone();
-    }
-
-    @Override
-    @Nullable
-    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
-      checkArgument(
-          readerViews.contains(view), "calling get(PCollectionView) with unknown view: " + view);
-      try {
-        final Future<Iterable<? extends WindowedValue<?>>> future = getViewFuture(view, window);
-        // Safe covariant cast
-        @SuppressWarnings("unchecked")
-        Iterable<WindowedValue<?>> values = (Iterable<WindowedValue<?>>) future.get();
-        return view.fromIterableInternal(values);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        return null;
-      } catch (ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    /**
-     * Gets the future containing the contents of the provided {@link PCollectionView} in the
-     * provided {@link BoundedWindow}, setting up a callback to populate the future with empty
-     * contents if necessary.
-     */
-    private <T> Future<Iterable<? extends WindowedValue<?>>> getViewFuture(
-        final PCollectionView<T> view, final BoundedWindow window)  {
-      PCollectionViewWindow<T> windowedView = PCollectionViewWindow.of(view, window);
-      final SettableFuture<Iterable<? extends WindowedValue<?>>> future =
-          viewByWindows.getUnchecked(windowedView);
-
-      WindowingStrategy<?, ?> windowingStrategy = view.getWindowingStrategyInternal();
-      evaluationContext.scheduleAfterOutputWouldBeProduced(
-          view, window, windowingStrategy, new WriteEmptyViewContents(view, window, future));
-      return future;
-    }
-
-    @Override
-    public <T> boolean contains(PCollectionView<T> view) {
-      return readerViews.contains(view);
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return readerViews.isEmpty();
-    }
-  }
-
-  private static class WriteEmptyViewContents implements Runnable {
-    private final PCollectionView<?> view;
-    private final BoundedWindow window;
-    private final SettableFuture<Iterable<? extends WindowedValue<?>>> future;
-
-    private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
-        SettableFuture<Iterable<? extends WindowedValue<?>>> future) {
-      this.future = future;
-      this.view = view;
-      this.window = window;
-    }
-
-    @Override
-    public void run() {
-      // The requested window has closed without producing elements, so reflect that in
-      // the PCollectionView. If set has already been called, will do nothing.
-      future.set(Collections.<WindowedValue<?>>emptyList());
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("view", view)
-          .add("window", window)
-          .toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
deleted file mode 100644
index 3422efd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTimerInternals.java
+++ /dev/null
@@ -1,84 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.TimerInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * An implementation of {@link TimerInternals} where all relevant data exists in memory.
- */
-public class InProcessTimerInternals implements TimerInternals {
-  private final Clock processingTimeClock;
-  private final TransformWatermarks watermarks;
-  private final TimerUpdateBuilder timerUpdateBuilder;
-
-  public static InProcessTimerInternals create(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    return new InProcessTimerInternals(clock, watermarks, timerUpdateBuilder);
-  }
-
-  private InProcessTimerInternals(
-      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
-    this.processingTimeClock = clock;
-    this.watermarks = watermarks;
-    this.timerUpdateBuilder = timerUpdateBuilder;
-  }
-
-  @Override
-  public void setTimer(TimerData timerKey) {
-    timerUpdateBuilder.setTimer(timerKey);
-  }
-
-  @Override
-  public void deleteTimer(TimerData timerKey) {
-    timerUpdateBuilder.deletedTimer(timerKey);
-  }
-
-  public TimerUpdate getTimerUpdate() {
-    return timerUpdateBuilder.build();
-  }
-
-  @Override
-  public Instant currentProcessingTime() {
-    return processingTimeClock.now();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentSynchronizedProcessingTime() {
-    return watermarks.getSynchronizedProcessingInputTime();
-  }
-
-  @Override
-  public Instant currentInputWatermarkTime() {
-    return watermarks.getInputWatermark();
-  }
-
-  @Override
-  @Nullable
-  public Instant currentOutputWatermarkTime() {
-    return watermarks.getOutputWatermark();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
deleted file mode 100644
index ed77f70..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/InProcessTransformResult.java
+++ /dev/null
@@ -1,77 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-
-/**
- * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}.
- */
-public interface InProcessTransformResult {
-  /**
-   * Returns the {@link AppliedPTransform} that produced this result.
-   */
-  AppliedPTransform<?, ?, ?> getTransform();
-
-  /**
-   * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
-   * will be committed by the evaluation context as part of completing this result.
-   */
-  Iterable<? extends UncommittedBundle<?>> getOutputBundles();
-
-  /**
-   * Returns the {@link CounterSet} used by this {@link PTransform}, or null if this transform did
-   * not use a {@link CounterSet}.
-   */
-  @Nullable CounterSet getCounters();
-
-  /**
-   * Returns the Watermark Hold for the transform at the time this result was produced.
-   *
-   * If the transform does not set any watermark hold, returns
-   * {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
-   */
-  Instant getWatermarkHold();
-
-  /**
-   * Returns the State used by the transform.
-   *
-   * If this evaluation did not access state, this may return null.
-   */
-  @Nullable
-  CopyOnAccessInMemoryStateInternals<?> getState();
-
-  /**
-   * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
-   * evaluation was triggered due to the delivery of one or more timers, those timers must be added
-   * to the builder before it is complete.
-   *
-   * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
-   */
-  TimerUpdate getTimerUpdate();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
deleted file mode 100644
index 0e6b7e8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/KeyedPValueTrackingVisitor.java
+++ /dev/null
@@ -1,96 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PValue;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
- * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that
- * produces keyed outputs is assumed to colocate output elements that share a key.
- *
- * <p>All {@link GroupByKey} transforms, or their runner-specific implementation primitive, produce
- * keyed output.
- */
-// TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
-// unkeyed
-class KeyedPValueTrackingVisitor implements PipelineVisitor {
-  @SuppressWarnings("rawtypes")
-  private final Set<Class<? extends PTransform>> producesKeyedOutputs;
-  private final Set<PValue> keyedValues;
-  private boolean finalized;
-
-  public static KeyedPValueTrackingVisitor create(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
-  }
-
-  private KeyedPValueTrackingVisitor(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    this.producesKeyedOutputs = producesKeyedOutputs;
-    this.keyedValues = new HashSet<>();
-  }
-
-  @Override
-  public void enterCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
-        KeyedPValueTrackingVisitor.class.getSimpleName(),
-        node);
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
-    checkState(
-        !finalized,
-        "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
-        KeyedPValueTrackingVisitor.class.getSimpleName(),
-        node);
-    if (node.isRootNode()) {
-      finalized = true;
-    } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
-      keyedValues.addAll(node.getExpandedOutputs());
-    }
-  }
-
-  @Override
-  public void visitTransform(TransformTreeNode node) {}
-
-  @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
-    if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
-      keyedValues.addAll(value.expand());
-    }
-  }
-
-  public Set<PValue> getKeyedPValues() {
-    checkState(
-        finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
-    return keyedValues;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
deleted file mode 100644
index 4a3d17a..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcement.java
+++ /dev/null
@@ -1,63 +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.runners.inprocess;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Enforcement tools that verify that executing code conforms to the model.
- *
- * <p>ModelEnforcement is performed on a per-element and per-bundle basis. The
- * {@link ModelEnforcement} is provided with the input bundle as part of
- * {@link ModelEnforcementFactory#forBundle(CommittedBundle, AppliedPTransform)}, each element
- * before and after that element is provided to an underlying {@link TransformEvaluator}, and the
- * output {@link InProcessTransformResult} and committed output bundles after the
- * {@link TransformEvaluator} has completed.
- *
- * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
- * of the input {@link PCollection} on construction, and then enforce per-element behavior
- * (such as the immutability of input elements). When the element is output or the bundle is
- * completed, the required conditions can be enforced across all elements.
- */
-public interface ModelEnforcement<T> {
-  /**
-   * Called before a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
-   * provided {@link WindowedValue}.
-   */
-  void beforeElement(WindowedValue<T> element);
-
-  /**
-   * Called after a call to {@link TransformEvaluator#processElement(WindowedValue)} on the
-   * provided {@link WindowedValue}.
-   */
-  void afterElement(WindowedValue<T> element);
-
-  /**
-   * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
-   * called, producing the provided {@link InProcessTransformResult} and
-   * {@link CommittedBundle output bundles}.
-   */
-  void afterFinish(
-      CommittedBundle<T> input,
-      InProcessTransformResult result,
-      Iterable<? extends CommittedBundle<?>> outputs);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
deleted file mode 100644
index 1fa36d6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ModelEnforcementFactory.java
+++ /dev/null
@@ -1,30 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-/**
- * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input
- * {@link CommittedBundle bundle}. {@link ModelEnforcement} instances are created before the
- * {@link TransformEvaluator} is created.
- */
-public interface ModelEnforcementFactory {
-  <T> ModelEnforcement<T> forBundle(CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
deleted file mode 100644
index 71039fa..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/NanosOffsetClock.java
+++ /dev/null
@@ -1,59 +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.runners.inprocess;
-
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time.
- */
-public class NanosOffsetClock implements Clock {
-  private final long baseMillis;
-  private final long nanosAtBaseMillis;
-
-  public static NanosOffsetClock create() {
-    return new NanosOffsetClock();
-  }
-
-  private NanosOffsetClock() {
-    baseMillis = System.currentTimeMillis();
-    nanosAtBaseMillis = System.nanoTime();
-  }
-
-  @Override
-  public Instant now() {
-    return new Instant(
-        baseMillis + (TimeUnit.MILLISECONDS.convert(
-            System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)));
-  }
-
-  /**
-   * Creates instances of {@link NanosOffsetClock}.
-   */
-  public static class Factory implements DefaultValueFactory<Clock> {
-    @Override
-    public Clock create(PipelineOptions options) {
-      return new NanosOffsetClock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
deleted file mode 100644
index 2b4bf09..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PTransformOverrideFactory.java
+++ /dev/null
@@ -1,33 +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.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-interface PTransformOverrideFactory {
-  /**
-   * Create a {@link PTransform} override for the provided {@link PTransform} if applicable.
-   * Otherwise, return the input {@link PTransform}.
-   *
-   * <p>The returned PTransform must be semantically equivalent to the input {@link PTransform}.
-   */
-  <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
deleted file mode 100644
index 35639bd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoInProcessEvaluator.java
+++ /dev/null
@@ -1,173 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-class ParDoInProcessEvaluator<T> implements TransformEvaluator<T> {
-  public static <InputT, OutputT> ParDoInProcessEvaluator<InputT> create(
-      InProcessEvaluationContext evaluationContext,
-      CommittedBundle<InputT> inputBundle,
-      AppliedPTransform<PCollection<InputT>, ?, ?> application,
-      DoFn<InputT, OutputT> fn,
-      List<PCollectionView<?>> sideInputs,
-      TupleTag<OutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      Map<TupleTag<?>, PCollection<?>> outputs) {
-    InProcessExecutionContext executionContext =
-        evaluationContext.getExecutionContext(application, inputBundle.getKey());
-    String stepName = evaluationContext.getStepName(application);
-    InProcessStepContext stepContext =
-        executionContext.getOrCreateStepContext(stepName, stepName);
-
-    CounterSet counters = evaluationContext.createCounterSet();
-
-    Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
-    for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
-      outputBundles.put(
-          outputEntry.getKey(),
-          evaluationContext.createBundle(inputBundle, outputEntry.getValue()));
-    }
-
-    DoFnRunner<InputT, OutputT> runner =
-        DoFnRunners.createDefault(
-            evaluationContext.getPipelineOptions(),
-            SerializableUtils.clone(fn),
-            evaluationContext.createSideInputReader(sideInputs),
-            BundleOutputManager.create(outputBundles),
-            mainOutputTag,
-            sideOutputTags,
-            stepContext,
-            counters.getAddCounterMutator(),
-            application.getInput().getWindowingStrategy());
-
-    try {
-      runner.startBundle();
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-
-    return new ParDoInProcessEvaluator<>(
-        runner, application, counters, outputBundles.values(), stepContext);
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  private final DoFnRunner<T, ?> fnRunner;
-  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
-  private final CounterSet counters;
-  private final Collection<UncommittedBundle<?>> outputBundles;
-  private final InProcessStepContext stepContext;
-
-  private ParDoInProcessEvaluator(
-      DoFnRunner<T, ?> fnRunner,
-      AppliedPTransform<PCollection<T>, ?, ?> transform,
-      CounterSet counters,
-      Collection<UncommittedBundle<?>> outputBundles,
-      InProcessStepContext stepContext) {
-    this.fnRunner = fnRunner;
-    this.transform = transform;
-    this.counters = counters;
-    this.outputBundles = outputBundles;
-    this.stepContext = stepContext;
-  }
-
-  @Override
-  public void processElement(WindowedValue<T> element) {
-    try {
-      fnRunner.processElement(element);
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-  }
-
-  @Override
-  public InProcessTransformResult finishBundle() {
-    try {
-      fnRunner.finishBundle();
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-    StepTransformResult.Builder resultBuilder;
-    CopyOnAccessInMemoryStateInternals<?> state = stepContext.commitState();
-    if (state != null) {
-      resultBuilder =
-          StepTransformResult.withHold(transform, state.getEarliestWatermarkHold())
-              .withState(state);
-    } else {
-      resultBuilder = StepTransformResult.withoutHold(transform);
-    }
-    return resultBuilder
-        .addOutput(outputBundles)
-        .withTimerUpdate(stepContext.getTimerUpdate())
-        .withCounters(counters)
-        .build();
-  }
-
-  static class BundleOutputManager implements OutputManager {
-    private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
-    private final Map<TupleTag<?>, List<?>> undeclaredOutputs;
-
-    public static BundleOutputManager create(Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
-      return new BundleOutputManager(outputBundles);
-    }
-
-    private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
-      this.bundles = bundles;
-      undeclaredOutputs = new HashMap<>();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-      @SuppressWarnings("rawtypes")
-      UncommittedBundle bundle = bundles.get(tag);
-      if (bundle == null) {
-        List undeclaredContents = undeclaredOutputs.get(tag);
-        if (undeclaredContents == null) {
-          undeclaredContents = new ArrayList<T>();
-          undeclaredOutputs.put(tag, undeclaredContents);
-        }
-        undeclaredContents.add(output);
-      } else {
-        bundle.add(output);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
deleted file mode 100644
index 299d3a8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoMultiEvaluatorFactory.java
+++ /dev/null
@@ -1,63 +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.runners.inprocess;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.Map;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link BoundMulti} primitive {@link PTransform}.
- */
-class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator =
-        createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private static <InT, OuT> ParDoInProcessEvaluator<InT> createMultiEvaluator(
-      AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
-      CommittedBundle<InT> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    Map<TupleTag<?>, PCollection<?>> outputs = application.getOutput().getAll();
-    DoFn<InT, OuT> fn = application.getTransform().getFn();
-
-    return ParDoInProcessEvaluator.create(
-        evaluationContext,
-        inputBundle,
-        application,
-        fn,
-        application.getTransform().getSideInputs(),
-        application.getTransform().getMainOutputTag(),
-        application.getTransform().getSideOutputTags().getAll(),
-        outputs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
deleted file mode 100644
index 4d38448..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ParDoSingleEvaluatorFactory.java
+++ /dev/null
@@ -1,63 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo.Bound;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Collections;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound ParDo.Bound} primitive {@link PTransform}.
- */
-class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      final AppliedPTransform<?, ?, ?> application,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator =
-        createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
-    return evaluator;
-  }
-
-  private static <InputT, OutputT> ParDoInProcessEvaluator<InputT> createSingleEvaluator(
-      @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
-          Bound<InputT, OutputT>> application,
-      CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
-    TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
-
-    return ParDoInProcessEvaluator.create(
-        evaluationContext,
-        inputBundle,
-        application,
-        application.getTransform().getFn(),
-        application.getTransform().getSideInputs(),
-        mainOutputTag,
-        Collections.<TupleTag<?>>emptyList(),
-        ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
deleted file mode 100644
index a90cd7b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/PassthroughTransformEvaluator.java
+++ /dev/null
@@ -1,49 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
-  public static <InputT> PassthroughTransformEvaluator<InputT> create(
-      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
-    return new PassthroughTransformEvaluator<>(transform, output);
-  }
-
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final UncommittedBundle<InputT> output;
-
-  private PassthroughTransformEvaluator(
-      AppliedPTransform<?, ?, ?> transform, UncommittedBundle<InputT> output) {
-    this.transform = transform;
-    this.output = output;
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> element) throws Exception {
-    output.add(element);
-  }
-
-  @Override
-  public InProcessTransformResult finishBundle() throws Exception {
-    return StepTransformResult.withoutHold(transform).addOutput(output).build();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
deleted file mode 100644
index 88630ad..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ShardControlledWrite.java
+++ /dev/null
@@ -1,81 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Partition;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PDone;
-
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * A write that explicitly controls its number of output shards.
- */
-abstract class ShardControlledWrite<InputT>
-    extends ForwardingPTransform<PCollection<InputT>, PDone> {
-  @Override
-  public PDone apply(PCollection<InputT> input) {
-    int numShards = getNumShards();
-    checkArgument(
-        numShards >= 1,
-        "%s should only be applied if the output has a controlled number of shards (> 1); got %s",
-        getClass().getSimpleName(),
-        getNumShards());
-    PCollectionList<InputT> shards =
-        input.apply(
-            "PartitionInto" + numShards + "Shards",
-            Partition.of(getNumShards(), new RandomSeedPartitionFn<InputT>()));
-    for (int i = 0; i < shards.size(); i++) {
-      PCollection<InputT> shard = shards.get(i);
-      PTransform<? super PCollection<InputT>, PDone> writeShard = getSingleShardTransform(i);
-      shard.apply(String.format("%s(Shard:%s)", writeShard.getName(), i), writeShard);
-    }
-    return PDone.in(input.getPipeline());
-  }
-
-  /**
-   * Returns the number of shards this {@link PTransform} should write to.
-   */
-  abstract int getNumShards();
-
-  /**
-   * Returns a {@link PTransform} that performs a write to the shard with the specified shard
-   * number.
-   *
-   * <p>This method will be called n times, where n is the value of {@link #getNumShards()}, for
-   * shard numbers {@code [0...n)}.
-   */
-  abstract PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum);
-
-  private static class RandomSeedPartitionFn<T> implements Partition.PartitionFn<T> {
-    int nextPartition = -1;
-    @Override
-    public int partitionFor(T elem, int numPartitions) {
-      if (nextPartition < 0) {
-        nextPartition = ThreadLocalRandom.current().nextInt(numPartitions);
-      }
-      nextPartition++;
-      nextPartition %= numPartitions;
-      return nextPartition;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
deleted file mode 100644
index 9c4d9aa..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepAndKey.java
+++ /dev/null
@@ -1,71 +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.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-
-import com.google.common.base.MoreObjects;
-
-import java.util.Objects;
-
-/**
- * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
- * per-step in a keyed manner (e.g. State).
- */
-final class StepAndKey {
-  private final AppliedPTransform<?, ?, ?> step;
-  private final Object key;
-
-  /**
-   * Create a new {@link StepAndKey} with the provided step and key.
-   */
-  public static StepAndKey of(AppliedPTransform<?, ?, ?> step, Object key) {
-    return new StepAndKey(step, key);
-  }
-
-  private StepAndKey(AppliedPTransform<?, ?, ?> step, Object key) {
-    this.step = step;
-    this.key = key;
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(StepAndKey.class)
-        .add("step", step.getFullName())
-        .add("key", key)
-        .toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(step, key);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == this) {
-      return true;
-    } else if (!(other instanceof StepAndKey)) {
-      return false;
-    } else {
-      StepAndKey that = (StepAndKey) other;
-      return Objects.equals(this.step, that.step)
-          && Objects.equals(this.key, that.key);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
deleted file mode 100644
index 8874eda..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/StepTransformResult.java
+++ /dev/null
@@ -1,165 +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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.runners.inprocess.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * An immutable {@link InProcessTransformResult}.
- */
-public class StepTransformResult implements InProcessTransformResult {
-  private final AppliedPTransform<?, ?, ?> transform;
-  private final Iterable<? extends UncommittedBundle<?>> bundles;
-  @Nullable private final CopyOnAccessInMemoryStateInternals<?> state;
-  private final TimerUpdate timerUpdate;
-  @Nullable private final CounterSet counters;
-  private final Instant watermarkHold;
-
-  private StepTransformResult(
-      AppliedPTransform<?, ?, ?> transform,
-      Iterable<? extends UncommittedBundle<?>> outputBundles,
-      CopyOnAccessInMemoryStateInternals<?> state,
-      TimerUpdate timerUpdate,
-      CounterSet counters,
-      Instant watermarkHold) {
-    this.transform = checkNotNull(transform);
-    this.bundles = checkNotNull(outputBundles);
-    this.state = state;
-    this.timerUpdate = checkNotNull(timerUpdate);
-    this.counters = counters;
-    this.watermarkHold = checkNotNull(watermarkHold);
-  }
-
-  @Override
-  public Iterable<? extends UncommittedBundle<?>> getOutputBundles() {
-    return bundles;
-  }
-
-  @Override
-  public CounterSet getCounters() {
-    return counters;
-  }
-
-  @Override
-  public AppliedPTransform<?, ?, ?> getTransform() {
-    return transform;
-  }
-
-  @Override
-  public Instant getWatermarkHold() {
-    return watermarkHold;
-  }
-
-  @Nullable
-  @Override
-  public CopyOnAccessInMemoryStateInternals<?> getState() {
-    return state;
-  }
-
-  @Override
-  public TimerUpdate getTimerUpdate() {
-    return timerUpdate;
-  }
-
-  public static Builder withHold(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
-    return new Builder(transform, watermarkHold);
-  }
-
-  public static Builder withoutHold(AppliedPTransform<?, ?, ?> transform) {
-    return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(StepTransformResult.class)
-        .add("transform", transform)
-        .toString();
-  }
-
-  /**
-   * A builder for creating instances of {@link StepTransformResult}.
-   */
-  public static class Builder {
-    private final AppliedPTransform<?, ?, ?> transform;
-    private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
-    private CopyOnAccessInMemoryStateInternals<?> state;
-    private TimerUpdate timerUpdate;
-    private CounterSet counters;
-    private final Instant watermarkHold;
-
-    private Builder(AppliedPTransform<?, ?, ?> transform, Instant watermarkHold) {
-      this.transform = transform;
-      this.watermarkHold = watermarkHold;
-      this.bundlesBuilder = ImmutableList.builder();
-      this.timerUpdate = TimerUpdate.builder(null).build();
-    }
-
-    public StepTransformResult build() {
-      return new StepTransformResult(
-          transform,
-          bundlesBuilder.build(),
-          state,
-          timerUpdate,
-          counters,
-          watermarkHold);
-    }
-
-    public Builder withCounters(CounterSet counters) {
-      this.counters = counters;
-      return this;
-    }
-
-    public Builder withState(CopyOnAccessInMemoryStateInternals<?> state) {
-      this.state = state;
-      return this;
-    }
-
-    public Builder withTimerUpdate(TimerUpdate timerUpdate) {
-      this.timerUpdate = timerUpdate;
-      return this;
-    }
-
-    public Builder addOutput(
-        UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
-      bundlesBuilder.add(outputBundle);
-      bundlesBuilder.add(outputBundles);
-      return this;
-    }
-
-    public Builder addOutput(Collection<UncommittedBundle<?>> outputBundles) {
-      bundlesBuilder.addAll(outputBundles);
-      return this;
-    }
-  }
-}


[15/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java
new file mode 100644
index 0000000..2efaad3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.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.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
+ * {@link PTransform}.
+ */
+class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+    TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
+            (AppliedPTransform) application, inputBundle, evaluationContext);
+    return evaluator;
+  }
+
+  private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
+      final AppliedPTransform<
+              PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
+          application,
+      final CommittedBundle<InputT> inputBundle,
+      final InProcessEvaluationContext evaluationContext) {
+    if (inputBundle == null) {
+      // it is impossible to call processElement on a flatten with no input bundle. A Flatten with
+      // no input bundle occurs as an output of Flatten.pcollections(PCollectionList.empty())
+      return new FlattenEvaluator<>(
+          null, StepTransformResult.withoutHold(application).build());
+    }
+    final UncommittedBundle<InputT> outputBundle =
+        evaluationContext.createBundle(inputBundle, application.getOutput());
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(application).addOutput(outputBundle).build();
+    return new FlattenEvaluator<>(outputBundle, result);
+  }
+
+  private static class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
+    private final UncommittedBundle<InputT> outputBundle;
+    private final InProcessTransformResult result;
+
+    public FlattenEvaluator(
+        UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
+      this.outputBundle = outputBundle;
+      this.result = result;
+    }
+
+    @Override
+    public void processElement(WindowedValue<InputT> element) {
+      outputBundle.add(element);
+    }
+
+    @Override
+    public InProcessTransformResult finishBundle() {
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
new file mode 100644
index 0000000..3160b58
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.TypedPValue;
+
+/**
+ * A base class for implementing {@link PTransform} overrides, which behave identically to the
+ * delegate transform but with overridden methods. Implementors are required to implement
+ * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}.
+ */
+public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
+    extends PTransform<InputT, OutputT> {
+  protected abstract PTransform<InputT, OutputT> delegate();
+
+  @Override
+  public OutputT apply(InputT input) {
+    return delegate().apply(input);
+  }
+
+  @Override
+  public void validate(InputT input) {
+    delegate().validate(input);
+  }
+
+  @Override
+  public String getName() {
+    return delegate().getName();
+  }
+
+  @Override
+  public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unused")
+      TypedPValue<T> output) throws CannotProvideCoderException {
+    return delegate().getDefaultOutputCoder(input, output);
+  }
+
+  @Override
+  public void populateDisplayData(DisplayData.Builder builder) {
+    delegate().populateDisplayData(builder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
new file mode 100644
index 0000000..874ec17
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.StepTransformResult.Builder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItemCoder;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKey}
+ * {@link PTransform}.
+ */
+class GroupByKeyEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      InProcessEvaluationContext evaluationContext) {
+    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+    TransformEvaluator<InputT> evaluator = createEvaluator(
+            (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
+    return evaluator;
+  }
+
+  private <K, V> TransformEvaluator<KV<K, WindowedValue<V>>> createEvaluator(
+      final AppliedPTransform<
+              PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+              InProcessGroupByKeyOnly<K, V>>
+          application,
+      final CommittedBundle<KV<K, V>> inputBundle,
+      final InProcessEvaluationContext evaluationContext) {
+    return new GroupByKeyEvaluator<K, V>(evaluationContext, inputBundle, application);
+  }
+
+  private static class GroupByKeyEvaluator<K, V>
+      implements TransformEvaluator<KV<K, WindowedValue<V>>> {
+    private final InProcessEvaluationContext evaluationContext;
+
+    private final CommittedBundle<KV<K, V>> inputBundle;
+    private final AppliedPTransform<
+            PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+            InProcessGroupByKeyOnly<K, V>>
+        application;
+    private final Coder<K> keyCoder;
+    private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
+
+    public GroupByKeyEvaluator(
+        InProcessEvaluationContext evaluationContext,
+        CommittedBundle<KV<K, V>> inputBundle,
+        AppliedPTransform<
+                PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>,
+                InProcessGroupByKeyOnly<K, V>>
+            application) {
+      this.evaluationContext = evaluationContext;
+      this.inputBundle = inputBundle;
+      this.application = application;
+
+      PCollection<KV<K, WindowedValue<V>>> input = application.getInput();
+      keyCoder = getKeyCoder(input.getCoder());
+      groupingMap = new HashMap<>();
+    }
+
+    private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
+      if (!(coder instanceof KvCoder)) {
+        throw new IllegalStateException();
+      }
+      @SuppressWarnings("unchecked")
+      Coder<K> keyCoder = ((KvCoder<K, WindowedValue<V>>) coder).getKeyCoder();
+      return keyCoder;
+    }
+
+    @Override
+    public void processElement(WindowedValue<KV<K, WindowedValue<V>>> element) {
+      KV<K, WindowedValue<V>> kv = element.getValue();
+      K key = kv.getKey();
+      byte[] encodedKey;
+      try {
+        encodedKey = encodeToByteArray(keyCoder, key);
+      } catch (CoderException exn) {
+        // TODO: Put in better element printing:
+        // truncate if too long.
+        throw new IllegalArgumentException(
+            String.format("unable to encode key %s of input to %s using %s", key, this, keyCoder),
+            exn);
+      }
+      GroupingKey<K> groupingKey = new GroupingKey<>(key, encodedKey);
+      List<WindowedValue<V>> values = groupingMap.get(groupingKey);
+      if (values == null) {
+        values = new ArrayList<WindowedValue<V>>();
+        groupingMap.put(groupingKey, values);
+      }
+      values.add(kv.getValue());
+    }
+
+    @Override
+    public InProcessTransformResult finishBundle() {
+      Builder resultBuilder = StepTransformResult.withoutHold(application);
+      for (Map.Entry<GroupingKey<K>, List<WindowedValue<V>>> groupedEntry :
+          groupingMap.entrySet()) {
+        K key = groupedEntry.getKey().key;
+        KeyedWorkItem<K, V> groupedKv =
+            KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
+        UncommittedBundle<KeyedWorkItem<K, V>> bundle =
+            evaluationContext.createKeyedBundle(inputBundle, key, application.getOutput());
+        bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
+        resultBuilder.addOutput(bundle);
+      }
+      return resultBuilder.build();
+    }
+
+    private static class GroupingKey<K> {
+      private K key;
+      private byte[] encodedKey;
+
+      public GroupingKey(K key, byte[] encodedKey) {
+        this.key = key;
+        this.encodedKey = encodedKey;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (o instanceof GroupingKey) {
+          GroupingKey<?> that = (GroupingKey<?>) o;
+          return Arrays.equals(this.encodedKey, that.encodedKey);
+        } else {
+          return false;
+        }
+      }
+
+      @Override
+      public int hashCode() {
+        return Arrays.hashCode(encodedKey);
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
+   */
+  public static final class InProcessGroupByKeyOverrideFactory
+      implements PTransformOverrideFactory {
+    @Override
+    public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
+        PTransform<InputT, OutputT> transform) {
+      if (transform instanceof GroupByKey) {
+        @SuppressWarnings({"rawtypes", "unchecked"})
+        PTransform<InputT, OutputT> override = new InProcessGroupByKey((GroupByKey) transform);
+        return override;
+      }
+      return transform;
+    }
+  }
+
+  /**
+   * An in-memory implementation of the {@link GroupByKey} primitive as a composite
+   * {@link PTransform}.
+   */
+  private static final class InProcessGroupByKey<K, V>
+      extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+    private final GroupByKey<K, V> original;
+
+    private InProcessGroupByKey(GroupByKey<K, V> from) {
+      this.original = from;
+    }
+
+    @Override
+    public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
+      return original;
+    }
+
+    @Override
+    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+      KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+
+      // This operation groups by the combination of key and window,
+      // merging windows as needed, using the windows assigned to the
+      // key/value input elements and the window merge operation of the
+      // window function associated with the input PCollection.
+      WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
+
+      // Use the default GroupAlsoByWindow implementation
+      DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow =
+          groupAlsoByWindow(windowingStrategy, inputCoder.getValueCoder());
+
+      // By default, implement GroupByKey via a series of lower-level operations.
+      return input
+          // Make each input element's timestamp and assigned windows
+          // explicit, in the value part.
+          .apply(new ReifyTimestampsAndWindows<K, V>())
+
+          .apply(new InProcessGroupByKeyOnly<K, V>())
+          .setCoder(KeyedWorkItemCoder.of(inputCoder.getKeyCoder(),
+              inputCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
+
+          // Group each key's values by window, merging windows as needed.
+          .apply("GroupAlsoByWindow", ParDo.of(groupAlsoByWindow))
+
+          // And update the windowing strategy as appropriate.
+          .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
+          .setCoder(
+              KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
+    }
+
+    private <W extends BoundedWindow>
+        DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> groupAlsoByWindow(
+            final WindowingStrategy<?, W> windowingStrategy, final Coder<V> inputCoder) {
+      return GroupAlsoByWindowViaWindowSetDoFn.create(
+          windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
+    }
+  }
+
+  /**
+   * An implementation primitive to use in the evaluation of a {@link GroupByKey}
+   * {@link PTransform}.
+   */
+  public static final class InProcessGroupByKeyOnly<K, V>
+      extends PTransform<PCollection<KV<K, WindowedValue<V>>>, PCollection<KeyedWorkItem<K, V>>> {
+    @Override
+    public PCollection<KeyedWorkItem<K, V>> apply(PCollection<KV<K, WindowedValue<V>>> input) {
+      return PCollection.<KeyedWorkItem<K, V>>createPrimitiveOutputInternal(
+          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
+    }
+
+    @VisibleForTesting
+    InProcessGroupByKeyOnly() {}
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
new file mode 100644
index 0000000..2103ad3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.api.client.util.Throwables;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.SetMultimap;
+
+import org.joda.time.Instant;
+
+/**
+ * A {@link BundleFactory} that ensures that elements added to it are not mutated after being
+ * output. Immutability checks are enforced at the time {@link UncommittedBundle#commit(Instant)} is
+ * called, checking the value at that time against the value at the time the element was added. All
+ * elements added to the bundle will be encoded by the {@link Coder} of the underlying
+ * {@link PCollection}.
+ *
+ * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
+ * after it is added to an output {@link PCollection}.
+ */
+class ImmutabilityCheckingBundleFactory implements BundleFactory {
+  /**
+   * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
+   * {@link BundleFactory} to create the output bundle.
+   */
+  public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
+    return new ImmutabilityCheckingBundleFactory(underlying);
+  }
+
+  private final BundleFactory underlying;
+
+  private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
+    this.underlying = checkNotNull(underlying);
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+    return new ImmutabilityEnforcingBundle<>(underlying.createRootBundle(output));
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+    return new ImmutabilityEnforcingBundle<>(underlying.createBundle(input, output));
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, Object key, PCollection<T> output) {
+    return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(input, key, output));
+  }
+
+  private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
+    private final UncommittedBundle<T> underlying;
+    private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
+    private Coder<T> coder;
+
+    public ImmutabilityEnforcingBundle(UncommittedBundle<T> underlying) {
+      this.underlying = underlying;
+      mutationDetectors = HashMultimap.create();
+      coder = getPCollection().getCoder();
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return underlying.getPCollection();
+    }
+
+    @Override
+    public UncommittedBundle<T> add(WindowedValue<T> element) {
+      try {
+        mutationDetectors.put(
+            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
+      } catch (CoderException e) {
+        throw Throwables.propagate(e);
+      }
+      underlying.add(element);
+      return this;
+    }
+
+    @Override
+    public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
+      for (MutationDetector detector : mutationDetectors.values()) {
+        try {
+          detector.verifyUnmodified();
+        } catch (IllegalMutationException exn) {
+          throw UserCodeException.wrap(
+              new IllegalMutationException(
+                  String.format(
+                      "PTransform %s mutated value %s after it was output (new value was %s)."
+                          + " Values must not be mutated in any way after being output.",
+                      underlying.getPCollection().getProducingTransformInternal().getFullName(),
+                      exn.getSavedValue(),
+                      exn.getNewValue()),
+                  exn.getSavedValue(),
+                  exn.getNewValue(),
+                  exn));
+        }
+      }
+      return underlying.commit(synchronizedProcessingTime);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
new file mode 100644
index 0000000..bfecc9d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.MutationDetector;
+import org.apache.beam.sdk.util.MutationDetectors;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * {@link ModelEnforcement} that enforces elements are not modified over the course of processing
+ * an element.
+ *
+ * <p>Implies {@link EncodabilityEnforcment}.
+ */
+class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
+  public static ModelEnforcementFactory create() {
+    return new ImmutabilityEnforcementFactory();
+  }
+
+  @Override
+  public <T> ModelEnforcement<T> forBundle(
+      CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+    return new ImmutabilityCheckingEnforcement<T>(input, consumer);
+  }
+
+  private static class ImmutabilityCheckingEnforcement<T> extends AbstractModelEnforcement<T> {
+    private final AppliedPTransform<?, ?, ?> transform;
+    private final Map<WindowedValue<T>, MutationDetector> mutationElements;
+    private final Coder<T> coder;
+
+    private ImmutabilityCheckingEnforcement(
+        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> transform) {
+      this.transform = transform;
+      coder = input.getPCollection().getCoder();
+      mutationElements = new IdentityHashMap<>();
+    }
+
+    @Override
+    public void beforeElement(WindowedValue<T> element) {
+      try {
+        mutationElements.put(
+            element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
+      } catch (CoderException e) {
+        throw UserCodeException.wrap(e);
+      }
+    }
+
+    @Override
+    public void afterElement(WindowedValue<T> element) {
+      verifyUnmodified(mutationElements.get(element));
+    }
+
+    @Override
+    public void afterFinish(
+        CommittedBundle<T> input,
+        InProcessTransformResult result,
+        Iterable<? extends CommittedBundle<?>> outputs) {
+      for (MutationDetector detector : mutationElements.values()) {
+        verifyUnmodified(detector);
+      }
+    }
+
+    private void verifyUnmodified(MutationDetector detector) {
+      try {
+        detector.verifyUnmodified();
+      } catch (IllegalMutationException e) {
+        throw new IllegalMutationException(
+            String.format(
+                "PTransform %s illegaly mutated value %s of class %s."
+                    + " Input values must not be mutated in any way.",
+                transform.getFullName(),
+                e.getSavedValue(),
+                e.getSavedValue().getClass()),
+            e.getSavedValue(),
+            e.getNewValue());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
new file mode 100644
index 0000000..07b6bb4
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
@@ -0,0 +1,1327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.SortedMultiset;
+import com.google.common.collect.TreeMultiset;
+
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+/**
+ * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of
+ * {@link AppliedPTransform AppliedPTransforms} to provide event-time and completion tracking for
+ * in-memory execution. {@link InMemoryWatermarkManager} is designed to update and return a
+ * consistent view of watermarks in the presence of concurrent updates.
+ *
+ * <p>An {@link InMemoryWatermarkManager} is provided with the collection of root
+ * {@link AppliedPTransform AppliedPTransforms} and a map of {@link PCollection PCollections} to
+ * all the {@link AppliedPTransform AppliedPTransforms} that consume them at construction time.
+ *
+ * <p>Whenever a root {@link AppliedPTransform transform} produces elements, the
+ * {@link InMemoryWatermarkManager} is provided with the produced elements and the output watermark
+ * of the producing {@link AppliedPTransform transform}. The
+ * {@link InMemoryWatermarkManager watermark manager} is responsible for computing the watermarks
+ * of all {@link AppliedPTransform transforms} that consume one or more
+ * {@link PCollection PCollections}.
+ *
+ * <p>Whenever a non-root {@link AppliedPTransform} finishes processing one or more in-flight
+ * elements (referred to as the input {@link CommittedBundle bundle}), the following occurs
+ * atomically:
+ * <ul>
+ *  <li>All of the in-flight elements are removed from the collection of pending elements for the
+ *      {@link AppliedPTransform}.</li>
+ *  <li>All of the elements produced by the {@link AppliedPTransform} are added to the collection
+ *      of pending elements for each {@link AppliedPTransform} that consumes them.</li>
+ *  <li>The input watermark for the {@link AppliedPTransform} becomes the maximum value of
+ *    <ul>
+ *      <li>the previous input watermark</li>
+ *      <li>the minimum of
+ *        <ul>
+ *          <li>the timestamps of all currently pending elements</li>
+ *          <li>all input {@link PCollection} watermarks</li>
+ *        </ul>
+ *      </li>
+ *    </ul>
+ *  </li>
+ *  <li>The output watermark for the {@link AppliedPTransform} becomes the maximum of
+ *    <ul>
+ *      <li>the previous output watermark</li>
+ *      <li>the minimum of
+ *        <ul>
+ *          <li>the current input watermark</li>
+ *          <li>the current watermark holds</li>
+ *        </ul>
+ *      </li>
+ *    </ul>
+ *  </li>
+ *  <li>The watermark of the output {@link PCollection} can be advanced to the output watermark of
+ *      the {@link AppliedPTransform}</li>
+ *  <li>The watermark of all downstream {@link AppliedPTransform AppliedPTransforms} can be
+ *      advanced.</li>
+ * </ul>
+ *
+ * <p>The watermark of a {@link PCollection} is equal to the output watermark of the
+ * {@link AppliedPTransform} that produces it.
+ *
+ * <p>The watermarks for a {@link PTransform} are updated as follows when output is committed:<pre>
+ * Watermark_In'  = MAX(Watermark_In, MIN(U(TS_Pending), U(Watermark_InputPCollection)))
+ * Watermark_Out' = MAX(Watermark_Out, MIN(Watermark_In', U(StateHold)))
+ * Watermark_PCollection = Watermark_Out_ProducingPTransform
+ * </pre>
+ */
+public class InMemoryWatermarkManager {
+  /**
+   * The watermark of some {@link Pipeline} element, usually a {@link PTransform} or a
+   * {@link PCollection}.
+   *
+   * <p>A watermark is a monotonically increasing value, which represents the point up to which the
+   * system believes it has received all of the data. Data that arrives with a timestamp that is
+   * before the watermark is considered late. {@link BoundedWindow#TIMESTAMP_MAX_VALUE} is a special
+   * timestamp which indicates we have received all of the data and there will be no more on-time or
+   * late data. This value is represented by {@link InMemoryWatermarkManager#THE_END_OF_TIME}.
+   */
+  private static interface Watermark {
+    /**
+     * Returns the current value of this watermark.
+     */
+    Instant get();
+
+    /**
+     * Refreshes the value of this watermark from its input watermarks and watermark holds.
+     *
+     * @return true if the value of the watermark has changed (and thus dependent watermark must
+     *         also be updated
+     */
+    WatermarkUpdate refresh();
+  }
+
+  /**
+   * The result of computing a {@link Watermark}.
+   */
+  private static enum WatermarkUpdate {
+    /** The watermark is later than the value at the previous time it was computed. */
+    ADVANCED(true),
+    /** The watermark is equal to the value at the previous time it was computed. */
+    NO_CHANGE(false);
+
+    private final boolean advanced;
+
+    private WatermarkUpdate(boolean advanced) {
+      this.advanced = advanced;
+    }
+
+    public boolean isAdvanced() {
+      return advanced;
+    }
+
+    /**
+     * Returns the {@link WatermarkUpdate} that is a result of combining the two watermark updates.
+     *
+     * If either of the input {@link WatermarkUpdate WatermarkUpdates} were advanced, the result
+     * {@link WatermarkUpdate} has been advanced.
+     */
+    public WatermarkUpdate union(WatermarkUpdate that) {
+      if (this.advanced) {
+        return this;
+      }
+      return that;
+    }
+
+    /**
+     * Returns the {@link WatermarkUpdate} based on the former and current
+     * {@link Instant timestamps}.
+     */
+    public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTime) {
+      if (currentTime.isAfter(oldTime)) {
+        return ADVANCED;
+      }
+      return NO_CHANGE;
+    }
+  }
+
+  /**
+   * The input {@link Watermark} of an {@link AppliedPTransform}.
+   *
+   * <p>At any point, the value of an {@link AppliedPTransformInputWatermark} is equal to the
+   * minimum watermark across all of its input {@link Watermark Watermarks}, and the minimum
+   * timestamp of all of the pending elements, restricted to be monotonically increasing.
+   *
+   * <p>See {@link #refresh()} for more information.
+   */
+  private static class AppliedPTransformInputWatermark implements Watermark {
+    private final Collection<? extends Watermark> inputWatermarks;
+    private final SortedMultiset<WindowedValue<?>> pendingElements;
+    private final Map<Object, NavigableSet<TimerData>> objectTimers;
+
+    private AtomicReference<Instant> currentWatermark;
+
+    public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
+      this.inputWatermarks = inputWatermarks;
+      this.pendingElements = TreeMultiset.create(new WindowedValueByTimestampComparator());
+      this.objectTimers = new HashMap<>();
+      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+    }
+
+    @Override
+    public Instant get() {
+      return currentWatermark.get();
+    }
+
+    /**
+     * {@inheritDoc}.
+     *
+     * <p>When refresh is called, the value of the {@link AppliedPTransformInputWatermark} becomes
+     * equal to the maximum value of
+     * <ul>
+     *   <li>the previous input watermark</li>
+     *   <li>the minimum of
+     *     <ul>
+     *       <li>the timestamps of all currently pending elements</li>
+     *       <li>all input {@link PCollection} watermarks</li>
+     *     </ul>
+     *   </li>
+     * </ul>
+     */
+    @Override
+    public synchronized WatermarkUpdate refresh() {
+      Instant oldWatermark = currentWatermark.get();
+      Instant minInputWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE;
+      for (Watermark inputWatermark : inputWatermarks) {
+        minInputWatermark = INSTANT_ORDERING.min(minInputWatermark, inputWatermark.get());
+      }
+      if (!pendingElements.isEmpty()) {
+        minInputWatermark = INSTANT_ORDERING.min(
+            minInputWatermark, pendingElements.firstEntry().getElement().getTimestamp());
+      }
+      Instant newWatermark = INSTANT_ORDERING.max(oldWatermark, minInputWatermark);
+      currentWatermark.set(newWatermark);
+      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
+    }
+
+    private synchronized void addPendingElements(Iterable<? extends WindowedValue<?>> newPending) {
+      for (WindowedValue<?> pendingElement : newPending) {
+        pendingElements.add(pendingElement);
+      }
+    }
+
+    private synchronized void removePendingElements(
+        Iterable<? extends WindowedValue<?>> finishedElements) {
+      for (WindowedValue<?> finishedElement : finishedElements) {
+        pendingElements.remove(finishedElement);
+      }
+    }
+
+    private synchronized void updateTimers(TimerUpdate update) {
+      NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
+      if (keyTimers == null) {
+        keyTimers = new TreeSet<>();
+        objectTimers.put(update.key, keyTimers);
+      }
+      for (TimerData timer : update.setTimers) {
+        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+          keyTimers.add(timer);
+        }
+      }
+      for (TimerData timer : update.deletedTimers) {
+        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+          keyTimers.remove(timer);
+        }
+      }
+      // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
+    }
+
+    private synchronized Map<Object, List<TimerData>> extractFiredEventTimeTimers() {
+      return extractFiredTimers(currentWatermark.get(), objectTimers);
+    }
+
+    @Override
+    public synchronized String toString() {
+      return MoreObjects.toStringHelper(AppliedPTransformInputWatermark.class)
+          .add("pendingElements", pendingElements)
+          .add("currentWatermark", currentWatermark)
+          .toString();
+    }
+  }
+
+  /**
+   * The output {@link Watermark} of an {@link AppliedPTransform}.
+   *
+   * <p>The value of an {@link AppliedPTransformOutputWatermark} is equal to the minimum of the
+   * current watermark hold and the {@link AppliedPTransformInputWatermark} for the same
+   * {@link AppliedPTransform}, restricted to be monotonically increasing. See
+   * {@link #refresh()} for more information.
+   */
+  private static class AppliedPTransformOutputWatermark implements Watermark {
+    private final Watermark inputWatermark;
+    private final PerKeyHolds holds;
+    private AtomicReference<Instant> currentWatermark;
+
+    public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
+      this.inputWatermark = inputWatermark;
+      holds = new PerKeyHolds();
+      currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+    }
+
+    public synchronized void updateHold(Object key, Instant newHold) {
+      if (newHold == null) {
+        holds.removeHold(key);
+      } else {
+        holds.updateHold(key, newHold);
+      }
+    }
+
+    @Override
+    public Instant get() {
+      return currentWatermark.get();
+    }
+
+    /**
+     * {@inheritDoc}.
+     *
+     * <p>When refresh is called, the value of the {@link AppliedPTransformOutputWatermark} becomes
+     * equal to the maximum value of:
+     * <ul>
+     *   <li>the previous output watermark</li>
+     *   <li>the minimum of
+     *     <ul>
+     *       <li>the current input watermark</li>
+     *       <li>the current watermark holds</li>
+     *     </ul>
+     *   </li>
+     * </ul>
+     */
+    @Override
+    public synchronized WatermarkUpdate refresh() {
+      Instant oldWatermark = currentWatermark.get();
+      Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
+      newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
+      currentWatermark.set(newWatermark);
+      return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);
+    }
+
+    @Override
+    public synchronized String toString() {
+      return MoreObjects.toStringHelper(AppliedPTransformOutputWatermark.class)
+          .add("holds", holds)
+          .add("currentWatermark", currentWatermark)
+          .toString();
+    }
+  }
+
+  /**
+   * The input {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
+   * {@link AppliedPTransform}.
+   *
+   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeInputWatermark} is equal
+   * to the minimum across all pending bundles at the {@link AppliedPTransform} and all upstream
+   * {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the input
+   * synchronized processing time at any step is equal to the maximum of:
+   * <ul>
+   *   <li>The most recently returned synchronized processing input time
+   *   <li>The minimum of
+   *     <ul>
+   *       <li>The current processing time
+   *       <li>The current synchronized processing time input hold
+   *     </ul>
+   * </ul>
+   */
+  private static class SynchronizedProcessingTimeInputWatermark implements Watermark {
+    private final Collection<? extends Watermark> inputWms;
+    private final Collection<CommittedBundle<?>> pendingBundles;
+    private final Map<Object, NavigableSet<TimerData>> processingTimers;
+    private final Map<Object, NavigableSet<TimerData>> synchronizedProcessingTimers;
+
+    private final PriorityQueue<TimerData> pendingTimers;
+
+    private AtomicReference<Instant> earliestHold;
+
+    public SynchronizedProcessingTimeInputWatermark(Collection<? extends Watermark> inputWms) {
+      this.inputWms = inputWms;
+      this.pendingBundles = new HashSet<>();
+      this.processingTimers = new HashMap<>();
+      this.synchronizedProcessingTimers = new HashMap<>();
+      this.pendingTimers = new PriorityQueue<>();
+      Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
+      for (Watermark wm : inputWms) {
+        initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
+      }
+      earliestHold = new AtomicReference<>(initialHold);
+    }
+
+    @Override
+    public Instant get() {
+      return earliestHold.get();
+    }
+
+    /**
+     * {@inheritDoc}.
+     *
+     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeInputWatermark}
+     * becomes equal to the minimum value of
+     * <ul>
+     *   <li>the timestamps of all currently pending bundles</li>
+     *   <li>all input {@link PCollection} synchronized processing time watermarks</li>
+     * </ul>
+     *
+     * <p>Note that this value is not monotonic, but the returned value for the synchronized
+     * processing time must be.
+     */
+    @Override
+    public synchronized WatermarkUpdate refresh() {
+      Instant oldHold = earliestHold.get();
+      Instant minTime = THE_END_OF_TIME.get();
+      for (Watermark input : inputWms) {
+        minTime = INSTANT_ORDERING.min(minTime, input.get());
+      }
+      for (CommittedBundle<?> bundle : pendingBundles) {
+        // TODO: Track elements in the bundle by the processing time they were output instead of
+        // entire bundles. Requried to support arbitrarily splitting and merging bundles between
+        // steps
+        minTime = INSTANT_ORDERING.min(minTime, bundle.getSynchronizedProcessingOutputWatermark());
+      }
+      earliestHold.set(minTime);
+      return WatermarkUpdate.fromTimestamps(oldHold, minTime);
+    }
+
+    public synchronized void addPending(CommittedBundle<?> bundle) {
+      pendingBundles.add(bundle);
+    }
+
+    public synchronized void removePending(CommittedBundle<?> bundle) {
+      pendingBundles.remove(bundle);
+    }
+
+    /**
+     * Return the earliest timestamp of the earliest timer that has not been completed. This is
+     * either the earliest timestamp across timers that have not been completed, or the earliest
+     * timestamp across timers that have been delivered but have not been completed.
+     */
+    public synchronized Instant getEarliestTimerTimestamp() {
+      Instant earliest = THE_END_OF_TIME.get();
+      for (NavigableSet<TimerData> timers : processingTimers.values()) {
+        if (!timers.isEmpty()) {
+          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
+        }
+      }
+      for (NavigableSet<TimerData> timers : synchronizedProcessingTimers.values()) {
+        if (!timers.isEmpty()) {
+          earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
+        }
+      }
+      if (!pendingTimers.isEmpty()) {
+        earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
+      }
+      return earliest;
+    }
+
+    private synchronized void updateTimers(TimerUpdate update) {
+      for (TimerData completedTimer : update.completedTimers) {
+        pendingTimers.remove(completedTimer);
+      }
+      Map<TimeDomain, NavigableSet<TimerData>> timerMap = timerMap(update.key);
+      for (TimerData addedTimer : update.setTimers) {
+        NavigableSet<TimerData> timerQueue = timerMap.get(addedTimer.getDomain());
+        if (timerQueue != null) {
+          timerQueue.add(addedTimer);
+        }
+      }
+      for (TimerData deletedTimer : update.deletedTimers) {
+        NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
+        if (timerQueue != null) {
+          timerQueue.remove(deletedTimer);
+        }
+      }
+    }
+
+    private synchronized Map<Object, List<TimerData>> extractFiredDomainTimers(
+        TimeDomain domain, Instant firingTime) {
+      Map<Object, List<TimerData>> firedTimers;
+      switch (domain) {
+        case PROCESSING_TIME:
+          firedTimers = extractFiredTimers(firingTime, processingTimers);
+          break;
+        case SYNCHRONIZED_PROCESSING_TIME:
+          firedTimers =
+              extractFiredTimers(
+                  INSTANT_ORDERING.min(firingTime, earliestHold.get()),
+                  synchronizedProcessingTimers);
+          break;
+        default:
+          throw new IllegalArgumentException(
+              "Called getFiredTimers on a Synchronized Processing Time watermark"
+                  + " and gave a non-processing time domain "
+                  + domain);
+      }
+      for (Map.Entry<Object, ? extends Collection<TimerData>> firedTimer : firedTimers.entrySet()) {
+        pendingTimers.addAll(firedTimer.getValue());
+      }
+      return firedTimers;
+    }
+
+    private Map<TimeDomain, NavigableSet<TimerData>> timerMap(Object key) {
+      NavigableSet<TimerData> processingQueue = processingTimers.get(key);
+      if (processingQueue == null) {
+        processingQueue = new TreeSet<>();
+        processingTimers.put(key, processingQueue);
+      }
+      NavigableSet<TimerData> synchronizedProcessingQueue =
+          synchronizedProcessingTimers.get(key);
+      if (synchronizedProcessingQueue == null) {
+        synchronizedProcessingQueue = new TreeSet<>();
+        synchronizedProcessingTimers.put(key, synchronizedProcessingQueue);
+      }
+      EnumMap<TimeDomain, NavigableSet<TimerData>> result = new EnumMap<>(TimeDomain.class);
+      result.put(TimeDomain.PROCESSING_TIME, processingQueue);
+      result.put(TimeDomain.SYNCHRONIZED_PROCESSING_TIME, synchronizedProcessingQueue);
+      return result;
+    }
+
+    @Override
+    public synchronized String toString() {
+      return MoreObjects.toStringHelper(SynchronizedProcessingTimeInputWatermark.class)
+          .add("earliestHold", earliestHold)
+          .toString();
+    }
+  }
+
+  /**
+   * The output {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} hold for an
+   * {@link AppliedPTransform}.
+   *
+   * <p>At any point, the hold value of an {@link SynchronizedProcessingTimeOutputWatermark} is
+   * equal to the minimum across all incomplete timers at the {@link AppliedPTransform} and all
+   * upstream {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} watermarks. The value of the output
+   * synchronized processing time at any step is equal to the maximum of:
+   * <ul>
+   *   <li>The most recently returned synchronized processing output time
+   *   <li>The minimum of
+   *     <ul>
+   *       <li>The current processing time
+   *       <li>The current synchronized processing time output hold
+   *     </ul>
+   * </ul>
+   */
+  private static class SynchronizedProcessingTimeOutputWatermark implements Watermark {
+    private final SynchronizedProcessingTimeInputWatermark inputWm;
+    private AtomicReference<Instant> latestRefresh;
+
+    public SynchronizedProcessingTimeOutputWatermark(
+        SynchronizedProcessingTimeInputWatermark inputWm) {
+      this.inputWm = inputWm;
+      this.latestRefresh = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
+    }
+
+    @Override
+    public Instant get() {
+      return latestRefresh.get();
+    }
+
+    /**
+     * {@inheritDoc}.
+     *
+     * <p>When refresh is called, the value of the {@link SynchronizedProcessingTimeOutputWatermark}
+     * becomes equal to the minimum value of:
+     * <ul>
+     *   <li>the current input watermark.
+     *   <li>all {@link TimeDomain#SYNCHRONIZED_PROCESSING_TIME} timers that are based on the input
+     *       watermark.
+     *   <li>all {@link TimeDomain#PROCESSING_TIME} timers that are based on the input watermark.
+     * </ul>
+     *
+     * <p>Note that this value is not monotonic, but the returned value for the synchronized
+     * processing time must be.
+     */
+    @Override
+    public synchronized WatermarkUpdate refresh() {
+      // Hold the output synchronized processing time to the input watermark, which takes into
+      // account buffered bundles, and the earliest pending timer, which determines what to hold
+      // downstream timers to.
+      Instant oldRefresh = latestRefresh.get();
+      Instant newTimestamp =
+          INSTANT_ORDERING.min(inputWm.get(), inputWm.getEarliestTimerTimestamp());
+      latestRefresh.set(newTimestamp);
+      return WatermarkUpdate.fromTimestamps(oldRefresh, newTimestamp);
+    }
+
+    @Override
+    public synchronized String toString() {
+      return MoreObjects.toStringHelper(SynchronizedProcessingTimeOutputWatermark.class)
+          .add("latestRefresh", latestRefresh)
+          .toString();
+    }
+  }
+
+  /**
+   * The {@code Watermark} that is after the latest time it is possible to represent in the global
+   * window. This is a distinguished value representing a complete {@link PTransform}.
+   */
+  private static final Watermark THE_END_OF_TIME = new Watermark() {
+        @Override
+        public WatermarkUpdate refresh() {
+          // THE_END_OF_TIME is a distinguished value that cannot be advanced.
+          return WatermarkUpdate.NO_CHANGE;
+        }
+
+        @Override
+        public Instant get() {
+          return BoundedWindow.TIMESTAMP_MAX_VALUE;
+        }
+      };
+
+  private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
+
+  /**
+   * A function that takes a WindowedValue and returns the exploded representation of that
+   * {@link WindowedValue}.
+   */
+  private static final Function<WindowedValue<?>, ? extends Iterable<? extends WindowedValue<?>>>
+      EXPLODE_WINDOWS_FN =
+          new Function<WindowedValue<?>, Iterable<? extends WindowedValue<?>>>() {
+            @Override
+            public Iterable<? extends WindowedValue<?>> apply(WindowedValue<?> input) {
+              return input.explodeWindows();
+            }
+          };
+
+  /**
+   * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
+   * latestTime argument and put in in the result with the same key, then remove all of the keys
+   * which have no more pending timers.
+   *
+   * The result collection retains ordering of timers (from earliest to latest).
+   */
+  private static Map<Object, List<TimerData>> extractFiredTimers(
+      Instant latestTime, Map<Object, NavigableSet<TimerData>> objectTimers) {
+    Map<Object, List<TimerData>> result = new HashMap<>();
+    Set<Object> emptyKeys = new HashSet<>();
+    for (Map.Entry<Object, NavigableSet<TimerData>> pendingTimers : objectTimers.entrySet()) {
+      NavigableSet<TimerData> timers = pendingTimers.getValue();
+      if (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
+        ArrayList<TimerData> keyFiredTimers = new ArrayList<>();
+        result.put(pendingTimers.getKey(), keyFiredTimers);
+        while (!timers.isEmpty() && timers.first().getTimestamp().isBefore(latestTime)) {
+          keyFiredTimers.add(timers.first());
+          timers.remove(timers.first());
+        }
+      }
+      if (timers.isEmpty()) {
+        emptyKeys.add(pendingTimers.getKey());
+      }
+    }
+    objectTimers.keySet().removeAll(emptyKeys);
+    return result;
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * The {@link Clock} providing the current time in the {@link TimeDomain#PROCESSING_TIME} domain.
+   */
+  private final Clock clock;
+
+  /**
+   * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
+   * that consume that {@link PCollection}.
+   */
+  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
+
+  /**
+   * The input and output watermark of each {@link AppliedPTransform}.
+   */
+  private final Map<AppliedPTransform<?, ?, ?>, TransformWatermarks> transformToWatermarks;
+
+  /**
+   * Creates a new {@link InMemoryWatermarkManager}. All watermarks within the newly created
+   * {@link InMemoryWatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
+   * minimum watermark, with no watermark holds or pending elements.
+   *
+   * @param rootTransforms the root-level transforms of the {@link Pipeline}
+   * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
+   *                  transforms that consume it as a part of their input
+   */
+  public static InMemoryWatermarkManager create(
+      Clock clock,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+    return new InMemoryWatermarkManager(clock, rootTransforms, consumers);
+  }
+
+  private InMemoryWatermarkManager(
+      Clock clock,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+    this.clock = clock;
+    this.consumers = consumers;
+
+    transformToWatermarks = new HashMap<>();
+
+    for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
+      getTransformWatermark(rootTransform);
+    }
+    for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
+      for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
+        getTransformWatermark(transform);
+      }
+    }
+  }
+
+  private TransformWatermarks getTransformWatermark(AppliedPTransform<?, ?, ?> transform) {
+    TransformWatermarks wms = transformToWatermarks.get(transform);
+    if (wms == null) {
+      List<Watermark> inputCollectionWatermarks = getInputWatermarks(transform);
+      AppliedPTransformInputWatermark inputWatermark =
+          new AppliedPTransformInputWatermark(inputCollectionWatermarks);
+      AppliedPTransformOutputWatermark outputWatermark =
+          new AppliedPTransformOutputWatermark(inputWatermark);
+
+      SynchronizedProcessingTimeInputWatermark inputProcessingWatermark =
+          new SynchronizedProcessingTimeInputWatermark(getInputProcessingWatermarks(transform));
+      SynchronizedProcessingTimeOutputWatermark outputProcessingWatermark =
+          new SynchronizedProcessingTimeOutputWatermark(inputProcessingWatermark);
+
+      wms =
+          new TransformWatermarks(
+              inputWatermark, outputWatermark, inputProcessingWatermark, outputProcessingWatermark);
+      transformToWatermarks.put(transform, wms);
+    }
+    return wms;
+  }
+
+  private Collection<Watermark> getInputProcessingWatermarks(
+      AppliedPTransform<?, ?, ?> transform) {
+    ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
+    Collection<? extends PValue> inputs = transform.getInput().expand();
+    if (inputs.isEmpty()) {
+      inputWmsBuilder.add(THE_END_OF_TIME);
+    }
+    for (PValue pvalue : inputs) {
+      Watermark producerOutputWatermark =
+          getTransformWatermark(pvalue.getProducingTransformInternal())
+              .synchronizedProcessingOutputWatermark;
+      inputWmsBuilder.add(producerOutputWatermark);
+    }
+    return inputWmsBuilder.build();
+  }
+
+  private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
+    ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
+    Collection<? extends PValue> inputs = transform.getInput().expand();
+    if (inputs.isEmpty()) {
+      inputWatermarksBuilder.add(THE_END_OF_TIME);
+    }
+    for (PValue pvalue : inputs) {
+      Watermark producerOutputWatermark =
+          getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
+      inputWatermarksBuilder.add(producerOutputWatermark);
+    }
+    List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
+    return inputCollectionWatermarks;
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Gets the input and output watermarks for an {@link AppliedPTransform}. If the
+   * {@link AppliedPTransform PTransform} has not processed any elements, return a watermark of
+   * {@link BoundedWindow#TIMESTAMP_MIN_VALUE}.
+   *
+   * @return a snapshot of the input watermark and output watermark for the provided transform
+   */
+  public TransformWatermarks getWatermarks(AppliedPTransform<?, ?, ?> transform) {
+    return transformToWatermarks.get(transform);
+  }
+
+  /**
+   * Updates the watermarks of a transform with one or more inputs.
+   *
+   * <p>Each transform has two monotonically increasing watermarks: the input watermark, which can,
+   * at any time, be updated to equal:
+   * <pre>
+   * MAX(CurrentInputWatermark, MIN(PendingElements, InputPCollectionWatermarks))
+   * </pre>
+   * and the output watermark, which can, at any time, be updated to equal:
+   * <pre>
+   * MAX(CurrentOutputWatermark, MIN(InputWatermark, WatermarkHolds))
+   * </pre>.
+   *
+   * @param completed the input that has completed
+   * @param transform the transform that has completed processing the input
+   * @param outputs the bundles the transform has output
+   * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
+   *                     is no hold
+   */
+  public void updateWatermarks(
+      @Nullable CommittedBundle<?> completed,
+      AppliedPTransform<?, ?, ?> transform,
+      TimerUpdate timerUpdate,
+      Iterable<? extends CommittedBundle<?>> outputs,
+      @Nullable Instant earliestHold) {
+    updatePending(completed, transform, timerUpdate, outputs);
+    TransformWatermarks transformWms = transformToWatermarks.get(transform);
+    transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold);
+    refreshWatermarks(transform);
+  }
+
+  private void refreshWatermarks(AppliedPTransform<?, ?, ?> transform) {
+    TransformWatermarks myWatermarks = transformToWatermarks.get(transform);
+    WatermarkUpdate updateResult = myWatermarks.refresh();
+    if (updateResult.isAdvanced()) {
+      for (PValue outputPValue : transform.getOutput().expand()) {
+        Collection<AppliedPTransform<?, ?, ?>> downstreamTransforms = consumers.get(outputPValue);
+        if (downstreamTransforms != null) {
+          for (AppliedPTransform<?, ?, ?> downstreamTransform : downstreamTransforms) {
+            refreshWatermarks(downstreamTransform);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Removes all of the completed Timers from the collection of pending timers, adds all new timers,
+   * and removes all deleted timers. Removes all elements consumed by the input bundle from the
+   * {@link PTransform PTransforms} collection of pending elements, and adds all elements produced
+   * by the {@link PTransform} to the pending queue of each consumer.
+   */
+  private void updatePending(
+      CommittedBundle<?> input,
+      AppliedPTransform<?, ?, ?> transform,
+      TimerUpdate timerUpdate,
+      Iterable<? extends CommittedBundle<?>> outputs) {
+    TransformWatermarks completedTransform = transformToWatermarks.get(transform);
+    completedTransform.updateTimers(timerUpdate);
+    if (input != null) {
+      completedTransform.removePending(input);
+    }
+
+    for (CommittedBundle<?> bundle : outputs) {
+      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
+        TransformWatermarks watermarks = transformToWatermarks.get(consumer);
+        watermarks.addPending(bundle);
+      }
+    }
+  }
+
+  /**
+   * Returns a map of each {@link PTransform} that has pending timers to those timers. All of the
+   * pending timers will be removed from this {@link InMemoryWatermarkManager}.
+   */
+  public Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> extractFiredTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<Object, FiredTimers>> allTimers = new HashMap<>();
+    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
+        transformToWatermarks.entrySet()) {
+      Map<Object, FiredTimers> keyFiredTimers = watermarksEntry.getValue().extractFiredTimers();
+      if (!keyFiredTimers.isEmpty()) {
+        allTimers.put(watermarksEntry.getKey(), keyFiredTimers);
+      }
+    }
+    return allTimers;
+  }
+
+  /**
+   * A (key, Instant) pair that holds the watermark. Holds are per-key, but the watermark is global,
+   * and as such the watermark manager must track holds and the release of holds on a per-key basis.
+   *
+   * <p>The {@link #compareTo(KeyedHold)} method of {@link KeyedHold} is not consistent with equals,
+   * as the key is arbitrarily ordered via identity, rather than object equality.
+   */
+  private static final class KeyedHold implements Comparable<KeyedHold> {
+    private static final Ordering<Object> KEY_ORDERING = Ordering.arbitrary().nullsLast();
+
+    private final Object key;
+    private final Instant timestamp;
+
+    /**
+     * Create a new KeyedHold with the specified key and timestamp.
+     */
+    public static KeyedHold of(Object key, Instant timestamp) {
+      return new KeyedHold(key, MoreObjects.firstNonNull(timestamp, THE_END_OF_TIME.get()));
+    }
+
+    private KeyedHold(Object key, Instant timestamp) {
+      this.key = key;
+      this.timestamp = timestamp;
+    }
+
+    @Override
+    public int compareTo(KeyedHold that) {
+      return ComparisonChain.start()
+          .compare(this.timestamp, that.timestamp)
+          .compare(this.key, that.key, KEY_ORDERING)
+          .result();
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(timestamp, key);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other == null || !(other instanceof KeyedHold)) {
+        return false;
+      }
+      KeyedHold that = (KeyedHold) other;
+      return Objects.equals(this.timestamp, that.timestamp) && Objects.equals(this.key, that.key);
+    }
+
+    /**
+     * Get the value of this {@link KeyedHold}.
+     */
+    public Instant getTimestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(KeyedHold.class)
+          .add("key", key)
+          .add("hold", timestamp)
+          .toString();
+    }
+  }
+
+  private static class PerKeyHolds {
+    private final Map<Object, KeyedHold> keyedHolds;
+    private final PriorityQueue<KeyedHold> allHolds;
+
+    private PerKeyHolds() {
+      this.keyedHolds = new HashMap<>();
+      this.allHolds = new PriorityQueue<>();
+    }
+
+    /**
+     * Gets the minimum hold across all keys in this {@link PerKeyHolds}, or THE_END_OF_TIME if
+     * there are no holds within this {@link PerKeyHolds}.
+     */
+    public Instant getMinHold() {
+      return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
+    }
+
+    /**
+     * Updates the hold of the provided key to the provided value, removing any other holds for
+     * the same key.
+     */
+    public void updateHold(@Nullable Object key, Instant newHold) {
+      removeHold(key);
+      KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
+      keyedHolds.put(key, newKeyedHold);
+      allHolds.offer(newKeyedHold);
+    }
+
+    /**
+     * Removes the hold of the provided key.
+     */
+    public void removeHold(Object key) {
+      KeyedHold oldHold = keyedHolds.get(key);
+      if (oldHold != null) {
+        allHolds.remove(oldHold);
+      }
+    }
+  }
+
+  /**
+   * A reference to the input and output watermarks of an {@link AppliedPTransform}.
+   */
+  public class TransformWatermarks {
+    private final AppliedPTransformInputWatermark inputWatermark;
+    private final AppliedPTransformOutputWatermark outputWatermark;
+
+    private final SynchronizedProcessingTimeInputWatermark synchronizedProcessingInputWatermark;
+    private final SynchronizedProcessingTimeOutputWatermark synchronizedProcessingOutputWatermark;
+
+    private Instant latestSynchronizedInputWm;
+    private Instant latestSynchronizedOutputWm;
+
+    private TransformWatermarks(
+        AppliedPTransformInputWatermark inputWatermark,
+        AppliedPTransformOutputWatermark outputWatermark,
+        SynchronizedProcessingTimeInputWatermark inputSynchProcessingWatermark,
+        SynchronizedProcessingTimeOutputWatermark outputSynchProcessingWatermark) {
+      this.inputWatermark = inputWatermark;
+      this.outputWatermark = outputWatermark;
+
+      this.synchronizedProcessingInputWatermark = inputSynchProcessingWatermark;
+      this.synchronizedProcessingOutputWatermark = outputSynchProcessingWatermark;
+      this.latestSynchronizedInputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      this.latestSynchronizedOutputWm = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Returns the input watermark of the {@link AppliedPTransform}.
+     */
+    public Instant getInputWatermark() {
+      return Preconditions.checkNotNull(inputWatermark.get());
+    }
+
+    /**
+     * Returns the output watermark of the {@link AppliedPTransform}.
+     */
+    public Instant getOutputWatermark() {
+      return outputWatermark.get();
+    }
+
+    /**
+     * Returns the synchronized processing input time of the {@link AppliedPTransform}.
+     *
+     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
+     * presence of holds, will increase as the system time progresses.
+     */
+    public synchronized Instant getSynchronizedProcessingInputTime() {
+      latestSynchronizedInputWm = INSTANT_ORDERING.max(
+          latestSynchronizedInputWm,
+          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingInputWatermark.get()));
+      return latestSynchronizedInputWm;
+    }
+
+    /**
+     * Returns the synchronized processing output time of the {@link AppliedPTransform}.
+     *
+     * <p>The returned value is guaranteed to be monotonically increasing, and outside of the
+     * presence of holds, will increase as the system time progresses.
+     */
+    public synchronized Instant getSynchronizedProcessingOutputTime() {
+      latestSynchronizedOutputWm = INSTANT_ORDERING.max(
+          latestSynchronizedOutputWm,
+          INSTANT_ORDERING.min(clock.now(), synchronizedProcessingOutputWatermark.get()));
+      return latestSynchronizedOutputWm;
+    }
+
+    private WatermarkUpdate refresh() {
+      inputWatermark.refresh();
+      synchronizedProcessingInputWatermark.refresh();
+      WatermarkUpdate eventOutputUpdate = outputWatermark.refresh();
+      WatermarkUpdate syncOutputUpdate = synchronizedProcessingOutputWatermark.refresh();
+      return eventOutputUpdate.union(syncOutputUpdate);
+    }
+
+    private void setEventTimeHold(Object key, Instant newHold) {
+      outputWatermark.updateHold(key, newHold);
+    }
+
+    private void removePending(CommittedBundle<?> bundle) {
+      inputWatermark.removePendingElements(elementsFromBundle(bundle));
+      synchronizedProcessingInputWatermark.removePending(bundle);
+    }
+
+    private void addPending(CommittedBundle<?> bundle) {
+      inputWatermark.addPendingElements(elementsFromBundle(bundle));
+      synchronizedProcessingInputWatermark.addPending(bundle);
+    }
+
+    private Iterable<? extends WindowedValue<?>> elementsFromBundle(CommittedBundle<?> bundle) {
+      return FluentIterable.from(bundle.getElements()).transformAndConcat(EXPLODE_WINDOWS_FN);
+    }
+
+    private Map<Object, FiredTimers> extractFiredTimers() {
+      Map<Object, List<TimerData>> eventTimeTimers = inputWatermark.extractFiredEventTimeTimers();
+      Map<Object, List<TimerData>> processingTimers;
+      Map<Object, List<TimerData>> synchronizedTimers;
+      if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) {
+        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
+        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+            TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE);
+      } else {
+        processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+            TimeDomain.PROCESSING_TIME, clock.now());
+        synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers(
+            TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime());
+      }
+      Map<Object, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
+      groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
+
+      Map<Object, FiredTimers> keyFiredTimers = new HashMap<>();
+      for (Map.Entry<Object, Map<TimeDomain, List<TimerData>>> firedTimers :
+          groupedTimers.entrySet()) {
+        keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
+      }
+      return keyFiredTimers;
+    }
+
+    @SafeVarargs
+    private final void groupFiredTimers(
+        Map<Object, Map<TimeDomain, List<TimerData>>> groupedToMutate,
+        Map<Object, List<TimerData>>... timersToGroup) {
+      for (Map<Object, List<TimerData>> subGroup : timersToGroup) {
+        for (Map.Entry<Object, List<TimerData>> newTimers : subGroup.entrySet()) {
+          Map<TimeDomain, List<TimerData>> grouped = groupedToMutate.get(newTimers.getKey());
+          if (grouped == null) {
+            grouped = new HashMap<>();
+            groupedToMutate.put(newTimers.getKey(), grouped);
+          }
+          grouped.put(newTimers.getValue().get(0).getDomain(), newTimers.getValue());
+        }
+      }
+    }
+
+    private void updateTimers(TimerUpdate update) {
+      inputWatermark.updateTimers(update);
+      synchronizedProcessingInputWatermark.updateTimers(update);
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(TransformWatermarks.class)
+          .add("inputWatermark", inputWatermark)
+          .add("outputWatermark", outputWatermark)
+          .add("inputProcessingTime", synchronizedProcessingInputWatermark)
+          .add("outputProcessingTime", synchronizedProcessingOutputWatermark)
+          .toString();
+    }
+  }
+
+  /**
+   * A collection of newly set, deleted, and completed timers.
+   *
+   * <p>setTimers and deletedTimers are collections of {@link TimerData} that have been added to the
+   * {@link TimerInternals} of an executed step. completedTimers are timers that were delivered as
+   * the input to the executed step.
+   */
+  public static class TimerUpdate {
+    private final Object key;
+    private final Iterable<? extends TimerData> completedTimers;
+
+    private final Iterable<? extends TimerData> setTimers;
+    private final Iterable<? extends TimerData> deletedTimers;
+
+    /**
+     * Returns a TimerUpdate for a null key with no timers.
+     */
+    public static TimerUpdate empty() {
+      return new TimerUpdate(
+          null,
+          Collections.<TimerData>emptyList(),
+          Collections.<TimerData>emptyList(),
+          Collections.<TimerData>emptyList());
+    }
+
+    /**
+     * Creates a new {@link TimerUpdate} builder with the provided completed timers that needs the
+     * set and deleted timers to be added to it.
+     */
+    public static TimerUpdateBuilder builder(Object key) {
+      return new TimerUpdateBuilder(key);
+    }
+
+    /**
+     * A {@link TimerUpdate} builder that needs to be provided with set timers and deleted timers.
+     */
+    public static final class TimerUpdateBuilder {
+      private final Object key;
+      private final Collection<TimerData> completedTimers;
+      private final Collection<TimerData> setTimers;
+      private final Collection<TimerData> deletedTimers;
+
+      private TimerUpdateBuilder(Object key) {
+        this.key = key;
+        this.completedTimers = new HashSet<>();
+        this.setTimers = new HashSet<>();
+        this.deletedTimers = new HashSet<>();
+      }
+
+      /**
+       * Adds all of the provided timers to the collection of completed timers, and returns this
+       * {@link TimerUpdateBuilder}.
+       */
+      public TimerUpdateBuilder withCompletedTimers(Iterable<TimerData> completedTimers) {
+        Iterables.addAll(this.completedTimers, completedTimers);
+        return this;
+      }
+
+      /**
+       * Adds the provided timer to the collection of set timers, removing it from deleted timers if
+       * it has previously been deleted. Returns this {@link TimerUpdateBuilder}.
+       */
+      public TimerUpdateBuilder setTimer(TimerData setTimer) {
+        deletedTimers.remove(setTimer);
+        setTimers.add(setTimer);
+        return this;
+      }
+
+      /**
+       * Adds the provided timer to the collection of deleted timers, removing it from set timers if
+       * it has previously been set. Returns this {@link TimerUpdateBuilder}.
+       */
+      public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
+        deletedTimers.add(deletedTimer);
+        setTimers.remove(deletedTimer);
+        return this;
+      }
+
+      /**
+       * Returns a new {@link TimerUpdate} with the most recently set completedTimers, setTimers,
+       * and deletedTimers.
+       */
+      public TimerUpdate build() {
+        return new TimerUpdate(
+            key,
+            ImmutableSet.copyOf(completedTimers),
+            ImmutableSet.copyOf(setTimers),
+            ImmutableSet.copyOf(deletedTimers));
+      }
+    }
+
+    private TimerUpdate(
+        Object key,
+        Iterable<? extends TimerData> completedTimers,
+        Iterable<? extends TimerData> setTimers,
+        Iterable<? extends TimerData> deletedTimers) {
+      this.key = key;
+      this.completedTimers = completedTimers;
+      this.setTimers = setTimers;
+      this.deletedTimers = deletedTimers;
+    }
+
+    @VisibleForTesting
+    Object getKey() {
+      return key;
+    }
+
+    @VisibleForTesting
+    Iterable<? extends TimerData> getCompletedTimers() {
+      return completedTimers;
+    }
+
+    @VisibleForTesting
+    Iterable<? extends TimerData> getSetTimers() {
+      return setTimers;
+    }
+
+    @VisibleForTesting
+    Iterable<? extends TimerData> getDeletedTimers() {
+      return deletedTimers;
+    }
+
+    /**
+     * Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
+     */
+    public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
+      return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(key, completedTimers, setTimers, deletedTimers);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other == null || !(other instanceof TimerUpdate)) {
+        return false;
+      }
+      TimerUpdate that = (TimerUpdate) other;
+      return Objects.equals(this.key, that.key)
+          && Objects.equals(this.completedTimers, that.completedTimers)
+          && Objects.equals(this.setTimers, that.setTimers)
+          && Objects.equals(this.deletedTimers, that.deletedTimers);
+    }
+  }
+
+  /**
+   * A pair of {@link TimerData} and key which can be delivered to the appropriate
+   * {@link AppliedPTransform}. A timer fires at the transform that set it with a specific key when
+   * the time domain in which it lives progresses past a specified time, as determined by the
+   * {@link InMemoryWatermarkManager}.
+   */
+  public static class FiredTimers {
+    private final Map<TimeDomain, ? extends Collection<TimerData>> timers;
+
+    private FiredTimers(Map<TimeDomain, ? extends Collection<TimerData>> timers) {
+      this.timers = timers;
+    }
+
+    /**
+     * Gets all of the timers that have fired within the provided {@link TimeDomain}. If no timers
+     * fired within the provided domain, return an empty collection.
+     *
+     * <p>Timers within a {@link TimeDomain} are guaranteed to be in order of increasing timestamp.
+     */
+    public Collection<TimerData> getTimers(TimeDomain domain) {
+      Collection<TimerData> domainTimers = timers.get(domain);
+      if (domainTimers == null) {
+        return Collections.emptyList();
+      }
+      return domainTimers;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(FiredTimers.class).add("timers", timers).toString();
+    }
+  }
+
+  private static class WindowedValueByTimestampComparator extends Ordering<WindowedValue<?>> {
+    @Override
+    public int compare(WindowedValue<?> o1, WindowedValue<?> o2) {
+      return ComparisonChain.start()
+          .compare(o1.getTimestamp(), o2.getTimestamp())
+          .result();
+    }
+  }
+
+  public Set<AppliedPTransform<?, ?, ?>> getCompletedTransforms() {
+    Set<AppliedPTransform<?, ?, ?>> result = new HashSet<>();
+    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> wms :
+        transformToWatermarks.entrySet()) {
+      if (wms.getValue().getOutputWatermark().equals(THE_END_OF_TIME.get())) {
+        result.add(wms.getKey());
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
new file mode 100644
index 0000000..bc9b04c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * A factory that produces bundles that perform no additional validation.
+ */
+class InProcessBundleFactory implements BundleFactory {
+  public static InProcessBundleFactory create() {
+    return new InProcessBundleFactory();
+  }
+
+  private InProcessBundleFactory() {}
+
+  @Override
+  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+    return InProcessBundle.create(output, null);
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+    return InProcessBundle.create(output, input.getKey());
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, @Nullable Object key, PCollection<T> output) {
+    return InProcessBundle.create(output, key);
+  }
+
+  /**
+   * A {@link UncommittedBundle} that buffers elements in memory.
+   */
+  private static final class InProcessBundle<T> implements UncommittedBundle<T> {
+    private final PCollection<T> pcollection;
+    @Nullable private final Object key;
+    private boolean committed = false;
+    private ImmutableList.Builder<WindowedValue<T>> elements;
+
+    /**
+     * Create a new {@link InProcessBundle} for the specified {@link PCollection}.
+     */
+    public static <T> InProcessBundle<T> create(PCollection<T> pcollection, @Nullable Object key) {
+      return new InProcessBundle<T>(pcollection, key);
+    }
+
+    private InProcessBundle(PCollection<T> pcollection, Object key) {
+      this.pcollection = pcollection;
+      this.key = key;
+      this.elements = ImmutableList.builder();
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return pcollection;
+    }
+
+    @Override
+    public InProcessBundle<T> add(WindowedValue<T> element) {
+      checkState(
+          !committed,
+          "Can't add element %s to committed bundle in PCollection %s",
+          element,
+          pcollection);
+      elements.add(element);
+      return this;
+    }
+
+    @Override
+    public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
+      checkState(!committed, "Can't commit already committed bundle %s", this);
+      committed = true;
+      final Iterable<WindowedValue<T>> committedElements = elements.build();
+      return new CommittedInProcessBundle<>(
+          pcollection, key, committedElements, synchronizedCompletionTime);
+    }
+  }
+
+  private static class CommittedInProcessBundle<T> implements CommittedBundle<T> {
+    public CommittedInProcessBundle(
+        PCollection<T> pcollection,
+        Object key,
+        Iterable<WindowedValue<T>> committedElements,
+        Instant synchronizedCompletionTime) {
+      this.pcollection = pcollection;
+      this.key = key;
+      this.committedElements = committedElements;
+      this.synchronizedCompletionTime = synchronizedCompletionTime;
+    }
+
+    private final PCollection<T> pcollection;
+    private final Object key;
+    private final Iterable<WindowedValue<T>> committedElements;
+    private final Instant synchronizedCompletionTime;
+
+    @Override
+    @Nullable
+    public Object getKey() {
+      return key;
+    }
+
+    @Override
+    public Iterable<WindowedValue<T>> getElements() {
+      return committedElements;
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return pcollection;
+    }
+
+    @Override
+    public Instant getSynchronizedProcessingOutputWatermark() {
+      return synchronizedCompletionTime;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .omitNullValues()
+          .add("pcollection", pcollection)
+          .add("key", key)
+          .add("elements", committedElements)
+          .toString();
+    }
+
+    @Override
+    public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
+      return new CommittedInProcessBundle<>(
+          pcollection, key, ImmutableList.copyOf(elements), synchronizedCompletionTime);
+    }
+  }
+}



[05/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
deleted file mode 100644
index fac5a40..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TextIOShardedWriteFactory.java
+++ /dev/null
@@ -1,78 +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.runners.inprocess;
-
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIO.Write.Bound;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class TextIOShardedWriteFactory implements PTransformOverrideFactory {
-
-  @Override
-  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform) {
-    if (transform instanceof TextIO.Write.Bound) {
-      @SuppressWarnings("unchecked")
-      TextIO.Write.Bound<InputT> originalWrite = (TextIO.Write.Bound<InputT>) transform;
-      if (originalWrite.getNumShards() > 1
-          || (originalWrite.getNumShards() == 1
-              && !"".equals(originalWrite.getShardNameTemplate()))) {
-        @SuppressWarnings("unchecked")
-        PTransform<InputT, OutputT> override =
-            (PTransform<InputT, OutputT>) new TextIOShardedWrite<InputT>(originalWrite);
-        return override;
-      }
-    }
-    return transform;
-  }
-
-  private static class TextIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
-    private final TextIO.Write.Bound<InputT> initial;
-
-    private TextIOShardedWrite(Bound<InputT> initial) {
-      this.initial = initial;
-    }
-
-    @Override
-    int getNumShards() {
-      return initial.getNumShards();
-    }
-
-    @Override
-    PTransform<PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
-      String shardName =
-          IOChannelUtils.constructName(
-              initial.getFilenamePrefix(),
-              initial.getShardTemplate(),
-              initial.getFilenameSuffix(),
-              shardNum,
-              getNumShards());
-      return TextIO.Write.withCoder(initial.getCoder()).to(shardName).withoutSharding();
-    }
-
-    @Override
-    protected PTransform<PCollection<InputT>, PDone> delegate() {
-      return initial;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
deleted file mode 100644
index e002329..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluator.java
+++ /dev/null
@@ -1,46 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An evaluator of a specific application of a transform. Will be used for at least one
- * {@link CommittedBundle}.
- *
- * @param <InputT> the type of elements that will be passed to {@link #processElement}
- */
-public interface TransformEvaluator<InputT> {
-  /**
-   * Process an element in the input {@link CommittedBundle}.
-   *
-   * @param element the element to process
-   */
-  void processElement(WindowedValue<InputT> element) throws Exception;
-
-  /**
-   * Finish processing the bundle of this {@link TransformEvaluator}.
-   *
-   * After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
-   * and no more elements will be processed.
-   *
-   * @return an {@link InProcessTransformResult} containing the results of this bundle evaluation.
-   */
-  InProcessTransformResult finishBundle() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.java
deleted file mode 100644
index a9f6759..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorFactory.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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import javax.annotation.Nullable;
-
-/**
- * A factory for creating instances of {@link TransformEvaluator} for the application of a
- * {@link PTransform}.
- */
-public interface TransformEvaluatorFactory {
-  /**
-   * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
-   *
-   * Any work that must be done before input elements are processed (such as calling
-   * {@link DoFn#startBundle(DoFn.Context)}) must be done before the {@link TransformEvaluator} is
-   * made available to the caller.
-   *
-   * @throws Exception whenever constructing the underlying evaluator throws an exception
-   */
-  <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
deleted file mode 100644
index f6542b8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformEvaluatorRegistry.java
+++ /dev/null
@@ -1,77 +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.runners.inprocess;
-
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
- * implementations based on the type of {@link PTransform} of the application.
- */
-class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
-  public static TransformEvaluatorRegistry defaultRegistry() {
-    @SuppressWarnings("rawtypes")
-    ImmutableMap<Class<? extends PTransform>, TransformEvaluatorFactory> primitives =
-        ImmutableMap.<Class<? extends PTransform>, TransformEvaluatorFactory>builder()
-            .put(Read.Bounded.class, new BoundedReadEvaluatorFactory())
-            .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory())
-            .put(ParDo.Bound.class, new ParDoSingleEvaluatorFactory())
-            .put(ParDo.BoundMulti.class, new ParDoMultiEvaluatorFactory())
-            .put(
-                GroupByKeyEvaluatorFactory.InProcessGroupByKeyOnly.class,
-                new GroupByKeyEvaluatorFactory())
-            .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory())
-            .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
-            .put(Window.Bound.class, new WindowEvaluatorFactory())
-            .build();
-    return new TransformEvaluatorRegistry(primitives);
-  }
-
-  // the TransformEvaluatorFactories can construct instances of all generic types of transform,
-  // so all instances of a primitive can be handled with the same evaluator factory.
-  @SuppressWarnings("rawtypes")
-  private final Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories;
-
-  private TransformEvaluatorRegistry(
-      @SuppressWarnings("rawtypes")
-      Map<Class<? extends PTransform>, TransformEvaluatorFactory> factories) {
-    this.factories = factories;
-  }
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws Exception {
-    TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
-    return factory.forApplication(application, inputBundle, evaluationContext);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.java
deleted file mode 100644
index a93c7b2..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutor.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.runners.inprocess;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-import com.google.common.base.Throwables;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a
- * {@link TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering
- * the result using a registered {@link CompletionCallback}.
- *
- * <p>A {@link TransformExecutor} that is currently executing also provides access to the thread
- * that it is being executed on.
- */
-class TransformExecutor<T> implements Callable<InProcessTransformResult> {
-  public static <T> TransformExecutor<T> create(
-      TransformEvaluatorFactory factory,
-      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
-      CommittedBundle<T> inputBundle,
-      AppliedPTransform<?, ?, ?> transform,
-      CompletionCallback completionCallback,
-      TransformExecutorService transformEvaluationState) {
-    return new TransformExecutor<>(
-        factory,
-        modelEnforcements,
-        evaluationContext,
-        inputBundle,
-        transform,
-        completionCallback,
-        transformEvaluationState);
-  }
-
-  private final TransformEvaluatorFactory evaluatorFactory;
-  private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
-
-  private final InProcessEvaluationContext evaluationContext;
-
-  /** The transform that will be evaluated. */
-  private final AppliedPTransform<?, ?, ?> transform;
-  /** The inputs this {@link TransformExecutor} will deliver to the transform. */
-  private final CommittedBundle<T> inputBundle;
-
-  private final CompletionCallback onComplete;
-  private final TransformExecutorService transformEvaluationState;
-
-  private final AtomicReference<Thread> thread;
-
-  private TransformExecutor(
-      TransformEvaluatorFactory factory,
-      Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
-      CommittedBundle<T> inputBundle,
-      AppliedPTransform<?, ?, ?> transform,
-      CompletionCallback completionCallback,
-      TransformExecutorService transformEvaluationState) {
-    this.evaluatorFactory = factory;
-    this.modelEnforcements = modelEnforcements;
-    this.evaluationContext = evaluationContext;
-
-    this.inputBundle = inputBundle;
-    this.transform = transform;
-
-    this.onComplete = completionCallback;
-
-    this.transformEvaluationState = transformEvaluationState;
-    this.thread = new AtomicReference<>();
-  }
-
-  @Override
-  public InProcessTransformResult call() {
-    checkState(
-        thread.compareAndSet(null, Thread.currentThread()),
-        "Tried to execute %s for %s on thread %s, but is already executing on thread %s",
-        TransformExecutor.class.getSimpleName(),
-        transform.getFullName(),
-        Thread.currentThread(),
-        thread.get());
-    try {
-      Collection<ModelEnforcement<T>> enforcements = new ArrayList<>();
-      for (ModelEnforcementFactory enforcementFactory : modelEnforcements) {
-        ModelEnforcement<T> enforcement = enforcementFactory.forBundle(inputBundle, transform);
-        enforcements.add(enforcement);
-      }
-      TransformEvaluator<T> evaluator =
-          evaluatorFactory.forApplication(transform, inputBundle, evaluationContext);
-
-      processElements(evaluator, enforcements);
-
-      InProcessTransformResult result = finishBundle(evaluator, enforcements);
-      return result;
-    } catch (Throwable t) {
-      onComplete.handleThrowable(inputBundle, t);
-      throw Throwables.propagate(t);
-    } finally {
-      transformEvaluationState.complete(this);
-    }
-  }
-
-  /**
-   * Processes all the elements in the input bundle using the transform evaluator, applying any
-   * necessary {@link ModelEnforcement ModelEnforcements}.
-   */
-  private void processElements(
-      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
-      throws Exception {
-    if (inputBundle != null) {
-      for (WindowedValue<T> value : inputBundle.getElements()) {
-        for (ModelEnforcement<T> enforcement : enforcements) {
-          enforcement.beforeElement(value);
-        }
-
-        evaluator.processElement(value);
-
-        for (ModelEnforcement<T> enforcement : enforcements) {
-          enforcement.afterElement(value);
-        }
-      }
-    }
-  }
-
-  /**
-   * Finishes processing the input bundle and commit the result using the
-   * {@link CompletionCallback}, applying any {@link ModelEnforcement} if necessary.
-   *
-   * @return the {@link InProcessTransformResult} produced by
-   *         {@link TransformEvaluator#finishBundle()}
-   */
-  private InProcessTransformResult finishBundle(
-      TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
-      throws Exception {
-    InProcessTransformResult result = evaluator.finishBundle();
-    CommittedResult outputs = onComplete.handleResult(inputBundle, result);
-    for (ModelEnforcement<T> enforcement : enforcements) {
-      enforcement.afterFinish(inputBundle, result, outputs.getOutputs());
-    }
-    return result;
-  }
-
-  /**
-   * If this {@link TransformExecutor} is currently executing, return the thread it is executing in.
-   * Otherwise, return null.
-   */
-  @Nullable
-  public Thread getThread() {
-    return thread.get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
deleted file mode 100644
index 600102f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorService.java
+++ /dev/null
@@ -1,35 +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.runners.inprocess;
-
-/**
- * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
- * appropriate for the {@link StepAndKey} the executor exists for.
- */
-interface TransformExecutorService {
-  /**
-   * Schedule the provided work to be eventually executed.
-   */
-  void schedule(TransformExecutor<?> work);
-
-  /**
-   * Finish executing the provided work. This may cause additional
-   * {@link TransformExecutor TransformExecutors} to be evaluated.
-   */
-  void complete(TransformExecutor<?> completed);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
deleted file mode 100644
index 3194340..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/TransformExecutorServices.java
+++ /dev/null
@@ -1,154 +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.runners.inprocess;
-
-import com.google.common.base.MoreObjects;
-
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Static factory methods for constructing instances of {@link TransformExecutorService}.
- */
-final class TransformExecutorServices {
-  private TransformExecutorServices() {
-    // Do not instantiate
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * parallel.
-   */
-  public static TransformExecutorService parallel(
-      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-    return new ParallelEvaluationState(executor, scheduled);
-  }
-
-  /**
-   * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
-   * serial.
-   */
-  public static TransformExecutorService serial(
-      ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-    return new SerialEvaluationState(executor, scheduled);
-  }
-
-  /**
-   * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
-   * scheduled will be immediately submitted to the {@link ExecutorService}.
-   *
-   * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
-   * processed in parallel.
-   */
-  private static class ParallelEvaluationState implements TransformExecutorService {
-    private final ExecutorService executor;
-    private final Map<TransformExecutor<?>, Boolean> scheduled;
-
-    private ParallelEvaluationState(
-        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-      this.executor = executor;
-      this.scheduled = scheduled;
-    }
-
-    @Override
-    public void schedule(TransformExecutor<?> work) {
-      executor.submit(work);
-      scheduled.put(work, true);
-    }
-
-    @Override
-    public void complete(TransformExecutor<?> completed) {
-      scheduled.remove(completed);
-    }
-  }
-
-  /**
-   * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
-   * scheduled will be placed on the work queue. Only one item of work will be submitted to the
-   * {@link ExecutorService} at any time.
-   *
-   * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair.
-   * Keyed computations are processed serially per step.
-   */
-  private static class SerialEvaluationState implements TransformExecutorService {
-    private final ExecutorService executor;
-    private final Map<TransformExecutor<?>, Boolean> scheduled;
-
-    private AtomicReference<TransformExecutor<?>> currentlyEvaluating;
-    private final Queue<TransformExecutor<?>> workQueue;
-
-    private SerialEvaluationState(
-        ExecutorService executor, Map<TransformExecutor<?>, Boolean> scheduled) {
-      this.scheduled = scheduled;
-      this.executor = executor;
-      this.currentlyEvaluating = new AtomicReference<>();
-      this.workQueue = new ConcurrentLinkedQueue<>();
-    }
-
-    /**
-     * Schedules the work, adding it to the work queue if there is a bundle currently being
-     * evaluated and scheduling it immediately otherwise.
-     */
-    @Override
-    public void schedule(TransformExecutor<?> work) {
-      workQueue.offer(work);
-      updateCurrentlyEvaluating();
-    }
-
-    @Override
-    public void complete(TransformExecutor<?> completed) {
-      if (!currentlyEvaluating.compareAndSet(completed, null)) {
-        throw new IllegalStateException(
-            "Finished work "
-                + completed
-                + " but could not complete due to unexpected currently executing "
-                + currentlyEvaluating.get());
-      }
-      scheduled.remove(completed);
-      updateCurrentlyEvaluating();
-    }
-
-    private void updateCurrentlyEvaluating() {
-      if (currentlyEvaluating.get() == null) {
-        // Only synchronize if we need to update what's currently evaluating
-        synchronized (this) {
-          TransformExecutor<?> newWork = workQueue.poll();
-          if (newWork != null) {
-            if (currentlyEvaluating.compareAndSet(null, newWork)) {
-              scheduled.put(newWork, true);
-              executor.submit(newWork);
-            } else {
-              workQueue.offer(newWork);
-            }
-          }
-        }
-      }
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(SerialEvaluationState.class)
-          .add("currentlyEvaluating", currentlyEvaluating)
-          .add("workQueue", workQueue)
-          .toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
deleted file mode 100644
index 0cebf43..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/UnboundedReadEvaluatorFactory.java
+++ /dev/null
@@ -1,177 +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.runners.inprocess;
-
-import org.apache.beam.sdk.io.Read.Unbounded;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
- * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}.
- */
-class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
-  /*
-   * An evaluator for a Source is stateful, to ensure the CheckpointMark is properly persisted.
-   * Evaluators are cached here to ensure that the checkpoint mark is appropriately reused
-   * and any splits are honored.
-   */
-  private final ConcurrentMap<EvaluatorKey, Queue<? extends UnboundedReadEvaluator<?>>>
-      sourceEvaluators = new ConcurrentHashMap<>();
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
-    return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
-  }
-
-  private <OutputT> TransformEvaluator<?> getTransformEvaluator(
-      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    UnboundedReadEvaluator<?> currentEvaluator =
-        getTransformEvaluatorQueue(transform, evaluationContext).poll();
-    if (currentEvaluator == null) {
-      return EmptyTransformEvaluator.create(transform);
-    }
-    return currentEvaluator;
-  }
-
-  /**
-   * Get the queue of {@link TransformEvaluator TransformEvaluators} that produce elements for the
-   * provided application of {@link Unbounded Read.Unbounded}, initializing it if required.
-   *
-   * <p>This method is thread-safe, and will only produce new evaluators if no other invocation has
-   * already done so.
-   */
-  @SuppressWarnings("unchecked")
-  private <OutputT> Queue<UnboundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
-      final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
-    // Key by the application and the context the evaluation is occurring in (which call to
-    // Pipeline#run).
-    EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
-    @SuppressWarnings("unchecked")
-    Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue =
-        (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-    if (evaluatorQueue == null) {
-      evaluatorQueue = new ConcurrentLinkedQueue<>();
-      if (sourceEvaluators.putIfAbsent(key, evaluatorQueue) == null) {
-        // If no queue existed in the evaluators, add an evaluator to initialize the evaluator
-        // factory for this transform
-        UnboundedSource<OutputT, ?> source = transform.getTransform().getSource();
-        UnboundedReadEvaluator<OutputT> evaluator =
-            new UnboundedReadEvaluator<OutputT>(
-                transform, evaluationContext, source, evaluatorQueue);
-        evaluatorQueue.offer(evaluator);
-      } else {
-        // otherwise return the existing Queue that arrived before us
-        evaluatorQueue = (Queue<UnboundedReadEvaluator<OutputT>>) sourceEvaluators.get(key);
-      }
-    }
-    return evaluatorQueue;
-  }
-
-  /**
-   * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource},
-   * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator
-   * creates the {@link UnboundedReader} and consumes some currently available input.
-   *
-   * <p>Calls to {@link UnboundedReadEvaluator} are not internally thread-safe, and should only be
-   * used by a single thread at a time. Each {@link UnboundedReadEvaluator} maintains its own
-   * checkpoint, and constructs its reader from the current checkpoint in each call to
-   * {@link #finishBundle()}.
-   */
-  private static class UnboundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
-    private static final int ARBITRARY_MAX_ELEMENTS = 10;
-    private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
-    private final Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue;
-    /**
-     * The source being read from by this {@link UnboundedReadEvaluator}. This may not be the same
-     * source as derived from {@link #transform} due to splitting.
-     */
-    private final UnboundedSource<OutputT, ?> source;
-    private CheckpointMark checkpointMark;
-
-    public UnboundedReadEvaluator(
-        AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
-        UnboundedSource<OutputT, ?> source,
-        Queue<UnboundedReadEvaluator<OutputT>> evaluatorQueue) {
-      this.transform = transform;
-      this.evaluationContext = evaluationContext;
-      this.evaluatorQueue = evaluatorQueue;
-      this.source = source;
-      this.checkpointMark = null;
-    }
-
-    @Override
-    public void processElement(WindowedValue<Object> element) {}
-
-    @Override
-    public InProcessTransformResult finishBundle() throws IOException {
-      UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
-      try (UnboundedReader<OutputT> reader =
-              createReader(source, evaluationContext.getPipelineOptions());) {
-        int numElements = 0;
-        if (reader.start()) {
-          do {
-            output.add(
-                WindowedValue.timestampedValueInGlobalWindow(
-                    reader.getCurrent(), reader.getCurrentTimestamp()));
-            numElements++;
-          } while (numElements < ARBITRARY_MAX_ELEMENTS && reader.advance());
-        }
-        checkpointMark = reader.getCheckpointMark();
-        checkpointMark.finalizeCheckpoint();
-        // TODO: When exercising create initial splits, make this the minimum watermark across all
-        // existing readers
-        StepTransformResult result =
-            StepTransformResult.withHold(transform, reader.getWatermark())
-                .addOutput(output)
-                .build();
-        evaluatorQueue.offer(this);
-        return result;
-      }
-    }
-
-    private <CheckpointMarkT extends CheckpointMark> UnboundedReader<OutputT> createReader(
-        UnboundedSource<OutputT, CheckpointMarkT> source, PipelineOptions options) {
-      @SuppressWarnings("unchecked")
-      CheckpointMarkT mark = (CheckpointMarkT) checkpointMark;
-      return source.createReader(options, mark);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
deleted file mode 100644
index 0b54ba8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/ViewEvaluatorFactory.java
+++ /dev/null
@@ -1,145 +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.runners.inprocess;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.PCollectionViewWriter;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link CreatePCollectionView} primitive {@link PTransform}.
- *
- * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
- * the {@link WriteView} {@link PTransform}, which is part of the
- * {@link InProcessCreatePCollectionView} composite transform. This transform is an override for the
- * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
- * written.
- */
-class ViewEvaluatorFactory implements TransformEvaluatorFactory {
-  @Override
-  public <T> TransformEvaluator<T> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      InProcessPipelineRunner.CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
-    TransformEvaluator<T> evaluator = createEvaluator(
-            (AppliedPTransform) application, evaluationContext);
-    return evaluator;
-  }
-
-  private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
-      final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
-          application,
-      InProcessEvaluationContext context) {
-    PCollection<Iterable<InT>> input = application.getInput();
-    final PCollectionViewWriter<InT, OuT> writer =
-        context.createPCollectionViewWriter(input, application.getOutput());
-    return new TransformEvaluator<Iterable<InT>>() {
-      private final List<WindowedValue<InT>> elements = new ArrayList<>();
-
-      @Override
-      public void processElement(WindowedValue<Iterable<InT>> element) {
-        for (InT input : element.getValue()) {
-          elements.add(element.withValue(input));
-        }
-      }
-
-      @Override
-      public InProcessTransformResult finishBundle() {
-        writer.add(elements);
-        return StepTransformResult.withoutHold(application).build();
-      }
-    };
-  }
-
-  public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
-    @Override
-    public <InputT extends PInput, OutputT extends POutput>
-        PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
-      if (transform instanceof CreatePCollectionView) {
-
-      }
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      PTransform<InputT, OutputT> createView =
-          (PTransform<InputT, OutputT>)
-              new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
-      return createView;
-    }
-  }
-
-  /**
-   * An in-process override for {@link CreatePCollectionView}.
-   */
-  private static class InProcessCreatePCollectionView<ElemT, ViewT>
-      extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
-    private final CreatePCollectionView<ElemT, ViewT> og;
-
-    private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
-      this.og = og;
-    }
-
-    @Override
-    public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
-      return input.apply(WithKeys.<Void, ElemT>of((Void) null))
-          .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()))
-          .apply(GroupByKey.<Void, ElemT>create())
-          .apply(Values.<Iterable<ElemT>>create())
-          .apply(new WriteView<ElemT, ViewT>(og));
-    }
-
-    @Override
-    protected PTransform<PCollection<ElemT>, PCollectionView<ViewT>> delegate() {
-      return og;
-    }
-  }
-
-  /**
-   * An in-process implementation of the {@link CreatePCollectionView} primitive.
-   *
-   * This implementation requires the input {@link PCollection} to be an iterable, which is provided
-   * to {@link PCollectionView#fromIterableInternal(Iterable)}.
-   */
-  public static final class WriteView<ElemT, ViewT>
-      extends PTransform<PCollection<Iterable<ElemT>>, PCollectionView<ViewT>> {
-    private final CreatePCollectionView<ElemT, ViewT> og;
-
-    WriteView(CreatePCollectionView<ElemT, ViewT> og) {
-      this.og = og;
-    }
-
-    @Override
-    public PCollectionView<ViewT> apply(PCollection<Iterable<ElemT>> input) {
-      return og.getView();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
deleted file mode 100644
index 3e4aca6..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WatermarkCallbackExecutor.java
+++ /dev/null
@@ -1,146 +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.runners.inprocess;
-
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowingStrategy;
-
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.Ordering;
-
-import org.joda.time.Instant;
-
-import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * Executes callbacks that occur based on the progression of the watermark per-step.
- *
- * <p>Callbacks are registered by calls to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)},
- * and are executed after a call to {@link #fireForWatermark(AppliedPTransform, Instant)} with the
- * same {@link AppliedPTransform} and a watermark sufficient to ensure that the trigger for the
- * windowing strategy would have been produced.
- *
- * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
- * {@link AppliedPTransform} - any call to
- * {@link #callOnGuaranteedFiring(AppliedPTransform, BoundedWindow, WindowingStrategy, Runnable)}
- * that could have potentially already fired should be followed by a call to
- * {@link #fireForWatermark(AppliedPTransform, Instant)} for the same transform with the current
- * value of the watermark.
- */
-class WatermarkCallbackExecutor {
-  /**
-   * Create a new {@link WatermarkCallbackExecutor}.
-   */
-  public static WatermarkCallbackExecutor create() {
-    return new WatermarkCallbackExecutor();
-  }
-
-  private final ConcurrentMap<AppliedPTransform<?, ?, ?>, PriorityQueue<WatermarkCallback>>
-      callbacks;
-  private final ExecutorService executor;
-
-  private WatermarkCallbackExecutor() {
-    this.callbacks = new ConcurrentHashMap<>();
-    this.executor = Executors.newSingleThreadExecutor();
-  }
-
-  /**
-   * Execute the provided {@link Runnable} after the next call to
-   * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window is guaranteed to have
-   * produced output.
-   */
-  public void callOnGuaranteedFiring(
-      AppliedPTransform<?, ?, ?> step,
-      BoundedWindow window,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Runnable runnable) {
-    WatermarkCallback callback =
-        WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
-
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
-      if (callbacks.putIfAbsent(step, callbackQueue) != null) {
-        callbackQueue = callbacks.get(step);
-      }
-    }
-
-    synchronized (callbackQueue) {
-      callbackQueue.offer(callback);
-    }
-  }
-
-  /**
-   * Schedule all pending callbacks that must have produced output by the time of the provided
-   * watermark.
-   */
-  public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark) {
-    PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
-    if (callbackQueue == null) {
-      return;
-    }
-    synchronized (callbackQueue) {
-      while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
-        executor.submit(callbackQueue.poll().getCallback());
-      }
-    }
-  }
-
-  private static class WatermarkCallback {
-    public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
-        BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
-      @SuppressWarnings("unchecked")
-      Instant firingAfter =
-          strategy.getTrigger().getSpec().getWatermarkThatGuaranteesFiring((W) window);
-      return new WatermarkCallback(firingAfter, callback);
-    }
-
-    private final Instant fireAfter;
-    private final Runnable callback;
-
-    private WatermarkCallback(Instant fireAfter, Runnable callback) {
-      this.fireAfter = fireAfter;
-      this.callback = callback;
-    }
-
-    public boolean shouldFire(Instant currentWatermark) {
-      return currentWatermark.isAfter(fireAfter)
-          || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
-    }
-
-    public Runnable getCallback() {
-      return callback;
-    }
-  }
-
-  private static class CallbackOrdering extends Ordering<WatermarkCallback> {
-    @Override
-    public int compare(WatermarkCallback left, WatermarkCallback right) {
-      return ComparisonChain.start()
-          .compare(left.fireAfter, right.fireAfter)
-          .compare(left.callback, right.callback, Ordering.arbitrary())
-          .result();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
deleted file mode 100644
index 4cdacec..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/WindowEvaluatorFactory.java
+++ /dev/null
@@ -1,131 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.Window.Bound;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-import javax.annotation.Nullable;
-
-/**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
- * {@link Bound Window.Bound} primitive {@link PTransform}.
- */
-class WindowEvaluatorFactory implements TransformEvaluatorFactory {
-
-  @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
-      AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
-      throws Exception {
-    return createTransformEvaluator(
-        (AppliedPTransform) application, inputBundle, evaluationContext);
-  }
-
-  private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
-      AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
-      CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
-    UncommittedBundle<InputT> outputBundle =
-        evaluationContext.createBundle(inputBundle, transform.getOutput());
-    if (fn == null) {
-      return PassthroughTransformEvaluator.create(transform, outputBundle);
-    }
-    return new WindowIntoEvaluator<>(transform, fn, outputBundle);
-  }
-
-  private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
-    private final AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>>
-        transform;
-    private final WindowFn<InputT, ?> windowFn;
-    private final UncommittedBundle<InputT> outputBundle;
-
-    @SuppressWarnings("unchecked")
-    public WindowIntoEvaluator(
-        AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
-        WindowFn<? super InputT, ?> windowFn,
-        UncommittedBundle<InputT> outputBundle) {
-      this.outputBundle = outputBundle;
-      this.transform = transform;
-      // Safe contravariant cast
-      this.windowFn = (WindowFn<InputT, ?>) windowFn;
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> element) throws Exception {
-      Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
-      outputBundle.add(
-          WindowedValue.<InputT>of(
-              element.getValue(), element.getTimestamp(), windows, PaneInfo.NO_FIRING));
-    }
-
-    private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
-        WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
-      WindowFn<InputT, W>.AssignContext assignContext =
-          new InProcessAssignContext<>(windowFn, element);
-      Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
-      return windows;
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() throws Exception {
-      return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
-    }
-  }
-
-  private static class InProcessAssignContext<InputT, W extends BoundedWindow>
-      extends WindowFn<InputT, W>.AssignContext {
-    private final WindowedValue<InputT> value;
-
-    public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
-      fn.super();
-      this.value = value;
-    }
-
-    @Override
-    public InputT element() {
-      return value.getValue();
-    }
-
-    @Override
-    public Instant timestamp() {
-      return value.getTimestamp();
-    }
-
-    @Override
-    public Collection<? extends BoundedWindow> windows() {
-      return value.getWindows();
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
deleted file mode 100644
index 43367dd..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,112 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroIOTest;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link AvroIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class AvroIOShardedWriteFactoryTest {
-
-  @Rule public TemporaryFolder tmp = new TemporaryFolder();
-  private AvroIOShardedWriteFactory factory;
-
-  @Before
-  public void setup() {
-    factory = new AvroIOShardedWriteFactory();
-  }
-
-  @Test
-  public void originalWithoutShardingReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withoutSharding();
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath());
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    AvroIO.Write.Bound<String> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(1);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    TestPipeline p = TestPipeline.create();
-    String[] elems = new String[] {"foo", "bar", "baz"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-
-    p.run();
-    AvroIOTest.assertTestOutputs(elems, 1, file.getAbsolutePath(), original.getShardNameTemplate());
-  }
-
-  @Test
-  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    AvroIO.Write.Bound<String> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(3);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    TestPipeline p = TestPipeline.create();
-    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-    p.run();
-    AvroIOTest.assertTestOutputs(elems, 3, file.getAbsolutePath(), original.getShardNameTemplate());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
deleted file mode 100644
index 146dd98..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/BoundedReadEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,290 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
-import org.apache.beam.sdk.io.CountingSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.Read.Bounded;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.collect.ImmutableList;
-
-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.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * Tests for {@link BoundedReadEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class BoundedReadEvaluatorFactoryTest {
-  private BoundedSource<Long> source;
-  private PCollection<Long> longs;
-  private TransformEvaluatorFactory factory;
-  @Mock private InProcessEvaluationContext context;
-  private BundleFactory bundleFactory;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    source = CountingSource.upTo(10L);
-    TestPipeline p = TestPipeline.create();
-    longs = p.apply(Read.from(source));
-
-    factory = new BoundedReadEvaluatorFactory();
-    bundleFactory = InProcessBundleFactory.create();
-  }
-
-  @Test
-  public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    assertThat(
-        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(),
-        containsInAnyOrder(
-            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-  }
-
-  /**
-   * Demonstrate that acquiring multiple {@link TransformEvaluator TransformEvaluators} for the same
-   * {@link Bounded Read.Bounded} application with the same evaluation context only produces the
-   * elements once.
-   */
-  @Test
-  public void boundedSourceInMemoryTransformEvaluatorAfterFinishIsEmpty() throws Exception {
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    Iterable<? extends WindowedValue<Long>> outputElements =
-        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
-    assertThat(
-        outputElements,
-        containsInAnyOrder(
-            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-
-    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(secondOutput);
-    TransformEvaluator<?> secondEvaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
-    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-    assertThat(secondResult.getOutputBundles(), emptyIterable());
-    assertThat(
-        secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
-    assertThat(
-        outputElements,
-        containsInAnyOrder(
-            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-  }
-
-  /**
-   * Demonstrates that acquiring multiple evaluators from the factory are independent, but
-   * the elements in the source are only produced once.
-   */
-  @Test
-  public void boundedSourceEvaluatorSimultaneousEvaluations() throws Exception {
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(longs);
-    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
-    when(context.createRootBundle(longs)).thenReturn(output).thenReturn(secondOutput);
-
-    // create both evaluators before finishing either.
-    TransformEvaluator<?> evaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    TransformEvaluator<?> secondEvaluator =
-        factory.forApplication(longs.getProducingTransformInternal(), null, context);
-
-    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
-
-    InProcessTransformResult result = evaluator.finishBundle();
-    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
-    Iterable<? extends WindowedValue<Long>> outputElements =
-        output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
-
-    assertThat(
-        outputElements,
-        containsInAnyOrder(
-            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
-    assertThat(secondResult.getOutputBundles(), emptyIterable());
-    assertThat(
-        secondOutput.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(), emptyIterable());
-    assertThat(
-        outputElements,
-        containsInAnyOrder(
-            gw(1L), gw(2L), gw(4L), gw(8L), gw(9L), gw(7L), gw(6L), gw(5L), gw(3L), gw(0L)));
-  }
-
-  @Test
-  public void boundedSourceEvaluatorClosesReader() throws Exception {
-    TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
-
-    TestPipeline p = TestPipeline.create();
-    PCollection<Long> pcollection = p.apply(Read.from(source));
-    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
-    when(context.createRootBundle(pcollection)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
-    evaluator.finishBundle();
-    CommittedBundle<Long> committed = output.commit(Instant.now());
-    assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L)));
-    assertThat(TestSource.readerClosed, is(true));
-  }
-
-  @Test
-  public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
-    TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of());
-
-    TestPipeline p = TestPipeline.create();
-    PCollection<Long> pcollection = p.apply(Read.from(source));
-    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
-
-    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
-    when(context.createRootBundle(pcollection)).thenReturn(output);
-
-    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
-    evaluator.finishBundle();
-    CommittedBundle<Long> committed = output.commit(Instant.now());
-    assertThat(committed.getElements(), emptyIterable());
-    assertThat(TestSource.readerClosed, is(true));
-  }
-
-  private static class TestSource<T> extends BoundedSource<T> {
-    private static boolean readerClosed;
-    private final Coder<T> coder;
-    private final T[] elems;
-
-    public TestSource(Coder<T> coder, T... elems) {
-      this.elems = elems;
-      this.coder = coder;
-      readerClosed = false;
-    }
-
-    @Override
-    public List<? extends BoundedSource<T>> splitIntoBundles(
-        long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
-      return ImmutableList.of(this);
-    }
-
-    @Override
-    public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
-      return 0;
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
-      return false;
-    }
-
-    @Override
-    public BoundedSource.BoundedReader<T> createReader(PipelineOptions options) throws IOException {
-      return new TestReader<>(this, elems);
-    }
-
-    @Override
-    public void validate() {
-    }
-
-    @Override
-    public Coder<T> getDefaultOutputCoder() {
-      return coder;
-    }
-  }
-
-  private static class TestReader<T> extends BoundedReader<T> {
-    private final BoundedSource<T> source;
-    private final List<T> elems;
-    private int index;
-
-    public TestReader(BoundedSource<T> source, T... elems) {
-      this.source = source;
-      this.elems = Arrays.asList(elems);
-      this.index = -1;
-    }
-
-    @Override
-    public BoundedSource<T> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      return advance();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      if (elems.size() > index + 1) {
-        index++;
-        return true;
-      }
-      return false;
-    }
-
-    @Override
-    public T getCurrent() throws NoSuchElementException {
-      return elems.get(index);
-    }
-
-    @Override
-    public void close() throws IOException {
-      TestSource.readerClosed = true;
-    }
-  }
-
-  private static WindowedValue<Long> gw(Long elem) {
-    return WindowedValue.valueInGlobalWindow(elem);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
deleted file mode 100644
index c888a65..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/CommittedResultTest.java
+++ /dev/null
@@ -1,77 +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.runners.inprocess;
-
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Tests for {@link CommittedResult}.
- */
-@RunWith(JUnit4.class)
-public class CommittedResultTest implements Serializable {
-  private transient TestPipeline p = TestPipeline.create();
-  private transient AppliedPTransform<?, ?, ?> transform =
-      AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
-      });
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
-
-  @Test
-  public void getTransformExtractsFromResult() {
-    CommittedResult result =
-        CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
-            Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
-
-    assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
-  }
-
-  @Test
-  public void getOutputsEqualInput() {
-    List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
-        ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
-            WindowingStrategy.globalDefault(),
-            PCollection.IsBounded.BOUNDED)).commit(Instant.now()),
-            bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
-                WindowingStrategy.globalDefault(),
-                PCollection.IsBounded.UNBOUNDED)).commit(Instant.now()));
-    CommittedResult result =
-        CommittedResult.create(StepTransformResult.withoutHold(transform).build(), outputs);
-
-    assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
deleted file mode 100644
index aef4845..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/inprocess/ConsumerTrackingPipelineVisitorTest.java
+++ /dev/null
@@ -1,272 +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.runners.inprocess;
-
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
-
-import org.hamcrest.Matchers;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * Tests for {@link ConsumerTrackingPipelineVisitor}.
- */
-@RunWith(JUnit4.class)
-public class ConsumerTrackingPipelineVisitorTest implements Serializable {
-  @Rule public transient ExpectedException thrown = ExpectedException.none();
-
-  private transient TestPipeline p = TestPipeline.create();
-  private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
-
-  @Test
-  public void getViewsReturnsViews() {
-    PCollectionView<List<String>> listView =
-        p.apply("listCreate", Create.of("foo", "bar"))
-            .apply(
-                ParDo.of(
-                    new DoFn<String, String>() {
-                      @Override
-                      public void processElement(DoFn<String, String>.ProcessContext c)
-                          throws Exception {
-                        c.output(Integer.toString(c.element().length()));
-                      }
-                    }))
-            .apply(View.<String>asList());
-    PCollectionView<Object> singletonView =
-        p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getViews(),
-        Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
-  }
-
-  @Test
-  public void getRootTransformsContainsPBegins() {
-    PCollection<String> created = p.apply(Create.of("foo", "bar"));
-    PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
-    PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getRootTransforms(),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            created.getProducingTransformInternal(),
-            counted.getProducingTransformInternal(),
-            unCounted.getProducingTransformInternal()));
-  }
-
-  @Test
-  public void getRootTransformsContainsEmptyFlatten() {
-    PCollection<String> empty =
-        PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getRootTransforms(),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            empty.getProducingTransformInternal()));
-  }
-
-  @Test
-  public void getValueToConsumersSucceeds() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @Override
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-
-    PCollection<String> flattened =
-        PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
-
-    p.traverseTopologically(visitor);
-
-    assertThat(
-        visitor.getValueToConsumers().get(created),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            transformed.getProducingTransformInternal(),
-            flattened.getProducingTransformInternal()));
-    assertThat(
-        visitor.getValueToConsumers().get(transformed),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            flattened.getProducingTransformInternal()));
-    assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
-  }
-
-  @Test
-  public void getUnfinalizedPValuesContainsDanglingOutputs() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @Override
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getUnfinalizedPValues(), Matchers.<PValue>contains(transformed));
-  }
-
-  @Test
-  public void getUnfinalizedPValuesEmpty() {
-    p.apply(Create.of("1", "2", "3"))
-        .apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @Override
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }))
-        .apply(
-            new PTransform<PInput, PDone>() {
-              @Override
-              public PDone apply(PInput input) {
-                return PDone.in(input.getPipeline());
-              }
-            });
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getUnfinalizedPValues(), emptyIterable());
-  }
-
-  @Test
-  public void getStepNamesContainsAllTransforms() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @Override
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-    PDone finished =
-        transformed.apply(
-            new PTransform<PInput, PDone>() {
-              @Override
-              public PDone apply(PInput input) {
-                return PDone.in(input.getPipeline());
-              }
-            });
-
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            created.getProducingTransformInternal(), "s0"));
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            transformed.getProducingTransformInternal(), "s1"));
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            finished.getProducingTransformInternal(), "s2"));
-  }
-
-  @Test
-  public void traverseMultipleTimesThrows() {
-    p.apply(Create.of(1, 2, 3));
-
-    p.traverseTopologically(visitor);
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    thrown.expectMessage("is finalized");
-    p.traverseTopologically(visitor);
-  }
-
-  @Test
-  public void traverseIndependentPathsSucceeds() {
-    p.apply("left", Create.of(1, 2, 3));
-    p.apply("right", Create.of("foo", "bar", "baz"));
-
-    p.traverseTopologically(visitor);
-  }
-
-  @Test
-  public void getRootTransformsWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getRootTransforms");
-    visitor.getRootTransforms();
-  }
-  @Test
-  public void getStepNamesWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getStepNames");
-    visitor.getStepNames();
-  }
-  @Test
-  public void getUnfinalizedPValuesWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getUnfinalizedPValues");
-    visitor.getUnfinalizedPValues();
-  }
-
-  @Test
-  public void getValueToConsumersWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getValueToConsumers");
-    visitor.getValueToConsumers();
-  }
-
-  @Test
-  public void getViewsWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getViews");
-    visitor.getViews();
-  }
-}


[09/17] incubator-beam git commit: Move InProcessRunner to its own module

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
new file mode 100644
index 0000000..236ad17
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+import org.hamcrest.Matchers;
+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 java.io.Serializable;
+
+/**
+ * Tests for {@link ParDoSingleEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ParDoSingleEvaluatorFactoryTest implements Serializable {
+  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void testParDoInMemoryTransformEvaluator() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+    PCollection<Integer> collection =
+        input.apply(
+            ParDo.of(
+                new DoFn<String, Integer>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(c.element().length());
+                  }
+                }));
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
+    when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, null, null, null);
+    when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    org.apache.beam.runners.direct.TransformEvaluator<String> evaluator =
+        new ParDoSingleEvaluatorFactory()
+            .forApplication(
+                collection.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    assertThat(result.getCounters(), equalTo(counters));
+
+    assertThat(
+        outputBundle.commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<Integer>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(3),
+            WindowedValue.timestampedValueInGlobalWindow(4, new Instant(1000)),
+            WindowedValue.valueInGlobalWindow(5, PaneInfo.ON_TIME_AND_ONLY_FIRING)));
+  }
+
+  @Test
+  public void testSideOutputToUndeclaredSideOutputSucceeds() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+    final TupleTag<Integer> sideOutputTag = new TupleTag<Integer>() {};
+    PCollection<Integer> collection =
+        input.apply(
+            ParDo.of(
+                new DoFn<String, Integer>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.sideOutput(sideOutputTag, c.element().length());
+                  }
+                }));
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
+    when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, null, null, null);
+    when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoSingleEvaluatorFactory()
+            .forApplication(
+                collection.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getOutputBundles(), Matchers.<UncommittedBundle<?>>containsInAnyOrder(outputBundle));
+    assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+    assertThat(result.getCounters(), equalTo(counters));
+  }
+
+  @Test
+  public void finishBundleWithStatePutsStateInResult() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
+        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
+    final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
+    final StateNamespace windowNs =
+        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
+    ParDo.Bound<String, KV<String, Integer>> pardo =
+        ParDo.of(
+            new DoFn<String, KV<String, Integer>>() {
+              @Override
+              public void processElement(ProcessContext c) {
+                c.windowingInternals()
+                    .stateInternals()
+                    .state(StateNamespaces.global(), watermarkTag)
+                    .add(new Instant(124443L - c.element().length()));
+                c.windowingInternals()
+                    .stateInternals()
+                    .state(
+                        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
+                        bagTag)
+                    .add(c.element());
+              }
+            });
+    PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, "myKey", null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    org.apache.beam.runners.direct.TransformEvaluator<String> evaluator =
+        new ParDoSingleEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+    evaluator.processElement(
+        WindowedValue.timestampedValueInGlobalWindow("bara", new Instant(1000)));
+    evaluator.processElement(
+        WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
+    assertThat(result.getState(), not(nullValue()));
+    assertThat(
+        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
+        equalTo(new Instant(124438L)));
+    assertThat(
+        result.getState().state(windowNs, bagTag).read(),
+        containsInAnyOrder("foo", "bara", "bazam"));
+  }
+
+  @Test
+  public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
+
+    final TimerData addedTimer =
+        TimerData.of(
+            StateNamespaces.window(
+                IntervalWindow.getCoder(),
+                new IntervalWindow(
+                    new Instant(0).plus(Duration.standardMinutes(5)),
+                    new Instant(1)
+                        .plus(Duration.standardMinutes(5))
+                        .plus(Duration.standardHours(1)))),
+            new Instant(54541L),
+            TimeDomain.EVENT_TIME);
+    final TimerData deletedTimer =
+        TimerData.of(
+            StateNamespaces.window(
+                IntervalWindow.getCoder(),
+                new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardHours(1)))),
+            new Instant(3400000),
+            TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+    ParDo.Bound<String, KV<String, Integer>> pardo =
+        ParDo.of(
+            new DoFn<String, KV<String, Integer>>() {
+              @Override
+              public void processElement(ProcessContext c) {
+                c.windowingInternals().stateInternals();
+                c.windowingInternals()
+                    .timerInternals()
+                    .setTimer(
+                        TimerData.of(
+                            StateNamespaces.window(
+                                IntervalWindow.getCoder(),
+                                new IntervalWindow(
+                                    new Instant(0).plus(Duration.standardMinutes(5)),
+                                    new Instant(1)
+                                        .plus(Duration.standardMinutes(5))
+                                        .plus(Duration.standardHours(1)))),
+                            new Instant(54541L),
+                            TimeDomain.EVENT_TIME));
+                c.windowingInternals()
+                    .timerInternals()
+                    .deleteTimer(
+                        TimerData.of(
+                            StateNamespaces.window(
+                                IntervalWindow.getCoder(),
+                                new IntervalWindow(
+                                    new Instant(0),
+                                    new Instant(0).plus(Duration.standardHours(1)))),
+                            new Instant(3400000),
+                            TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+              }
+            });
+    PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+
+    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    UncommittedBundle<KV<String, Integer>> mainOutputBundle =
+        bundleFactory.createRootBundle(mainOutput);
+
+    when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
+
+    InProcessExecutionContext executionContext =
+        new InProcessExecutionContext(null, "myKey", null, null);
+    when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(), null))
+        .thenReturn(executionContext);
+    CounterSet counters = new CounterSet();
+    when(evaluationContext.createCounterSet()).thenReturn(counters);
+
+    TransformEvaluator<String> evaluator =
+        new ParDoSingleEvaluatorFactory()
+            .forApplication(
+                mainOutput.getProducingTransformInternal(), inputBundle, evaluationContext);
+
+    evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getTimerUpdate(),
+        equalTo(
+            TimerUpdate.builder("myKey").setTimer(addedTimer).deletedTimer(deletedTimer).build()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
new file mode 100644
index 0000000..49c9061
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.TextIOTest;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+
+/**
+ * Tests for {@link TextIOShardedWriteFactory}.
+ */
+@RunWith(JUnit4.class)
+public class TextIOShardedWriteFactoryTest {
+  @Rule public TemporaryFolder tmp = new TemporaryFolder();
+  private TextIOShardedWriteFactory factory;
+
+  @Before
+  public void setup() {
+    factory = new TextIOShardedWriteFactory();
+  }
+
+  @Test
+  public void originalWithoutShardingReturnsOriginal() throws Exception {
+    File file = tmp.newFile("foo");
+    PTransform<PCollection<String>, PDone> original =
+        TextIO.Write.to(file.getAbsolutePath()).withoutSharding();
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, theInstance(original));
+  }
+
+  @Test
+  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
+    File file = tmp.newFile("foo");
+    PTransform<PCollection<String>, PDone> original = TextIO.Write.to(file.getAbsolutePath());
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, theInstance(original));
+  }
+
+  @Test
+  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
+    File file = tmp.newFile("foo");
+    TextIO.Write.Bound<String> original =
+        TextIO.Write.to(file.getAbsolutePath()).withNumShards(1);
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+    TestPipeline p = TestPipeline.create();
+    String[] elems = new String[] {"foo", "bar", "baz"};
+    p.apply(Create.<String>of(elems)).apply(overridden);
+
+    file.delete();
+
+    p.run();
+    TextIOTest.assertOutputFiles(
+        elems, StringUtf8Coder.of(), 1, tmp, "foo", original.getShardNameTemplate());
+  }
+
+  @Test
+  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
+    File file = tmp.newFile("foo");
+    TextIO.Write.Bound<String> original = TextIO.Write.to(file.getAbsolutePath()).withNumShards(3);
+    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
+
+    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
+
+    TestPipeline p = TestPipeline.create();
+    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
+    p.apply(Create.<String>of(elems)).apply(overridden);
+
+    file.delete();
+    p.run();
+    TextIOTest.assertOutputFiles(
+        elems, StringUtf8Coder.of(), 3, tmp, "foo", original.getShardNameTemplate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
new file mode 100644
index 0000000..7dd5830
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.any;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
+import com.google.common.util.concurrent.MoreExecutors;
+
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * Tests for {@link TransformExecutorServices}.
+ */
+@RunWith(JUnit4.class)
+public class TransformExecutorServicesTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private ExecutorService executorService;
+  private Map<TransformExecutor<?>, Boolean> scheduled;
+
+  @Before
+  public void setup() {
+    executorService = MoreExecutors.newDirectExecutorService();
+    scheduled = new ConcurrentHashMap<>();
+  }
+
+  @Test
+  public void parallelScheduleMultipleSchedulesBothImmediately() {
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> first = mock(TransformExecutor.class);
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+    TransformExecutorService parallel =
+        TransformExecutorServices.parallel(executorService, scheduled);
+    parallel.schedule(first);
+    parallel.schedule(second);
+
+    verify(first).call();
+    verify(second).call();
+    assertThat(
+        scheduled,
+        Matchers.allOf(
+            Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true),
+            Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true)));
+
+    parallel.complete(first);
+    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
+    assertThat(
+        scheduled,
+        not(
+            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+                Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
+    parallel.complete(second);
+    assertThat(scheduled.isEmpty(), is(true));
+  }
+
+  @Test
+  public void serialScheduleTwoWaitsForFirstToComplete() {
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> first = mock(TransformExecutor.class);
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+    TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
+    serial.schedule(first);
+    verify(first).call();
+
+    serial.schedule(second);
+    verify(second, never()).call();
+
+    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(first, true));
+    assertThat(
+        scheduled,
+        not(
+            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+                Matchers.<TransformExecutor<?>>equalTo(second), any(Boolean.class))));
+
+    serial.complete(first);
+    verify(second).call();
+    assertThat(scheduled, Matchers.<TransformExecutor<?>, Boolean>hasEntry(second, true));
+    assertThat(
+        scheduled,
+        not(
+            Matchers.<TransformExecutor<?>, Boolean>hasEntry(
+                Matchers.<TransformExecutor<?>>equalTo(first), any(Boolean.class))));
+
+    serial.complete(second);
+  }
+
+  @Test
+  public void serialCompleteNotExecutingTaskThrows() {
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> first = mock(TransformExecutor.class);
+    @SuppressWarnings("unchecked")
+    TransformExecutor<Object> second = mock(TransformExecutor.class);
+
+    TransformExecutorService serial = TransformExecutorServices.serial(executorService, scheduled);
+    serial.schedule(first);
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("unexpected currently executing");
+
+    serial.complete(second);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
new file mode 100644
index 0000000..959e9d3
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
@@ -0,0 +1,538 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.IllegalMutationException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.util.concurrent.MoreExecutors;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Tests for {@link TransformExecutor}.
+ */
+@RunWith(JUnit4.class)
+public class TransformExecutorTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private PCollection<String> created;
+  private PCollection<KV<Integer, String>> downstream;
+
+  private CountDownLatch evaluatorCompleted;
+
+  private RegisteringCompletionCallback completionCallback;
+  private TransformExecutorService transformEvaluationState;
+  private BundleFactory bundleFactory;
+  @Mock private InProcessEvaluationContext evaluationContext;
+  @Mock private TransformEvaluatorRegistry registry;
+  private Map<TransformExecutor<?>, Boolean> scheduled;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+
+    bundleFactory = InProcessBundleFactory.create();
+
+    scheduled = new HashMap<>();
+    transformEvaluationState =
+        TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService(), scheduled);
+
+    evaluatorCompleted = new CountDownLatch(1);
+    completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
+
+    TestPipeline p = TestPipeline.create();
+    created = p.apply(Create.of("foo", "spam", "third"));
+    downstream = created.apply(WithKeys.<Integer, String>of(3));
+  }
+
+  @Test
+  public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    final AtomicBoolean finishCalled = new AtomicBoolean(false);
+    TransformEvaluator<Object> evaluator =
+        new TransformEvaluator<Object>() {
+          @Override
+          public void processElement(WindowedValue<Object> element) throws Exception {
+            throw new IllegalArgumentException("Shouldn't be called");
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            finishCalled.set(true);
+            return result;
+          }
+        };
+
+    when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<Object> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>emptyList(),
+            evaluationContext,
+            null,
+            created.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+    executor.call();
+
+    assertThat(finishCalled.get(), is(true));
+    assertThat(completionCallback.handledResult, equalTo(result));
+    assertThat(completionCallback.handledThrowable, is(nullValue()));
+    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+  }
+
+  @Test
+  public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+    final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
+    TransformEvaluator<String> evaluator =
+        new TransformEvaluator<String>() {
+          @Override
+          public void processElement(WindowedValue<String> element) throws Exception {
+            elementsProcessed.add(element);
+            return;
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            return result;
+          }
+        };
+
+    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
+    WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
+    WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(created).add(foo).add(spam).add(third).commit(Instant.now());
+    when(
+            registry.<String>forApplication(
+                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<String> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>emptyList(),
+            evaluationContext,
+            inputBundle,
+            downstream.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+
+    Executors.newSingleThreadExecutor().submit(executor);
+
+    evaluatorCompleted.await();
+
+    assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
+    assertThat(completionCallback.handledResult, equalTo(result));
+    assertThat(completionCallback.handledThrowable, is(nullValue()));
+    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+  }
+
+  @Test
+  public void processElementThrowsExceptionCallsback() throws Exception {
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+    final Exception exception = new Exception();
+    TransformEvaluator<String> evaluator =
+        new TransformEvaluator<String>() {
+          @Override
+          public void processElement(WindowedValue<String> element) throws Exception {
+            throw exception;
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            return result;
+          }
+        };
+
+    WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(created).add(foo).commit(Instant.now());
+    when(
+            registry.<String>forApplication(
+                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<String> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>emptyList(),
+            evaluationContext,
+            inputBundle,
+            downstream.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+    Executors.newSingleThreadExecutor().submit(executor);
+
+    evaluatorCompleted.await();
+
+    assertThat(completionCallback.handledResult, is(nullValue()));
+    assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
+    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+  }
+
+  @Test
+  public void finishBundleThrowsExceptionCallsback() throws Exception {
+    final Exception exception = new Exception();
+    TransformEvaluator<String> evaluator =
+        new TransformEvaluator<String>() {
+          @Override
+          public void processElement(WindowedValue<String> element) throws Exception {}
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            throw exception;
+          }
+        };
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(created).commit(Instant.now());
+    when(
+            registry.<String>forApplication(
+                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<String> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>emptyList(),
+            evaluationContext,
+            inputBundle,
+            downstream.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+    Executors.newSingleThreadExecutor().submit(executor);
+
+    evaluatorCompleted.await();
+
+    assertThat(completionCallback.handledResult, is(nullValue()));
+    assertThat(completionCallback.handledThrowable, Matchers.<Throwable>equalTo(exception));
+    assertThat(scheduled, not(Matchers.<TransformExecutor<?>>hasKey(executor)));
+  }
+
+  @Test
+  public void duringCallGetThreadIsNonNull() throws Exception {
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+    final CountDownLatch testLatch = new CountDownLatch(1);
+    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+    TransformEvaluator<Object> evaluator =
+        new TransformEvaluator<Object>() {
+          @Override
+          public void processElement(WindowedValue<Object> element) throws Exception {
+            throw new IllegalArgumentException("Shouldn't be called");
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            testLatch.countDown();
+            evaluatorLatch.await();
+            return result;
+          }
+        };
+
+    when(registry.forApplication(created.getProducingTransformInternal(), null, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<String> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>emptyList(),
+            evaluationContext,
+            null,
+            created.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+
+    Executors.newSingleThreadExecutor().submit(executor);
+    testLatch.await();
+    assertThat(executor.getThread(), not(nullValue()));
+
+    // Finish the execution so everything can get closed down cleanly.
+    evaluatorLatch.countDown();
+  }
+
+  @Test
+  public void callWithEnforcementAppliesEnforcement() throws Exception {
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+
+    TransformEvaluator<Object> evaluator =
+        new TransformEvaluator<Object>() {
+          @Override
+          public void processElement(WindowedValue<Object> element) throws Exception {
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            return result;
+          }
+        };
+
+    WindowedValue<String> fooElem = WindowedValue.valueInGlobalWindow("foo");
+    WindowedValue<String> barElem = WindowedValue.valueInGlobalWindow("bar");
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(created).add(fooElem).add(barElem).commit(Instant.now());
+    when(
+            registry.forApplication(
+                downstream.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TestEnforcementFactory enforcement = new TestEnforcementFactory();
+    TransformExecutor<String> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>singleton(enforcement),
+            evaluationContext,
+            inputBundle,
+            downstream.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+
+    executor.call();
+    TestEnforcement<?> testEnforcement = enforcement.instance;
+    assertThat(
+        testEnforcement.beforeElements,
+        Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
+    assertThat(
+        testEnforcement.afterElements,
+        Matchers.<WindowedValue<?>>containsInAnyOrder(barElem, fooElem));
+    assertThat(testEnforcement.finishedBundles, contains(result));
+  }
+
+  @Test
+  public void callWithEnforcementThrowsOnFinishPropagates() throws Exception {
+    PCollection<byte[]> pcBytes =
+        created.apply(
+            new PTransform<PCollection<String>, PCollection<byte[]>>() {
+              @Override
+              public PCollection<byte[]> apply(PCollection<String> input) {
+                return PCollection.<byte[]>createPrimitiveOutputInternal(
+                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+                    .setCoder(ByteArrayCoder.of());
+              }
+            });
+
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
+    final CountDownLatch testLatch = new CountDownLatch(1);
+    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+
+    TransformEvaluator<Object> evaluator =
+        new TransformEvaluator<Object>() {
+          @Override
+          public void processElement(WindowedValue<Object> element) throws Exception {}
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            testLatch.countDown();
+            evaluatorLatch.await();
+            return result;
+          }
+        };
+
+    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
+    CommittedBundle<byte[]> inputBundle =
+        bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
+    when(
+            registry.forApplication(
+                pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<byte[]> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+            evaluationContext,
+            inputBundle,
+            pcBytes.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+
+    Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
+    testLatch.await();
+    fooBytes.getValue()[0] = 'b';
+    evaluatorLatch.countDown();
+
+    thrown.expectCause(isA(IllegalMutationException.class));
+    task.get();
+  }
+
+  @Test
+  public void callWithEnforcementThrowsOnElementPropagates() throws Exception {
+    PCollection<byte[]> pcBytes =
+        created.apply(
+            new PTransform<PCollection<String>, PCollection<byte[]>>() {
+              @Override
+              public PCollection<byte[]> apply(PCollection<String> input) {
+                return PCollection.<byte[]>createPrimitiveOutputInternal(
+                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
+                    .setCoder(ByteArrayCoder.of());
+              }
+            });
+
+    final InProcessTransformResult result =
+        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
+    final CountDownLatch testLatch = new CountDownLatch(1);
+    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+
+    TransformEvaluator<Object> evaluator =
+        new TransformEvaluator<Object>() {
+          @Override
+          public void processElement(WindowedValue<Object> element) throws Exception {
+            testLatch.countDown();
+            evaluatorLatch.await();
+          }
+
+          @Override
+          public InProcessTransformResult finishBundle() throws Exception {
+            return result;
+          }
+        };
+
+    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
+    CommittedBundle<byte[]> inputBundle =
+        bundleFactory.createRootBundle(pcBytes).add(fooBytes).commit(Instant.now());
+    when(
+            registry.forApplication(
+                pcBytes.getProducingTransformInternal(), inputBundle, evaluationContext))
+        .thenReturn(evaluator);
+
+    TransformExecutor<byte[]> executor =
+        TransformExecutor.create(
+            registry,
+            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+            evaluationContext,
+            inputBundle,
+            pcBytes.getProducingTransformInternal(),
+            completionCallback,
+            transformEvaluationState);
+
+    Future<InProcessTransformResult> task = Executors.newSingleThreadExecutor().submit(executor);
+    testLatch.await();
+    fooBytes.getValue()[0] = 'b';
+    evaluatorLatch.countDown();
+
+    thrown.expectCause(isA(IllegalMutationException.class));
+    task.get();
+  }
+
+  private static class RegisteringCompletionCallback implements CompletionCallback {
+    private InProcessTransformResult handledResult = null;
+    private Throwable handledThrowable = null;
+    private final CountDownLatch onMethod;
+
+    private RegisteringCompletionCallback(CountDownLatch onMethod) {
+      this.onMethod = onMethod;
+    }
+
+    @Override
+    public CommittedResult handleResult(
+        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+      handledResult = result;
+      onMethod.countDown();
+      return CommittedResult.create(result, Collections.<CommittedBundle<?>>emptyList());
+    }
+
+    @Override
+    public void handleThrowable(CommittedBundle<?> inputBundle, Throwable t) {
+      handledThrowable = t;
+      onMethod.countDown();
+    }
+  }
+
+  private static class TestEnforcementFactory implements ModelEnforcementFactory {
+    private TestEnforcement<?> instance;
+    @Override
+    public <T> TestEnforcement<T> forBundle(
+        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+      TestEnforcement<T> newEnforcement = new TestEnforcement<>();
+      instance = newEnforcement;
+      return newEnforcement;
+    }
+  }
+
+  private static class TestEnforcement<T> implements ModelEnforcement<T> {
+    private final List<WindowedValue<T>> beforeElements = new ArrayList<>();
+    private final List<WindowedValue<T>> afterElements = new ArrayList<>();
+    private final List<InProcessTransformResult> finishedBundles = new ArrayList<>();
+
+    @Override
+    public void beforeElement(WindowedValue<T> element) {
+      beforeElements.add(element);
+    }
+
+    @Override
+    public void afterElement(WindowedValue<T> element) {
+      afterElements.add(element);
+    }
+
+    @Override
+    public void afterFinish(
+        CommittedBundle<T> input,
+        InProcessTransformResult result,
+        Iterable<? extends CommittedBundle<?>> outputs) {
+      finishedBundles.add(result);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
new file mode 100644
index 0000000..9f3909e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.joda.time.ReadableInstant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+/**
+ * Tests for {@link UnboundedReadEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class UnboundedReadEvaluatorFactoryTest {
+  private PCollection<Long> longs;
+  private TransformEvaluatorFactory factory;
+  private InProcessEvaluationContext context;
+  private UncommittedBundle<Long> output;
+
+  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Before
+  public void setup() {
+    UnboundedSource<Long, ?> source =
+        CountingSource.unboundedWithTimestampFn(new LongToInstantFn());
+    TestPipeline p = TestPipeline.create();
+    longs = p.apply(Read.from(source));
+
+    factory = new UnboundedReadEvaluatorFactory();
+    context = mock(InProcessEvaluationContext.class);
+    output = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(output);
+  }
+
+  @Test
+  public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws Exception {
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+    assertThat(
+        output.commit(Instant.now()).getElements(),
+        containsInAnyOrder(
+            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+            tgw(0L)));
+  }
+
+  /**
+   * Demonstrate that multiple sequential creations will produce additional elements if the source
+   * can provide them.
+   */
+  @Test
+  public void unboundedSourceInMemoryTransformEvaluatorMultipleSequentialCalls() throws Exception {
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+    InProcessTransformResult result = evaluator.finishBundle();
+    assertThat(
+        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+    assertThat(
+        output.commit(Instant.now()).getElements(),
+        containsInAnyOrder(
+            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+            tgw(0L)));
+
+    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+    when(context.createRootBundle(longs)).thenReturn(secondOutput);
+    TransformEvaluator<?> secondEvaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+    assertThat(
+        secondResult.getWatermarkHold(),
+        Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+    assertThat(
+        secondOutput.commit(Instant.now()).getElements(),
+        containsInAnyOrder(tgw(11L), tgw(12L), tgw(14L), tgw(18L), tgw(19L), tgw(17L), tgw(16L),
+            tgw(15L), tgw(13L), tgw(10L)));
+  }
+
+  @Test
+  public void boundedSourceEvaluatorClosesReader() throws Exception {
+    TestUnboundedSource<Long> source =
+        new TestUnboundedSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<Long> pcollection = p.apply(Read.from(source));
+    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+    when(context.createRootBundle(pcollection)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+    evaluator.finishBundle();
+    CommittedBundle<Long> committed = output.commit(Instant.now());
+    assertThat(ImmutableList.copyOf(committed.getElements()), hasSize(3));
+    assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
+  }
+
+  @Test
+  public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
+    TestUnboundedSource<Long> source = new TestUnboundedSource<>(BigEndianLongCoder.of());
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<Long> pcollection = p.apply(Read.from(source));
+    AppliedPTransform<?, ?, ?> sourceTransform = pcollection.getProducingTransformInternal();
+
+    UncommittedBundle<Long> output = bundleFactory.createRootBundle(pcollection);
+    when(context.createRootBundle(pcollection)).thenReturn(output);
+
+    TransformEvaluator<?> evaluator = factory.forApplication(sourceTransform, null, context);
+    evaluator.finishBundle();
+    CommittedBundle<Long> committed = output.commit(Instant.now());
+    assertThat(committed.getElements(), emptyIterable());
+    assertThat(TestUnboundedSource.readerClosedCount, equalTo(1));
+  }
+
+  // TODO: Once the source is split into multiple sources before evaluating, this test will have to
+  // be updated.
+  /**
+   * Demonstrate that only a single unfinished instance of TransformEvaluator can be created at a
+   * time, with other calls returning an empty evaluator.
+   */
+  @Test
+  public void unboundedSourceWithMultipleSimultaneousEvaluatorsIndependent() throws Exception {
+    UncommittedBundle<Long> secondOutput = bundleFactory.createRootBundle(longs);
+
+    TransformEvaluator<?> evaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+    TransformEvaluator<?> secondEvaluator =
+        factory.forApplication(longs.getProducingTransformInternal(), null, context);
+
+    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+    InProcessTransformResult result = evaluator.finishBundle();
+
+    assertThat(
+        result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
+    assertThat(
+        output.commit(Instant.now()).getElements(),
+        containsInAnyOrder(
+            tgw(1L), tgw(2L), tgw(4L), tgw(8L), tgw(9L), tgw(7L), tgw(6L), tgw(5L), tgw(3L),
+            tgw(0L)));
+
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+    assertThat(secondOutput.commit(Instant.now()).getElements(), emptyIterable());
+  }
+
+  /**
+   * A terse alias for producing timestamped longs in the {@link GlobalWindow}, where
+   * the timestamp is the epoch offset by the value of the element.
+   */
+  private static WindowedValue<Long> tgw(Long elem) {
+    return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem));
+  }
+
+  private static class LongToInstantFn implements SerializableFunction<Long, Instant> {
+    @Override
+    public Instant apply(Long input) {
+      return new Instant(input);
+    }
+  }
+
+  private static class TestUnboundedSource<T> extends UnboundedSource<T, TestCheckpointMark> {
+    static int readerClosedCount;
+    private final Coder<T> coder;
+    private final List<T> elems;
+
+    public TestUnboundedSource(Coder<T> coder, T... elems) {
+      readerClosedCount = 0;
+      this.coder = coder;
+      this.elems = Arrays.asList(elems);
+    }
+
+    @Override
+    public List<? extends UnboundedSource<T, TestCheckpointMark>> generateInitialSplits(
+        int desiredNumSplits, PipelineOptions options) throws Exception {
+      return ImmutableList.of(this);
+    }
+
+    @Override
+    public UnboundedSource.UnboundedReader<T> createReader(
+        PipelineOptions options, TestCheckpointMark checkpointMark) {
+      return new TestUnboundedReader(elems);
+    }
+
+    @Override
+    @Nullable
+    public Coder<TestCheckpointMark> getCheckpointMarkCoder() {
+      return new TestCheckpointMark.Coder();
+    }
+
+    @Override
+    public void validate() {}
+
+    @Override
+    public Coder<T> getDefaultOutputCoder() {
+      return coder;
+    }
+
+    private class TestUnboundedReader extends UnboundedReader<T> {
+      private final List<T> elems;
+      private int index;
+
+      public TestUnboundedReader(List<T> elems) {
+        this.elems = elems;
+        this.index = -1;
+      }
+
+      @Override
+      public boolean start() throws IOException {
+        return advance();
+      }
+
+      @Override
+      public boolean advance() throws IOException {
+        if (index + 1 < elems.size()) {
+          index++;
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public Instant getWatermark() {
+        return Instant.now();
+      }
+
+      @Override
+      public CheckpointMark getCheckpointMark() {
+        return new TestCheckpointMark();
+      }
+
+      @Override
+      public UnboundedSource<T, ?> getCurrentSource() {
+        TestUnboundedSource<T> source = TestUnboundedSource.this;
+        return source;
+      }
+
+      @Override
+      public T getCurrent() throws NoSuchElementException {
+        return elems.get(index);
+      }
+
+      @Override
+      public Instant getCurrentTimestamp() throws NoSuchElementException {
+        return Instant.now();
+      }
+
+      @Override
+      public void close() throws IOException {
+        readerClosedCount++;
+      }
+    }
+  }
+
+  private static class TestCheckpointMark implements CheckpointMark {
+    @Override
+    public void finalizeCheckpoint() throws IOException {}
+
+    public static class Coder extends AtomicCoder<TestCheckpointMark> {
+      @Override
+      public void encode(
+          TestCheckpointMark value,
+          OutputStream outStream,
+          org.apache.beam.sdk.coders.Coder.Context context)
+          throws CoderException, IOException {}
+
+      @Override
+      public TestCheckpointMark decode(
+          InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
+          throws CoderException, IOException {
+        return new TestCheckpointMark();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
new file mode 100644
index 0000000..859418b
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.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.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link ViewEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ViewEvaluatorFactoryTest {
+  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+
+  @Test
+  public void testInMemoryEvaluator() throws Exception {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<String> input = p.apply(Create.of("foo", "bar"));
+    CreatePCollectionView<String, Iterable<String>> createView =
+        CreatePCollectionView.of(
+            PCollectionViews.iterableView(p, input.getWindowingStrategy(), StringUtf8Coder.of()));
+    PCollection<Iterable<String>> concat =
+        input.apply(WithKeys.<Void, String>of((Void) null))
+            .setCoder(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of()))
+            .apply(GroupByKey.<Void, String>create())
+            .apply(Values.<Iterable<String>>create());
+    PCollectionView<Iterable<String>> view =
+        concat.apply(new ViewEvaluatorFactory.WriteView<>(createView));
+
+    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+    TestViewWriter<String, Iterable<String>> viewWriter = new TestViewWriter<>();
+    when(context.createPCollectionViewWriter(concat, view)).thenReturn(viewWriter);
+
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createRootBundle(input).commit(Instant.now());
+    TransformEvaluator<Iterable<String>> evaluator =
+        new ViewEvaluatorFactory()
+            .forApplication(view.getProducingTransformInternal(), inputBundle, context);
+
+    evaluator.processElement(
+        WindowedValue.<Iterable<String>>valueInGlobalWindow(ImmutableList.of("foo", "bar")));
+    assertThat(viewWriter.latest, nullValue());
+
+    evaluator.finishBundle();
+    assertThat(
+        viewWriter.latest,
+        containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar")));
+  }
+
+  private static class TestViewWriter<ElemT, ViewT> implements PCollectionViewWriter<ElemT, ViewT> {
+    private Iterable<WindowedValue<ElemT>> latest;
+
+    @Override
+    public void add(Iterable<WindowedValue<ElemT>> values) {
+      latest = values;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
new file mode 100644
index 0000000..d47cf5e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+
+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 java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for {@link WatermarkCallbackExecutor}.
+ */
+@RunWith(JUnit4.class)
+public class WatermarkCallbackExecutorTest {
+  private WatermarkCallbackExecutor executor = WatermarkCallbackExecutor.create();
+  private AppliedPTransform<?, ?, ?> create;
+  private AppliedPTransform<?, ?, ?> sum;
+
+  @Before
+  public void setup() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
+    create = created.getProducingTransformInternal();
+    sum = created.apply(Sum.integersGlobally()).getProducingTransformInternal();
+  }
+
+  @Test
+  public void onGuaranteedFiringFiresAfterTrigger() throws Exception {
+    CountDownLatch latch = new CountDownLatch(1);
+    executor.callOnGuaranteedFiring(
+        create,
+        GlobalWindow.INSTANCE,
+        WindowingStrategy.globalDefault(),
+        new CountDownLatchCallback(latch));
+
+    executor.fireForWatermark(create, BoundedWindow.TIMESTAMP_MAX_VALUE);
+    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
+  }
+
+  @Test
+  public void multipleCallbacksShouldFireFires() throws Exception {
+    CountDownLatch latch = new CountDownLatch(2);
+    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+    IntervalWindow window =
+        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+    executor.callOnGuaranteedFiring(
+        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+    executor.callOnGuaranteedFiring(
+        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(10)));
+    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true));
+  }
+
+  @Test
+  public void noCallbacksShouldFire() throws Exception {
+    CountDownLatch latch = new CountDownLatch(1);
+    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+    IntervalWindow window =
+        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+    executor.callOnGuaranteedFiring(
+        create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(5)));
+    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
+  }
+
+  @Test
+  public void unrelatedStepShouldNotFire() throws Exception {
+    CountDownLatch latch = new CountDownLatch(1);
+    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(Duration.standardMinutes(10));
+    IntervalWindow window =
+        new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10)));
+    executor.callOnGuaranteedFiring(
+        sum, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch));
+
+    executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(20)));
+    assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false));
+  }
+
+  private static class CountDownLatchCallback implements Runnable {
+    private final CountDownLatch latch;
+
+    public CountDownLatchCallback(CountDownLatch latch) {
+      this.latch = latch;
+    }
+
+    @Override
+    public void run() {
+      latch.countDown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
new file mode 100644
index 0000000..64eb8ea
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+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.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.Window.Bound;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+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.MockitoAnnotations;
+
+/**
+ * Tests for {@link WindowEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class WindowEvaluatorFactoryTest {
+  private static final Instant EPOCH = new Instant(0);
+
+  private PCollection<Long> input;
+  private WindowEvaluatorFactory factory;
+
+  @Mock private InProcessEvaluationContext evaluationContext;
+
+  private BundleFactory bundleFactory;
+
+  private WindowedValue<Long> first =
+      WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L));
+  private WindowedValue<Long> second =
+      WindowedValue.timestampedValueInGlobalWindow(
+          Long.valueOf(1L), EPOCH.plus(Duration.standardDays(3)));
+  private WindowedValue<Long> third =
+      WindowedValue.of(
+          Long.valueOf(2L),
+          new Instant(-10L),
+          new IntervalWindow(new Instant(-100), EPOCH),
+          PaneInfo.NO_FIRING);
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    TestPipeline p = TestPipeline.create();
+    input = p.apply(Create.of(1L, 2L, 3L));
+
+    bundleFactory = InProcessBundleFactory.create();
+    factory = new WindowEvaluatorFactory();
+  }
+
+  @Test
+  public void nullWindowFunSucceeds() throws Exception {
+    Bound<Long> transform =
+        Window.<Long>triggering(
+                AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(1)))
+            .accumulatingFiredPanes();
+    PCollection<Long> triggering = input.apply(transform);
+
+    CommittedBundle<Long> inputBundle = createInputBundle();
+
+    UncommittedBundle<Long> outputBundle = createOutputBundle(triggering, inputBundle);
+
+    InProcessTransformResult result = runEvaluator(triggering, inputBundle, transform);
+
+    assertThat(
+        Iterables.getOnlyElement(result.getOutputBundles()),
+        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+    assertThat(committed.getElements(), containsInAnyOrder(third, first, second));
+  }
+
+  @Test
+  public void singleWindowFnSucceeds() throws Exception {
+    Duration windowDuration = Duration.standardDays(7);
+    Bound<Long> transform = Window.<Long>into(FixedWindows.of(windowDuration));
+    PCollection<Long> windowed = input.apply(transform);
+
+    CommittedBundle<Long> inputBundle = createInputBundle();
+
+    UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
+
+    BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
+    BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
+
+    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+
+    assertThat(
+        Iterables.getOnlyElement(result.getOutputBundles()),
+        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+
+    WindowedValue<Long> expectedNewFirst =
+        WindowedValue.of(3L, new Instant(2L), firstSecondWindow, PaneInfo.NO_FIRING);
+    WindowedValue<Long> expectedNewSecond =
+        WindowedValue.of(
+            1L, EPOCH.plus(Duration.standardDays(3)), firstSecondWindow, PaneInfo.NO_FIRING);
+    WindowedValue<Long> expectedNewThird =
+        WindowedValue.of(2L, new Instant(-10L), thirdWindow, PaneInfo.NO_FIRING);
+    assertThat(
+        committed.getElements(),
+        containsInAnyOrder(expectedNewFirst, expectedNewSecond, expectedNewThird));
+  }
+
+  @Test
+  public void multipleWindowsWindowFnSucceeds() throws Exception {
+    Duration windowDuration = Duration.standardDays(6);
+    Duration slidingBy = Duration.standardDays(3);
+    Bound<Long> transform = Window.into(SlidingWindows.of(windowDuration).every(slidingBy));
+    PCollection<Long> windowed = input.apply(transform);
+
+    CommittedBundle<Long> inputBundle = createInputBundle();
+    UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
+
+    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+
+    assertThat(
+        Iterables.getOnlyElement(result.getOutputBundles()),
+        Matchers.<UncommittedBundle<?>>equalTo(outputBundle));
+    CommittedBundle<Long> committed = outputBundle.commit(Instant.now());
+
+    BoundedWindow w1 = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
+    BoundedWindow w2 =
+        new IntervalWindow(EPOCH.plus(slidingBy), EPOCH.plus(slidingBy).plus(windowDuration));
+    BoundedWindow wMinus1 = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
+    BoundedWindow wMinusSlide =
+        new IntervalWindow(EPOCH.minus(windowDuration).plus(slidingBy), EPOCH.plus(slidingBy));
+
+    WindowedValue<Long> expectedFirst =
+        WindowedValue.of(
+            first.getValue(),
+            first.getTimestamp(),
+            ImmutableSet.of(w1, wMinusSlide),
+            PaneInfo.NO_FIRING);
+    WindowedValue<Long> expectedSecond =
+        WindowedValue.of(
+            second.getValue(), second.getTimestamp(), ImmutableSet.of(w1, w2), PaneInfo.NO_FIRING);
+    WindowedValue<Long> expectedThird =
+        WindowedValue.of(
+            third.getValue(),
+            third.getTimestamp(),
+            ImmutableSet.of(wMinus1, wMinusSlide),
+            PaneInfo.NO_FIRING);
+
+    assertThat(
+        committed.getElements(), containsInAnyOrder(expectedFirst, expectedSecond, expectedThird));
+  }
+
+  private CommittedBundle<Long> createInputBundle() {
+    CommittedBundle<Long> inputBundle =
+        bundleFactory
+            .createRootBundle(input)
+            .add(first)
+            .add(second)
+            .add(third)
+            .commit(Instant.now());
+    return inputBundle;
+  }
+
+  private UncommittedBundle<Long> createOutputBundle(
+      PCollection<Long> output, CommittedBundle<Long> inputBundle) {
+    UncommittedBundle<Long> outputBundle = bundleFactory.createBundle(inputBundle, output);
+    when(evaluationContext.createBundle(inputBundle, output)).thenReturn(outputBundle);
+    return outputBundle;
+  }
+
+  private InProcessTransformResult runEvaluator(
+      PCollection<Long> windowed,
+      CommittedBundle<Long> inputBundle,
+      Window.Bound<Long> windowTransform /* Required while Window.Bound is a composite */)
+      throws Exception {
+    TransformEvaluator<Long> evaluator =
+        factory.forApplication(
+            AppliedPTransform.of("Window", input, windowed, windowTransform),
+            inputBundle,
+            evaluationContext);
+
+    evaluator.processElement(first);
+    evaluator.processElement(second);
+    evaluator.processElement(third);
+    InProcessTransformResult result = evaluator.finishBundle();
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 7b2f356..74812e8 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -35,6 +35,7 @@
   <name>Apache Beam :: Runners</name>
 
   <modules>
+    <module>direct-java</module>
     <module>flink</module>
     <module>spark</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
deleted file mode 100644
index fe9c165..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AbstractModelEnforcement.java
+++ /dev/null
@@ -1,38 +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.runners.inprocess;
-
-import org.apache.beam.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An abstract {@link ModelEnforcement} that provides default empty implementations for each method.
- */
-abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
-  @Override
-  public void beforeElement(WindowedValue<T> element) {}
-
-  @Override
-  public void afterElement(WindowedValue<T> element) {}
-
-  @Override
-  public void afterFinish(
-      CommittedBundle<T> input,
-      InProcessTransformResult result,
-      Iterable<? extends CommittedBundle<?>> outputs) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e13cacb8/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
deleted file mode 100644
index e19ffe4..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/inprocess/AvroIOShardedWriteFactory.java
+++ /dev/null
@@ -1,76 +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.runners.inprocess;
-
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class AvroIOShardedWriteFactory implements PTransformOverrideFactory {
-  @Override
-  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform) {
-    if (transform instanceof AvroIO.Write.Bound) {
-      @SuppressWarnings("unchecked")
-      AvroIO.Write.Bound<InputT> originalWrite = (AvroIO.Write.Bound<InputT>) transform;
-      if (originalWrite.getNumShards() > 1
-          || (originalWrite.getNumShards() == 1
-              && !"".equals(originalWrite.getShardNameTemplate()))) {
-        @SuppressWarnings("unchecked")
-        PTransform<InputT, OutputT> override =
-            (PTransform<InputT, OutputT>) new AvroIOShardedWrite<InputT>(originalWrite);
-        return override;
-      }
-    }
-    return transform;
-  }
-
-  private class AvroIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
-    private final AvroIO.Write.Bound<InputT> initial;
-
-    private AvroIOShardedWrite(AvroIO.Write.Bound<InputT> initial) {
-      this.initial = initial;
-    }
-
-    @Override
-    int getNumShards() {
-      return initial.getNumShards();
-    }
-
-    @Override
-    PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
-      String shardName =
-          IOChannelUtils.constructName(
-              initial.getFilenamePrefix(),
-              initial.getShardNameTemplate(),
-              initial.getFilenameSuffix(),
-              shardNum,
-              getNumShards());
-      return initial.withoutSharding().to(shardName).withSuffix("");
-    }
-
-    @Override
-    protected PTransform<PCollection<InputT>, PDone> delegate() {
-      return initial;
-    }
-  }
-}