You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2016/06/20 22:15:54 UTC

[01/50] [abbrv] incubator-beam git commit: Remove the DirectPipelineRunner from the Core SDK

Repository: incubator-beam
Updated Branches:
  refs/heads/python-sdk 0e5c662b4 -> f166b16b8


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index ea708e5..8abfb05 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -28,8 +28,8 @@ import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.testing.CrashingRunner;
 import org.apache.beam.sdk.testing.ExpectedLogs;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -157,7 +157,7 @@ public class PipelineTest {
   @Test
   public void testToString() {
     PipelineOptions options = PipelineOptionsFactory.as(PipelineOptions.class);
-    options.setRunner(DirectPipelineRunner.class);
+    options.setRunner(CrashingRunner.class);
     Pipeline pipeline = Pipeline.create(options);
     assertEquals("Pipeline#" + pipeline.hashCode(), pipeline.toString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
index 774968f..cabfc21 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
@@ -18,14 +18,12 @@
 package org.apache.beam.sdk.io;
 
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
-
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -123,10 +121,6 @@ public class BoundedReadFromUnboundedSourceTest implements Serializable{
   private void test(boolean dedup, boolean timeBound) throws Exception {
     Pipeline p = TestPipeline.create();
 
-    if (p.getOptions().getRunner() == DirectPipelineRunner.class) {
-      finalizeTracker = new ArrayList<>();
-      TestCountingSource.setFinalizeTracker(finalizeTracker);
-    }
     TestCountingSource source = new TestCountingSource(Integer.MAX_VALUE).withoutSplitting();
     if (dedup) {
       source = source.withDedup();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.java
deleted file mode 100644
index 92c4835..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRegistrarTest.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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-
-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 DirectPipelineRegistrar}. */
-@RunWith(JUnit4.class)
-public class DirectPipelineRegistrarTest {
-  @Test
-  public void testCorrectOptionsAreReturned() {
-    assertEquals(ImmutableList.of(DirectPipelineOptions.class),
-        new DirectPipelineRegistrar.Options().getPipelineOptions());
-  }
-
-  @Test
-  public void testCorrectRunnersAreReturned() {
-    assertEquals(ImmutableList.of(DirectPipelineRunner.class),
-        new DirectPipelineRegistrar.Runner().getPipelineRunners());
-  }
-
-  @Test
-  public void testServiceLoaderForOptions() {
-    for (PipelineOptionsRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
-      if (registrar instanceof DirectPipelineRegistrar.Options) {
-        return;
-      }
-    }
-    fail("Expected to find " + DirectPipelineRegistrar.Options.class);
-  }
-
-  @Test
-  public void testServiceLoaderForRunner() {
-    for (PipelineRunnerRegistrar registrar :
-        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
-      if (registrar instanceof DirectPipelineRegistrar.Runner) {
-        return;
-      }
-    }
-    fail("Expected to find " + DirectPipelineRegistrar.Runner.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.java
deleted file mode 100644
index edf6996..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/DirectPipelineRunnerTest.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;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.AvroCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.ShardNameTemplate;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-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.ParDo;
-import org.apache.beam.sdk.util.IOChannelUtils;
-
-import com.google.common.collect.Iterables;
-import com.google.common.io.Files;
-
-import org.apache.avro.file.DataFileReader;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-
-/** Tests for {@link DirectPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class DirectPipelineRunnerTest implements Serializable {
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-
-  @Test
-  public void testToString() {
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    DirectPipelineRunner runner = DirectPipelineRunner.fromOptions(options);
-    assertEquals("DirectPipelineRunner#" + runner.hashCode(),
-        runner.toString());
-  }
-
-  /** A {@link Coder} that fails during decoding. */
-  private static class CrashingCoder<T> extends AtomicCoder<T> {
-    @Override
-    public void encode(T value, OutputStream stream, Context context) throws CoderException {
-      throw new CoderException("Called CrashingCoder.encode");
-    }
-
-    @Override
-    public T decode(
-        InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context)
-            throws CoderException {
-      throw new CoderException("Called CrashingCoder.decode");
-    }
-  }
-
-  /** A {@link DoFn} that outputs {@code 'hello'}. */
-  private static class HelloDoFn extends DoFn<Integer, String> {
-    @Override
-    public void processElement(DoFn<Integer, String>.ProcessContext c) throws Exception {
-      c.output("hello");
-    }
-  }
-
-  @Test
-  public void testCoderException() {
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-
-    p.apply("CreateTestData", Create.of(42))
-        .apply("CrashDuringCoding", ParDo.of(new HelloDoFn()))
-        .setCoder(new CrashingCoder<String>());
-
-    expectedException.expect(RuntimeException.class);
-    expectedException.expectCause(isA(CoderException.class));
-    p.run();
-  }
-
-  @Test
-  public void testDirectPipelineOptions() {
-    DirectPipelineOptions options = PipelineOptionsFactory.create().as(DirectPipelineOptions.class);
-    assertNull(options.getDirectPipelineRunnerRandomSeed());
-  }
-
-  @Test
-  public void testTextIOWriteWithDefaultShardingStrategy() throws Exception {
-    String prefix = IOChannelUtils.resolve(Files.createTempDir().toString(), "output");
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    String[] expectedElements = new String[]{ "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-    p.apply(Create.of(expectedElements))
-     .apply(TextIO.Write.to(prefix).withSuffix("txt"));
-    p.run();
-
-    String filename =
-        IOChannelUtils.constructName(prefix, ShardNameTemplate.INDEX_OF_MAX, ".txt", 0, 1);
-    List<String> fileContents =
-        Files.readLines(new File(filename), StandardCharsets.UTF_8);
-    // Ensure that each file got at least one record
-    assertFalse(fileContents.isEmpty());
-
-    assertThat(fileContents, containsInAnyOrder(expectedElements));
-  }
-
-  @Test
-  public void testTextIOWriteWithLimitedNumberOfShards() throws Exception {
-    final int numShards = 3;
-    String prefix = IOChannelUtils.resolve(Files.createTempDir().toString(), "shardedOutput");
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    String[] expectedElements = new String[]{ "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-    p.apply(Create.of(expectedElements))
-     .apply(TextIO.Write.to(prefix).withNumShards(numShards).withSuffix("txt"));
-    p.run();
-
-    List<String> allContents = new ArrayList<>();
-    for (int i = 0; i < numShards; ++i) {
-      String shardFileName =
-          IOChannelUtils.constructName(prefix, ShardNameTemplate.INDEX_OF_MAX, ".txt", i, 3);
-      List<String> shardFileContents =
-          Files.readLines(new File(shardFileName), StandardCharsets.UTF_8);
-
-      // Ensure that each file got at least one record
-      assertFalse(shardFileContents.isEmpty());
-
-      allContents.addAll(shardFileContents);
-    }
-
-    assertThat(allContents, containsInAnyOrder(expectedElements));
-  }
-
-  @Test
-  public void testAvroIOWriteWithDefaultShardingStrategy() throws Exception {
-    String prefix = IOChannelUtils.resolve(Files.createTempDir().toString(), "output");
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    String[] expectedElements = new String[]{ "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-    p.apply(Create.of(expectedElements))
-     .apply(AvroIO.Write.withSchema(String.class).to(prefix).withSuffix(".avro"));
-    p.run();
-
-    String filename =
-        IOChannelUtils.constructName(prefix, ShardNameTemplate.INDEX_OF_MAX, ".avro", 0, 1);
-    List<String> fileContents = new ArrayList<>();
-    Iterables.addAll(fileContents, DataFileReader.openReader(
-        new File(filename), AvroCoder.of(String.class).createDatumReader()));
-
-    // Ensure that each file got at least one record
-    assertFalse(fileContents.isEmpty());
-
-    assertThat(fileContents, containsInAnyOrder(expectedElements));
-  }
-
-  @Test
-  public void testAvroIOWriteWithLimitedNumberOfShards() throws Exception {
-    final int numShards = 3;
-    String prefix = IOChannelUtils.resolve(Files.createTempDir().toString(), "shardedOutput");
-    PipelineOptions options = PipelineOptionsFactory.create();
-    options.setRunner(DirectPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-    String[] expectedElements = new String[]{ "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-    p.apply(Create.of(expectedElements))
-     .apply(AvroIO.Write.withSchema(String.class).to(prefix)
-                        .withNumShards(numShards).withSuffix(".avro"));
-    p.run();
-
-    List<String> allContents = new ArrayList<>();
-    for (int i = 0; i < numShards; ++i) {
-      String shardFileName =
-          IOChannelUtils.constructName(prefix, ShardNameTemplate.INDEX_OF_MAX, ".avro", i, 3);
-      List<String> shardFileContents = new ArrayList<>();
-      Iterables.addAll(shardFileContents, DataFileReader.openReader(
-          new File(shardFileName), AvroCoder.of(String.class).createDatumReader()));
-
-      // Ensure that each file got at least one record
-      assertFalse(shardFileContents.isEmpty());
-
-      allContents.addAll(shardFileContents);
-    }
-
-    assertThat(allContents, containsInAnyOrder(expectedElements));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index 9313439..5d2e69d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.DirectPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.CrashingRunner;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.TestCredential;
 
@@ -53,10 +54,10 @@ public class PipelineRunnerTest {
     options.setAppName("test");
     options.setProject("test");
     options.setGcsUtil(mockGcsUtil);
-    options.setRunner(DirectPipelineRunner.class);
+    options.setRunner(CrashingRunner.class);
     options.setGcpCredential(new TestCredential());
     PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
-    assertTrue(runner instanceof DirectPipelineRunner);
+    assertTrue(runner instanceof CrashingRunner);
   }
 
   @Test
@@ -66,10 +67,10 @@ public class PipelineRunnerTest {
     options.setAppName("test");
     options.setProject("test");
     options.setGcsUtil(mockGcsUtil);
-    options.setRunner(DirectPipelineRunner.class);
+    options.setRunner(CrashingRunner.class);
     options.setGcpCredential(new TestCredential());
     PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
-    assertTrue(runner instanceof DirectPipelineRunner);
+    assertTrue(runner instanceof CrashingRunner);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
index a0b508c..b0ca70b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
@@ -39,7 +39,6 @@ import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -56,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.Sessions;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.util.PerKeyCombineFnRunners;
 import org.apache.beam.sdk.util.PropertyNames;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.KV;
@@ -87,7 +85,6 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
-import java.util.Random;
 import java.util.Set;
 
 /**
@@ -516,24 +513,6 @@ public class CombineTest implements Serializable {
     assertThat(sum.getName(), Matchers.startsWith("Sum"));
   }
 
-  @Test
-  public void testAddInputsRandomly() {
-    TestCounter counter = new TestCounter();
-    Combine.KeyedCombineFn<
-        String, Integer, TestCounter.Counter, Iterable<Long>> fn =
-        counter.asKeyedFn();
-
-    List<TestCounter.Counter> accums = DirectPipelineRunner.TestCombineDoFn.addInputsRandomly(
-        PerKeyCombineFnRunners.create(fn), "bob", Arrays.asList(NUMBERS), new Random(42),
-        processContext);
-
-    assertThat(accums, Matchers.contains(
-        counter.new Counter(3, 2, 0, 0),
-        counter.new Counter(131, 5, 0, 0),
-        counter.new Counter(8, 2, 0, 0),
-        counter.new Counter(1, 1, 0, 0)));
-  }
-
   private static final SerializableFunction<String, Integer> hotKeyFanout =
       new SerializableFunction<String, Integer>() {
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
index 4ce025d..299def7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
@@ -30,9 +30,6 @@ import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.MapCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -244,15 +241,9 @@ public class GroupByKeyTest {
                     Duration.standardMinutes(1)))));
   }
 
-  private Pipeline createTestDirectRunner() {
-    DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class);
-    options.setRunner(DirectPipelineRunner.class);
-    return Pipeline.create(options);
-  }
-
   @Test
   public void testInvalidWindowsDirect() {
-    Pipeline p = createTestDirectRunner();
+    Pipeline p = TestPipeline.create();
 
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
@@ -297,7 +288,7 @@ public class GroupByKeyTest {
 
   @Test
   public void testGroupByKeyDirectUnbounded() {
-    Pipeline p = createTestDirectRunner();
+    Pipeline p = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input =
         p.apply(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 18d39d7..5e6e6a3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -33,9 +33,6 @@ import org.apache.beam.sdk.coders.NullableCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -1335,12 +1332,6 @@ public class ViewTest implements Serializable {
     assertEquals("View.AsMultimap", View.<String, Integer>asMultimap().getName());
   }
 
-  private Pipeline createTestDirectRunner() {
-    DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class);
-    options.setRunner(DirectPipelineRunner.class);
-    return Pipeline.create(options);
-  }
-
   private void testViewUnbounded(
       Pipeline pipeline,
       PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
@@ -1378,51 +1369,51 @@ public class ViewTest implements Serializable {
 
   @Test
   public void testViewUnboundedAsSingletonDirect() {
-    testViewUnbounded(createTestDirectRunner(), View.<KV<String, Integer>>asSingleton());
+    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asSingleton());
   }
 
   @Test
   public void testViewUnboundedAsIterableDirect() {
-    testViewUnbounded(createTestDirectRunner(), View.<KV<String, Integer>>asIterable());
+    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asIterable());
   }
 
   @Test
   public void testViewUnboundedAsListDirect() {
-    testViewUnbounded(createTestDirectRunner(), View.<KV<String, Integer>>asList());
+    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asList());
   }
 
   @Test
   public void testViewUnboundedAsMapDirect() {
-    testViewUnbounded(createTestDirectRunner(), View.<String, Integer>asMap());
+    testViewUnbounded(TestPipeline.create(), View.<String, Integer>asMap());
   }
 
   @Test
   public void testViewUnboundedAsMultimapDirect() {
-    testViewUnbounded(createTestDirectRunner(), View.<String, Integer>asMultimap());
+    testViewUnbounded(TestPipeline.create(), View.<String, Integer>asMultimap());
   }
 
   @Test
   public void testViewNonmergingAsSingletonDirect() {
-    testViewNonmerging(createTestDirectRunner(), View.<KV<String, Integer>>asSingleton());
+    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asSingleton());
   }
 
   @Test
   public void testViewNonmergingAsIterableDirect() {
-    testViewNonmerging(createTestDirectRunner(), View.<KV<String, Integer>>asIterable());
+    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asIterable());
   }
 
   @Test
   public void testViewNonmergingAsListDirect() {
-    testViewNonmerging(createTestDirectRunner(), View.<KV<String, Integer>>asList());
+    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asList());
   }
 
   @Test
   public void testViewNonmergingAsMapDirect() {
-    testViewNonmerging(createTestDirectRunner(), View.<String, Integer>asMap());
+    testViewNonmerging(TestPipeline.create(), View.<String, Integer>asMap());
   }
 
   @Test
   public void testViewNonmergingAsMultimapDirect() {
-    testViewNonmerging(createTestDirectRunner(), View.<String, Integer>asMultimap());
+    testViewNonmerging(TestPipeline.create(), View.<String, Integer>asMultimap());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
index 4914d4c..76df4d4 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
@@ -17,6 +17,7 @@
  */
 package ${package}.common;
 
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
 import org.apache.beam.runners.dataflow.DataflowPipelineJob;
 import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
@@ -39,7 +40,6 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.transforms.IntraBundleParallelization;
 import org.apache.beam.sdk.util.Transport;
 import com.google.common.collect.Lists;
@@ -250,17 +250,8 @@ public class DataflowExampleUtils {
     }
   }
 
-  /**
-   * Do some runner setup: check that the DirectPipelineRunner is not used in conjunction with
-   * streaming, and if streaming is specified, use the DataflowPipelineRunner. Return the streaming
-   * flag value.
-   */
   public void setupRunner() {
-    if (options.isStreaming()) {
-      if (options.getRunner() == DirectPipelineRunner.class) {
-        throw new IllegalArgumentException(
-          "Processing of unbounded input sources is not supported with the DirectPipelineRunner.");
-      }
+    if (options.isStreaming() && options.getRunner().equals(BlockingDataflowPipelineRunner.class)) {
       // In order to cancel the pipelines automatically,
       // {@literal DataflowPipelineRunner} is forced to be used.
       options.setRunner(DataflowPipelineRunner.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/testing/travis/test_wordcount.sh
----------------------------------------------------------------------
diff --git a/testing/travis/test_wordcount.sh b/testing/travis/test_wordcount.sh
index 40e2724..b00b0d6 100755
--- a/testing/travis/test_wordcount.sh
+++ b/testing/travis/test_wordcount.sh
@@ -70,7 +70,7 @@ function run_via_mvn {
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='mvn exec:java -f pom.xml -pl examples/java \
     -Dexec.mainClass=org.apache.beam.examples.WordCount \
-    -Dexec.args="--runner=DirectPipelineRunner --inputFile='"$input"' --output='"$outfile_prefix"'"'
+    -Dexec.args="--runner=InProcessPipelineRunner --inputFile='"$input"' --output='"$outfile_prefix"'"'
   echo "$name: Running $cmd" >&2
   sh -c "$cmd"
   check_result_hash "$name" "$outfile_prefix" "$expected_hash"
@@ -84,7 +84,7 @@ function run_bundled {
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='java -cp '"$JAR_FILE"' \
     org.apache.beam.examples.WordCount \
-    --runner=DirectPipelineRunner \
+    --runner=InProcessPipelineRunner \
     --inputFile='"'$input'"' \
     --output='"$outfile_prefix"
   echo "$name: Running $cmd" >&2


[06/50] [abbrv] incubator-beam git commit: Rename DoFnTester#processBatch to processBundle

Posted by da...@apache.org.
Rename DoFnTester#processBatch to processBundle

DoFns process elements in bundles, not batches.


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

Branch: refs/heads/python-sdk
Commit: 0065851b96644f2c75b8e51c95ebf0e79c5865f5
Parents: ecf6ab8
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 14 09:27:55 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/examples/WordCountTest.java |  6 +++---
 .../cookbook/BigQueryTornadoesTest.java         |  8 ++++----
 .../cookbook/CombinePerKeyExamplesTest.java     |  4 ++--
 .../examples/cookbook/FilterExamplesTest.java   |  4 ++--
 .../examples/cookbook/JoinExamplesTest.java     |  4 ++--
 .../cookbook/MaxPerKeyExamplesTest.java         |  4 ++--
 .../examples/cookbook/TriggerExampleTest.java   |  4 ++--
 .../examples/complete/game/UserScoreTest.java   |  2 +-
 .../dataflow/DataflowPipelineRunnerTest.java    | 20 ++++++++++----------
 .../apache/beam/sdk/transforms/DoFnTester.java  | 16 ++++++++--------
 .../beam/sdk/transforms/DoFnTesterTest.java     | 10 +++++-----
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  2 +-
 .../src/test/java/WordCountTest.java            |  6 +++---
 13 files changed, 45 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
index 1382612..ff117dc 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
@@ -52,11 +52,11 @@ public class WordCountTest {
     DoFnTester<String, String> extractWordsFn =
         DoFnTester.of(new ExtractWordsFn());
 
-    Assert.assertThat(extractWordsFn.processBatch(" some  input  words "),
+    Assert.assertThat(extractWordsFn.processBundle(" some  input  words "),
                       CoreMatchers.hasItems("some", "input", "words"));
-    Assert.assertThat(extractWordsFn.processBatch(" "),
+    Assert.assertThat(extractWordsFn.processBundle(" "),
                       CoreMatchers.<String>hasItems());
-    Assert.assertThat(extractWordsFn.processBatch(" some ", " input", " words"),
+    Assert.assertThat(extractWordsFn.processBundle(" some ", " input", " words"),
                       CoreMatchers.hasItems("some", "input", "words"));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
index 36be568..b986c0b 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java
@@ -45,7 +45,7 @@ public class BigQueryTornadoesTest {
           .set("tornado", true);
     DoFnTester<TableRow, Integer> extractWordsFn =
         DoFnTester.of(new ExtractTornadoesFn());
-    Assert.assertThat(extractWordsFn.processBatch(row),
+    Assert.assertThat(extractWordsFn.processBundle(row),
                       CoreMatchers.hasItems(6));
   }
 
@@ -56,7 +56,7 @@ public class BigQueryTornadoesTest {
           .set("tornado", false);
     DoFnTester<TableRow, Integer> extractWordsFn =
         DoFnTester.of(new ExtractTornadoesFn());
-    Assert.assertTrue(extractWordsFn.processBatch(row).isEmpty());
+    Assert.assertTrue(extractWordsFn.processBundle(row).isEmpty());
   }
 
   @Test
@@ -65,12 +65,12 @@ public class BigQueryTornadoesTest {
     DoFnTester<KV<Integer, Long>, TableRow> formatCountsFn =
         DoFnTester.of(new FormatCountsFn());
     KV empty[] = {};
-    List<TableRow> results = formatCountsFn.processBatch(empty);
+    List<TableRow> results = formatCountsFn.processBundle(empty);
     Assert.assertTrue(results.size() == 0);
     KV input[] = { KV.of(3, 0L),
                    KV.of(4, Long.MAX_VALUE),
                    KV.of(5, Long.MIN_VALUE) };
-    results = formatCountsFn.processBatch(input);
+    results = formatCountsFn.processBundle(input);
     Assert.assertEquals(results.size(), 3);
     Assert.assertEquals(results.get(0).get("month"), 3);
     Assert.assertEquals(results.get(0).get("tornado_count"), 0L);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
index 5d6456a..6d0b167 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java
@@ -75,7 +75,7 @@ public class CombinePerKeyExamplesTest {
   public void testExtractLargeWordsFn() throws Exception {
     DoFnTester<TableRow, KV<String, String>> extractLargeWordsFn =
         DoFnTester.of(new ExtractLargeWordsFn());
-    List<KV<String, String>> results = extractLargeWordsFn.processBatch(ROWS_ARRAY);
+    List<KV<String, String>> results = extractLargeWordsFn.processBundle(ROWS_ARRAY);
     Assert.assertThat(results, CoreMatchers.hasItem(tuple1));
     Assert.assertThat(results, CoreMatchers.hasItem(tuple2));
     Assert.assertThat(results, CoreMatchers.hasItem(tuple3));
@@ -85,7 +85,7 @@ public class CombinePerKeyExamplesTest {
   public void testFormatShakespeareOutputFn() throws Exception {
     DoFnTester<KV<String, String>, TableRow> formatShakespeareOutputFn =
         DoFnTester.of(new FormatShakespeareOutputFn());
-    List<TableRow> results = formatShakespeareOutputFn.processBatch(COMBINED_TUPLES_ARRAY);
+    List<TableRow> results = formatShakespeareOutputFn.processBundle(COMBINED_TUPLES_ARRAY);
     Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
     Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
index 13beab0..2598a97 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java
@@ -71,7 +71,7 @@ public class FilterExamplesTest {
   public void testProjectionFn() throws Exception {
     DoFnTester<TableRow, TableRow> projectionFn =
         DoFnTester.of(new ProjectionFn());
-    List<TableRow> results = projectionFn.processBatch(ROWS_ARRAY);
+    List<TableRow> results = projectionFn.processBundle(ROWS_ARRAY);
     Assert.assertThat(results, CoreMatchers.hasItem(outRow1));
     Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
     Assert.assertThat(results, CoreMatchers.hasItem(outRow3));
@@ -81,7 +81,7 @@ public class FilterExamplesTest {
   public void testFilterSingleMonthDataFn() throws Exception {
     DoFnTester<TableRow, TableRow> filterSingleMonthDataFn =
         DoFnTester.of(new FilterSingleMonthDataFn(7));
-    List<TableRow> results = filterSingleMonthDataFn.processBatch(PROJROWS_ARRAY);
+    List<TableRow> results = filterSingleMonthDataFn.processBundle(PROJROWS_ARRAY);
     Assert.assertThat(results, CoreMatchers.hasItem(outRow2));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
index 22fe6a1..9b04667 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
@@ -87,7 +87,7 @@ public class JoinExamplesTest {
   public void testExtractEventDataFn() throws Exception {
     DoFnTester<TableRow, KV<String, String>> extractEventDataFn =
         DoFnTester.of(new ExtractEventDataFn());
-    List<KV<String, String>> results = extractEventDataFn.processBatch(EVENTS);
+    List<KV<String, String>> results = extractEventDataFn.processBundle(EVENTS);
     Assert.assertThat(results, CoreMatchers.hasItem(kv1));
     Assert.assertThat(results, CoreMatchers.hasItem(kv2));
   }
@@ -96,7 +96,7 @@ public class JoinExamplesTest {
   public void testExtractCountryInfoFn() throws Exception {
     DoFnTester<TableRow, KV<String, String>> extractCountryInfoFn =
         DoFnTester.of(new ExtractCountryInfoFn());
-    List<KV<String, String>> results = extractCountryInfoFn.processBatch(CCS);
+    List<KV<String, String>> results = extractCountryInfoFn.processBundle(CCS);
     Assert.assertThat(results, CoreMatchers.hasItem(kv3));
     Assert.assertThat(results, CoreMatchers.hasItem(kv4));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
index 9e129a1..1d5bcf4 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java
@@ -69,7 +69,7 @@ public class MaxPerKeyExamplesTest {
   public void testExtractTempFn() throws Exception {
     DoFnTester<TableRow, KV<Integer, Double>> extractTempFn =
         DoFnTester.of(new ExtractTempFn());
-    List<KV<Integer, Double>> results = extractTempFn.processBatch(TEST_ROWS);
+    List<KV<Integer, Double>> results = extractTempFn.processBundle(TEST_ROWS);
     Assert.assertThat(results, CoreMatchers.hasItem(kv1));
     Assert.assertThat(results, CoreMatchers.hasItem(kv2));
     Assert.assertThat(results, CoreMatchers.hasItem(kv3));
@@ -79,7 +79,7 @@ public class MaxPerKeyExamplesTest {
   public void testFormatMaxesFn() throws Exception {
     DoFnTester<KV<Integer, Double>, TableRow> formatMaxesFnFn =
         DoFnTester.of(new FormatMaxesFn());
-    List<TableRow> results = formatMaxesFnFn.processBatch(TEST_KVS);
+    List<TableRow> results = formatMaxesFnFn.processBundle(TEST_KVS);
     Assert.assertThat(results, CoreMatchers.hasItem(resultRow1));
     Assert.assertThat(results, CoreMatchers.hasItem(resultRow2));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
index cddce7f..6f58389 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
@@ -101,12 +101,12 @@ public class TriggerExampleTest {
     DoFnTester<String, KV<String, Integer>> extractFlowInfow = DoFnTester
         .of(new ExtractFlowInfo());
 
-    List<KV<String, Integer>> results = extractFlowInfow.processBatch(INPUT);
+    List<KV<String, Integer>> results = extractFlowInfow.processBundle(INPUT);
     Assert.assertEquals(results.size(), 1);
     Assert.assertEquals(results.get(0).getKey(), "94");
     Assert.assertEquals(results.get(0).getValue(), new Integer(29));
 
-    List<KV<String, Integer>> output = extractFlowInfow.processBatch("");
+    List<KV<String, Integer>> output = extractFlowInfow.processBundle("");
     Assert.assertEquals(output.size(), 0);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
index 6f58114..cc3e7fa 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
@@ -89,7 +89,7 @@ public class UserScoreTest implements Serializable {
     DoFnTester<String, GameActionInfo> parseEventFn =
         DoFnTester.of(new ParseEventFn());
 
-    List<GameActionInfo> results = parseEventFn.processBatch(GAME_EVENTS_ARRAY);
+    List<GameActionInfo> results = parseEventFn.processBundle(GAME_EVENTS_ARRAY);
     Assert.assertEquals(results.size(), 8);
     Assert.assertEquals(results.get(0).getUser(), "user0_MagentaKangaroo");
     Assert.assertEquals(results.get(0).getTeam(), "MagentaKangaroo");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
index f7068b0..38d4c96 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
@@ -961,7 +961,7 @@ public class DataflowPipelineRunnerTest {
                    <String, GlobalWindow>(GlobalWindow.Coder.INSTANCE));
 
     assertThat(
-        doFnTester.processBatch(
+        doFnTester.processBundle(
             ImmutableList.of(KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(
                 0, ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")))))),
         contains(IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE), valueInGlobalWindow("a"))));
@@ -978,7 +978,7 @@ public class DataflowPipelineRunnerTest {
 
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("found for singleton within window");
-    doFnTester.processBatch(ImmutableList.of(
+    doFnTester.processBundle(ImmutableList.of(
         KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(0,
             ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")),
                 KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("b"))))));
@@ -990,7 +990,7 @@ public class DataflowPipelineRunnerTest {
         DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
 
     // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(ImmutableList.of("a", "b", "c")), contains(
+    assertThat(doFnTester.processBundle(ImmutableList.of("a", "b", "c")), contains(
         IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
         IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
         IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
@@ -1028,7 +1028,7 @@ public class DataflowPipelineRunnerTest {
                 )));
 
     // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
+    assertThat(doFnTester.processBundle(inputElements), contains(
         IsmRecord.of(ImmutableList.of(windowA, 0L),
             WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
         IsmRecord.of(ImmutableList.of(windowA, 1L),
@@ -1100,7 +1100,7 @@ public class DataflowPipelineRunnerTest {
                     WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
 
     // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
+    assertThat(doFnTester.processBundle(inputElements), contains(
         IsmRecord.of(
             ImmutableList.of(1L, windowA, 0L),
             WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
@@ -1188,7 +1188,7 @@ public class DataflowPipelineRunnerTest {
 
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("Unique keys are expected but found key");
-    doFnTester.processBatch(inputElements);
+    doFnTester.processBundle(inputElements);
   }
 
   @Test
@@ -1231,7 +1231,7 @@ public class DataflowPipelineRunnerTest {
                     KV.of(windowC, 9L))));
 
     // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
+    assertThat(doFnTester.processBundle(inputElements), contains(
         IsmRecord.<WindowedValue<Long>>meta(
             ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
             CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
@@ -1286,7 +1286,7 @@ public class DataflowPipelineRunnerTest {
                     KV.of(windowC, 3L))));
 
     // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBatch(inputElements), contains(
+    assertThat(doFnTester.processBundle(inputElements), contains(
         IsmRecord.<WindowedValue<Long>>meta(
             ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
             CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
@@ -1339,7 +1339,7 @@ public class DataflowPipelineRunnerTest {
     List<IsmRecord<WindowedValue<TransformedMap<Long,
                                                 WindowedValue<Long>,
                                                 Long>>>> output =
-                                                doFnTester.processBatch(inputElements);
+                                                doFnTester.processBundle(inputElements);
     assertEquals(3, output.size());
     Map<Long, Long> outputMap;
 
@@ -1396,7 +1396,7 @@ public class DataflowPipelineRunnerTest {
     List<IsmRecord<WindowedValue<TransformedMap<Long,
                                                 Iterable<WindowedValue<Long>>,
                                                 Iterable<Long>>>>> output =
-                                                doFnTester.processBatch(inputElements);
+                                                doFnTester.processBundle(inputElements);
     assertEquals(3, output.size());
     Map<Long, Iterable<Long>> outputMap;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 415af95..3ec749e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -57,18 +57,18 @@ import java.util.Map;
  *
  * DoFnTester<InputT, OutputT> fnTester = DoFnTester.of(fn);
  *
- * // Set arguments shared across all batches:
+ * // Set arguments shared across all bundles:
  * fnTester.setSideInputs(...);      // If fn takes side inputs.
  * fnTester.setSideOutputTags(...);  // If fn writes to side outputs.
  *
- * // Process a batch containing a single input element:
+ * // Process a bundle containing a single input element:
  * Input testInput = ...;
- * List<OutputT> testOutputs = fnTester.processBatch(testInput);
+ * List<OutputT> testOutputs = fnTester.processBundle(testInput);
  * Assert.assertThat(testOutputs,
  *                   JUnitMatchers.hasItems(...));
  *
- * // Process a bigger batch:
- * Assert.assertThat(fnTester.processBatch(i1, i2, ...),
+ * // Process a bigger bundle:
+ * Assert.assertThat(fnTester.processBundle(i1, i2, ...),
  *                   JUnitMatchers.hasItems(...));
  * } </pre>
  *
@@ -164,7 +164,7 @@ public class DoFnTester<InputT, OutputT> {
    * calls {@link #finishBundle}, then returns the result of
    * {@link #takeOutputElements}.
    */
-  public List<OutputT> processBatch(Iterable <? extends InputT> inputElements) throws Exception {
+  public List<OutputT> processBundle(Iterable <? extends InputT> inputElements) throws Exception {
     startBundle();
     for (InputT inputElement : inputElements) {
       processElement(inputElement);
@@ -185,8 +185,8 @@ public class DoFnTester<InputT, OutputT> {
    * </ol>
    */
   @SafeVarargs
-  public final List<OutputT> processBatch(InputT... inputElements) throws Exception {
-    return processBatch(Arrays.asList(inputElements));
+  public final List<OutputT> processBundle(InputT... inputElements) throws Exception {
+    return processBundle(Arrays.asList(inputElements));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
index 3261f85..b391671 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
@@ -127,8 +127,8 @@ public class DoFnTesterTest {
     CounterDoFn counterDoFn = new CounterDoFn();
     DoFnTester<Long, String> tester = DoFnTester.of(counterDoFn);
 
-    // processBatch() returns all the output like takeOutputElements().
-    List<String> take = tester.processBatch(1L, 2L, 3L, 4L);
+    // processBundle() returns all the output like takeOutputElements().
+    List<String> take = tester.processBundle(1L, 2L, 3L, 4L);
 
     assertThat(take, hasItems("1", "2", "3", "4"));
 
@@ -182,7 +182,7 @@ public class DoFnTesterTest {
   public void getAggregatorValuesShouldGetValueOfCounter() throws Exception {
     CounterDoFn counterDoFn = new CounterDoFn();
     DoFnTester<Long, String> tester = DoFnTester.of(counterDoFn);
-    tester.processBatch(1L, 2L, 4L, 8L);
+    tester.processBundle(1L, 2L, 4L, 8L);
 
     Long aggregatorVal = tester.getAggregatorValue(counterDoFn.agg);
 
@@ -193,7 +193,7 @@ public class DoFnTesterTest {
   public void getAggregatorValuesWithEmptyCounterShouldSucceed() throws Exception {
     CounterDoFn counterDoFn = new CounterDoFn();
     DoFnTester<Long, String> tester = DoFnTester.of(counterDoFn);
-    tester.processBatch();
+    tester.processBundle();
     Long aggregatorVal = tester.getAggregatorValue(counterDoFn.agg);
     // empty bundle
     assertThat(aggregatorVal, equalTo(0L));
@@ -203,7 +203,7 @@ public class DoFnTesterTest {
   public void getAggregatorValuesInStartFinishBundleShouldGetValues() throws Exception {
     CounterDoFn fn = new CounterDoFn(1L, 2L);
     DoFnTester<Long, String> tester = DoFnTester.of(fn);
-    tester.processBatch(0L, 0L);
+    tester.processBundle(0L, 0L);
 
     Long aggValue = tester.getAggregatorValue(fn.agg);
     assertThat(aggValue, equalTo(1L + 2L));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
index d99c536..10a2a7e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
@@ -438,7 +438,7 @@ public class CoGroupByKeyTest implements Serializable {
                 purchasesTag,
                 addressesTag,
                 namesTag))
-                .processBatch(
+                .processBundle(
                     KV.of(1, result1),
                     KV.of(2, result2),
                     KV.of(3, result3),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0065851b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
index 4b26198..debfc78 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
@@ -52,11 +52,11 @@ public class WordCountTest {
     DoFnTester<String, String> extractWordsFn =
         DoFnTester.of(new ExtractWordsFn());
 
-    Assert.assertThat(extractWordsFn.processBatch(" some  input  words "),
+    Assert.assertThat(extractWordsFn.processBundle(" some  input  words "),
                       CoreMatchers.hasItems("some", "input", "words"));
-    Assert.assertThat(extractWordsFn.processBatch(" "),
+    Assert.assertThat(extractWordsFn.processBundle(" "),
                       CoreMatchers.<String>hasItems());
-    Assert.assertThat(extractWordsFn.processBatch(" some ", " input", " words"),
+    Assert.assertThat(extractWordsFn.processBundle(" some ", " input", " words"),
                       CoreMatchers.hasItems("some", "input", "words"));
   }
 


[32/50] [abbrv] incubator-beam git commit: Finish removing DirectPipelineRunner references

Posted by da...@apache.org.
Finish removing DirectPipelineRunner references


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

Branch: refs/heads/python-sdk
Commit: 340d09845959340f73577512437ebe0939bdeff9
Parents: 09bf9b3
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 13:22:26 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../examples/common/DataflowExampleUtils.java   |  4 +-
 .../examples/cookbook/DatastoreWordCount.java   |  4 +-
 .../translation/TransformTranslatorTest.java    |  2 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |  2 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  2 +-
 .../beam/sdk/options/DirectPipelineOptions.java | 74 --------------------
 .../beam/sdk/options/PipelineOptions.java       |  7 +-
 .../beam/sdk/util/BigQueryTableRowIterator.java |  2 +-
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  2 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    | 24 ++++---
 .../main/java/common/DataflowExampleUtils.java  |  2 +-
 .../src/main/java/StarterPipeline.java          |  2 +-
 .../src/main/java/it/pkg/StarterPipeline.java   |  2 +-
 13 files changed, 28 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
index 5b1af6d..46b8af3 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -315,7 +315,7 @@ public class DataflowExampleUtils {
   }
 
   /**
-   * Do some runner setup: check that the DirectPipelineRunner is not used in conjunction with
+   * Do some runner setup: check that the DirectRunner is not used in conjunction with
    * streaming, and if streaming is specified, use the DataflowPipelineRunner.
    */
   public void setupRunner() {
@@ -413,7 +413,7 @@ public class DataflowExampleUtils {
       }
     } else {
       // Do nothing if the given PipelineResult doesn't support waitToFinish(),
-      // such as EvaluationResults returned by DirectPipelineRunner.
+      // such as EvaluationResults returned by DirectRunner.
       tearDown();
       printPendingMessages();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
index bfaecdf..2d1f88c 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
@@ -193,7 +193,7 @@ public class DatastoreWordCount {
 
   /**
    * An example that creates a pipeline to populate DatastoreIO from a
-   * text input.  Forces use of DirectPipelineRunner for local execution mode.
+   * text input.  Forces use of DirectRunner for local execution mode.
    */
   public static void writeDataToDatastore(Options options) {
       Pipeline p = Pipeline.create(options);
@@ -247,7 +247,7 @@ public class DatastoreWordCount {
 
   /**
    * An example to demo how to use {@link DatastoreIO}.  The runner here is
-   * customizable, which means users could pass either {@code DirectPipelineRunner}
+   * customizable, which means users could pass either {@code DirectRunner}
    * or {@code DataflowPipelineRunner} in the pipeline options.
    */
   public static void main(String args[]) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
index 5fdfb49..b593316 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
@@ -53,7 +53,7 @@ public class TransformTranslatorTest {
 
   /**
    * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline
-   * in DirectPipelineRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
+   * in DirectRunner and on SparkPipelineRunner, with the mapped dataflow-to-spark
    * transforms. Finally it makes sure that the results are the same for both runs.
    */
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
index 2a5698c..c6de8b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
@@ -364,7 +364,7 @@ public class PubsubIO {
    * the stream.
    *
    * <p>When running with a {@link PipelineRunner} that only supports bounded
-   * {@link PCollection PCollections} (such as {@link DirectPipelineRunner}),
+   * {@link PCollection PCollections} (such as {@link DirectRunner}),
    * only a bounded portion of the input Pub/Sub stream can be processed. As such, either
    * {@link Bound#maxNumRecords(int)} or {@link Bound#maxReadTime(Duration)} must be set.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index bbef072..dc50a8c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -111,7 +111,7 @@ import javax.annotation.Nullable;
  * }</pre>
  *
  * <h3>Permissions</h3>
- * <p>When run using the {@link DirectPipelineRunner}, your pipeline can read and write text files
+ * <p>When run using the {@link DirectRunner}, your pipeline can read and write text files
  * on your local drive and remote text files on Google Cloud Storage that you have access to using
  * your {@code gcloud} credentials. When running in the Dataflow service, the pipeline can only
  * read and write files from GCS. For more information about permissions, see the Cloud Dataflow

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
deleted file mode 100644
index c2095e3..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.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.options;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-/**
- * Options that can be used to configure the {@link DirectPipelineRunner}.
- */
-public interface DirectPipelineOptions
-    extends ApplicationNameOptions, BigQueryOptions, GcsOptions, GcpOptions, PipelineOptions,
-        StreamingOptions {
-
-  /**
-   * The random seed to use for pseudorandom behaviors in the {@link DirectPipelineRunner}.
-   * If not explicitly specified, a random seed will be generated.
-   */
-  @JsonIgnore
-  @Description("The random seed to use for pseudorandom behaviors in the DirectPipelineRunner."
-      + " If not explicitly specified, a random seed will be generated.")
-  Long getDirectPipelineRunnerRandomSeed();
-  void setDirectPipelineRunnerRandomSeed(Long value);
-
-  /**
-   * Controls whether the runner should ensure that all of the elements of
-   * the pipeline, such as DoFns, can be serialized.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should ensure that all of the elements of the "
-      + "pipeline, such as DoFns, can be serialized.")
-  @Default.Boolean(true)
-  boolean isTestSerializability();
-  void setTestSerializability(boolean testSerializability);
-
-  /**
-   * Controls whether the runner should ensure that all of the elements of
-   * every {@link PCollection} can be encoded using the appropriate
-   * {@link Coder}.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should ensure that all of the elements of every "
-      + "PCollection can be encoded using the appropriate Coder.")
-  @Default.Boolean(true)
-  boolean isTestEncodability();
-  void setTestEncodability(boolean testEncodability);
-
-  /**
-   * Controls whether the runner should randomize the order of each
-   * {@link PCollection}.
-   */
-  @JsonIgnore
-  @Description("Controls whether the runner should randomize the order of each PCollection.")
-  @Default.Boolean(true)
-  boolean isTestUnorderedness();
-  void setTestUnorderedness(boolean testUnorderedness);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index 456b6ae..e89e5ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -70,10 +70,9 @@ import javax.annotation.concurrent.ThreadSafe;
  *   p.run();
  * }
  *
- * // To create options for the DirectPipeline:
- * DirectPipelineOptions directPipelineOptions =
- *     PipelineOptionsFactory.as(DirectPipelineOptions.class);
- * directPipelineOptions.setStreaming(true);
+ * // To create options for the DirectRunner:
+ * DirectOptions directRunnerOptions =
+ *     PipelineOptionsFactory.as(DirectOptions.class);
  *
  * // To cast from one type to another using the as(Class) method:
  * DataflowPipelineOptions dataflowPipelineOptions =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
index ca1ac69..ad41a3f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
@@ -322,7 +322,7 @@ public class BigQueryTableRowIterator implements AutoCloseable {
 
       if (convertedValue == null) {
         // BigQuery does not include null values when the export operation (to JSON) is used.
-        // To match that behavior, BigQueryTableRowiterator, and the DirectPipelineRunner,
+        // To match that behavior, BigQueryTableRowiterator, and the DirectRunner,
         // intentionally omits columns with null values.
         continue;
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
index 58b10a7..1ebe72b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -91,7 +91,7 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
 
   /**
    * An implementation of {@code OutputManager} using simple lists, for testing and in-memory
-   * contexts such as the {@link DirectPipelineRunner}.
+   * contexts such as the {@link DirectRunner}.
    */
   public static class ListOutputManager implements OutputManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index 5d2e69d..fb8bb72 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -20,7 +20,9 @@ package org.apache.beam.sdk.runners;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.DirectPipelineOptions;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.CrashingRunner;
 import org.apache.beam.sdk.util.GcsUtil;
@@ -50,12 +52,12 @@ public class PipelineRunnerTest {
   @Test
   public void testLongName() {
     // Check we can create a pipeline runner using the full class name.
-    DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class);
-    options.setAppName("test");
-    options.setProject("test");
-    options.setGcsUtil(mockGcsUtil);
+    PipelineOptions options = PipelineOptionsFactory.create();
+    options.as(ApplicationNameOptions.class).setAppName("test");
+    options.as(GcpOptions.class).setProject("test");
+    options.as(GcsOptions.class).setGcsUtil(mockGcsUtil);
     options.setRunner(CrashingRunner.class);
-    options.setGcpCredential(new TestCredential());
+    options.as(GcpOptions.class).setGcpCredential(new TestCredential());
     PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
     assertTrue(runner instanceof CrashingRunner);
   }
@@ -63,12 +65,12 @@ public class PipelineRunnerTest {
   @Test
   public void testShortName() {
     // Check we can create a pipeline runner using the short class name.
-    DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class);
-    options.setAppName("test");
-    options.setProject("test");
-    options.setGcsUtil(mockGcsUtil);
+    PipelineOptions options = PipelineOptionsFactory.create();
+    options.as(ApplicationNameOptions.class).setAppName("test");
+    options.as(GcpOptions.class).setProject("test");
+    options.as(GcsOptions.class).setGcsUtil(mockGcsUtil);
     options.setRunner(CrashingRunner.class);
-    options.setGcpCredential(new TestCredential());
+    options.as(GcpOptions.class).setGcpCredential(new TestCredential());
     PipelineRunner<?> runner = PipelineRunner.fromOptions(options);
     assertTrue(runner instanceof CrashingRunner);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
index 76df4d4..6ec4540 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
@@ -315,7 +315,7 @@ public class DataflowExampleUtils {
       }
     } else {
       // Do nothing if the given PipelineResult doesn't support waitToFinish(),
-      // such as EvaluationResults returned by DirectPipelineRunner.
+      // such as EvaluationResults returned by DirectRunner.
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
index 2146b77..027431f 100644
--- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
@@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory;
  * <p>The example takes two strings, converts them to their upper-case
  * representation and logs them.
  *
- * <p>To run this starter example locally using DirectPipelineRunner, just
+ * <p>To run this starter example locally using DirectRunner, just
  * execute it without any additional parameters from your favorite development
  * environment.
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340d0984/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
index 6cd27e7..bb86b0d 100644
--- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
@@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory;
  * <p>The example takes two strings, converts them to their upper-case
  * representation and logs them.
  *
- * <p>To run this starter example locally using DirectPipelineRunner, just
+ * <p>To run this starter example locally using DirectRunner, just
  * execute it without any additional parameters from your favorite development
  * environment.
  *


[39/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
deleted file mode 100644
index 38d4c96..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,1417 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow;
-
-import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.startsWith;
-import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsList;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsMap;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsMultimap;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsSingleton;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.TransformedMap;
-import org.apache.beam.runners.dataflow.internal.IsmFormat;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.MetadataKeyCoder;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroSource;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFnTester;
-import org.apache.beam.sdk.transforms.PTransform;
-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.CoderUtils;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.ReleaseInfo;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.apache.beam.sdk.values.KV;
-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.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import org.hamcrest.Description;
-import org.hamcrest.Matchers;
-import org.hamcrest.TypeSafeMatcher;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-/**
- * Tests for the {@link DataflowPipelineRunner}.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineRunnerTest {
-
-  private static final String PROJECT_ID = "some-project";
-
-  @Rule
-  public TemporaryFolder tmpFolder = new TemporaryFolder();
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  // Asserts that the given Job has all expected fields set.
-  private static void assertValidJob(Job job) {
-    assertNull(job.getId());
-    assertNull(job.getCurrentState());
-    assertTrue(Pattern.matches("[a-z]([-a-z0-9]*[a-z0-9])?", job.getName()));
-  }
-
-  private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) {
-    options.setStableUniqueNames(CheckEnabled.ERROR);
-    options.setRunner(DataflowPipelineRunner.class);
-    Pipeline p = Pipeline.create(options);
-
-    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
-
-    return p;
-  }
-
-  private static Dataflow buildMockDataflow(
-      final ArgumentCaptor<Job> jobCaptor) throws IOException {
-    Dataflow mockDataflowClient = mock(Dataflow.class);
-    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
-    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
-    Dataflow.Projects.Jobs.Create mockRequest =
-        mock(Dataflow.Projects.Jobs.Create.class);
-    Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
-
-    when(mockDataflowClient.projects()).thenReturn(mockProjects);
-    when(mockProjects.jobs()).thenReturn(mockJobs);
-    when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
-        .thenReturn(mockRequest);
-    when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
-    when(mockList.setPageToken(anyString())).thenReturn(mockList);
-    when(mockList.execute())
-        .thenReturn(
-            new ListJobsResponse()
-                .setJobs(
-                    Arrays.asList(
-                        new Job()
-                            .setName("oldjobname")
-                            .setId("oldJobId")
-                            .setCurrentState("JOB_STATE_RUNNING"))));
-
-    Job resultJob = new Job();
-    resultJob.setId("newid");
-    when(mockRequest.execute()).thenReturn(resultJob);
-    return mockDataflowClient;
-  }
-
-  private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
-    GcsUtil mockGcsUtil = mock(GcsUtil.class);
-    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
-        .then(new Answer<SeekableByteChannel>() {
-              @Override
-              public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
-                return FileChannel.open(
-                    Files.createTempFile("channel-", ".tmp"),
-                    StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
-              }
-            });
-
-    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
-    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
-      @Override
-      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
-        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
-      }
-    });
-    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
-    return mockGcsUtil;
-  }
-
-  private DataflowPipelineOptions buildPipelineOptions() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-    return buildPipelineOptions(jobCaptor);
-  }
-
-  private DataflowPipelineOptions buildPipelineOptions(
-      ArgumentCaptor<Job> jobCaptor) throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject(PROJECT_ID);
-    options.setTempLocation("gs://somebucket/some/path");
-    // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
-    options.setFilesToStage(new LinkedList<String>());
-    options.setDataflowClient(buildMockDataflow(jobCaptor));
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-    return options;
-  }
-
-  @Test
-  public void testFromOptionsWithUppercaseConvertsToLowercase() throws Exception {
-    String mixedCase = "ThisJobNameHasMixedCase";
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setJobName(mixedCase);
-
-    DataflowPipelineRunner runner = DataflowPipelineRunner.fromOptions(options);
-    assertThat(options.getJobName(), equalTo(mixedCase.toLowerCase()));
-  }
-
-  @Test
-  public void testRun() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    Pipeline p = buildDataflowPipeline(options);
-    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
-    assertEquals("newid", job.getJobId());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testRunReturnDifferentRequestId() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    Dataflow mockDataflowClient = options.getDataflowClient();
-    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
-    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
-        .thenReturn(mockRequest);
-    Job resultJob = new Job();
-    resultJob.setId("newid");
-    // Return a different request id.
-    resultJob.setClientRequestId("different_request_id");
-    when(mockRequest.execute()).thenReturn(resultJob);
-
-    Pipeline p = buildDataflowPipeline(options);
-    try {
-      p.run();
-      fail("Expected DataflowJobAlreadyExistsException");
-    } catch (DataflowJobAlreadyExistsException expected) {
-      assertThat(expected.getMessage(),
-          containsString("If you want to submit a second job, try again by setting a "
-            + "different name using --jobName."));
-      assertEquals(expected.getJob().getJobId(), resultJob.getId());
-    }
-  }
-
-  @Test
-  public void testUpdate() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setUpdate(true);
-    options.setJobName("oldJobName");
-    Pipeline p = buildDataflowPipeline(options);
-    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
-    assertEquals("newid", job.getJobId());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testUpdateNonExistentPipeline() throws IOException {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Could not find running job named badjobname");
-
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setUpdate(true);
-    options.setJobName("badJobName");
-    Pipeline p = buildDataflowPipeline(options);
-    p.run();
-  }
-
-  @Test
-  public void testUpdateAlreadyUpdatedPipeline() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setUpdate(true);
-    options.setJobName("oldJobName");
-    Dataflow mockDataflowClient = options.getDataflowClient();
-    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
-    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
-        .thenReturn(mockRequest);
-    final Job resultJob = new Job();
-    resultJob.setId("newid");
-    // Return a different request id.
-    resultJob.setClientRequestId("different_request_id");
-    when(mockRequest.execute()).thenReturn(resultJob);
-
-    Pipeline p = buildDataflowPipeline(options);
-
-    thrown.expect(DataflowJobAlreadyUpdatedException.class);
-    thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
-      @Override
-      public void describeTo(Description description) {
-        description.appendText("Expected job ID: " + resultJob.getId());
-      }
-
-      @Override
-      protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
-        return resultJob.getId().equals(item.getJob().getJobId());
-      }
-    });
-    thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
-        + "into job id: newid and cannot be updated again.");
-    p.run();
-  }
-
-  @Test
-  public void testRunWithFiles() throws IOException {
-    // Test that the function DataflowPipelineRunner.stageFiles works as
-    // expected.
-    GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
-    final String gcsStaging = "gs://somebucket/some/path";
-    final String gcsTemp = "gs://somebucket/some/temp/path";
-    final String cloudDataflowDataset = "somedataset";
-
-    // Create some temporary files.
-    File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt");
-    temp1.deleteOnExit();
-    File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt");
-    temp2.deleteOnExit();
-
-    String overridePackageName = "alias.txt";
-
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setFilesToStage(ImmutableList.of(
-        temp1.getAbsolutePath(),
-        overridePackageName + "=" + temp2.getAbsolutePath()));
-    options.setStagingLocation(gcsStaging);
-    options.setTempLocation(gcsTemp);
-    options.setTempDatasetId(cloudDataflowDataset);
-    options.setProject(PROJECT_ID);
-    options.setJobName("job");
-    options.setDataflowClient(buildMockDataflow(jobCaptor));
-    options.setGcsUtil(mockGcsUtil);
-    options.setGcpCredential(new TestCredential());
-
-    Pipeline p = buildDataflowPipeline(options);
-
-    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
-    assertEquals("newid", job.getJobId());
-
-    Job workflowJob = jobCaptor.getValue();
-    assertValidJob(workflowJob);
-
-    assertEquals(
-        2,
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
-    DataflowPackage workflowPackage1 =
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
-    assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
-    DataflowPackage workflowPackage2 =
-        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
-    assertEquals(overridePackageName, workflowPackage2.getName());
-
-    assertEquals(
-        "storage.googleapis.com/somebucket/some/temp/path",
-        workflowJob.getEnvironment().getTempStoragePrefix());
-    assertEquals(
-        cloudDataflowDataset,
-        workflowJob.getEnvironment().getDataset());
-    assertEquals(
-        ReleaseInfo.getReleaseInfo().getName(),
-        workflowJob.getEnvironment().getUserAgent().get("name"));
-    assertEquals(
-        ReleaseInfo.getReleaseInfo().getVersion(),
-        workflowJob.getEnvironment().getUserAgent().get("version"));
-  }
-
-  @Test
-  public void runWithDefaultFilesToStage() throws Exception {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setFilesToStage(null);
-    DataflowPipelineRunner.fromOptions(options);
-    assertTrue(!options.getFilesToStage().isEmpty());
-  }
-
-  @Test
-  public void detectClassPathResourceWithFileResources() throws Exception {
-    File file = tmpFolder.newFile("file");
-    File file2 = tmpFolder.newFile("file2");
-    URLClassLoader classLoader = new URLClassLoader(new URL[]{
-        file.toURI().toURL(),
-        file2.toURI().toURL()
-    });
-
-    assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
-        DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader));
-  }
-
-  @Test
-  public void detectClassPathResourcesWithUnsupportedClassLoader() {
-    ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
-
-    DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader);
-  }
-
-  @Test
-  public void detectClassPathResourceWithNonFileResources() throws Exception {
-    String url = "http://www.google.com/all-the-secrets.jar";
-    URLClassLoader classLoader = new URLClassLoader(new URL[]{
-        new URL(url)
-    });
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Unable to convert url (" + url + ") to file.");
-
-    DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader);
-  }
-
-  @Test
-  public void testGcsStagingLocationInitialization() throws Exception {
-    // Test that the staging location is initialized correctly.
-    String gcsTemp = "gs://somebucket/some/temp/path";
-
-    // Set temp location (required), and check that staging location is set.
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setTempLocation(gcsTemp);
-    options.setProject(PROJECT_ID);
-    options.setGcpCredential(new TestCredential());
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setRunner(DataflowPipelineRunner.class);
-
-    DataflowPipelineRunner.fromOptions(options);
-
-    assertNotNull(options.getStagingLocation());
-  }
-
-  @Test
-  public void testNonGcsFilePathInReadFailure() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
-
-    thrown.expectCause(Matchers.allOf(
-        instanceOf(IllegalArgumentException.class),
-        ThrowableMessageMatcher.hasMessage(
-            containsString("expected a valid 'gs://' path but was given"))));
-    p.run();
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNonGcsFilePathInWriteFailure() throws IOException {
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
-    pc.apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file"));
-  }
-
-  @Test
-  public void testMultiSlashGcsFileReadPath() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadInvalidGcsFile")
-        .from("gs://bucket/tmp//file"));
-
-    thrown.expectCause(Matchers.allOf(
-        instanceOf(IllegalArgumentException.class),
-        ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
-    p.run();
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testMultiSlashGcsFileWritePath() throws IOException {
-    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("consecutive slashes");
-    pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
-  }
-
-  @Test
-  public void testInvalidTempLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setTempLocation("file://temp/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testInvalidStagingLocation() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    options.setStagingLocation("file://my/staging/location");
-    try {
-      DataflowPipelineRunner.fromOptions(options);
-      fail("fromOptions should have failed");
-    } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
-    }
-    options.setStagingLocation("my/staging/location");
-    try {
-      DataflowPipelineRunner.fromOptions(options);
-      fail("fromOptions should have failed");
-    } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
-    }
-  }
-
-  @Test
-  public void testNonExistentTempLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setGcsUtil(mockGcsUtil);
-    options.setTempLocation("gs://non-existent-bucket/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString(
-        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNonExistentStagingLocation() throws IOException {
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    options.setGcsUtil(mockGcsUtil);
-    options.setStagingLocation("gs://non-existent-bucket/location");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString(
-        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
-    DataflowPipelineRunner.fromOptions(options);
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testNoProjectFails() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-
-    options.setRunner(DataflowPipelineRunner.class);
-    // Explicitly set to null to prevent the default instance factory from reading credentials
-    // from a user's environment, causing this test to fail.
-    options.setProject(null);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project id");
-    thrown.expectMessage("when running a Dataflow in the cloud");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectId() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectPrefix() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("google.com:some-project-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-    options.setGcpCredential(new TestCredential());
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectNumber() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project ID");
-    thrown.expectMessage("project number");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testProjectDescription() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("some project");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Project ID");
-    thrown.expectMessage("project description");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-12345");
-
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Number of worker harness threads");
-    thrown.expectMessage("Please make sure the value is non-negative.");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testNoStagingLocationAndNoTempLocationFails() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setProject("foo-project");
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setGcpCredential(new TestCredential());
-    options.setProject("foo-project");
-    options.setStagingLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setGcpCredential(new TestCredential());
-    options.setProject("foo-project");
-    options.setTempLocation("gs://spam/ham/eggs");
-    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
-    DataflowPipelineRunner.fromOptions(options);
-  }
-
-  @Test
-  public void testInvalidJobName() throws IOException {
-    List<String> invalidNames = Arrays.asList(
-        "invalid_name",
-        "0invalid",
-        "invalid-");
-    List<String> expectedReason = Arrays.asList(
-        "JobName invalid",
-        "JobName invalid",
-        "JobName invalid");
-
-    for (int i = 0; i < invalidNames.size(); ++i) {
-      DataflowPipelineOptions options = buildPipelineOptions();
-      options.setJobName(invalidNames.get(i));
-
-      try {
-        DataflowPipelineRunner.fromOptions(options);
-        fail("Expected IllegalArgumentException for jobName "
-            + options.getJobName());
-      } catch (IllegalArgumentException e) {
-        assertThat(e.getMessage(),
-            containsString(expectedReason.get(i)));
-      }
-    }
-  }
-
-  @Test
-  public void testValidJobName() throws IOException {
-    List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
-        "this-one-is-fairly-long-01234567890123456789");
-
-    for (String name : names) {
-      DataflowPipelineOptions options = buildPipelineOptions();
-      options.setJobName(name);
-
-      DataflowPipelineRunner runner = DataflowPipelineRunner
-          .fromOptions(options);
-      assertNotNull(runner);
-    }
-  }
-
-  /**
-   * A fake PTransform for testing.
-   */
-  public static class TestTransform
-      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
-    public boolean translated = false;
-
-    @Override
-    public PCollection<Integer> apply(PCollection<Integer> input) {
-      return PCollection.<Integer>createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          input.isBounded());
-    }
-
-    @Override
-    protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
-      return input.getCoder();
-    }
-  }
-
-  @Test
-  public void testTransformTranslatorMissing() throws IOException {
-    // Test that we throw if we don't provide a translation.
-    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
-    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
-    Pipeline p = Pipeline.create(options);
-
-    p.apply(Create.of(Arrays.asList(1, 2, 3)))
-     .apply(new TestTransform());
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(Matchers.containsString("no translator registered"));
-    DataflowPipelineTranslator.fromOptions(options)
-        .translate(
-            p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
-    assertValidJob(jobCaptor.getValue());
-  }
-
-  @Test
-  public void testTransformTranslator() throws IOException {
-    // Test that we can provide a custom translation
-    DataflowPipelineOptions options = buildPipelineOptions();
-    Pipeline p = Pipeline.create(options);
-    TestTransform transform = new TestTransform();
-
-    p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
-        .apply(transform);
-
-    DataflowPipelineTranslator translator = DataflowPipelineRunner
-        .fromOptions(options).getTranslator();
-
-    DataflowPipelineTranslator.registerTransformTranslator(
-        TestTransform.class,
-        new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
-          @SuppressWarnings("unchecked")
-          @Override
-          public void translate(
-              TestTransform transform,
-              DataflowPipelineTranslator.TranslationContext context) {
-            transform.translated = true;
-
-            // Note: This is about the minimum needed to fake out a
-            // translation. This obviously isn't a real translation.
-            context.addStep(transform, "TestTranslate");
-            context.addOutput("output", context.getOutput(transform));
-          }
-        });
-
-    translator.translate(
-        p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
-    assertTrue(transform.translated);
-  }
-
-  /** Records all the composite transforms visited within the Pipeline. */
-  private static class CompositeTransformRecorder extends PipelineVisitor.Defaults {
-    private List<PTransform<?, ?>> transforms = new ArrayList<>();
-
-    @Override
-    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
-      if (node.getTransform() != null) {
-        transforms.add(node.getTransform());
-      }
-      return CompositeBehavior.ENTER_TRANSFORM;
-    }
-
-    public List<PTransform<?, ?>> getCompositeTransforms() {
-      return transforms;
-    }
-  }
-
-  @Test
-  public void testApplyIsScopedToExactClass() throws IOException {
-    DataflowPipelineOptions options = buildPipelineOptions();
-    Pipeline p = Pipeline.create(options);
-
-    Create.TimestampedValues<String> transform =
-        Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
-    p.apply(transform);
-
-    CompositeTransformRecorder recorder = new CompositeTransformRecorder();
-    p.traverseTopologically(recorder);
-
-    // The recorder will also have seen a Create.Values composite as well, but we can't obtain that
-    // transform.
-    assertThat(
-        "Expected to have seen CreateTimestamped composite transform.",
-        recorder.getCompositeTransforms(),
-        hasItem(transform));
-    assertThat(
-        "Expected to have two composites, CreateTimestamped and Create.Values",
-        recorder.getCompositeTransforms(),
-        hasItem(Matchers.<PTransform<?, ?>>isA((Class) Create.Values.class)));
-  }
-
-  @Test
-  public void testToString() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    assertEquals(
-        "DataflowPipelineRunner#testjobname",
-        DataflowPipelineRunner.fromOptions(options).toString());
-  }
-
-  private static PipelineOptions makeOptions(boolean streaming) {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
-    options.setStreaming(streaming);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    return options;
-  }
-
-  private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
-      throws Exception {
-    String mode = streaming ? "streaming" : "batch";
-    thrown.expect(UnsupportedOperationException.class);
-    thrown.expectMessage(
-        "The DataflowPipelineRunner in " + mode + " mode does not support " + name);
-
-    Pipeline p = Pipeline.create(makeOptions(streaming));
-    p.apply(source);
-    p.run();
-  }
-
-  @Test
-  public void testBoundedSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true);
-  }
-
-  @Test
-  public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true);
-  }
-
-  @Test
-  public void testAvroIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        AvroIO.Read.from("foo"), "AvroIO.Read", true);
-  }
-
-  @Test
-  public void testTextIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true);
-  }
-
-  @Test
-  public void testReadBoundedSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true);
-  }
-
-  @Test
-  public void testReadUnboundedUnsupportedInBatch() throws Exception {
-    testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false);
-  }
-
-  private void testUnsupportedSink(
-      PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
-          throws Exception {
-    thrown.expect(UnsupportedOperationException.class);
-    thrown.expectMessage(
-        "The DataflowPipelineRunner in streaming mode does not support " + name);
-
-    Pipeline p = Pipeline.create(makeOptions(streaming));
-    p.apply(Create.of("foo")).apply(sink);
-    p.run();
-  }
-
-  @Test
-  public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
-    testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
-  }
-
-  @Test
-  public void testTextIOSinkUnsupportedInStreaming() throws Exception {
-    testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
-  }
-
-  @Test
-  public void testBatchViewAsSingletonToIsmRecord() throws Exception {
-    DoFnTester<KV<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>,
-               IsmRecord<WindowedValue<String>>> doFnTester =
-               DoFnTester.of(
-                   new BatchViewAsSingleton.IsmRecordForSingularValuePerWindowDoFn
-                   <String, GlobalWindow>(GlobalWindow.Coder.INSTANCE));
-
-    assertThat(
-        doFnTester.processBundle(
-            ImmutableList.of(KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(
-                0, ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")))))),
-        contains(IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE), valueInGlobalWindow("a"))));
-  }
-
-  @Test
-  public void testBatchViewAsSingletonToIsmRecordWithMultipleValuesThrowsException()
-      throws Exception {
-    DoFnTester<KV<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>,
-    IsmRecord<WindowedValue<String>>> doFnTester =
-    DoFnTester.of(
-        new BatchViewAsSingleton.IsmRecordForSingularValuePerWindowDoFn
-        <String, GlobalWindow>(GlobalWindow.Coder.INSTANCE));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("found for singleton within window");
-    doFnTester.processBundle(ImmutableList.of(
-        KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(0,
-            ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")),
-                KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("b"))))));
-  }
-
-  @Test
-  public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
-    DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
-        DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBundle(ImmutableList.of("a", "b", "c")), contains(
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
-        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
-  }
-
-  @Test
-  public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(
-            new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
-                IntervalWindow.getCoder()));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(
-                    windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-                KV.of(
-                    windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
-                )),
-            KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(
-                    windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
-                )));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBundle(inputElements), contains(
-        IsmRecord.of(ImmutableList.of(windowA, 0L),
-            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowA, 1L),
-            WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowA, 2L),
-            WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowB, 0L),
-            WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowB, 1L),
-            WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(ImmutableList.of(windowC, 0L),
-            WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
-  }
-
-  @Test
-  public void testToIsmRecordForMapLikeDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
-            outputForSizeTag,
-            outputForEntrySetTag,
-            windowCoder,
-            keyCoder,
-            ismCoder,
-            false /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                // same window same key as to previous
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
-                // same window different key as to previous
-                KV.of(KV.of(2L, windowA),
-                    WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                // different window same key as to previous
-                KV.of(KV.of(2L, windowB),
-                    WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
-                // different window and different key as to previous
-                KV.of(KV.of(3L, windowB),
-                    WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                // different shard
-                KV.of(KV.of(4L, windowC),
-                    WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBundle(inputElements), contains(
-        IsmRecord.of(
-            ImmutableList.of(1L, windowA, 0L),
-            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(1L, windowA, 1L),
-            WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(2L, windowA, 0L),
-            WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(2L, windowB, 0L),
-            WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(3L, windowB, 0L),
-            WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
-        IsmRecord.of(
-            ImmutableList.of(4L, windowC, 0L),
-            WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
-
-    // Verify the number of unique keys per window.
-    assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
-            KV.of(windowC, 1L))
-        ));
-
-    // Verify the output for the unique keys.
-    assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 1L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
-            KV.of(windowA, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 2L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-            KV.of(windowB, 3L)),
-        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
-            KV.of(windowC, 4L))
-        ));
-  }
-
-  @Test
-  public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
-            outputForSizeTag,
-            outputForEntrySetTag,
-            windowCoder,
-            keyCoder,
-            ismCoder,
-            true /* unique keys */));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-
-    Iterable<KV<Integer,
-                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
-                // same window same key as to previous
-                KV.of(KV.of(1L, windowA),
-                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
-
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("Unique keys are expected but found key");
-    doFnTester.processBundle(inputElements);
-  }
-
-  @Test
-  public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
-        new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
-            windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowA, 2L),
-                    KV.of(windowA, 3L),
-                    KV.of(windowB, 7L))),
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowC, 9L))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBundle(inputElements), contains(
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
-        ));
-  }
-
-  @Test
-  public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
-    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
-    Coder<Long> keyCoder = VarLongCoder.of();
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
-        1,
-        2,
-        ImmutableList.<Coder<?>>of(
-            MetadataKeyCoder.of(keyCoder),
-            IntervalWindow.getCoder(),
-            BigEndianLongCoder.of()),
-        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
-               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
-        new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
-            keyCoder, windowCoder));
-    doFnTester.setSideOutputTags(TupleTagList.of(
-        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowA, 2L),
-                    // same window as previous
-                    KV.of(windowA, 3L),
-                    // different window as previous
-                    KV.of(windowB, 3L))),
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
-                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
-                    KV.of(windowC, 3L))));
-
-    // The order of the output elements is important relative to processing order
-    assertThat(doFnTester.processBundle(inputElements), contains(
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
-        IsmRecord.<WindowedValue<Long>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
-        ));
-  }
-
-  @Test
-  public void testToMapDoFn() throws Exception {
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
-                  IsmRecord<WindowedValue<TransformedMap<Long,
-                                                         WindowedValue<Long>,
-                                                         Long>>>> doFnTester =
-        DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowC, WindowedValue.of(
-                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    List<IsmRecord<WindowedValue<TransformedMap<Long,
-                                                WindowedValue<Long>,
-                                                Long>>>> output =
-                                                doFnTester.processBundle(inputElements);
-    assertEquals(3, output.size());
-    Map<Long, Long> outputMap;
-
-    outputMap = output.get(0).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
-
-    outputMap = output.get(1).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
-
-    outputMap = output.get(2).getValue().getValue();
-    assertEquals(1, outputMap.size());
-    assertEquals(ImmutableMap.of(4L, 41L), outputMap);
-  }
-
-  @Test
-  public void testToMultimapDoFn() throws Exception {
-    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
-    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
-                  IsmRecord<WindowedValue<TransformedMap<Long,
-                                                         Iterable<WindowedValue<Long>>,
-                                                         Iterable<Long>>>>> doFnTester =
-        DoFnTester.of(
-            new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
-    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
-    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
-    Iterable<KV<Integer,
-             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
-        ImmutableList.of(
-            KV.of(1,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowA, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
-                    KV.of(windowB, WindowedValue.of(
-                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
-            KV.of(2,
-                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
-                    KV.of(windowC, WindowedValue.of(
-                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
-    // The order of the output elements is important relative to processing order
-    List<IsmRecord<WindowedValue<TransformedMap<Long,
-                                                Iterable<WindowedValue<Long>>,
-                                                Iterable<Long>>>>> output =
-                                                doFnTester.processBundle(inputElements);
-    assertEquals(3, output.size());
-    Map<Long, Iterable<Long>> outputMap;
-
-    outputMap = output.get(0).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
-    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
-
-    outputMap = output.get(1).getValue().getValue();
-    assertEquals(2, outputMap.size());
-    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
-    assertThat(outputMap.get(3L), containsInAnyOrder(31L));
-
-    outputMap = output.get(2).getValue().getValue();
-    assertEquals(1, outputMap.size());
-    assertThat(outputMap.get(4L), containsInAnyOrder(41L));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
index 165d2b5..261ba99 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -124,7 +124,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
   }
 
   private Pipeline buildPipeline(DataflowPipelineOptions options) {
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     Pipeline p = Pipeline.create(options);
 
     p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
@@ -164,7 +164,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
 
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setGcpCredential(new TestCredential());
     options.setJobName("some-job-name");
     options.setProject("some-project");
@@ -178,14 +178,14 @@ public class DataflowPipelineTranslatorTest implements Serializable {
   @Test
   public void testSettingOfSdkPipelineOptions() throws IOException {
     DataflowPipelineOptions options = buildPipelineOptions();
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
 
     Pipeline p = buildPipeline(options);
     p.traverseTopologically(new RecordingPipelineVisitor());
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     // Note that the contents of this materialized map may be changed by the act of reading an
@@ -196,7 +196,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     settings.put("project", "some-project");
     settings.put("pathValidatorClass",
         "org.apache.beam.runners.dataflow.util.DataflowPathValidator");
-    settings.put("runner", "org.apache.beam.runners.dataflow.DataflowPipelineRunner");
+    settings.put("runner", "org.apache.beam.runners.dataflow.DataflowRunner");
     settings.put("jobName", "some-job-name");
     settings.put("tempLocation", "gs://somebucket/some/path");
     settings.put("stagingLocation", "gs://somebucket/some/path/staging");
@@ -222,7 +222,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -239,7 +239,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -258,7 +258,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -275,7 +275,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -291,7 +291,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -327,7 +327,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -362,7 +362,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -396,7 +396,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -416,7 +416,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -437,7 +437,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     Job job =
         DataflowPipelineTranslator.fromOptions(options)
             .translate(
-                p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+                p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
             .getJob();
 
     assertEquals(1, job.getEnvironment().getWorkerPools().size());
@@ -466,7 +466,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -520,7 +520,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -655,7 +655,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
 
     // Check that translation doesn't fail.
     t.translate(
-        p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+        p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
   }
 
   @Test
@@ -698,7 +698,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     // Check that translation doesn't fail.
     t.translate(
         pipeline,
-        (DataflowPipelineRunner) pipeline.getRunner(),
+        (DataflowRunner) pipeline.getRunner(),
         Collections.<DataflowPackage>emptyList());
   }
 
@@ -724,7 +724,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage"))));
     t.translate(
         pipeline,
-        (DataflowPipelineRunner) pipeline.getRunner(),
+        (DataflowRunner) pipeline.getRunner(),
         Collections.<DataflowPackage>emptyList());
   }
 
@@ -745,7 +745,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -777,7 +777,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -807,7 +807,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -840,7 +840,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 
@@ -903,7 +903,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
         translator
             .translate(
                 pipeline,
-                (DataflowPipelineRunner) pipeline.getRunner(),
+                (DataflowRunner) pipeline.getRunner(),
                 Collections.<DataflowPackage>emptyList())
             .getJob();
 


[37/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
deleted file mode 100644
index fbaf116..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,601 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow.testing;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
-import org.apache.beam.runners.dataflow.util.TimeUtil;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.SerializableMatcher;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.TestPipelineOptions;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.Transport;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.Json;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-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.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-
-/** Tests for {@link TestDataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class TestDataflowPipelineRunnerTest {
-  @Rule public ExpectedException expectedException = ExpectedException.none();
-  @Mock private MockHttpTransport transport;
-  @Mock private MockLowLevelHttpRequest request;
-  @Mock private GcsUtil mockGcsUtil;
-
-  private TestDataflowPipelineOptions options;
-  private Dataflow service;
-
-  @Before
-  public void setUp() throws Exception {
-    MockitoAnnotations.initMocks(this);
-    when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
-    doCallRealMethod().when(request).getContentAsString();
-    service = new Dataflow(transport, Transport.getJsonFactory(), null);
-
-    options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
-    options.setAppName("TestAppName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setTempRoot("gs://test");
-    options.setGcpCredential(new TestCredential());
-    options.setDataflowClient(service);
-    options.setRunner(TestDataflowPipelineRunner.class);
-    options.setPathValidatorClass(NoopPathValidator.class);
-  }
-
-  @Test
-  public void testToString() {
-    assertEquals("TestDataflowPipelineRunner#TestAppName",
-        new TestDataflowPipelineRunner(options).toString());
-  }
-
-  @Test
-  public void testRunBatchJobThatSucceeds() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    assertEquals(mockJob, runner.run(p, mockRunner));
-  }
-
-  @Test
-  public void testRunBatchJobThatFails() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.FAILED);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testBatchPipelineFailsIfException() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenAnswer(new Answer<State>() {
-          @Override
-          public State answer(InvocationOnMock invocation) {
-            JobMessage message = new JobMessage();
-            message.setMessageText("FooException");
-            message.setTime(TimeUtil.toCloudTime(Instant.now()));
-            message.setMessageImportance("JOB_MESSAGE_ERROR");
-            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
-                .process(Arrays.asList(message));
-            return State.CANCELLED;
-          }
-        });
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      assertThat(expected.getMessage(), containsString("FooException"));
-      verify(mockJob, atLeastOnce()).cancel();
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testRunStreamingJobThatSucceeds() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testRunStreamingJobThatFails() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    doReturn(State.DONE).when(job).getState();
-    assertEquals(Optional.of(true), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testCheckingForSuccessWhenPAssertFails() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    doReturn(State.DONE).when(job).getState();
-    assertEquals(Optional.of(false), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, false /* tentative */));
-    doReturn(State.RUNNING).when(job).getState();
-    assertEquals(Optional.absent(), runner.checkForSuccess(job));
-  }
-
-  private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
-      throws Exception {
-    MetricStructuredName name = new MetricStructuredName();
-    name.setName(success ? "PAssertSuccess" : "PAssertFailure");
-    name.setContext(
-        tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
-
-    MetricUpdate metric = new MetricUpdate();
-    metric.setName(name);
-    metric.setScalar(BigDecimal.ONE);
-
-    MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
-    response.setContentType(Json.MEDIA_TYPE);
-    JobMetrics jobMetrics = new JobMetrics();
-    jobMetrics.setMetrics(Lists.newArrayList(metric));
-    // N.B. Setting the factory is necessary in order to get valid JSON.
-    jobMetrics.setFactory(Transport.getJsonFactory());
-    response.setContent(jobMetrics.toPrettyString());
-    return response;
-  }
-
-  @Test
-  public void testStreamingPipelineFailsIfServiceFails() throws Exception {
-    DataflowPipelineJob job =
-        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, false /* tentative */));
-    doReturn(State.FAILED).when(job).getState();
-    assertEquals(Optional.of(false), runner.checkForSuccess(job));
-  }
-
-  @Test
-  public void testStreamingPipelineFailsIfException() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.RUNNING);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenAnswer(new Answer<State>() {
-          @Override
-          public State answer(InvocationOnMock invocation) {
-            JobMessage message = new JobMessage();
-            message.setMessageText("FooException");
-            message.setTime(TimeUtil.toCloudTime(Instant.now()));
-            message.setMessageImportance("JOB_MESSAGE_ERROR");
-            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
-                .process(Arrays.asList(message));
-            return State.CANCELLED;
-          }
-        });
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      assertThat(expected.getMessage(), containsString("FooException"));
-      verify(mockJob, atLeastOnce()).cancel();
-      return;
-    }
-    // Note that fail throws an AssertionError which is why it is placed out here
-    // instead of inside the try-catch block.
-    fail("AssertionError expected");
-  }
-
-  @Test
-  public void testBatchOnCreateMatcher() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testStreamingOnCreateMatcher() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
-
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenReturn(State.DONE);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.DONE);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
-
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenReturn(State.DONE);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(true /* success */, true /* tentative */));
-    runner.run(p, mockRunner);
-  }
-
-  @Test
-  public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception {
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.FAILED);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnSuccessMatcher(new TestFailureMatcher());
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
-          any(JobMessagesHandler.class));
-      return;
-    }
-    fail("Expected an exception on pipeline failure.");
-  }
-
-  @Test
-  public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception {
-    options.setStreaming(true);
-    Pipeline p = TestPipeline.create(options);
-    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
-    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
-    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
-    when(mockJob.getDataflowClient()).thenReturn(service);
-    when(mockJob.getState()).thenReturn(State.FAILED);
-    when(mockJob.getProjectId()).thenReturn("test-project");
-    when(mockJob.getJobId()).thenReturn("test-job");
-
-    DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
-    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
-    p.getOptions().as(TestPipelineOptions.class)
-        .setOnSuccessMatcher(new TestFailureMatcher());
-
-    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
-        .thenReturn(State.FAILED);
-
-    when(request.execute()).thenReturn(
-        generateMockMetricResponse(false /* success */, true /* tentative */));
-    try {
-      runner.run(p, mockRunner);
-    } catch (AssertionError expected) {
-      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
-          any(JobMessagesHandler.class));
-      return;
-    }
-    fail("Expected an exception on pipeline failure.");
-  }
-
-  static class TestSuccessMatcher extends BaseMatcher<PipelineResult> implements
-      SerializableMatcher<PipelineResult> {
-    private final DataflowPipelineJob mockJob;
-    private final int called;
-
-    public TestSuccessMatcher(DataflowPipelineJob job, int times) {
-      this.mockJob = job;
-      this.called = times;
-    }
-
-    @Override
-    public boolean matches(Object o) {
-      if (!(o instanceof PipelineResult)) {
-        fail(String.format("Expected PipelineResult but received %s", o));
-      }
-      try {
-        verify(mockJob, Mockito.times(called)).waitToFinish(any(Long.class), any(TimeUnit.class),
-            any(JobMessagesHandler.class));
-      } catch (IOException | InterruptedException e) {
-        throw new AssertionError(e);
-      }
-      assertSame(mockJob, o);
-      return true;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-    }
-  }
-
-  static class TestFailureMatcher extends BaseMatcher<PipelineResult> implements
-      SerializableMatcher<PipelineResult> {
-    @Override
-    public boolean matches(Object o) {
-      fail("OnSuccessMatcher should not be called on pipeline failure.");
-      return false;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
new file mode 100644
index 0000000..4067f08
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
@@ -0,0 +1,601 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow.testing;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
+import org.apache.beam.runners.dataflow.util.TimeUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.SerializableMatcher;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.Json;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+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.io.IOException;
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+/** Tests for {@link TestDataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class TestDataflowRunnerTest {
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+  @Mock private MockHttpTransport transport;
+  @Mock private MockLowLevelHttpRequest request;
+  @Mock private GcsUtil mockGcsUtil;
+
+  private TestDataflowPipelineOptions options;
+  private Dataflow service;
+
+  @Before
+  public void setUp() throws Exception {
+    MockitoAnnotations.initMocks(this);
+    when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
+    doCallRealMethod().when(request).getContentAsString();
+    service = new Dataflow(transport, Transport.getJsonFactory(), null);
+
+    options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+    options.setAppName("TestAppName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setTempRoot("gs://test");
+    options.setGcpCredential(new TestCredential());
+    options.setDataflowClient(service);
+    options.setRunner(TestDataflowPipelineRunner.class);
+    options.setPathValidatorClass(NoopPathValidator.class);
+  }
+
+  @Test
+  public void testToString() {
+    assertEquals("TestDataflowPipelineRunner#TestAppName",
+        new TestDataflowPipelineRunner(options).toString());
+  }
+
+  @Test
+  public void testRunBatchJobThatSucceeds() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    assertEquals(mockJob, runner.run(p, mockRunner));
+  }
+
+  @Test
+  public void testRunBatchJobThatFails() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testBatchPipelineFailsIfException() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenAnswer(new Answer<State>() {
+          @Override
+          public State answer(InvocationOnMock invocation) {
+            JobMessage message = new JobMessage();
+            message.setMessageText("FooException");
+            message.setTime(TimeUtil.toCloudTime(Instant.now()));
+            message.setMessageImportance("JOB_MESSAGE_ERROR");
+            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+                .process(Arrays.asList(message));
+            return State.CANCELLED;
+          }
+        });
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      assertThat(expected.getMessage(), containsString("FooException"));
+      verify(mockJob, atLeastOnce()).cancel();
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testRunStreamingJobThatSucceeds() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testRunStreamingJobThatFails() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    doReturn(State.DONE).when(job).getState();
+    assertEquals(Optional.of(true), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testCheckingForSuccessWhenPAssertFails() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    doReturn(State.DONE).when(job).getState();
+    assertEquals(Optional.of(false), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, false /* tentative */));
+    doReturn(State.RUNNING).when(job).getState();
+    assertEquals(Optional.absent(), runner.checkForSuccess(job));
+  }
+
+  private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
+      throws Exception {
+    MetricStructuredName name = new MetricStructuredName();
+    name.setName(success ? "PAssertSuccess" : "PAssertFailure");
+    name.setContext(
+        tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
+
+    MetricUpdate metric = new MetricUpdate();
+    metric.setName(name);
+    metric.setScalar(BigDecimal.ONE);
+
+    MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
+    response.setContentType(Json.MEDIA_TYPE);
+    JobMetrics jobMetrics = new JobMetrics();
+    jobMetrics.setMetrics(Lists.newArrayList(metric));
+    // N.B. Setting the factory is necessary in order to get valid JSON.
+    jobMetrics.setFactory(Transport.getJsonFactory());
+    response.setContent(jobMetrics.toPrettyString());
+    return response;
+  }
+
+  @Test
+  public void testStreamingPipelineFailsIfServiceFails() throws Exception {
+    DataflowPipelineJob job =
+        spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, false /* tentative */));
+    doReturn(State.FAILED).when(job).getState();
+    assertEquals(Optional.of(false), runner.checkForSuccess(job));
+  }
+
+  @Test
+  public void testStreamingPipelineFailsIfException() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.RUNNING);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenAnswer(new Answer<State>() {
+          @Override
+          public State answer(InvocationOnMock invocation) {
+            JobMessage message = new JobMessage();
+            message.setMessageText("FooException");
+            message.setTime(TimeUtil.toCloudTime(Instant.now()));
+            message.setMessageImportance("JOB_MESSAGE_ERROR");
+            ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+                .process(Arrays.asList(message));
+            return State.CANCELLED;
+          }
+        });
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      assertThat(expected.getMessage(), containsString("FooException"));
+      verify(mockJob, atLeastOnce()).cancel();
+      return;
+    }
+    // Note that fail throws an AssertionError which is why it is placed out here
+    // instead of inside the try-catch block.
+    fail("AssertionError expected");
+  }
+
+  @Test
+  public void testBatchOnCreateMatcher() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testStreamingOnCreateMatcher() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.DONE);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.DONE);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.DONE);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(true /* success */, true /* tentative */));
+    runner.run(p, mockRunner);
+  }
+
+  @Test
+  public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception {
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestFailureMatcher());
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
+          any(JobMessagesHandler.class));
+      return;
+    }
+    fail("Expected an exception on pipeline failure.");
+  }
+
+  @Test
+  public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception {
+    options.setStreaming(true);
+    Pipeline p = TestPipeline.create(options);
+    PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+    PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+    final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+    when(mockJob.getDataflowClient()).thenReturn(service);
+    when(mockJob.getState()).thenReturn(State.FAILED);
+    when(mockJob.getProjectId()).thenReturn("test-project");
+    when(mockJob.getJobId()).thenReturn("test-job");
+
+    DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
+    when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    p.getOptions().as(TestPipelineOptions.class)
+        .setOnSuccessMatcher(new TestFailureMatcher());
+
+    when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+        .thenReturn(State.FAILED);
+
+    when(request.execute()).thenReturn(
+        generateMockMetricResponse(false /* success */, true /* tentative */));
+    try {
+      runner.run(p, mockRunner);
+    } catch (AssertionError expected) {
+      verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class),
+          any(JobMessagesHandler.class));
+      return;
+    }
+    fail("Expected an exception on pipeline failure.");
+  }
+
+  static class TestSuccessMatcher extends BaseMatcher<PipelineResult> implements
+      SerializableMatcher<PipelineResult> {
+    private final DataflowPipelineJob mockJob;
+    private final int called;
+
+    public TestSuccessMatcher(DataflowPipelineJob job, int times) {
+      this.mockJob = job;
+      this.called = times;
+    }
+
+    @Override
+    public boolean matches(Object o) {
+      if (!(o instanceof PipelineResult)) {
+        fail(String.format("Expected PipelineResult but received %s", o));
+      }
+      try {
+        verify(mockJob, Mockito.times(called)).waitToFinish(any(Long.class), any(TimeUnit.class),
+            any(JobMessagesHandler.class));
+      } catch (IOException | InterruptedException e) {
+        throw new AssertionError(e);
+      }
+      assertSame(mockJob, o);
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+    }
+  }
+
+  static class TestFailureMatcher extends BaseMatcher<PipelineResult> implements
+      SerializableMatcher<PipelineResult> {
+    @Override
+    public boolean matches(Object o) {
+      fail("OnSuccessMatcher should not be called on pipeline failure.");
+      return false;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowDisplayDataEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowDisplayDataEvaluator.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowDisplayDataEvaluator.java
index 0b865c3..d809cc6 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowDisplayDataEvaluator.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowDisplayDataEvaluator.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.transforms;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.options.GcpOptions;
@@ -30,7 +30,7 @@ import com.google.common.collect.Lists;
 
 /**
  * Factory methods for creating {@link DisplayDataEvaluator} instances against the
- * {@link DataflowPipelineRunner}.
+ * {@link DataflowRunner}.
  */
 public final class DataflowDisplayDataEvaluator {
   /** Do not instantiate. */
@@ -43,7 +43,7 @@ public final class DataflowDisplayDataEvaluator {
   public static DataflowPipelineOptions getDefaultOptions() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
 
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setProject("foobar");
     options.setTempLocation("gs://bucket/tmpLocation");
     options.setFilesToStage(Lists.<String>newArrayList());
@@ -56,7 +56,7 @@ public final class DataflowDisplayDataEvaluator {
 
   /**
    * Create a {@link DisplayDataEvaluator} instance to evaluate pipeline display data against
-   * the {@link DataflowPipelineRunner}.
+   * the {@link DataflowRunner}.
    */
   public static DisplayDataEvaluator create() {
     return create(getDefaultOptions());
@@ -64,7 +64,7 @@ public final class DataflowDisplayDataEvaluator {
 
   /**
    * Create a {@link DisplayDataEvaluator} instance to evaluate pipeline display data against
-   * the {@link DataflowPipelineRunner} with the specified {@code options}.
+   * the {@link DataflowRunner} with the specified {@code options}.
    */
   public static DisplayDataEvaluator create(DataflowPipelineOptions options) {
     return DisplayDataEvaluator.create(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
index f0e677e..a44b8a7 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.transforms;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
@@ -46,20 +46,20 @@ import org.junit.runners.JUnit4;
 import java.util.Arrays;
 import java.util.List;
 
-/** Tests for {@link GroupByKey} for the {@link DataflowPipelineRunner}. */
+/** Tests for {@link GroupByKey} for the {@link DataflowRunner}. */
 @RunWith(JUnit4.class)
 public class DataflowGroupByKeyTest {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   /**
-   * Create a test pipeline that uses the {@link DataflowPipelineRunner} so that {@link GroupByKey}
+   * Create a test pipeline that uses the {@link DataflowRunner} so that {@link GroupByKey}
    * is not expanded. This is used for verifying that even without expansion the proper errors show
    * up.
    */
   private Pipeline createTestServiceRunner() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setProject("someproject");
     options.setStagingLocation("gs://staging");
     options.setPathValidatorClass(NoopPathValidator.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
index d787500..1b263d2 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.transforms;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -44,7 +44,7 @@ import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Tests for {@link View} for a {@link DataflowPipelineRunner}. */
+/** Tests for {@link View} for a {@link DataflowRunner}. */
 @RunWith(JUnit4.class)
 public class DataflowViewTest {
   @Rule
@@ -52,7 +52,7 @@ public class DataflowViewTest {
 
   private Pipeline createTestBatchRunner() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setProject("someproject");
     options.setStagingLocation("gs://staging");
     options.setPathValidatorClass(NoopPathValidator.class);
@@ -62,7 +62,7 @@ public class DataflowViewTest {
 
   private Pipeline createTestStreamingRunner() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setStreaming(true);
     options.setProject("someproject");
     options.setStagingLocation("gs://staging");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
index 5587986..a91f56c 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
@@ -21,7 +21,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.when;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.GcsUtil;
@@ -52,7 +52,7 @@ public class DataflowPathValidatorTest {
     when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
     options.setGcpCredential(new TestCredential());
-    options.setRunner(DataflowPipelineRunner.class);
+    options.setRunner(DataflowRunner.class);
     options.setGcsUtil(mockGcsUtil);
     validator = new DataflowPathValidator(options);
   }
@@ -66,7 +66,7 @@ public class DataflowPathValidatorTest {
   public void testInvalidFilePattern() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+        "DataflowRunner expected a valid 'gs://' path but was given '/local/path'");
     validator.validateInputFilePatternSupported("/local/path");
   }
 
@@ -88,7 +88,7 @@ public class DataflowPathValidatorTest {
   public void testInvalidOutputPrefix() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+        "DataflowRunner expected a valid 'gs://' path but was given '/local/path'");
     validator.validateOutputFilePrefixSupported("/local/path");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
index 61ad24f..2b4464d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
@@ -113,7 +113,7 @@ public class SimpleWordCountTest {
       String[] words = WORD_BOUNDARY.split(c.element());
 
       // Keep track of the number of lines without any words encountered while tokenizing.
-      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
+      // This aggregator is visible in the monitoring UI when run using DataflowRunner.
       if (words.length == 0) {
         emptyLines.addValue(1L);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
index 22a2241..de3c152 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java
@@ -158,7 +158,7 @@ public class SerializationTest {
       String[] words = WORD_BOUNDARY.split(c.element().toString());
 
       // Keep track of the number of lines without any words encountered while tokenizing.
-      // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner.
+      // This aggregator is visible in the monitoring UI when run using DataflowRunner.
       if (words.length == 0) {
         emptyLines.addValue(1L);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
index 479090f..8719384 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java
@@ -17,16 +17,16 @@
  */
 /**
  * Defines runners for executing Pipelines in different modes, including
- * {@link org.apache.beam.sdk.runners.DirectPipelineRunner} and
- * {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
+ * {@link org.apache.beam.sdk.runners.DirectRunner} and
+ * {@link org.apache.beam.sdk.runners.DataflowRunner}.
  *
- * <p>{@link org.apache.beam.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline}
+ * <p>{@link org.apache.beam.sdk.runners.DirectRunner} executes a {@code Pipeline}
  * locally, without contacting the Dataflow service.
- * {@link org.apache.beam.sdk.runners.DataflowPipelineRunner} submits a
+ * {@link org.apache.beam.sdk.runners.DataflowRunner} submits a
  * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine
- * instances. {@code DataflowPipelineRunner} returns
+ * instances. {@code DataflowRunner} returns
  * as soon as the {@code Pipeline} has been submitted. Use
- * {@link org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner} to have execution
+ * {@link org.apache.beam.sdk.runners.BlockingDataflowRunner} to have execution
  * updates printed to the console.
  *
  * <p>The runner is specified as part {@link org.apache.beam.sdk.options.PipelineOptions}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 0dba043..b901268 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -60,7 +60,7 @@ import javax.annotation.Nullable;
  *   <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
  *   options. For example:
  *   <pre>{@code [
- *     "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowPipelineRunner",
+ *     "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner",
  *     "--project=mygcpproject",
  *     "--stagingLocation=gs://mygcsbucket/path"
  *     ]}</pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
index 4c98123..329dec5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
@@ -28,7 +28,7 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn;
  * {@code Aggregator} by calling {@link Aggregator#addValue}.
  *
  * <p>Aggregators are visible in the monitoring UI, when the pipeline is run
- * using DataflowPipelineRunner or BlockingDataflowPipelineRunner, along with
+ * using DataflowRunner or BlockingDataflowRunner, along with
  * their current value. Aggregators may not become visible until the system
  * begins executing the ParDo transform that created them and/or their initial
  * value is changed.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
index ad41a3f..3865654 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableRowIterator.java
@@ -156,7 +156,7 @@ public class BigQueryTableRowIterator implements AutoCloseable {
       if (iteratorOverCurrentBatch != null && iteratorOverCurrentBatch.hasNext()) {
         // Embed schema information into the raw row, so that values have an
         // associated key.  This matches how rows are read when using the
-        // DataflowPipelineRunner.
+        // DataflowRunner.
         current = getTypedTableRow(schema.getFields(), iteratorOverCurrentBatch.next());
         return true;
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
index fb8bb72..f9ce018 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java
@@ -37,7 +37,7 @@ import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
 /**
- * Tests for DataflowPipelineRunner.
+ * Tests for DataflowRunner.
  */
 @RunWith(JUnit4.class)
 public class PipelineRunnerTest {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
index 43c990a..3306cb4 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
@@ -70,7 +70,7 @@ import java.util.regex.Pattern;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
  * </pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
index 3e4fc86..98af2e7 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
@@ -17,7 +17,7 @@
  */
 package ${package};
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
@@ -63,7 +63,7 @@ public class MinimalWordCount {
     // in Google Cloud Storage to stage files.
     DataflowPipelineOptions options = PipelineOptionsFactory.create()
       .as(DataflowPipelineOptions.class);
-    options.setRunner(BlockingDataflowPipelineRunner.class);
+    options.setRunner(BlockingDataflowRunner.class);
     // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud.
     options.setProject("SET_YOUR_PROJECT_ID_HERE");
     // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
index 7dea9fe..8e56b03 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
@@ -82,7 +82,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
index fc1f4b5..07ed6d0 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
@@ -77,7 +77,7 @@ import org.apache.beam.sdk.values.PCollection;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and an output prefix on GCS:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
index 6ec4540..82f0eff 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
@@ -17,9 +17,9 @@
  */
 package ${package}.common;
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil;
 
@@ -251,10 +251,10 @@ public class DataflowExampleUtils {
   }
 
   public void setupRunner() {
-    if (options.isStreaming() && options.getRunner().equals(BlockingDataflowPipelineRunner.class)) {
+    if (options.isStreaming() && options.getRunner().equals(BlockingDataflowRunner.class)) {
       // In order to cancel the pipelines automatically,
-      // {@literal DataflowPipelineRunner} is forced to be used.
-      options.setRunner(DataflowPipelineRunner.class);
+      // {@literal DataflowRunner} is forced to be used.
+      options.setRunner(DataflowRunner.class);
     }
   }
 
@@ -268,7 +268,7 @@ public class DataflowExampleUtils {
     DataflowPipelineOptions copiedOptions = options.cloneAs(DataflowPipelineOptions.class);
     copiedOptions.setStreaming(false);
     copiedOptions.setWorkerHarnessContainerImage(
-        DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
+        DataflowRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
     copiedOptions.setNumWorkers(
         options.as(ExamplePubsubTopicOptions.class).getInjectorNumWorkers());
     copiedOptions.setJobName(options.getJobName() + "-injector");
@@ -298,7 +298,7 @@ public class DataflowExampleUtils {
   }
 
   /**
-   * If {@literal DataflowPipelineRunner} or {@literal BlockingDataflowPipelineRunner} is used,
+   * If {@literal DataflowRunner} or {@literal BlockingDataflowRunner} is used,
    * waits for the pipeline to finish and cancels it (and the injector) before the program exists.
    */
   public void waitToFinish(PipelineResult result) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
index 027431f..9a75bb7 100644
--- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  * Platform, you should specify the following command-line options:
  *   --project=<YOUR_PROJECT_ID>
  *   --stagingLocation=<STAGING_LOCATION_IN_CLOUD_STORAGE>
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  */
 public class StarterPipeline {
   private static final Logger LOG = LoggerFactory.getLogger(StarterPipeline.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
index bb86b0d..8c71d9d 100644
--- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
+++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  * Platform, you should specify the following command-line options:
  *   --project=<YOUR_PROJECT_ID>
  *   --stagingLocation=<STAGING_LOCATION_IN_CLOUD_STORAGE>
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  */
 public class StarterPipeline {
   private static final Logger LOG = LoggerFactory.getLogger(StarterPipeline.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 09fe7d9..55aea6a 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -37,7 +37,7 @@
     <module>core</module>
     <module>io</module>
     <!-- sdks/java/maven-archtypes has several dependencies on the
-         DataflowPipelineRunner. Until these are refactored out or
+         DataflowRunner. Until these are refactored out or
          a released artifact exists, we need to modify the build order.
     <module>maven-archetypes</module> -->
     <module>extensions</module>


[15/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 4dd1475..ef31ba7 100644
--- 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
@@ -42,7 +42,7 @@ class TransformExecutor<T> implements Runnable {
   public static <T> TransformExecutor<T> create(
       TransformEvaluatorFactory factory,
       Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
+      EvaluationContext evaluationContext,
       CommittedBundle<T> inputBundle,
       AppliedPTransform<?, ?, ?> transform,
       CompletionCallback completionCallback,
@@ -60,7 +60,7 @@ class TransformExecutor<T> implements Runnable {
   private final TransformEvaluatorFactory evaluatorFactory;
   private final Iterable<? extends ModelEnforcementFactory> modelEnforcements;
 
-  private final InProcessEvaluationContext evaluationContext;
+  private final EvaluationContext evaluationContext;
 
   /** The transform that will be evaluated. */
   private final AppliedPTransform<?, ?, ?> transform;
@@ -75,7 +75,7 @@ class TransformExecutor<T> implements Runnable {
   private TransformExecutor(
       TransformEvaluatorFactory factory,
       Iterable<? extends ModelEnforcementFactory> modelEnforcements,
-      InProcessEvaluationContext evaluationContext,
+      EvaluationContext evaluationContext,
       CommittedBundle<T> inputBundle,
       AppliedPTransform<?, ?, ?> transform,
       CompletionCallback completionCallback,
@@ -117,7 +117,7 @@ class TransformExecutor<T> implements Runnable {
 
       processElements(evaluator, enforcements);
 
-      InProcessTransformResult result = finishBundle(evaluator, enforcements);
+      TransformResult result = finishBundle(evaluator, enforcements);
     } catch (Throwable t) {
       onComplete.handleThrowable(inputBundle, t);
       if (t instanceof RuntimeException) {
@@ -155,13 +155,13 @@ class TransformExecutor<T> implements Runnable {
    * 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
+   * @return the {@link TransformResult} produced by
    *         {@link TransformEvaluator#finishBundle()}
    */
-  private InProcessTransformResult finishBundle(
+  private TransformResult finishBundle(
       TransformEvaluator<T> evaluator, Collection<ModelEnforcement<T>> enforcements)
       throws Exception {
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     CommittedResult outputs = onComplete.handleResult(inputBundle, result);
     for (ModelEnforcement<T> enforcement : enforcements) {
       enforcement.afterFinish(inputBundle, result, outputs.getOutputs());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java
new file mode 100644
index 0000000..c1e502d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.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.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
+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.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 TransformResult {
+  /**
+   * 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 elements that were provided to the {@link TransformEvaluator} as input but were not
+   * processed.
+   */
+  Iterable<? extends WindowedValue<?>> getUnprocessedElements();
+
+  /**
+   * 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/babddbbc/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
index 5030730..3fb773e 100644
--- 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
@@ -69,13 +69,13 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   @Override
   @Nullable
   public <InputT> TransformEvaluator<InputT> forApplication(AppliedPTransform<?, ?, ?> application,
-      @Nullable CommittedBundle<?> inputBundle, InProcessEvaluationContext evaluationContext) {
+      @Nullable CommittedBundle<?> inputBundle, EvaluationContext evaluationContext) {
     return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
   }
 
   private <OutputT> TransformEvaluator<?> getTransformEvaluator(
       final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
+      final EvaluationContext evaluationContext) {
     return getTransformEvaluatorQueue(transform, evaluationContext).poll();
   }
 
@@ -90,7 +90,7 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   private <OutputT, CheckpointMarkT extends CheckpointMark>
   Queue<UnboundedReadEvaluator<OutputT, CheckpointMarkT>> getTransformEvaluatorQueue(
       final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
+      final EvaluationContext evaluationContext) {
     // Key by the application and the context the evaluation is occurring in (which call to
     // Pipeline#run).
     EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
@@ -134,7 +134,7 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
     private static final int ARBITRARY_MAX_ELEMENTS = 10;
 
     private final AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
+    private final EvaluationContext evaluationContext;
     private final ConcurrentLinkedQueue<UnboundedReadEvaluator<OutputT, CheckpointMarkT>>
         evaluatorQueue;
     /**
@@ -153,7 +153,7 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
 
     public UnboundedReadEvaluator(
         AppliedPTransform<?, PCollection<OutputT>, Unbounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
+        EvaluationContext evaluationContext,
         UnboundedSource<OutputT, CheckpointMarkT> source,
         ConcurrentLinkedQueue<UnboundedReadEvaluator<OutputT, CheckpointMarkT>> evaluatorQueue) {
       this.transform = transform;
@@ -168,7 +168,7 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
     public void processElement(WindowedValue<Object> element) {}
 
     @Override
-    public InProcessTransformResult finishBundle() throws IOException {
+    public TransformResult finishBundle() throws IOException {
       UncommittedBundle<OutputT> output = evaluationContext.createRootBundle(transform.getOutput());
       try {
         boolean elementAvailable = startReader();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java
new file mode 100644
index 0000000..570dc90
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.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.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.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 DirectRunner}.
+ */
+public class UncommittedBundleOutputManager implements OutputManager {
+  private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;
+
+  public static UncommittedBundleOutputManager create(
+      Map<TupleTag<?>, UncommittedBundle<?>> outputBundles) {
+    return new UncommittedBundleOutputManager(outputBundles);
+  }
+
+  public UncommittedBundleOutputManager(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/babddbbc/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
index f4260f5..47eaae7 100644
--- 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
@@ -41,7 +41,7 @@ import java.util.List;
  *
  * <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 DirectCreatePCollectionView} composite transform. This transform is an override for the
  * {@link CreatePCollectionView} transform that applies windowing and triggers before the view is
  * written.
  */
@@ -50,7 +50,7 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
   public <T> TransformEvaluator<T> forApplication(
       AppliedPTransform<?, ?, ?> application,
       DirectRunner.CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     @SuppressWarnings({"cast", "unchecked", "rawtypes"})
     TransformEvaluator<T> evaluator = createEvaluator(
             (AppliedPTransform) application, evaluationContext);
@@ -60,7 +60,7 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
   private <InT, OuT> TransformEvaluator<Iterable<InT>> createEvaluator(
       final AppliedPTransform<PCollection<Iterable<InT>>, PCollectionView<OuT>, WriteView<InT, OuT>>
           application,
-      InProcessEvaluationContext context) {
+      EvaluationContext context) {
     PCollection<Iterable<InT>> input = application.getInput();
     final PCollectionViewWriter<InT, OuT> writer =
         context.createPCollectionViewWriter(input, application.getOutput());
@@ -75,14 +75,14 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
       }
 
       @Override
-      public InProcessTransformResult finishBundle() {
+      public TransformResult finishBundle() {
         writer.add(elements);
         return StepTransformResult.withoutHold(application).build();
       }
     };
   }
 
-  public static class InProcessViewOverrideFactory implements PTransformOverrideFactory {
+  public static class ViewOverrideFactory implements PTransformOverrideFactory {
     @Override
     public <InputT extends PInput, OutputT extends POutput>
         PTransform<InputT, OutputT> override(PTransform<InputT, OutputT> transform) {
@@ -92,7 +92,7 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
       @SuppressWarnings({"rawtypes", "unchecked"})
       PTransform<InputT, OutputT> createView =
           (PTransform<InputT, OutputT>)
-              new InProcessCreatePCollectionView<>((CreatePCollectionView) transform);
+              new DirectCreatePCollectionView<>((CreatePCollectionView) transform);
       return createView;
     }
   }
@@ -100,11 +100,11 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
   /**
    * An in-process override for {@link CreatePCollectionView}.
    */
-  private static class InProcessCreatePCollectionView<ElemT, ViewT>
+  private static class DirectCreatePCollectionView<ElemT, ViewT>
       extends ForwardingPTransform<PCollection<ElemT>, PCollectionView<ViewT>> {
     private final CreatePCollectionView<ElemT, ViewT> og;
 
-    private InProcessCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
+    private DirectCreatePCollectionView(CreatePCollectionView<ElemT, ViewT> og) {
       this.og = og;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
new file mode 100644
index 0000000..b8f9987
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -0,0 +1,1420 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.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.auto.value.AutoValue;
+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.ConcurrentLinkedQueue;
+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 WatermarkManager} is designed to update and return a
+ * consistent view of watermarks in the presence of concurrent updates.
+ *
+ * <p>An {@link WatermarkManager} 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 WatermarkManager} is provided with the produced elements and the output watermark
+ * of the producing {@link AppliedPTransform transform}. The
+ * {@link WatermarkManager 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 WatermarkManager {
+  /**
+   * 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 WatermarkManager#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<StructuralKey<?>, 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<StructuralKey<?>, 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<StructuralKey<?>, NavigableSet<TimerData>> processingTimers;
+    private final Map<StructuralKey<?>, 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) {
+      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 completedTimer : update.completedTimers) {
+        pendingTimers.remove(completedTimer);
+      }
+      for (TimerData deletedTimer : update.deletedTimers) {
+        NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
+        if (timerQueue != null) {
+          timerQueue.remove(deletedTimer);
+        }
+      }
+    }
+
+    private synchronized Map<StructuralKey<?>, List<TimerData>> extractFiredDomainTimers(
+        TimeDomain domain, Instant firingTime) {
+      Map<StructuralKey<?>, 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<StructuralKey<?>, ? extends Collection<TimerData>> firedTimer :
+          firedTimers.entrySet()) {
+        pendingTimers.addAll(firedTimer.getValue());
+      }
+      return firedTimers;
+    }
+
+    private Map<TimeDomain, NavigableSet<TimerData>> timerMap(StructuralKey<?> 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<StructuralKey<?>, List<TimerData>> extractFiredTimers(
+      Instant latestTime, Map<StructuralKey<?>, NavigableSet<TimerData>> objectTimers) {
+    Map<StructuralKey<?>, List<TimerData>> result = new HashMap<>();
+    Set<StructuralKey<?>> emptyKeys = new HashSet<>();
+    for (Map.Entry<StructuralKey<?>, 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;
+
+  /**
+   * A queue of pending updates to the state of this {@link WatermarkManager}.
+   */
+  private final ConcurrentLinkedQueue<PendingWatermarkUpdate> pendingUpdates;
+
+  /**
+   * A queue of pending {@link AppliedPTransform AppliedPTransforms} that have potentially
+   * stale data.
+   */
+  private final ConcurrentLinkedQueue<AppliedPTransform<?, ?, ?>> pendingRefreshes;
+
+  /**
+   * Creates a new {@link WatermarkManager}. All watermarks within the newly created
+   * {@link WatermarkManager} 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 WatermarkManager create(
+      Clock clock,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+    return new WatermarkManager(clock, rootTransforms, consumers);
+  }
+
+  private WatermarkManager(
+      Clock clock,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+    this.clock = clock;
+    this.consumers = consumers;
+    this.pendingUpdates = new ConcurrentLinkedQueue<>();
+    this.pendingRefreshes = new ConcurrentLinkedQueue<>();
+
+    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 timerUpdate the timers that were added, removed, and completed as part of producing
+   *                    this update
+   * @param result the result that was produced by processing the input
+   * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
+   *                     is no hold
+   */
+  public void updateWatermarks(
+      @Nullable CommittedBundle<?> completed,
+      TimerUpdate timerUpdate,
+      CommittedResult result,
+      Instant earliestHold) {
+    pendingUpdates.offer(PendingWatermarkUpdate.create(completed,
+        timerUpdate,
+        result,
+        earliestHold));
+  }
+
+  /**
+   * Applies all pending updates to this {@link WatermarkManager}, causing the pending state
+   * of all {@link TransformWatermarks} to be advanced as far as possible.
+   */
+  private void applyPendingUpdates() {
+    Set<AppliedPTransform<?, ?, ?>> updatedTransforms = new HashSet<>();
+    PendingWatermarkUpdate pending = pendingUpdates.poll();
+    while (pending != null) {
+      applyPendingUpdate(pending);
+      updatedTransforms.add(pending.getTransform());
+      pending = pendingUpdates.poll();
+    }
+    pendingRefreshes.addAll(updatedTransforms);
+  }
+
+  private void applyPendingUpdate(PendingWatermarkUpdate pending) {
+    CommittedResult result = pending.getResult();
+    AppliedPTransform transform = result.getTransform();
+    CommittedBundle<?> inputBundle = pending.getInputBundle();
+
+    updatePending(inputBundle, pending.getTimerUpdate(), result);
+
+    TransformWatermarks transformWms = transformToWatermarks.get(transform);
+    transformWms.setEventTimeHold(inputBundle == null ? null : inputBundle.getKey(),
+        pending.getEarliestHold());
+  }
+
+  /**
+   * First adds all produced elements to the queue of pending elements for each consumer, then adds
+   * all pending timers to the collection of pending timers, then removes all completed and deleted
+   * timers from the collection of pending timers, then removes all completed elements from the
+   * pending queue of the transform.
+   *
+   * <p>It is required that all newly pending elements are added to the queue of pending elements
+   * for each consumer prior to the completed elements being removed, as doing otherwise could cause
+   * a Watermark to appear in a state in which the upstream (completed) element does not hold the
+   * watermark but the element it produced is not yet pending. This can cause the watermark to
+   * erroneously advance.
+   */
+  private void updatePending(
+      CommittedBundle<?> input,
+      TimerUpdate timerUpdate,
+      CommittedResult result) {
+    // Newly pending elements must be added before completed elements are removed, as the two
+    // do not share a Mutex within this call and thus can be interleaved with external calls to
+    // refresh.
+    for (CommittedBundle<?> bundle : result.getOutputs()) {
+      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
+        TransformWatermarks watermarks = transformToWatermarks.get(consumer);
+        watermarks.addPending(bundle);
+      }
+    }
+
+    TransformWatermarks completedTransform = transformToWatermarks.get(result.getTransform());
+    if (input != null) {
+      // Add the unprocessed inputs
+      completedTransform.addPending(result.getUnprocessedInputs());
+    }
+    completedTransform.updateTimers(timerUpdate);
+    if (input != null) {
+      completedTransform.removePending(input);
+    }
+  }
+
+  /**
+   * Refresh the watermarks contained within this {@link WatermarkManager}, causing all
+   * watermarks to be advanced as far as possible.
+   */
+  synchronized void refreshAll() {
+    applyPendingUpdates();
+    while (!pendingRefreshes.isEmpty()) {
+      refreshWatermarks(pendingRefreshes.poll());
+    }
+  }
+
+  private void refreshWatermarks(AppliedPTransform<?, ?, ?> toRefresh) {
+    TransformWatermarks myWatermarks = transformToWatermarks.get(toRefresh);
+    WatermarkUpdate updateResult = myWatermarks.refresh();
+    Set<AppliedPTransform<?, ?, ?>> additionalRefreshes = new HashSet<>();
+    if (updateResult.isAdvanced()) {
+      for (PValue outputPValue : toRefresh.getOutput().expand()) {
+        additionalRefreshes.addAll(consumers.get(outputPValue));
+      }
+    }
+    pendingRefreshes.addAll(additionalRefreshes);
+  }
+
+  /**
+   * 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 WatermarkManager}.
+   */
+  public Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> extractFiredTimers() {
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> allTimers = new HashMap<>();
+    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
+        transformToWatermarks.entrySet()) {
+      Map<StructuralKey<?>, 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<StructuralKey<?>, FiredTimers> extractFiredTimers() {
+      Map<StructuralKey<?>, List<TimerData>> eventTimeTimers =
+          inputWatermark.extractFiredEventTimeTimers();
+      Map<StructuralKey<?>, List<TimerData>> processingTimers;
+      Map<StructuralKey<?>, 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<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
+      groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
+
+      Map<StructuralKey<?>, FiredTimers> keyFiredTimers = new HashMap<>();
+      for (Map.Entry<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> firedTimers :
+          groupedTimers.entrySet()) {
+        keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
+      }
+      return keyFiredTimers;
+    }
+
+    @SafeVarargs
+    private final void groupFiredTimers(
+        Map<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> groupedToMutate,
+        Map<StructuralKey<?>, List<TimerData>>... timersToGroup) {
+      for (Map<StructuralKey<?>, List<TimerData>> subGroup : timersToGroup) {
+        for (Map.Entry<StructuralKey<?>, 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 StructuralKey<?> 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(StructuralKey<?> 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 StructuralKey<?> key;
+      private final Collection<TimerData> completedTimers;
+      private final Collection<TimerData> setTimers;
+      private final Collection<TimerData> deletedTimers;
+
+      private TimerUpdateBuilder(StructuralKey<?> 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(
+        StructuralKey<?> 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
+    StructuralKey<?> 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 WatermarkManager}.
+   */
+  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;
+  }
+
+  @AutoValue
+  abstract static class PendingWatermarkUpdate {
+    @Nullable
+    public abstract CommittedBundle<?> getInputBundle();
+    public abstract TimerUpdate getTimerUpdate();
+    public abstract CommittedResult getResult();
+    public abstract Instant getEarliestHold();
+
+    /**
+     * Gets the {@link AppliedPTransform} that generated this result.
+     */
+    public AppliedPTransform<?, ?, ?> getTransform() {
+      return getResult().getTransform();
+    }
+
+    public static PendingWatermarkUpdate create(
+        CommittedBundle<?> inputBundle,
+        TimerUpdate timerUpdate,
+        CommittedResult result, Instant earliestHold) {
+      return new AutoValue_WatermarkManager_PendingWatermarkUpdate(inputBundle,
+          timerUpdate,
+          result,
+          earliestHold);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 89866cc..b07b58a 100644
--- 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
@@ -45,7 +45,7 @@ class WindowEvaluatorFactory implements TransformEvaluatorFactory {
   public <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application,
       @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
+      EvaluationContext evaluationContext)
       throws Exception {
     return createTransformEvaluator(
         (AppliedPTransform) application, inputBundle, evaluationContext);
@@ -54,7 +54,7 @@ class WindowEvaluatorFactory implements TransformEvaluatorFactory {
   private <InputT> TransformEvaluator<InputT> createTransformEvaluator(
       AppliedPTransform<PCollection<InputT>, PCollection<InputT>, Window.Bound<InputT>> transform,
       CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     WindowFn<? super InputT, ?> fn = transform.getTransform().getWindowFn();
     UncommittedBundle<InputT> outputBundle =
         evaluationContext.createBundle(inputBundle, transform.getOutput());
@@ -92,22 +92,22 @@ class WindowEvaluatorFactory implements TransformEvaluatorFactory {
     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);
+          new DirectAssignContext<>(windowFn, element);
       Collection<? extends BoundedWindow> windows = windowFn.assignWindows(assignContext);
       return windows;
     }
 
     @Override
-    public InProcessTransformResult finishBundle() throws Exception {
+    public TransformResult finishBundle() throws Exception {
       return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build();
     }
   }
 
-  private static class InProcessAssignContext<InputT, W extends BoundedWindow>
+  private static class DirectAssignContext<InputT, W extends BoundedWindow>
       extends WindowFn<InputT, W>.AssignContext {
     private final WindowedValue<InputT> value;
 
-    public InProcessAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
+    public DirectAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
       fn.super();
       this.value = value;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index e26f860..9bc4f7b 100644
--- 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
@@ -64,7 +64,7 @@ public class BoundedReadEvaluatorFactoryTest {
   private BoundedSource<Long> source;
   private PCollection<Long> longs;
   private TransformEvaluatorFactory factory;
-  @Mock private InProcessEvaluationContext context;
+  @Mock private EvaluationContext context;
   private BundleFactory bundleFactory;
 
   @Before
@@ -75,7 +75,7 @@ public class BoundedReadEvaluatorFactoryTest {
     longs = p.apply(Read.from(source));
 
     factory = new BoundedReadEvaluatorFactory();
-    bundleFactory = InProcessBundleFactory.create();
+    bundleFactory = ImmutableListBundleFactory.create();
   }
 
   @Test
@@ -85,7 +85,7 @@ public class BoundedReadEvaluatorFactoryTest {
 
     TransformEvaluator<?> evaluator =
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
     assertThat(
         output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements(),
@@ -105,7 +105,7 @@ public class BoundedReadEvaluatorFactoryTest {
 
     TransformEvaluator<?> evaluator =
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
     Iterable<? extends WindowedValue<Long>> outputElements =
         output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();
@@ -138,7 +138,7 @@ public class BoundedReadEvaluatorFactoryTest {
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
     assertThat(secondEvaluator, nullValue());
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
     Iterable<? extends WindowedValue<Long>> outputElements =
         output.commit(BoundedWindow.TIMESTAMP_MAX_VALUE).getElements();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 4969a30..1e51b55 100644
--- 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
@@ -53,7 +53,7 @@ public class CommittedResultTest implements Serializable {
   private transient AppliedPTransform<?, ?, ?> transform =
       AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
       });
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void getTransformExtractsFromResult() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
index cd44b7e..5c89f1b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
@@ -20,7 +20,7 @@ package org.apache.beam.runners.direct;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import org.apache.beam.runners.direct.DirectRegistrar.InProcessRunner;
+import org.apache.beam.runners.direct.DirectRegistrar.DirectRunner;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
@@ -33,42 +33,42 @@ import org.junit.runners.JUnit4;
 
 import java.util.ServiceLoader;
 
-/** Tests for {@link InProcessRunner}. */
+/** Tests for {@link DirectRunner}. */
 @RunWith(JUnit4.class)
 public class DirectRegistrarTest {
   @Test
   public void testCorrectOptionsAreReturned() {
     assertEquals(
         ImmutableList.of(DirectOptions.class),
-        new DirectRegistrar.InProcessOptions().getPipelineOptions());
+        new DirectRegistrar.DirectOptions().getPipelineOptions());
   }
 
   @Test
   public void testCorrectRunnersAreReturned() {
     assertEquals(
-        ImmutableList.of(DirectRunner.class),
-        new DirectRegistrar.InProcessRunner().getPipelineRunners());
+        ImmutableList.of(org.apache.beam.runners.direct.DirectRunner.class),
+        new DirectRunner().getPipelineRunners());
   }
 
   @Test
   public void testServiceLoaderForOptions() {
     for (PipelineOptionsRegistrar registrar :
         Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
-      if (registrar instanceof DirectRegistrar.InProcessOptions) {
+      if (registrar instanceof DirectRegistrar.DirectOptions) {
         return;
       }
     }
-    fail("Expected to find " + DirectRegistrar.InProcessOptions.class);
+    fail("Expected to find " + DirectRegistrar.DirectOptions.class);
   }
 
   @Test
   public void testServiceLoaderForRunner() {
     for (PipelineRunnerRegistrar registrar :
         Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
-      if (registrar instanceof DirectRegistrar.InProcessRunner) {
+      if (registrar instanceof DirectRunner) {
         return;
       }
     }
-    fail("Expected to find " + DirectRegistrar.InProcessRunner.class);
+    fail("Expected to find " + DirectRunner.class);
   }
 }


[16/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index 1cfa544..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.runners.direct;
-
-import org.apache.beam.runners.direct.DirectRunner.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/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
deleted file mode 100644
index 53b93d0..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupAlsoByWindow;
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.sdk.coders.Coder;
-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.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.sdk.util.KeyedWorkItem;
-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.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.util.Collections;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
- * {@link GroupByKeyOnly} {@link PTransform}.
- */
-class InProcessGroupAlsoByWindowEvaluatorFactory 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<KeyedWorkItem<K, V>> createEvaluator(
-      AppliedPTransform<
-              PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>,
-              InProcessGroupAlsoByWindow<K, V>>
-          application,
-      CommittedBundle<KeyedWorkItem<K, V>> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
-    return new InProcessGroupAlsoByWindowEvaluator<K, V>(
-        evaluationContext, inputBundle, application);
-  }
-
-  /**
-   * A transform evaluator for the pseudo-primitive {@link GroupAlsoByWindow}. Windowing is ignored;
-   * all input should be in the global window since all output will be as well.
-   *
-   * @see GroupByKeyViaGroupByKeyOnly
-   */
-  private static class InProcessGroupAlsoByWindowEvaluator<K, V>
-      implements TransformEvaluator<KeyedWorkItem<K, V>> {
-
-    private final TransformEvaluator<KeyedWorkItem<K, V>> gabwParDoEvaluator;
-
-    public InProcessGroupAlsoByWindowEvaluator(
-        final InProcessEvaluationContext evaluationContext,
-        CommittedBundle<KeyedWorkItem<K, V>> inputBundle,
-        final AppliedPTransform<
-                PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>,
-                InProcessGroupAlsoByWindow<K, V>>
-            application) {
-
-      Coder<V> valueCoder =
-          application.getTransform().getValueCoder(inputBundle.getPCollection().getCoder());
-
-      @SuppressWarnings("unchecked")
-      WindowingStrategy<?, BoundedWindow> windowingStrategy =
-          (WindowingStrategy<?, BoundedWindow>) application.getTransform().getWindowingStrategy();
-
-      DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> gabwDoFn =
-          GroupAlsoByWindowViaWindowSetDoFn.create(
-              windowingStrategy,
-              SystemReduceFn.<K, V, BoundedWindow>buffering(valueCoder));
-
-      TupleTag<KV<K, Iterable<V>>> mainOutputTag = new TupleTag<KV<K, Iterable<V>>>() {};
-
-      // Not technically legit, as the application is not a ParDo
-      this.gabwParDoEvaluator =
-          ParDoInProcessEvaluator.create(
-              evaluationContext,
-              inputBundle,
-              application,
-              gabwDoFn,
-              Collections.<PCollectionView<?>>emptyList(),
-              mainOutputTag,
-              Collections.<TupleTag<?>>emptyList(),
-              ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
-    }
-
-    @Override
-    public void processElement(WindowedValue<KeyedWorkItem<K, V>> element) throws Exception {
-      gabwParDoEvaluator.processElement(element);
-    }
-
-    @Override
-    public InProcessTransformResult finishBundle() throws Exception {
-      return gabwParDoEvaluator.finishBundle();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKey.java
deleted file mode 100644
index 026b4d5..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKey.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.direct;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-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.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-class InProcessGroupByKey<K, V>
-    extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-  private final GroupByKey<K, V> original;
-
-  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) {
-    @SuppressWarnings("unchecked")
-    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();
-
-    // 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", new InProcessGroupAlsoByWindow<K, V>(windowingStrategy))
-
-        // And update the windowing strategy as appropriate.
-        .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
-        .setCoder(
-            KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
-  }
-
-  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());
-    }
-
-    InProcessGroupByKeyOnly() {}
-  }
-
-  static final class InProcessGroupAlsoByWindow<K, V>
-      extends PTransform<PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
-    private final WindowingStrategy<?, ?> windowingStrategy;
-
-    public InProcessGroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
-      this.windowingStrategy = windowingStrategy;
-    }
-
-    public WindowingStrategy<?, ?> getWindowingStrategy() {
-      return windowingStrategy;
-    }
-
-    private KeyedWorkItemCoder<K, V> getKeyedWorkItemCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
-      // Coder<KV<...>> --> KvCoder<...>
-      checkArgument(
-          inputCoder instanceof KeyedWorkItemCoder,
-          "%s requires a %s<...> but got %s",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName(),
-          inputCoder);
-      @SuppressWarnings("unchecked")
-      KeyedWorkItemCoder<K, V> kvCoder = (KeyedWorkItemCoder<K, V>) inputCoder;
-      return kvCoder;
-    }
-
-    public Coder<K> getKeyCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
-      return getKeyedWorkItemCoder(inputCoder).getKeyCoder();
-    }
-
-    public Coder<V> getValueCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
-      return getKeyedWorkItemCoder(inputCoder).getElementCoder();
-    }
-
-    @Override
-    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KeyedWorkItem<K, V>> input) {
-      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
-          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
deleted file mode 100644
index 3604bca..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.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.KvCoder;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-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 java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
- * {@link GroupByKeyOnly} {@link PTransform}.
- */
-class InProcessGroupByKeyOnlyEvaluatorFactory 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, WindowedValue<V>>> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
-    return new InProcessGroupByKeyOnlyEvaluator<K, V>(evaluationContext, inputBundle, application);
-  }
-
-  /**
-   * A transform evaluator for the pseudo-primitive {@link GroupByKeyOnly}. Windowing is ignored;
-   * all input should be in the global window since all output will be as well.
-   *
-   * @see GroupByKeyViaGroupByKeyOnly
-   */
-  private static class InProcessGroupByKeyOnlyEvaluator<K, V>
-      implements TransformEvaluator<KV<K, WindowedValue<V>>> {
-    private final InProcessEvaluationContext evaluationContext;
-
-    private final CommittedBundle<KV<K, WindowedValue<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 InProcessGroupByKeyOnlyEvaluator(
-        InProcessEvaluationContext evaluationContext,
-        CommittedBundle<KV<K, WindowedValue<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;
-      this.keyCoder = getKeyCoder(application.getInput().getCoder());
-      this.groupingMap = new HashMap<>();
-    }
-
-    private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
-      checkState(
-          coder instanceof KvCoder,
-          "%s requires a coder of class %s."
-              + " This is an internal error; this is checked during pipeline construction"
-              + " but became corrupted.",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName());
-      @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,
-            StructuralKey.of(key, keyCoder),
-            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);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOverrideFactory.java
deleted file mode 100644
index 1d84bc9..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOverrideFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-/**
- * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
- */
-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 =
-          (PTransform) new InProcessGroupByKey((GroupByKey) transform);
-      return override;
-    }
-    return transform;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index f53f590..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java
+++ /dev/null
@@ -1,277 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-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.ReadyCheckingSideInputReader;
-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.Optional;
-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.Iterables;
-import com.google.common.collect.Sets;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-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 Collection<PCollectionView<?>> containedViews;
-  private final LoadingCache<
-          PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
-      viewByWindows;
-
-  /**
-   * Create a new {@link InProcessSideInputContainer} with the provided views and the provided
-   * context.
-   */
-  public static InProcessSideInputContainer create(
-      final InProcessEvaluationContext context, Collection<PCollectionView<?>> containedViews) {
-    LoadingCache<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
-        viewByWindows = CacheBuilder.newBuilder().build(new CallbackSchedulingLoader(context));
-    return new InProcessSideInputContainer(containedViews, viewByWindows);
-  }
-
-  private InProcessSideInputContainer(
-      Collection<PCollectionView<?>> containedViews,
-      LoadingCache<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
-          viewByWindows) {
-    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);
-    AtomicReference<Iterable<? extends WindowedValue<?>>> contents =
-        viewByWindows.getUnchecked(windowedView);
-    if (contents.compareAndSet(null, windowValues)) {
-      // the value had never been set, so we set it and are done.
-      return;
-    }
-    PaneInfo newPane = windowValues.iterator().next().getPane();
-
-    Iterable<? extends WindowedValue<?>> existingValues;
-    long existingPane;
-    do {
-      existingValues = contents.get();
-      existingPane =
-          Iterables.isEmpty(existingValues)
-              ? -1L
-              : existingValues.iterator().next().getPane().getIndex();
-    } while (newPane.getIndex() > existingPane
-        && !contents.compareAndSet(existingValues, windowValues));
-  }
-
-  private static class CallbackSchedulingLoader extends
-      CacheLoader<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>> {
-    private final InProcessEvaluationContext context;
-
-    public CallbackSchedulingLoader(
-        InProcessEvaluationContext context) {
-      this.context = context;
-    }
-
-    @Override
-    public AtomicReference<Iterable<? extends WindowedValue<?>>>
-        load(PCollectionViewWindow<?> view) {
-
-      AtomicReference<Iterable<? extends WindowedValue<?>>> contents = new AtomicReference<>();
-      WindowingStrategy<?, ?> windowingStrategy = view.getView().getWindowingStrategyInternal();
-
-      context.scheduleAfterOutputWouldBeProduced(view.getView(),
-          view.getWindow(),
-          windowingStrategy,
-          new WriteEmptyViewContents(view.getView(), view.getWindow(), contents));
-      return contents;
-    }
-  }
-
-  private static class WriteEmptyViewContents implements Runnable {
-    private final PCollectionView<?> view;
-    private final BoundedWindow window;
-    private final AtomicReference<Iterable<? extends WindowedValue<?>>> contents;
-
-    private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
-        AtomicReference<Iterable<? extends WindowedValue<?>>> contents) {
-      this.contents = contents;
-      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.
-      contents.compareAndSet(null, Collections.<WindowedValue<?>>emptyList());
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("view", view)
-          .add("window", window)
-          .toString();
-    }
-  }
-
-  private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
-    private final Collection<PCollectionView<?>> readerViews;
-    private final LoadingCache<
-        PCollectionViewWindow<?>, Optional<? extends Iterable<? extends WindowedValue<?>>>>
-        viewContents;
-
-    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
-      this.readerViews = ImmutableSet.copyOf(readerViews);
-      this.viewContents = CacheBuilder.newBuilder().build(new CurrentViewContentsLoader());
-    }
-
-    @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 viewContents.getUnchecked(PCollectionViewWindow.of(view, window)).isPresent();
-    }
-
-    @Override
-    @Nullable
-    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
-      checkArgument(readerViews.contains(view),
-          "call to get(PCollectionView) with unknown view: %s",
-          view);
-      checkArgument(
-          isReady(view, window),
-          "calling get() on PCollectionView %s that is not ready in window %s",
-          view,
-          window);
-      // Safe covariant cast
-      @SuppressWarnings("unchecked") Iterable<WindowedValue<?>> values =
-          (Iterable<WindowedValue<?>>) viewContents.getUnchecked(PCollectionViewWindow.of(view,
-              window)).get();
-      return view.fromIterableInternal(values);
-    }
-
-    @Override
-    public <T> boolean contains(PCollectionView<T> view) {
-      return readerViews.contains(view);
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return readerViews.isEmpty();
-    }
-  }
-
-  /**
-   * A {@link CacheLoader} that loads the current contents of a {@link PCollectionViewWindow} into
-   * an optional.
-   */
-  private class CurrentViewContentsLoader extends CacheLoader<
-      PCollectionViewWindow<?>, Optional<? extends Iterable<? extends WindowedValue<?>>>> {
-
-    @Override
-    public Optional<? extends Iterable<? extends WindowedValue<?>>>
-        load(PCollectionViewWindow<?> key) {
-      return Optional.fromNullable(viewByWindows.getUnchecked(key).get());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index cd54f59..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.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/babddbbc/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
deleted file mode 100644
index 92127b4..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessTransformResult.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.runners.direct;
-
-import org.apache.beam.runners.direct.InMemoryWatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.DirectRunner.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.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 elements that were provided to the {@link TransformEvaluator} as input but were not
-   * processed.
-   */
-  Iterable<? extends WindowedValue<?>> getUnprocessedElements();
-
-  /**
-   * 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/babddbbc/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
index 758ee24..074619a 100644
--- 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
@@ -30,7 +30,7 @@ import org.apache.beam.sdk.values.PCollection;
  * {@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
+ * output {@link TransformResult} and committed output bundles after the
  * {@link TransformEvaluator} has completed.
  *
  * <p>Typically, {@link ModelEnforcement} will obtain required metadata (such as the {@link Coder}
@@ -53,11 +53,11 @@ public interface ModelEnforcement<T> {
 
   /**
    * Called after a bundle has been completed and {@link TransformEvaluator#finishBundle()} has been
-   * called, producing the provided {@link InProcessTransformResult} and
+   * called, producing the provided {@link TransformResult} and
    * {@link CommittedBundle output bundles}.
    */
   void afterFinish(
       CommittedBundle<T> input,
-      InProcessTransformResult result,
+      TransformResult result,
       Iterable<? extends CommittedBundle<?>> outputs);
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
new file mode 100644
index 0000000..58cee4d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectExecutionContext.DirectStepContext;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.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.PushbackSideInputDoFnRunner;
+import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
+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 com.google.common.collect.ImmutableList;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class ParDoEvaluator<T> implements TransformEvaluator<T> {
+  public static <InputT, OutputT> ParDoEvaluator<InputT> create(
+      EvaluationContext 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) {
+    DirectExecutionContext executionContext =
+        evaluationContext.getExecutionContext(application, inputBundle.getKey());
+    String stepName = evaluationContext.getStepName(application);
+    DirectStepContext 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()));
+    }
+
+    ReadyCheckingSideInputReader sideInputReader =
+        evaluationContext.createSideInputReader(sideInputs);
+    DoFnRunner<InputT, OutputT> underlying =
+        DoFnRunners.createDefault(
+            evaluationContext.getPipelineOptions(),
+            fn,
+            sideInputReader,
+            BundleOutputManager.create(outputBundles),
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            counters.getAddCounterMutator(),
+            application.getInput().getWindowingStrategy());
+    PushbackSideInputDoFnRunner<InputT, OutputT> runner =
+        PushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader);
+
+    try {
+      runner.startBundle();
+    } catch (Exception e) {
+      throw UserCodeException.wrap(e);
+    }
+
+    return new ParDoEvaluator<>(
+        runner, application, counters, outputBundles.values(), stepContext);
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private final PushbackSideInputDoFnRunner<T, ?> fnRunner;
+  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
+  private final CounterSet counters;
+  private final Collection<UncommittedBundle<?>> outputBundles;
+  private final DirectStepContext stepContext;
+
+  private final ImmutableList.Builder<WindowedValue<T>> unprocessedElements;
+
+  private ParDoEvaluator(
+      PushbackSideInputDoFnRunner<T, ?> fnRunner,
+      AppliedPTransform<PCollection<T>, ?, ?> transform,
+      CounterSet counters,
+      Collection<UncommittedBundle<?>> outputBundles,
+      DirectStepContext stepContext) {
+    this.fnRunner = fnRunner;
+    this.transform = transform;
+    this.counters = counters;
+    this.outputBundles = outputBundles;
+    this.stepContext = stepContext;
+
+    this.unprocessedElements = ImmutableList.builder();
+  }
+
+  @Override
+  public void processElement(WindowedValue<T> element) {
+    try {
+      Iterable<WindowedValue<T>> unprocessed = fnRunner.processElementInReadyWindows(element);
+      unprocessedElements.addAll(unprocessed);
+    } catch (Exception e) {
+      throw UserCodeException.wrap(e);
+    }
+  }
+
+  @Override
+  public TransformResult 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)
+        .addUnprocessedElements(unprocessedElements.build())
+        .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/babddbbc/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
deleted file mode 100644
index b9f4808..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoInProcessEvaluator.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import org.apache.beam.runners.direct.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.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.PushbackSideInputDoFnRunner;
-import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
-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 com.google.common.collect.ImmutableList;
-
-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()));
-    }
-
-    ReadyCheckingSideInputReader sideInputReader =
-        evaluationContext.createSideInputReader(sideInputs);
-    DoFnRunner<InputT, OutputT> underlying =
-        DoFnRunners.createDefault(
-            evaluationContext.getPipelineOptions(),
-            fn,
-            sideInputReader,
-            BundleOutputManager.create(outputBundles),
-            mainOutputTag,
-            sideOutputTags,
-            stepContext,
-            counters.getAddCounterMutator(),
-            application.getInput().getWindowingStrategy());
-    PushbackSideInputDoFnRunner<InputT, OutputT> runner =
-        PushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader);
-
-    try {
-      runner.startBundle();
-    } catch (Exception e) {
-      throw UserCodeException.wrap(e);
-    }
-
-    return new ParDoInProcessEvaluator<>(
-        runner, application, counters, outputBundles.values(), stepContext);
-  }
-
-  ////////////////////////////////////////////////////////////////////////////////////////////////
-
-  private final PushbackSideInputDoFnRunner<T, ?> fnRunner;
-  private final AppliedPTransform<PCollection<T>, ?, ?> transform;
-  private final CounterSet counters;
-  private final Collection<UncommittedBundle<?>> outputBundles;
-  private final InProcessStepContext stepContext;
-
-  private final ImmutableList.Builder<WindowedValue<T>> unprocessedElements;
-
-  private ParDoInProcessEvaluator(
-      PushbackSideInputDoFnRunner<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;
-
-    this.unprocessedElements = ImmutableList.builder();
-  }
-
-  @Override
-  public void processElement(WindowedValue<T> element) {
-    try {
-      Iterable<WindowedValue<T>> unprocessed = fnRunner.processElementInReadyWindows(element);
-      unprocessedElements.addAll(unprocessed);
-    } 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)
-        .addUnprocessedElements(unprocessedElements.build())
-        .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/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
index 58d6f00..e008bdc 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -47,7 +47,7 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
   public <T> TransformEvaluator<T> forApplication(
       AppliedPTransform<?, ?, ?> application,
       CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     @SuppressWarnings({"unchecked", "rawtypes"})
     TransformEvaluator<T> evaluator =
         createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
@@ -57,14 +57,14 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
   private <InT, OuT> TransformEvaluator<InT> createMultiEvaluator(
       AppliedPTransform<PCollection<InT>, PCollectionTuple, BoundMulti<InT, OuT>> application,
       CommittedBundle<InT> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     Map<TupleTag<?>, PCollection<?>> outputs = application.getOutput().getAll();
     DoFn<InT, OuT> fn = application.getTransform().getFn();
 
     @SuppressWarnings({"unchecked", "rawtypes"}) ThreadLocal<DoFn<InT, OuT>> fnLocal =
         (ThreadLocal) fnClones.getUnchecked(application.getTransform().getFn());
     try {
-      TransformEvaluator<InT> parDoEvaluator = ParDoInProcessEvaluator.create(evaluationContext,
+      TransformEvaluator<InT> parDoEvaluator = ParDoEvaluator.create(evaluationContext,
           inputBundle,
           application,
           fnLocal.get(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
index afbb6ed..0f7fc83 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -47,7 +47,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
   public <T> TransformEvaluator<T> forApplication(
       final AppliedPTransform<?, ?, ?> application,
       CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     @SuppressWarnings({"unchecked", "rawtypes"})
     TransformEvaluator<T> evaluator =
         createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext);
@@ -57,13 +57,13 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
   private <InputT, OutputT> TransformEvaluator<InputT> createSingleEvaluator(
       @SuppressWarnings("rawtypes") AppliedPTransform<PCollection<InputT>, PCollection<OutputT>,
           Bound<InputT, OutputT>> application,
-      CommittedBundle<InputT> inputBundle, InProcessEvaluationContext evaluationContext) {
+      CommittedBundle<InputT> inputBundle, EvaluationContext evaluationContext) {
     TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
 
     @SuppressWarnings({"unchecked", "rawtypes"}) ThreadLocal<DoFn<InputT, OutputT>> fnLocal =
         (ThreadLocal) fnClones.getUnchecked(application.getTransform().getFn());
     try {
-      ParDoInProcessEvaluator<InputT> parDoEvaluator = ParDoInProcessEvaluator.create(
+      ParDoEvaluator<InputT> parDoEvaluator = ParDoEvaluator.create(
           evaluationContext,
           inputBundle,
           application,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index ba792d3..c6e10e5 100644
--- 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
@@ -42,7 +42,7 @@ class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT
   }
 
   @Override
-  public InProcessTransformResult finishBundle() throws Exception {
+  public TransformResult finishBundle() throws Exception {
     return StepTransformResult.withoutHold(transform).addOutput(output).build();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java
new file mode 100644
index 0000000..76df11c
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.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.DirectRunner.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 PipelineExecutor {
+  /**
+   * 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/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
new file mode 100644
index 0000000..7a19ed9
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.PCollectionViewWindow;
+import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
+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.Optional;
+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.Iterables;
+import com.google.common.collect.Sets;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+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 SideInputContainer {
+  private final Collection<PCollectionView<?>> containedViews;
+  private final LoadingCache<
+          PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
+      viewByWindows;
+
+  /**
+   * Create a new {@link SideInputContainer} with the provided views and the provided
+   * context.
+   */
+  public static SideInputContainer create(
+      final EvaluationContext context, Collection<PCollectionView<?>> containedViews) {
+    LoadingCache<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
+        viewByWindows = CacheBuilder.newBuilder().build(new CallbackSchedulingLoader(context));
+    return new SideInputContainer(containedViews, viewByWindows);
+  }
+
+  private SideInputContainer(
+      Collection<PCollectionView<?>> containedViews,
+      LoadingCache<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>>
+          viewByWindows) {
+    this.containedViews = ImmutableSet.copyOf(containedViews);
+    this.viewByWindows = viewByWindows;
+  }
+
+  /**
+   * Return a view of this {@link SideInputContainer} that contains only the views in the
+   * provided argument. The returned {@link SideInputContainer} is unmodifiable without
+   * casting, but will change as this {@link SideInputContainer} 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);
+    AtomicReference<Iterable<? extends WindowedValue<?>>> contents =
+        viewByWindows.getUnchecked(windowedView);
+    if (contents.compareAndSet(null, windowValues)) {
+      // the value had never been set, so we set it and are done.
+      return;
+    }
+    PaneInfo newPane = windowValues.iterator().next().getPane();
+
+    Iterable<? extends WindowedValue<?>> existingValues;
+    long existingPane;
+    do {
+      existingValues = contents.get();
+      existingPane =
+          Iterables.isEmpty(existingValues)
+              ? -1L
+              : existingValues.iterator().next().getPane().getIndex();
+    } while (newPane.getIndex() > existingPane
+        && !contents.compareAndSet(existingValues, windowValues));
+  }
+
+  private static class CallbackSchedulingLoader extends
+      CacheLoader<PCollectionViewWindow<?>, AtomicReference<Iterable<? extends WindowedValue<?>>>> {
+    private final EvaluationContext context;
+
+    public CallbackSchedulingLoader(
+        EvaluationContext context) {
+      this.context = context;
+    }
+
+    @Override
+    public AtomicReference<Iterable<? extends WindowedValue<?>>>
+        load(PCollectionViewWindow<?> view) {
+
+      AtomicReference<Iterable<? extends WindowedValue<?>>> contents = new AtomicReference<>();
+      WindowingStrategy<?, ?> windowingStrategy = view.getView().getWindowingStrategyInternal();
+
+      context.scheduleAfterOutputWouldBeProduced(view.getView(),
+          view.getWindow(),
+          windowingStrategy,
+          new WriteEmptyViewContents(view.getView(), view.getWindow(), contents));
+      return contents;
+    }
+  }
+
+  private static class WriteEmptyViewContents implements Runnable {
+    private final PCollectionView<?> view;
+    private final BoundedWindow window;
+    private final AtomicReference<Iterable<? extends WindowedValue<?>>> contents;
+
+    private WriteEmptyViewContents(PCollectionView<?> view, BoundedWindow window,
+        AtomicReference<Iterable<? extends WindowedValue<?>>> contents) {
+      this.contents = contents;
+      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.
+      contents.compareAndSet(null, Collections.<WindowedValue<?>>emptyList());
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("view", view)
+          .add("window", window)
+          .toString();
+    }
+  }
+
+  private final class SideInputContainerSideInputReader implements ReadyCheckingSideInputReader {
+    private final Collection<PCollectionView<?>> readerViews;
+    private final LoadingCache<
+        PCollectionViewWindow<?>, Optional<? extends Iterable<? extends WindowedValue<?>>>>
+        viewContents;
+
+    private SideInputContainerSideInputReader(Collection<PCollectionView<?>> readerViews) {
+      this.readerViews = ImmutableSet.copyOf(readerViews);
+      this.viewContents = CacheBuilder.newBuilder().build(new CurrentViewContentsLoader());
+    }
+
+    @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 viewContents.getUnchecked(PCollectionViewWindow.of(view, window)).isPresent();
+    }
+
+    @Override
+    @Nullable
+    public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
+      checkArgument(readerViews.contains(view),
+          "call to get(PCollectionView) with unknown view: %s",
+          view);
+      checkArgument(
+          isReady(view, window),
+          "calling get() on PCollectionView %s that is not ready in window %s",
+          view,
+          window);
+      // Safe covariant cast
+      @SuppressWarnings("unchecked") Iterable<WindowedValue<?>> values =
+          (Iterable<WindowedValue<?>>) viewContents.getUnchecked(PCollectionViewWindow.of(view,
+              window)).get();
+      return view.fromIterableInternal(values);
+    }
+
+    @Override
+    public <T> boolean contains(PCollectionView<T> view) {
+      return readerViews.contains(view);
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return readerViews.isEmpty();
+    }
+  }
+
+  /**
+   * A {@link CacheLoader} that loads the current contents of a {@link PCollectionViewWindow} into
+   * an optional.
+   */
+  private class CurrentViewContentsLoader extends CacheLoader<
+      PCollectionViewWindow<?>, Optional<? extends Iterable<? extends WindowedValue<?>>>> {
+
+    @Override
+    public Optional<? extends Iterable<? extends WindowedValue<?>>>
+        load(PCollectionViewWindow<?> key) {
+      return Optional.fromNullable(viewByWindows.getUnchecked(key).get());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index eacea91..5706b2a 100644
--- 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
@@ -19,8 +19,8 @@ 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.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -37,9 +37,9 @@ import java.util.Collection;
 import javax.annotation.Nullable;
 
 /**
- * An immutable {@link InProcessTransformResult}.
+ * An immutable {@link TransformResult}.
  */
-public class StepTransformResult implements InProcessTransformResult {
+public class StepTransformResult implements TransformResult {
   private final AppliedPTransform<?, ?, ?> transform;
   private final Iterable<? extends UncommittedBundle<?>> bundles;
   private final Iterable<? extends WindowedValue<?>> unprocessedElements;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java
index 36bdff4..d8a6bf9 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java
@@ -52,7 +52,7 @@ class ThreadLocalInvalidatingTransformEvaluator<InputT>
   }
 
   @Override
-  public InProcessTransformResult finishBundle() throws Exception {
+  public TransformResult finishBundle() throws Exception {
     try {
       return underlying.finishBundle();
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 1fec9d8..6c8e48b 100644
--- 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
@@ -40,7 +40,7 @@ public interface TransformEvaluator<InputT> {
    * 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.
+   * @return an {@link TransformResult} containing the results of this bundle evaluation.
    */
-  InProcessTransformResult finishBundle() throws Exception;
+  TransformResult finishBundle() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index b12a34c..1973a2f 100644
--- 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
@@ -46,5 +46,5 @@ public interface TransformEvaluatorFactory {
    */
   @Nullable <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application, @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) throws Exception;
+      EvaluationContext evaluationContext) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index dfc1753..f0afc3b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupAlsoByWindow;
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
@@ -50,8 +50,8 @@ class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
             .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory())
             .put(Window.Bound.class, new WindowEvaluatorFactory())
             // Runner-specific primitives used in expansion of GroupByKey
-            .put(InProcessGroupByKeyOnly.class, new InProcessGroupByKeyOnlyEvaluatorFactory())
-            .put(InProcessGroupAlsoByWindow.class, new InProcessGroupAlsoByWindowEvaluatorFactory())
+            .put(DirectGroupByKeyOnly.class, new GroupByKeyOnlyEvaluatorFactory())
+            .put(DirectGroupAlsoByWindow.class, new GroupAlsoByWindowEvaluatorFactory())
             .build();
     return new TransformEvaluatorRegistry(primitives);
   }
@@ -71,7 +71,7 @@ class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
   public <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application,
       @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
+      EvaluationContext evaluationContext)
       throws Exception {
     TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass());
     return factory.forApplication(application, inputBundle, evaluationContext);



[42/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
deleted file mode 100644
index 4d37966..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.dataflow.model.Environment;
-
-/**
- * An instance of this class can be passed to the
- * {@link DataflowPipelineRunner} to add user defined hooks to be
- * invoked at various times during pipeline execution.
- */
-@Experimental
-public class DataflowPipelineRunnerHooks {
-  /**
-   * Allows the user to modify the environment of their job before their job is submitted
-   * to the service for execution.
-   *
-   * @param environment The environment of the job. Users can make change to this instance in order
-   *     to change the environment with which their job executes on the service.
-   */
-  public void modifyEnvironmentBeforeSubmission(Environment environment) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index f5fefc0..e15b9d2 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -30,7 +30,7 @@ import static org.apache.beam.sdk.util.Structs.getString;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
+import org.apache.beam.runners.dataflow.DataflowRunner.GroupByKeyAndSortValuesOnly;
 import org.apache.beam.runners.dataflow.internal.ReadTranslator;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.util.DoFnInfo;
@@ -139,7 +139,7 @@ public class DataflowPipelineTranslator {
    */
   public JobSpecification translate(
       Pipeline pipeline,
-      DataflowPipelineRunner runner,
+      DataflowRunner runner,
       List<DataflowPackage> packages) {
 
     Translator translator = new Translator(pipeline, runner);
@@ -224,7 +224,7 @@ public class DataflowPipelineTranslator {
 
   /**
    * The interface provided to registered callbacks for interacting
-   * with the {@link DataflowPipelineRunner}, including reading and writing the
+   * with the {@link DataflowRunner}, including reading and writing the
    * values of {@link PCollection}s and side inputs ({@link PCollectionView}s).
    */
   public interface TranslationContext {
@@ -350,7 +350,7 @@ public class DataflowPipelineTranslator {
     private final Pipeline pipeline;
 
     /** The runner which will execute the pipeline. */
-    private final DataflowPipelineRunner runner;
+    private final DataflowRunner runner;
 
     /** The Cloud Dataflow Job representation. */
     private final Job job = new Job();
@@ -385,7 +385,7 @@ public class DataflowPipelineTranslator {
      * Constructs a Translator that will translate the specified
      * Pipeline into Dataflow objects.
      */
-    public Translator(Pipeline pipeline, DataflowPipelineRunner runner) {
+    public Translator(Pipeline pipeline, DataflowRunner runner) {
       this.pipeline = pipeline;
       this.runner = runner;
     }


[23/50] [abbrv] incubator-beam git commit: Improve BigQueryIO validation for streaming WriteDisposition

Posted by da...@apache.org.
Improve BigQueryIO validation for streaming WriteDisposition


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

Branch: refs/heads/python-sdk
Commit: 0a7246d268969cb1b7f46149e38361802c95e70a
Parents: cfa217a
Author: Scott Wegner <sw...@google.com>
Authored: Mon Jun 13 11:05:52 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0a7246d2/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 38009bf..1c666ed 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -201,8 +201,8 @@ import javax.annotation.Nullable;
  *
  * <p>See {@link BigQueryIO.Write} for details on how to specify if a write should
  * append to an existing table, replace the table, or verify that the table is
- * empty. Note that the dataset being written to must already exist. Write
- * dispositions are not supported in streaming mode.
+ * empty. Note that the dataset being written to must already exist. Unbounded PCollections can only
+ * be written using {@link WriteDisposition#WRITE_EMPTY} or {@link WriteDisposition#WRITE_APPEND}.
  *
  * <h3>Sharding BigQuery output tables</h3>
  * <p>A common use case is to dynamically generate BigQuery table names based on
@@ -1674,7 +1674,7 @@ public class BigQueryIO {
           }
         }
 
-        if (options.isStreaming() || tableRefFunction != null) {
+        if (input.isBounded() == PCollection.IsBounded.UNBOUNDED || tableRefFunction != null) {
           // We will use BigQuery's streaming write API -- validate supported dispositions.
           checkArgument(
               createDisposition != CreateDisposition.CREATE_NEVER,


[21/50] [abbrv] incubator-beam git commit: Replace GcsPath by IOChannelFactory in WordCount.

Posted by da...@apache.org.
Replace GcsPath by IOChannelFactory in WordCount.


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

Branch: refs/heads/python-sdk
Commit: 605833071a7034aa3b723776a0f9e24330f64c8b
Parents: e90a1b9
Author: Pei He <pe...@google.com>
Authored: Mon Jun 13 16:58:01 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/examples/WordCount.java | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60583307/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
index f5611d3..03ed5d7 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
@@ -32,10 +32,15 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.util.IOChannelFactory;
+import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
+import com.google.common.base.Strings;
+
+import java.io.IOException;
+
 /**
  * An example that counts words in Shakespeare and includes Beam best practices.
  *
@@ -176,9 +181,15 @@ public class WordCount {
     public static class OutputFactory implements DefaultValueFactory<String> {
       @Override
       public String create(PipelineOptions options) {
-        if (options.getTempLocation() != null) {
-          return GcsPath.fromUri(options.getTempLocation())
-              .resolve("counts.txt").toString();
+        String tempLocation = options.getTempLocation();
+        if (!Strings.isNullOrEmpty(tempLocation)) {
+          try {
+            IOChannelFactory factory = IOChannelUtils.getFactory(tempLocation);
+            return factory.resolve(tempLocation, "counts.txt");
+          } catch (IOException e) {
+            throw new RuntimeException(
+                String.format("Failed to resolve temp location: %s", tempLocation));
+          }
         } else {
           throw new IllegalArgumentException("Must specify --output or --tempLocation");
         }


[38/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
new file mode 100644
index 0000000..e094d0d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -0,0 +1,1417 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.startsWith;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsList;
+import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMap;
+import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMultimap;
+import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsSingleton;
+import org.apache.beam.runners.dataflow.DataflowRunner.TransformedMap;
+import org.apache.beam.runners.dataflow.internal.IsmFormat;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.AvroSource;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.PTransform;
+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.CoderUtils;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+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.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.hamcrest.Description;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Tests for the {@link DataflowRunner}.
+ */
+@RunWith(JUnit4.class)
+public class DataflowRunnerTest {
+
+  private static final String PROJECT_ID = "some-project";
+
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  // Asserts that the given Job has all expected fields set.
+  private static void assertValidJob(Job job) {
+    assertNull(job.getId());
+    assertNull(job.getCurrentState());
+    assertTrue(Pattern.matches("[a-z]([-a-z0-9]*[a-z0-9])?", job.getName()));
+  }
+
+  private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) {
+    options.setStableUniqueNames(CheckEnabled.ERROR);
+    options.setRunner(DataflowRunner.class);
+    Pipeline p = Pipeline.create(options);
+
+    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
+        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+
+    return p;
+  }
+
+  private static Dataflow buildMockDataflow(
+      final ArgumentCaptor<Job> jobCaptor) throws IOException {
+    Dataflow mockDataflowClient = mock(Dataflow.class);
+    Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
+    Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
+    Dataflow.Projects.Jobs.Create mockRequest =
+        mock(Dataflow.Projects.Jobs.Create.class);
+    Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
+
+    when(mockDataflowClient.projects()).thenReturn(mockProjects);
+    when(mockProjects.jobs()).thenReturn(mockJobs);
+    when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
+        .thenReturn(mockRequest);
+    when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
+    when(mockList.setPageToken(anyString())).thenReturn(mockList);
+    when(mockList.execute())
+        .thenReturn(
+            new ListJobsResponse()
+                .setJobs(
+                    Arrays.asList(
+                        new Job()
+                            .setName("oldjobname")
+                            .setId("oldJobId")
+                            .setCurrentState("JOB_STATE_RUNNING"))));
+
+    Job resultJob = new Job();
+    resultJob.setId("newid");
+    when(mockRequest.execute()).thenReturn(resultJob);
+    return mockDataflowClient;
+  }
+
+  private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
+    GcsUtil mockGcsUtil = mock(GcsUtil.class);
+    when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+        .then(new Answer<SeekableByteChannel>() {
+              @Override
+              public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+                return FileChannel.open(
+                    Files.createTempFile("channel-", ".tmp"),
+                    StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+              }
+            });
+
+    when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
+    when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
+      @Override
+      public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+        return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+      }
+    });
+    when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
+    return mockGcsUtil;
+  }
+
+  private DataflowPipelineOptions buildPipelineOptions() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+    return buildPipelineOptions(jobCaptor);
+  }
+
+  private DataflowPipelineOptions buildPipelineOptions(
+      ArgumentCaptor<Job> jobCaptor) throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject(PROJECT_ID);
+    options.setTempLocation("gs://somebucket/some/path");
+    // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
+    options.setFilesToStage(new LinkedList<String>());
+    options.setDataflowClient(buildMockDataflow(jobCaptor));
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+    return options;
+  }
+
+  @Test
+  public void testFromOptionsWithUppercaseConvertsToLowercase() throws Exception {
+    String mixedCase = "ThisJobNameHasMixedCase";
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setJobName(mixedCase);
+
+    DataflowRunner runner = DataflowRunner.fromOptions(options);
+    assertThat(options.getJobName(), equalTo(mixedCase.toLowerCase()));
+  }
+
+  @Test
+  public void testRun() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    Pipeline p = buildDataflowPipeline(options);
+    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+    assertEquals("newid", job.getJobId());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testRunReturnDifferentRequestId() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    Dataflow mockDataflowClient = options.getDataflowClient();
+    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+        .thenReturn(mockRequest);
+    Job resultJob = new Job();
+    resultJob.setId("newid");
+    // Return a different request id.
+    resultJob.setClientRequestId("different_request_id");
+    when(mockRequest.execute()).thenReturn(resultJob);
+
+    Pipeline p = buildDataflowPipeline(options);
+    try {
+      p.run();
+      fail("Expected DataflowJobAlreadyExistsException");
+    } catch (DataflowJobAlreadyExistsException expected) {
+      assertThat(expected.getMessage(),
+          containsString("If you want to submit a second job, try again by setting a "
+            + "different name using --jobName."));
+      assertEquals(expected.getJob().getJobId(), resultJob.getId());
+    }
+  }
+
+  @Test
+  public void testUpdate() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setUpdate(true);
+    options.setJobName("oldJobName");
+    Pipeline p = buildDataflowPipeline(options);
+    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+    assertEquals("newid", job.getJobId());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testUpdateNonExistentPipeline() throws IOException {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Could not find running job named badjobname");
+
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setUpdate(true);
+    options.setJobName("badJobName");
+    Pipeline p = buildDataflowPipeline(options);
+    p.run();
+  }
+
+  @Test
+  public void testUpdateAlreadyUpdatedPipeline() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setUpdate(true);
+    options.setJobName("oldJobName");
+    Dataflow mockDataflowClient = options.getDataflowClient();
+    Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+    when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+        .thenReturn(mockRequest);
+    final Job resultJob = new Job();
+    resultJob.setId("newid");
+    // Return a different request id.
+    resultJob.setClientRequestId("different_request_id");
+    when(mockRequest.execute()).thenReturn(resultJob);
+
+    Pipeline p = buildDataflowPipeline(options);
+
+    thrown.expect(DataflowJobAlreadyUpdatedException.class);
+    thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("Expected job ID: " + resultJob.getId());
+      }
+
+      @Override
+      protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
+        return resultJob.getId().equals(item.getJob().getJobId());
+      }
+    });
+    thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
+        + "into job id: newid and cannot be updated again.");
+    p.run();
+  }
+
+  @Test
+  public void testRunWithFiles() throws IOException {
+    // Test that the function DataflowRunner.stageFiles works as
+    // expected.
+    GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
+    final String gcsStaging = "gs://somebucket/some/path";
+    final String gcsTemp = "gs://somebucket/some/temp/path";
+    final String cloudDataflowDataset = "somedataset";
+
+    // Create some temporary files.
+    File temp1 = File.createTempFile("DataflowRunnerTest", "txt");
+    temp1.deleteOnExit();
+    File temp2 = File.createTempFile("DataflowRunnerTest2", "txt");
+    temp2.deleteOnExit();
+
+    String overridePackageName = "alias.txt";
+
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setFilesToStage(ImmutableList.of(
+        temp1.getAbsolutePath(),
+        overridePackageName + "=" + temp2.getAbsolutePath()));
+    options.setStagingLocation(gcsStaging);
+    options.setTempLocation(gcsTemp);
+    options.setTempDatasetId(cloudDataflowDataset);
+    options.setProject(PROJECT_ID);
+    options.setJobName("job");
+    options.setDataflowClient(buildMockDataflow(jobCaptor));
+    options.setGcsUtil(mockGcsUtil);
+    options.setGcpCredential(new TestCredential());
+
+    Pipeline p = buildDataflowPipeline(options);
+
+    DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+    assertEquals("newid", job.getJobId());
+
+    Job workflowJob = jobCaptor.getValue();
+    assertValidJob(workflowJob);
+
+    assertEquals(
+        2,
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
+    DataflowPackage workflowPackage1 =
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
+    assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
+    DataflowPackage workflowPackage2 =
+        workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
+    assertEquals(overridePackageName, workflowPackage2.getName());
+
+    assertEquals(
+        "storage.googleapis.com/somebucket/some/temp/path",
+        workflowJob.getEnvironment().getTempStoragePrefix());
+    assertEquals(
+        cloudDataflowDataset,
+        workflowJob.getEnvironment().getDataset());
+    assertEquals(
+        ReleaseInfo.getReleaseInfo().getName(),
+        workflowJob.getEnvironment().getUserAgent().get("name"));
+    assertEquals(
+        ReleaseInfo.getReleaseInfo().getVersion(),
+        workflowJob.getEnvironment().getUserAgent().get("version"));
+  }
+
+  @Test
+  public void runWithDefaultFilesToStage() throws Exception {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setFilesToStage(null);
+    DataflowRunner.fromOptions(options);
+    assertTrue(!options.getFilesToStage().isEmpty());
+  }
+
+  @Test
+  public void detectClassPathResourceWithFileResources() throws Exception {
+    File file = tmpFolder.newFile("file");
+    File file2 = tmpFolder.newFile("file2");
+    URLClassLoader classLoader = new URLClassLoader(new URL[]{
+        file.toURI().toURL(),
+        file2.toURI().toURL()
+    });
+
+    assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
+        DataflowRunner.detectClassPathResourcesToStage(classLoader));
+  }
+
+  @Test
+  public void detectClassPathResourcesWithUnsupportedClassLoader() {
+    ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
+
+    DataflowRunner.detectClassPathResourcesToStage(mockClassLoader);
+  }
+
+  @Test
+  public void detectClassPathResourceWithNonFileResources() throws Exception {
+    String url = "http://www.google.com/all-the-secrets.jar";
+    URLClassLoader classLoader = new URLClassLoader(new URL[]{
+        new URL(url)
+    });
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Unable to convert url (" + url + ") to file.");
+
+    DataflowRunner.detectClassPathResourcesToStage(classLoader);
+  }
+
+  @Test
+  public void testGcsStagingLocationInitialization() throws Exception {
+    // Test that the staging location is initialized correctly.
+    String gcsTemp = "gs://somebucket/some/temp/path";
+
+    // Set temp location (required), and check that staging location is set.
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setTempLocation(gcsTemp);
+    options.setProject(PROJECT_ID);
+    options.setGcpCredential(new TestCredential());
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setRunner(DataflowRunner.class);
+
+    DataflowRunner.fromOptions(options);
+
+    assertNotNull(options.getStagingLocation());
+  }
+
+  @Test
+  public void testNonGcsFilePathInReadFailure() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+    p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
+
+    thrown.expectCause(Matchers.allOf(
+        instanceOf(IllegalArgumentException.class),
+        ThrowableMessageMatcher.hasMessage(
+            containsString("expected a valid 'gs://' path but was given"))));
+    p.run();
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNonGcsFilePathInWriteFailure() throws IOException {
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+    pc.apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file"));
+  }
+
+  @Test
+  public void testMultiSlashGcsFileReadPath() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+    p.apply(TextIO.Read.named("ReadInvalidGcsFile")
+        .from("gs://bucket/tmp//file"));
+
+    thrown.expectCause(Matchers.allOf(
+        instanceOf(IllegalArgumentException.class),
+        ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
+    p.run();
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testMultiSlashGcsFileWritePath() throws IOException {
+    Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("consecutive slashes");
+    pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
+  }
+
+  @Test
+  public void testInvalidTempLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setTempLocation("file://temp/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+    DataflowRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testInvalidStagingLocation() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    options.setStagingLocation("file://my/staging/location");
+    try {
+      DataflowRunner.fromOptions(options);
+      fail("fromOptions should have failed");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+    }
+    options.setStagingLocation("my/staging/location");
+    try {
+      DataflowRunner.fromOptions(options);
+      fail("fromOptions should have failed");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+    }
+  }
+
+  @Test
+  public void testNonExistentTempLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setGcsUtil(mockGcsUtil);
+    options.setTempLocation("gs://non-existent-bucket/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString(
+        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+    DataflowRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNonExistentStagingLocation() throws IOException {
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    options.setGcsUtil(mockGcsUtil);
+    options.setStagingLocation("gs://non-existent-bucket/location");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(containsString(
+        "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+    DataflowRunner.fromOptions(options);
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testNoProjectFails() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+
+    options.setRunner(DataflowRunner.class);
+    // Explicitly set to null to prevent the default instance factory from reading credentials
+    // from a user's environment, causing this test to fail.
+    options.setProject(null);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project id");
+    thrown.expectMessage("when running a Dataflow in the cloud");
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectId() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("foo-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectPrefix() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("google.com:some-project-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+    options.setGcpCredential(new TestCredential());
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectNumber() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project ID");
+    thrown.expectMessage("project number");
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testProjectDescription() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("some project");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Project ID");
+    thrown.expectMessage("project description");
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("foo-12345");
+
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Number of worker harness threads");
+    thrown.expectMessage("Please make sure the value is non-negative.");
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testNoStagingLocationAndNoTempLocationFails() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setProject("foo-project");
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setGcpCredential(new TestCredential());
+    options.setProject("foo-project");
+    options.setStagingLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setGcpCredential(new TestCredential());
+    options.setProject("foo-project");
+    options.setTempLocation("gs://spam/ham/eggs");
+    options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+    DataflowRunner.fromOptions(options);
+  }
+
+  @Test
+  public void testInvalidJobName() throws IOException {
+    List<String> invalidNames = Arrays.asList(
+        "invalid_name",
+        "0invalid",
+        "invalid-");
+    List<String> expectedReason = Arrays.asList(
+        "JobName invalid",
+        "JobName invalid",
+        "JobName invalid");
+
+    for (int i = 0; i < invalidNames.size(); ++i) {
+      DataflowPipelineOptions options = buildPipelineOptions();
+      options.setJobName(invalidNames.get(i));
+
+      try {
+        DataflowRunner.fromOptions(options);
+        fail("Expected IllegalArgumentException for jobName "
+            + options.getJobName());
+      } catch (IllegalArgumentException e) {
+        assertThat(e.getMessage(),
+            containsString(expectedReason.get(i)));
+      }
+    }
+  }
+
+  @Test
+  public void testValidJobName() throws IOException {
+    List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
+        "this-one-is-fairly-long-01234567890123456789");
+
+    for (String name : names) {
+      DataflowPipelineOptions options = buildPipelineOptions();
+      options.setJobName(name);
+
+      DataflowRunner runner = DataflowRunner
+          .fromOptions(options);
+      assertNotNull(runner);
+    }
+  }
+
+  /**
+   * A fake PTransform for testing.
+   */
+  public static class TestTransform
+      extends PTransform<PCollection<Integer>, PCollection<Integer>> {
+    public boolean translated = false;
+
+    @Override
+    public PCollection<Integer> apply(PCollection<Integer> input) {
+      return PCollection.<Integer>createPrimitiveOutputInternal(
+          input.getPipeline(),
+          WindowingStrategy.globalDefault(),
+          input.isBounded());
+    }
+
+    @Override
+    protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
+      return input.getCoder();
+    }
+  }
+
+  @Test
+  public void testTransformTranslatorMissing() throws IOException {
+    // Test that we throw if we don't provide a translation.
+    ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+    DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+    Pipeline p = Pipeline.create(options);
+
+    p.apply(Create.of(Arrays.asList(1, 2, 3)))
+     .apply(new TestTransform());
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(Matchers.containsString("no translator registered"));
+    DataflowPipelineTranslator.fromOptions(options)
+        .translate(
+            p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+    assertValidJob(jobCaptor.getValue());
+  }
+
+  @Test
+  public void testTransformTranslator() throws IOException {
+    // Test that we can provide a custom translation
+    DataflowPipelineOptions options = buildPipelineOptions();
+    Pipeline p = Pipeline.create(options);
+    TestTransform transform = new TestTransform();
+
+    p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
+        .apply(transform);
+
+    DataflowPipelineTranslator translator = DataflowRunner
+        .fromOptions(options).getTranslator();
+
+    DataflowPipelineTranslator.registerTransformTranslator(
+        TestTransform.class,
+        new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
+          @SuppressWarnings("unchecked")
+          @Override
+          public void translate(
+              TestTransform transform,
+              DataflowPipelineTranslator.TranslationContext context) {
+            transform.translated = true;
+
+            // Note: This is about the minimum needed to fake out a
+            // translation. This obviously isn't a real translation.
+            context.addStep(transform, "TestTranslate");
+            context.addOutput("output", context.getOutput(transform));
+          }
+        });
+
+    translator.translate(
+        p, (DataflowRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+    assertTrue(transform.translated);
+  }
+
+  /** Records all the composite transforms visited within the Pipeline. */
+  private static class CompositeTransformRecorder extends PipelineVisitor.Defaults {
+    private List<PTransform<?, ?>> transforms = new ArrayList<>();
+
+    @Override
+    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+      if (node.getTransform() != null) {
+        transforms.add(node.getTransform());
+      }
+      return CompositeBehavior.ENTER_TRANSFORM;
+    }
+
+    public List<PTransform<?, ?>> getCompositeTransforms() {
+      return transforms;
+    }
+  }
+
+  @Test
+  public void testApplyIsScopedToExactClass() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    Pipeline p = Pipeline.create(options);
+
+    Create.TimestampedValues<String> transform =
+        Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
+    p.apply(transform);
+
+    CompositeTransformRecorder recorder = new CompositeTransformRecorder();
+    p.traverseTopologically(recorder);
+
+    // The recorder will also have seen a Create.Values composite as well, but we can't obtain that
+    // transform.
+    assertThat(
+        "Expected to have seen CreateTimestamped composite transform.",
+        recorder.getCompositeTransforms(),
+        hasItem(transform));
+    assertThat(
+        "Expected to have two composites, CreateTimestamped and Create.Values",
+        recorder.getCompositeTransforms(),
+        hasItem(Matchers.<PTransform<?, ?>>isA((Class) Create.Values.class)));
+  }
+
+  @Test
+  public void testToString() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setRunner(DataflowRunner.class);
+    assertEquals(
+        "DataflowRunner#testjobname",
+        DataflowRunner.fromOptions(options).toString());
+  }
+
+  private static PipelineOptions makeOptions(boolean streaming) {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setRunner(DataflowRunner.class);
+    options.setStreaming(streaming);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    return options;
+  }
+
+  private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
+      throws Exception {
+    String mode = streaming ? "streaming" : "batch";
+    thrown.expect(UnsupportedOperationException.class);
+    thrown.expectMessage(
+        "The DataflowRunner in " + mode + " mode does not support " + name);
+
+    Pipeline p = Pipeline.create(makeOptions(streaming));
+    p.apply(source);
+    p.run();
+  }
+
+  @Test
+  public void testBoundedSourceUnsupportedInStreaming() throws Exception {
+    testUnsupportedSource(
+        AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true);
+  }
+
+  @Test
+  public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception {
+    testUnsupportedSource(
+        BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true);
+  }
+
+  @Test
+  public void testAvroIOSourceUnsupportedInStreaming() throws Exception {
+    testUnsupportedSource(
+        AvroIO.Read.from("foo"), "AvroIO.Read", true);
+  }
+
+  @Test
+  public void testTextIOSourceUnsupportedInStreaming() throws Exception {
+    testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true);
+  }
+
+  @Test
+  public void testReadBoundedSourceUnsupportedInStreaming() throws Exception {
+    testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true);
+  }
+
+  @Test
+  public void testReadUnboundedUnsupportedInBatch() throws Exception {
+    testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false);
+  }
+
+  private void testUnsupportedSink(
+      PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
+          throws Exception {
+    thrown.expect(UnsupportedOperationException.class);
+    thrown.expectMessage(
+        "The DataflowRunner in streaming mode does not support " + name);
+
+    Pipeline p = Pipeline.create(makeOptions(streaming));
+    p.apply(Create.of("foo")).apply(sink);
+    p.run();
+  }
+
+  @Test
+  public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
+    testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
+  }
+
+  @Test
+  public void testTextIOSinkUnsupportedInStreaming() throws Exception {
+    testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
+  }
+
+  @Test
+  public void testBatchViewAsSingletonToIsmRecord() throws Exception {
+    DoFnTester<KV<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>,
+               IsmRecord<WindowedValue<String>>> doFnTester =
+               DoFnTester.of(
+                   new BatchViewAsSingleton.IsmRecordForSingularValuePerWindowDoFn
+                   <String, GlobalWindow>(GlobalWindow.Coder.INSTANCE));
+
+    assertThat(
+        doFnTester.processBundle(
+            ImmutableList.of(KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(
+                0, ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")))))),
+        contains(IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE), valueInGlobalWindow("a"))));
+  }
+
+  @Test
+  public void testBatchViewAsSingletonToIsmRecordWithMultipleValuesThrowsException()
+      throws Exception {
+    DoFnTester<KV<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>,
+    IsmRecord<WindowedValue<String>>> doFnTester =
+    DoFnTester.of(
+        new BatchViewAsSingleton.IsmRecordForSingularValuePerWindowDoFn
+        <String, GlobalWindow>(GlobalWindow.Coder.INSTANCE));
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("found for singleton within window");
+    doFnTester.processBundle(ImmutableList.of(
+        KV.<Integer, Iterable<KV<GlobalWindow, WindowedValue<String>>>>of(0,
+            ImmutableList.of(KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("a")),
+                KV.of(GlobalWindow.INSTANCE, valueInGlobalWindow("b"))))));
+  }
+
+  @Test
+  public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
+    DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
+        DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBundle(ImmutableList.of("a", "b", "c")), contains(
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
+        IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
+  }
+
+  @Test
+  public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(
+            new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
+                IntervalWindow.getCoder()));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(
+                    windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+                KV.of(
+                    windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
+                )),
+            KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(
+                    windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
+                )));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBundle(inputElements), contains(
+        IsmRecord.of(ImmutableList.of(windowA, 0L),
+            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowA, 1L),
+            WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowA, 2L),
+            WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowB, 0L),
+            WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowB, 1L),
+            WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(ImmutableList.of(windowC, 0L),
+            WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
+  }
+
+  @Test
+  public void testToIsmRecordForMapLikeDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+            outputForSizeTag,
+            outputForEntrySetTag,
+            windowCoder,
+            keyCoder,
+            ismCoder,
+            false /* unique keys */));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                // same window same key as to previous
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+                // same window different key as to previous
+                KV.of(KV.of(2L, windowA),
+                    WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                // different window same key as to previous
+                KV.of(KV.of(2L, windowB),
+                    WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+                // different window and different key as to previous
+                KV.of(KV.of(3L, windowB),
+                    WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                // different shard
+                KV.of(KV.of(4L, windowC),
+                    WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBundle(inputElements), contains(
+        IsmRecord.of(
+            ImmutableList.of(1L, windowA, 0L),
+            WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(1L, windowA, 1L),
+            WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(2L, windowA, 0L),
+            WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(2L, windowB, 0L),
+            WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(3L, windowB, 0L),
+            WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+        IsmRecord.of(
+            ImmutableList.of(4L, windowC, 0L),
+            WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
+
+    // Verify the number of unique keys per window.
+    assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+            KV.of(windowC, 1L))
+        ));
+
+    // Verify the output for the unique keys.
+    assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 1L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+            KV.of(windowA, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 2L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+            KV.of(windowB, 3L)),
+        KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+            KV.of(windowC, 4L))
+        ));
+  }
+
+  @Test
+  public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+            outputForSizeTag,
+            outputForEntrySetTag,
+            windowCoder,
+            keyCoder,
+            ismCoder,
+            true /* unique keys */));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+
+    Iterable<KV<Integer,
+                Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+                // same window same key as to previous
+                KV.of(KV.of(1L, windowA),
+                    WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("Unique keys are expected but found key");
+    doFnTester.processBundle(inputElements);
+  }
+
+  @Test
+  public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+        new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
+            windowCoder));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowA, 2L),
+                    KV.of(windowA, 3L),
+                    KV.of(windowB, 7L))),
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowC, 9L))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBundle(inputElements), contains(
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
+        ));
+  }
+
+  @Test
+  public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+    TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+    Coder<Long> keyCoder = VarLongCoder.of();
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+        1,
+        2,
+        ImmutableList.<Coder<?>>of(
+            MetadataKeyCoder.of(keyCoder),
+            IntervalWindow.getCoder(),
+            BigEndianLongCoder.of()),
+        FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+               IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+        new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
+            keyCoder, windowCoder));
+    doFnTester.setSideOutputTags(TupleTagList.of(
+        ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowA, 2L),
+                    // same window as previous
+                    KV.of(windowA, 3L),
+                    // different window as previous
+                    KV.of(windowB, 3L))),
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+                (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+                    KV.of(windowC, 3L))));
+
+    // The order of the output elements is important relative to processing order
+    assertThat(doFnTester.processBundle(inputElements), contains(
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+        IsmRecord.<WindowedValue<Long>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
+        ));
+  }
+
+  @Test
+  public void testToMapDoFn() throws Exception {
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+                  IsmRecord<WindowedValue<TransformedMap<Long,
+                                                         WindowedValue<Long>,
+                                                         Long>>>> doFnTester =
+        DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowC, WindowedValue.of(
+                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    List<IsmRecord<WindowedValue<TransformedMap<Long,
+                                                WindowedValue<Long>,
+                                                Long>>>> output =
+                                                doFnTester.processBundle(inputElements);
+    assertEquals(3, output.size());
+    Map<Long, Long> outputMap;
+
+    outputMap = output.get(0).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
+
+    outputMap = output.get(1).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
+
+    outputMap = output.get(2).getValue().getValue();
+    assertEquals(1, outputMap.size());
+    assertEquals(ImmutableMap.of(4L, 41L), outputMap);
+  }
+
+  @Test
+  public void testToMultimapDoFn() throws Exception {
+    Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+    DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+                  IsmRecord<WindowedValue<TransformedMap<Long,
+                                                         Iterable<WindowedValue<Long>>,
+                                                         Iterable<Long>>>>> doFnTester =
+        DoFnTester.of(
+            new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+    IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+    IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+    IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+    Iterable<KV<Integer,
+             Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+        ImmutableList.of(
+            KV.of(1,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowA, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+                    KV.of(windowB, WindowedValue.of(
+                        KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+            KV.of(2,
+                (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+                    KV.of(windowC, WindowedValue.of(
+                        KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+    // The order of the output elements is important relative to processing order
+    List<IsmRecord<WindowedValue<TransformedMap<Long,
+                                                Iterable<WindowedValue<Long>>,
+                                                Iterable<Long>>>>> output =
+                                                doFnTester.processBundle(inputElements);
+    assertEquals(3, output.size());
+    Map<Long, Iterable<Long>> outputMap;
+
+    outputMap = output.get(0).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
+    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+
+    outputMap = output.get(1).getValue().getValue();
+    assertEquals(2, outputMap.size());
+    assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+    assertThat(outputMap.get(3L), containsInAnyOrder(31L));
+
+    outputMap = output.get(2).getValue().getValue();
+    assertEquals(1, outputMap.size());
+    assertThat(outputMap.get(4L), containsInAnyOrder(41L));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowAvroIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowAvroIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowAvroIOTest.java
index 614affb..006daa9 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowAvroIOTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowAvroIOTest.java
@@ -22,7 +22,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
 import org.apache.beam.sdk.io.AvroIO;
 import org.apache.beam.sdk.transforms.display.DisplayData;
@@ -36,7 +36,7 @@ import org.junit.runners.JUnit4;
 import java.util.Set;
 
 /**
- * {@link DataflowPipelineRunner} specific tests for {@link AvroIO} transforms.
+ * {@link DataflowRunner} specific tests for {@link AvroIO} transforms.
  */
 @RunWith(JUnit4.class)
 public class DataflowAvroIOTest {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowPubsubIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowPubsubIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowPubsubIOTest.java
index 3df9cdb..27bc2d9 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowPubsubIOTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowPubsubIOTest.java
@@ -22,7 +22,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
 import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.transforms.display.DisplayData;
@@ -35,7 +35,7 @@ import org.junit.runners.JUnit4;
 import java.util.Set;
 
 /**
- * {@link DataflowPipelineRunner} specific tests for {@link PubsubIO} transforms.
+ * {@link DataflowRunner} specific tests for {@link PubsubIO} transforms.
  */
 @RunWith(JUnit4.class)
 public class DataflowPubsubIOTest {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
index 0340435..727ffdc 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
@@ -23,7 +23,7 @@ import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.startsWith;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
 import org.apache.beam.sdk.io.TextIO;
@@ -39,7 +39,7 @@ import org.junit.runners.JUnit4;
 import java.util.Set;
 
 /**
- * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write transforms.
+ * {@link DataflowRunner} specific tests for TextIO Read and Write transforms.
  */
 @RunWith(JUnit4.class)
 public class DataflowTextIOTest {


[25/50] [abbrv] incubator-beam git commit: Roll-forwards: Base PAssert on GBK instead of side inputs

Posted by da...@apache.org.
Roll-forwards: Base PAssert on GBK instead of side inputs

Previously PAssert - hence all RunnableOnService/NeedsRunner
tests - required side input support. This created a very steep
on ramp for new runners.

GroupByKey is a bit more fundamental and most backends will be
able to group by key in the global window very quickly. So switching
the primitive used to gather all the contents of a PCollection for
assertions should make it a bit easier to get early feedback during
runner development.


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

Branch: refs/heads/python-sdk
Commit: e90a1b9d74cbc06d7818bae8dfe2af81acd73222
Parents: 8278e5f
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Jun 8 15:07:52 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../testing/TestDataflowPipelineRunner.java     |   3 +-
 .../org/apache/beam/sdk/testing/PAssert.java    | 857 +++++++++++--------
 .../apache/beam/sdk/testing/PAssertTest.java    |  27 -
 3 files changed, 491 insertions(+), 396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e90a1b9d/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
index 3e8d903..c940e9a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
@@ -166,7 +166,8 @@ public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJ
   public <OutputT extends POutput, InputT extends PInput> OutputT apply(
       PTransform<InputT, OutputT> transform, InputT input) {
     if (transform instanceof PAssert.OneSideInputAssert
-        || transform instanceof PAssert.TwoSideInputAssert) {
+        || transform instanceof PAssert.GroupThenAssert
+        || transform instanceof PAssert.GroupThenAssertForSingleton) {
       expectedNumberOfAssertions += 1;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e90a1b9d/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
index c2cd598..a29a56d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.testing;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.not;
@@ -34,46 +36,46 @@ import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Aggregator;
 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.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Never;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 
-import com.google.common.base.Optional;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
 /**
- * An assertion on the contents of a {@link PCollection}
- * incorporated into the pipeline.  Such an assertion
- * can be checked no matter what kind of {@link PipelineRunner} is
- * used.
+ * An assertion on the contents of a {@link PCollection} incorporated into the pipeline. Such an
+ * assertion can be checked no matter what kind of {@link PipelineRunner} is used.
  *
- * <p>Note that the {@code PAssert} call must precede the call
- * to {@link Pipeline#run}.
+ * <p>Note that the {@code PAssert} call must precede the call to {@link Pipeline#run}.
  *
- * <p>Examples of use:
- * <pre>{@code
+ * <p>Examples of use: <pre>{@code
  * Pipeline p = TestPipeline.create();
  * ...
  * PCollection<String> output =
@@ -103,34 +105,92 @@ public class PAssert {
 
   private static int assertCount = 0;
 
+  private static String nextAssertionName() {
+    return "PAssert$" + (assertCount++);
+  }
+
   // Do not instantiate.
   private PAssert() {}
 
   /**
-   * Constructs an {@link IterableAssert} for the elements of the provided
-   * {@link PCollection}.
+   * Builder interface for assertions applicable to iterables and PCollection contents.
+   */
+  public interface IterableAssert<T> {
+
+    /**
+     * Asserts that the iterable in question contains the provided elements.
+     *
+     * @return the same {@link IterableAssert} builder for further assertions
+     */
+    IterableAssert<T> containsInAnyOrder(T... expectedElements);
+
+    /**
+     * Asserts that the iterable in question contains the provided elements.
+     *
+     * @return the same {@link IterableAssert} builder for further assertions
+     */
+    IterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements);
+
+    /**
+     * Asserts that the iterable in question is empty.
+     *
+     * @return the same {@link IterableAssert} builder for further assertions
+     */
+    IterableAssert<T> empty();
+
+    /**
+     * Applies the provided checking function (presumably containing assertions) to the
+     * iterable in question.
+     *
+     * @return the same {@link IterableAssert} builder for further assertions
+     */
+    IterableAssert<T> satisfies(SerializableFunction<Iterable<T>, Void> checkerFn);
+  }
+
+  /**
+   * Builder interface for assertions applicable to a single value.
+   */
+  public interface SingletonAssert<T> {
+    /**
+     * Asserts that the value in question is equal to the provided value, according to
+     * {@link Object#equals}.
+     *
+     * @return the same {@link SingletonAssert} builder for further assertions
+     */
+    SingletonAssert<T> isEqualTo(T expected);
+
+    /**
+     * Asserts that the value in question is not equal to the provided value, according
+     * to {@link Object#equals}.
+     *
+     * @return the same {@link SingletonAssert} builder for further assertions
+     */
+    SingletonAssert<T> notEqualTo(T notExpected);
+
+    /**
+     * Applies the provided checking function (presumably containing assertions) to the
+     * value in question.
+     *
+     * @return the same {@link SingletonAssert} builder for further assertions
+     */
+    SingletonAssert<T> satisfies(SerializableFunction<T, Void> checkerFn);
+  }
+
+  /**
+   * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}.
    */
   public static <T> IterableAssert<T> that(PCollection<T> actual) {
-    return new IterableAssert<>(
-        new CreateActual<T, Iterable<T>>(actual, View.<T>asIterable()),
-         actual.getPipeline())
-         .setCoder(actual.getCoder());
+    return new PCollectionContentsAssert<>(actual);
   }
 
   /**
-   * Constructs an {@link IterableAssert} for the value of the provided
-   * {@link PCollection} which must contain a single {@code Iterable<T>}
-   * value.
+   * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection} which
+   * must contain a single {@code Iterable<T>} value.
    */
-  public static <T> IterableAssert<T>
-      thatSingletonIterable(PCollection<? extends Iterable<T>> actual) {
+  public static <T> IterableAssert<T> thatSingletonIterable(
+      PCollection<? extends Iterable<T>> actual) {
 
-    List<? extends Coder<?>> maybeElementCoder = actual.getCoder().getCoderArguments();
-    Coder<T> tCoder;
     try {
-      @SuppressWarnings("unchecked")
-      Coder<T> tCoderTmp = (Coder<T>) Iterables.getOnlyElement(maybeElementCoder);
-      tCoder = tCoderTmp;
     } catch (NoSuchElementException | IllegalArgumentException exc) {
       throw new IllegalArgumentException(
           "PAssert.<T>thatSingletonIterable requires a PCollection<Iterable<T>>"
@@ -141,19 +201,7 @@ public class PAssert {
     @SuppressWarnings("unchecked") // Safe covariant cast
     PCollection<Iterable<T>> actualIterables = (PCollection<Iterable<T>>) actual;
 
-    return new IterableAssert<>(
-        new CreateActual<Iterable<T>, Iterable<T>>(
-            actualIterables, View.<Iterable<T>>asSingleton()),
-        actual.getPipeline())
-        .setCoder(tCoder);
-  }
-
-  /**
-   * Constructs an {@link IterableAssert} for the value of the provided
-   * {@code PCollectionView PCollectionView<Iterable<T>>}.
-   */
-  public static <T> IterableAssert<T> thatIterable(PCollectionView<Iterable<T>> actual) {
-    return new IterableAssert<>(new PreExisting<Iterable<T>>(actual), actual.getPipeline());
+    return new PCollectionSingletonIterableAssert<>(actualIterables);
   }
 
   /**
@@ -161,93 +209,96 @@ public class PAssert {
    * {@code PCollection PCollection<T>}, which must be a singleton.
    */
   public static <T> SingletonAssert<T> thatSingleton(PCollection<T> actual) {
-    return new SingletonAssert<>(
-        new CreateActual<T, T>(actual, View.<T>asSingleton()), actual.getPipeline())
-        .setCoder(actual.getCoder());
+    return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder());
   }
 
   /**
    * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}.
    *
-   * <p>Note that the actual value must be coded by a {@link KvCoder},
-   * not just any {@code Coder<K, V>}.
+   * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
+   * {@code Coder<K, V>}.
    */
-  public static <K, V> SingletonAssert<Map<K, Iterable<V>>>
-      thatMultimap(PCollection<KV<K, V>> actual) {
+  public static <K, V> SingletonAssert<Map<K, Iterable<V>>> thatMultimap(
+      PCollection<KV<K, V>> actual) {
     @SuppressWarnings("unchecked")
     KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
-
-    return new SingletonAssert<>(
-        new CreateActual<>(actual, View.<K, V>asMultimap()), actual.getPipeline())
-        .setCoder(MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())));
+    return new PCollectionViewAssert<>(
+        actual,
+        View.<K, V>asMultimap(),
+        MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())));
   }
 
   /**
-   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection},
-   * which must have at most one value per key.
+   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}, which
+   * must have at most one value per key.
    *
-   * <p>Note that the actual value must be coded by a {@link KvCoder},
-   * not just any {@code Coder<K, V>}.
+   * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
+   * {@code Coder<K, V>}.
    */
   public static <K, V> SingletonAssert<Map<K, V>> thatMap(PCollection<KV<K, V>> actual) {
     @SuppressWarnings("unchecked")
     KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
-
-    return new SingletonAssert<>(
-        new CreateActual<>(actual, View.<K, V>asMap()), actual.getPipeline())
-        .setCoder(MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()));
+    return new PCollectionViewAssert<>(
+        actual, View.<K, V>asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()));
   }
 
   ////////////////////////////////////////////////////////////
 
   /**
-   * An assertion about the contents of a {@link PCollectionView} yielding an {@code Iterable<T>}.
+   * An {@link IterableAssert} about the contents of a {@link PCollection}. This does not require
+   * the runner to support side inputs.
    */
-  public static class IterableAssert<T> implements Serializable {
-    private final Pipeline pipeline;
-    private final PTransform<PBegin, PCollectionView<Iterable<T>>> createActual;
-    private Optional<Coder<T>> coder;
+  private static class PCollectionContentsAssert<T> implements IterableAssert<T> {
+    private final PCollection<T> actual;
 
-    protected IterableAssert(
-        PTransform<PBegin, PCollectionView<Iterable<T>>> createActual, Pipeline pipeline) {
-      this.createActual = createActual;
-      this.pipeline = pipeline;
-      this.coder = Optional.absent();
+    public PCollectionContentsAssert(PCollection<T> actual) {
+      this.actual = actual;
     }
 
     /**
-     * Sets the coder to use for elements of type {@code T}, as needed for internal purposes.
+     * Checks that the {@code Iterable} contains the expected elements, in any order.
      *
      * <p>Returns this {@code IterableAssert}.
      */
-    public IterableAssert<T> setCoder(Coder<T> coderOrNull) {
-      this.coder = Optional.fromNullable(coderOrNull);
-      return this;
+    @Override
+    @SafeVarargs
+    public final PCollectionContentsAssert<T> containsInAnyOrder(T... expectedElements) {
+      return containsInAnyOrder(Arrays.asList(expectedElements));
     }
 
     /**
-     * Gets the coder, which may yet be absent.
+     * Checks that the {@code Iterable} contains the expected elements, in any order.
+     *
+     * <p>Returns this {@code IterableAssert}.
      */
-    public Coder<T> getCoder() {
-      if (coder.isPresent()) {
-        return coder.get();
-      } else {
-        throw new IllegalStateException(
-            "Attempting to access the coder of an IterableAssert"
-                + " that has not been set yet.");
-      }
+    @Override
+    public PCollectionContentsAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
+      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
+    }
+
+    @Override
+    public PCollectionContentsAssert<T> empty() {
+      containsInAnyOrder(Collections.<T>emptyList());
+      return this;
+    }
+
+    @Override
+    public PCollectionContentsAssert<T> satisfies(
+        SerializableFunction<Iterable<T>, Void> checkerFn) {
+      actual.apply(nextAssertionName(), new GroupThenAssert<>(checkerFn));
+      return this;
     }
 
     /**
-     * Applies a {@link SerializableFunction} to check the elements of the {@code Iterable}.
+     * Checks that the {@code Iterable} contains elements that match the provided matchers, in any
+     * order.
      *
      * <p>Returns this {@code IterableAssert}.
      */
-    public IterableAssert<T> satisfies(SerializableFunction<Iterable<T>, Void> checkerFn) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new OneSideInputAssert<Iterable<T>>(createActual, checkerFn));
-      return this;
+    @SafeVarargs
+    final PCollectionContentsAssert<T> containsInAnyOrder(
+        SerializableMatcher<? super T>... elementMatchers) {
+      return satisfies(SerializableMatchers.<T>containsInAnyOrder(elementMatchers));
     }
 
     /**
@@ -255,17 +306,11 @@ public class PAssert {
      *
      * <p>Returns this {@code IterableAssert}.
      */
-    public IterableAssert<T> satisfies(
-        AssertRelation<Iterable<T>, Iterable<T>> relation,
-        final Iterable<T> expectedElements) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new TwoSideInputAssert<Iterable<T>, Iterable<T>>(
-              createActual,
-              new CreateExpected<T, Iterable<T>>(expectedElements, coder, View.<T>asIterable()),
-              relation));
-
-      return this;
+    private PCollectionContentsAssert<T> satisfies(
+        AssertRelation<Iterable<T>, Iterable<T>> relation, Iterable<T> expectedElements) {
+      return satisfies(
+          new CheckRelationAgainstExpected<Iterable<T>>(
+              relation, expectedElements, IterableCoder.of(actual.getCoder())));
     }
 
     /**
@@ -273,15 +318,14 @@ public class PAssert {
      *
      * <p>Returns this {@code IterableAssert}.
      */
-    IterableAssert<T> satisfies(final SerializableMatcher<Iterable<? extends T>> matcher) {
+    PCollectionContentsAssert<T> satisfies(
+        final SerializableMatcher<Iterable<? extends T>> matcher) {
       // Safe covariant cast. Could be elided by changing a lot of this file to use
       // more flexible bounds.
       @SuppressWarnings({"rawtypes", "unchecked"})
       SerializableFunction<Iterable<T>, Void> checkerFn =
-        (SerializableFunction) new MatcherCheckerFn<>(matcher);
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new OneSideInputAssert<Iterable<T>>(createActual, checkerFn));
+          (SerializableFunction) new MatcherCheckerFn<>(matcher);
+      actual.apply("PAssert$" + (assertCount++), new GroupThenAssert<>(checkerFn));
       return this;
     }
 
@@ -300,19 +344,9 @@ public class PAssert {
     }
 
     /**
-     * Checks that the {@code Iterable} is empty.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> empty() {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), Collections.<T>emptyList());
-    }
-
-    /**
      * @throws UnsupportedOperationException always
-     * @deprecated {@link Object#equals(Object)} is not supported on PAssert objects.
-     *    If you meant to test object equality, use a variant of {@link #containsInAnyOrder}
-     *    instead.
+     * @deprecated {@link Object#equals(Object)} is not supported on PAssert objects. If you meant
+     * to test object equality, use a variant of {@link #containsInAnyOrder} instead.
      */
     @Deprecated
     @Override
@@ -331,169 +365,129 @@ public class PAssert {
       throw new UnsupportedOperationException(
           String.format("%s.hashCode() is not supported.", IterableAssert.class.getSimpleName()));
     }
+  }
 
-    /**
-     * Checks that the {@code Iterable} contains the expected elements, in any
-     * order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public IterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
-    }
+  /**
+   * An {@link IterableAssert} for an iterable that is the sole element of a {@link PCollection}.
+   * This does not require the runner to support side inputs.
+   */
+  private static class PCollectionSingletonIterableAssert<T> implements IterableAssert<T> {
+    private final PCollection<Iterable<T>> actual;
+    private final Coder<T> elementCoder;
 
-    /**
-     * Checks that the {@code Iterable} contains the expected elements, in any
-     * order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    @SafeVarargs
-    public final IterableAssert<T> containsInAnyOrder(T... expectedElements) {
-      return satisfies(
-        new AssertContainsInAnyOrderRelation<T>(),
-        Arrays.asList(expectedElements));
+    public PCollectionSingletonIterableAssert(PCollection<Iterable<T>> actual) {
+      this.actual = actual;
+
+      @SuppressWarnings("unchecked")
+      Coder<T> typedCoder = (Coder<T>) actual.getCoder().getCoderArguments().get(0);
+      this.elementCoder = typedCoder;
     }
 
-    /**
-     * Checks that the {@code Iterable} contains elements that match the provided matchers,
-     * in any order.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
+    @Override
     @SafeVarargs
-    final IterableAssert<T> containsInAnyOrder(
-        SerializableMatcher<? super T>... elementMatchers) {
-      return satisfies(SerializableMatchers.<T>containsInAnyOrder(elementMatchers));
+    public final PCollectionSingletonIterableAssert<T> containsInAnyOrder(T... expectedElements) {
+      return containsInAnyOrder(Arrays.asList(expectedElements));
     }
-  }
 
-  /**
-   * An assertion about the single value of type {@code T}
-   * associated with a {@link PCollectionView}.
-   */
-  public static class SingletonAssert<T> implements Serializable {
-    private final Pipeline pipeline;
-    private final CreateActual<?, T> createActual;
-    private Optional<Coder<T>> coder;
-
-    protected SingletonAssert(
-        CreateActual<?, T> createActual, Pipeline pipeline) {
-      this.pipeline = pipeline;
-      this.createActual = createActual;
-      this.coder = Optional.absent();
+    @Override
+    public PCollectionSingletonIterableAssert<T> empty() {
+      return containsInAnyOrder(Collections.<T>emptyList());
     }
 
-    /**
-     * Always throws an {@link UnsupportedOperationException}: users are probably looking for
-     * {@link #isEqualTo}.
-     */
-    @Deprecated
     @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException(
-          String.format(
-              "tests for Java equality of the %s object, not the PCollection in question. "
-                  + "Call a test method, such as isEqualTo.",
-              getClass().getSimpleName()));
+    public PCollectionSingletonIterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
+      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
     }
 
-    /**
-     * @throws UnsupportedOperationException always.
-     * @deprecated {@link Object#hashCode()} is not supported on PAssert objects.
-     */
-    @Deprecated
     @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException(
-          String.format("%s.hashCode() is not supported.", SingletonAssert.class.getSimpleName()));
+    public PCollectionSingletonIterableAssert<T> satisfies(
+        SerializableFunction<Iterable<T>, Void> checkerFn) {
+      actual.apply("PAssert$" + (assertCount++), new GroupThenAssertForSingleton<>(checkerFn));
+      return this;
     }
 
-    /**
-     * Sets the coder to use for elements of type {@code T}, as needed
-     * for internal purposes.
-     *
-     * <p>Returns this {@code IterableAssert}.
-     */
-    public SingletonAssert<T> setCoder(Coder<T> coderOrNull) {
-      this.coder = Optional.fromNullable(coderOrNull);
-      return this;
+    private PCollectionSingletonIterableAssert<T> satisfies(
+        AssertRelation<Iterable<T>, Iterable<T>> relation, Iterable<T> expectedElements) {
+      return satisfies(
+          new CheckRelationAgainstExpected<Iterable<T>>(
+              relation, expectedElements, IterableCoder.of(elementCoder)));
     }
+  }
 
-    /**
-     * Gets the coder, which may yet be absent.
-     */
-    public Coder<T> getCoder() {
-      if (coder.isPresent()) {
-        return coder.get();
-      } else {
-        throw new IllegalStateException(
-            "Attempting to access the coder of an IterableAssert that has not been set yet.");
-      }
+  /**
+   * An assertion about the contents of a {@link PCollection} when it is viewed as a single value
+   * of type {@code ViewT}. This requires side input support from the runner.
+   */
+  private static class PCollectionViewAssert<ElemT, ViewT> implements SingletonAssert<ViewT> {
+    private final PCollection<ElemT> actual;
+    private final PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view;
+    private final Coder<ViewT> coder;
+
+    protected PCollectionViewAssert(
+        PCollection<ElemT> actual,
+        PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
+        Coder<ViewT> coder) {
+      this.actual = actual;
+      this.view = view;
+      this.coder = coder;
     }
 
-    /**
-     * Applies a {@link SerializableFunction} to check the value of this
-     * {@code SingletonAssert}'s view.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> satisfies(SerializableFunction<T, Void> checkerFn) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++), new OneSideInputAssert<T>(createActual, checkerFn));
-      return this;
+    @Override
+    public PCollectionViewAssert<ElemT, ViewT> isEqualTo(ViewT expectedValue) {
+      return satisfies(new AssertIsEqualToRelation<ViewT>(), expectedValue);
     }
 
-    /**
-     * Applies an {@link AssertRelation} to check the provided relation against the
-     * value of this assert and the provided expected value.
-     *
-     * <p>Returns this {@code SingletonAssert}.
-     */
-    public SingletonAssert<T> satisfies(
-        AssertRelation<T, T> relation,
-        final T expectedValue) {
-      pipeline.apply(
-          "PAssert$" + (assertCount++),
-          new TwoSideInputAssert<T, T>(
-              createActual,
-              new CreateExpected<T, T>(Arrays.asList(expectedValue), coder, View.<T>asSingleton()),
-              relation));
+    @Override
+    public PCollectionViewAssert<ElemT, ViewT> notEqualTo(ViewT expectedValue) {
+      return satisfies(new AssertNotEqualToRelation<ViewT>(), expectedValue);
+    }
 
+    @Override
+    public PCollectionViewAssert<ElemT, ViewT> satisfies(
+        SerializableFunction<ViewT, Void> checkerFn) {
+      actual
+          .getPipeline()
+          .apply(
+              "PAssert$" + (assertCount++),
+              new OneSideInputAssert<ViewT>(CreateActual.from(actual, view), checkerFn));
       return this;
     }
 
     /**
-     * Checks that the value of this {@code SingletonAssert}'s view is equal
-     * to the expected value.
+     * Applies an {@link AssertRelation} to check the provided relation against the value of this
+     * assert and the provided expected value.
      *
      * <p>Returns this {@code SingletonAssert}.
      */
-    public SingletonAssert<T> isEqualTo(T expectedValue) {
-      return satisfies(new AssertIsEqualToRelation<T>(), expectedValue);
+    private PCollectionViewAssert<ElemT, ViewT> satisfies(
+        AssertRelation<ViewT, ViewT> relation, final ViewT expectedValue) {
+      return satisfies(new CheckRelationAgainstExpected<ViewT>(relation, expectedValue, coder));
     }
 
     /**
-     * Checks that the value of this {@code SingletonAssert}'s view is not equal
-     * to the expected value.
-     *
-     * <p>Returns this {@code SingletonAssert}.
+     * Always throws an {@link UnsupportedOperationException}: users are probably looking for
+     * {@link #isEqualTo}.
      */
-    public SingletonAssert<T> notEqualTo(T expectedValue) {
-      return satisfies(new AssertNotEqualToRelation<T>(), expectedValue);
+    @Deprecated
+    @Override
+    public boolean equals(Object o) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "tests for Java equality of the %s object, not the PCollection in question. "
+                  + "Call a test method, such as isEqualTo.",
+              getClass().getSimpleName()));
     }
 
     /**
-     * Checks that the value of this {@code SingletonAssert}'s view is equal to
-     * the expected value.
-     *
-     * @deprecated replaced by {@link #isEqualTo}
+     * @throws UnsupportedOperationException always.
+     * @deprecated {@link Object#hashCode()} is not supported on {@link PAssert} objects.
      */
     @Deprecated
-    public SingletonAssert<T> is(T expectedValue) {
-      return isEqualTo(expectedValue);
+    @Override
+    public int hashCode() {
+      throw new UnsupportedOperationException(
+          String.format("%s.hashCode() is not supported.", SingletonAssert.class.getSimpleName()));
     }
-
   }
 
   ////////////////////////////////////////////////////////////////////////
@@ -504,8 +498,13 @@ public class PAssert {
     private final transient PCollection<T> actual;
     private final transient PTransform<PCollection<T>, PCollectionView<ActualT>> actualView;
 
-    private CreateActual(PCollection<T> actual,
-        PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
+    public static <T, ActualT> CreateActual<T, ActualT> from(
+        PCollection<T> actual, PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
+      return new CreateActual<>(actual, actualView);
+    }
+
+    private CreateActual(
+        PCollection<T> actual, PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
       this.actual = actual;
       this.actualView = actualView;
     }
@@ -515,73 +514,198 @@ public class PAssert {
       final Coder<T> coder = actual.getCoder();
       return actual
           .apply(Window.<T>into(new GlobalWindows()))
-          .apply(ParDo.of(new DoFn<T, T>() {
-            @Override
-            public void processElement(ProcessContext context) throws CoderException {
-              context.output(CoderUtils.clone(coder, context.element()));
-            }
-          }))
+          .apply(
+              ParDo.of(
+                  new DoFn<T, T>() {
+                    @Override
+                    public void processElement(ProcessContext context) throws CoderException {
+                      context.output(CoderUtils.clone(coder, context.element()));
+                    }
+                  }))
           .apply(actualView);
     }
   }
 
-  private static class CreateExpected<T, ExpectedT>
-      extends PTransform<PBegin, PCollectionView<ExpectedT>> {
-
-    private final Iterable<T> elements;
-    private final Optional<Coder<T>> coder;
-    private final transient PTransform<PCollection<T>, PCollectionView<ExpectedT>> view;
+  /**
+   * A partially applied {@link AssertRelation}, where one value is provided along with a coder to
+   * serialize/deserialize them.
+   */
+  private static class CheckRelationAgainstExpected<T> implements SerializableFunction<T, Void> {
+    private final AssertRelation<T, T> relation;
+    private final byte[] encodedExpected;
+    private final Coder<T> coder;
 
-    private CreateExpected(Iterable<T> elements, Optional<Coder<T>> coder,
-        PTransform<PCollection<T>, PCollectionView<ExpectedT>> view) {
-      this.elements = elements;
+    public CheckRelationAgainstExpected(AssertRelation<T, T> relation, T expected, Coder<T> coder) {
+      this.relation = relation;
       this.coder = coder;
-      this.view = view;
+
+      try {
+        this.encodedExpected = CoderUtils.encodeToByteArray(coder, expected);
+      } catch (IOException coderException) {
+        throw new RuntimeException(coderException);
+      }
     }
 
     @Override
-    public PCollectionView<ExpectedT> apply(PBegin input) {
-      Create.Values<T> createTransform = Create.<T>of(elements);
-      if (coder.isPresent()) {
-        createTransform = createTransform.withCoder(coder.get());
+    public Void apply(T actual) {
+      try {
+        T expected = CoderUtils.decodeFromByteArray(coder, encodedExpected);
+        return relation.assertFor(expected).apply(actual);
+      } catch (IOException coderException) {
+        throw new RuntimeException(coderException);
       }
-      return input.apply(createTransform).apply(view);
     }
   }
 
-  private static class PreExisting<T> extends PTransform<PBegin, PCollectionView<T>> {
+  /**
+   * A transform that gathers the contents of a {@link PCollection} into a single main input
+   * iterable in the global window. This requires a runner to support {@link GroupByKey} in the
+   * global window, but not side inputs or other windowing or triggers.
+   *
+   * <p>If the {@link PCollection} is empty, this transform returns a {@link PCollection} containing
+   * a single empty iterable, even though in practice most runners will not produce any element.
+   */
+  private static class GroupGlobally<T> extends PTransform<PCollection<T>, PCollection<Iterable<T>>>
+      implements Serializable {
 
-    private final PCollectionView<T> view;
+    public GroupGlobally() {}
 
-    private PreExisting(PCollectionView<T> view) {
-      this.view = view;
+    @Override
+    public PCollection<Iterable<T>> apply(PCollection<T> input) {
+
+      final int contentsKey = 0;
+      final int dummyKey = 1;
+      final int combinedKey = 42;
+
+      // Group the contents by key. If it is empty, this PCollection will be empty, too.
+      // Then key it again with a dummy key.
+      PCollection<KV<Integer, KV<Integer, Iterable<T>>>> doubleKeyedGroupedInput =
+          input
+              .apply("GloballyWindow", Window.<T>into(new GlobalWindows()))
+              .apply("ContentsWithKeys", WithKeys.<Integer, T>of(contentsKey))
+              .apply(
+                  "NeverTriggerContents",
+                  Window.<KV<Integer, T>>triggering(Never.ever()).discardingFiredPanes())
+              .apply("ContentsGBK", GroupByKey.<Integer, T>create())
+              .apply(
+                  "DoubleKeyContents", WithKeys.<Integer, KV<Integer, Iterable<T>>>of(combinedKey));
+
+      // Create another non-empty PCollection that is keyed with a distinct dummy key
+      PCollection<KV<Integer, KV<Integer, Iterable<T>>>> doubleKeyedDummy =
+          input
+              .getPipeline()
+              .apply(
+                  Create.of(
+                          KV.of(
+                              combinedKey,
+                              KV.of(dummyKey, (Iterable<T>) Collections.<T>emptyList())))
+                      .withCoder(doubleKeyedGroupedInput.getCoder()))
+              .setWindowingStrategyInternal(doubleKeyedGroupedInput.getWindowingStrategy());
+
+      // Flatten them together and group by the combined key to get a single element
+      PCollection<KV<Integer, Iterable<KV<Integer, Iterable<T>>>>> dummyAndContents =
+          PCollectionList.<KV<Integer, KV<Integer, Iterable<T>>>>of(doubleKeyedGroupedInput)
+              .and(doubleKeyedDummy)
+              .apply(
+                  "FlattenDummyAndContents",
+                  Flatten.<KV<Integer, KV<Integer, Iterable<T>>>>pCollections())
+              .apply(
+                  "GroupDummyAndContents", GroupByKey.<Integer, KV<Integer, Iterable<T>>>create());
+
+      // Extract the contents if they exist else empty contents.
+      return dummyAndContents
+          .apply(
+              "GetContents",
+              ParDo.of(
+                  new DoFn<KV<Integer, Iterable<KV<Integer, Iterable<T>>>>, Iterable<T>>() {
+                    @Override
+                    public void processElement(ProcessContext ctx) {
+                      Iterable<KV<Integer, Iterable<T>>> groupedDummyAndContents =
+                          ctx.element().getValue();
+
+                      if (Iterables.size(groupedDummyAndContents) == 1) {
+                        // Only the dummy value, so just output empty
+                        ctx.output(Collections.<T>emptyList());
+                      } else {
+                        checkState(
+                            Iterables.size(groupedDummyAndContents) == 2,
+                            "Internal error: PAssert grouped contents with a"
+                                + " dummy value resulted in more than 2 groupings: %s",
+                                groupedDummyAndContents);
+
+                        if (Iterables.get(groupedDummyAndContents, 0).getKey() == contentsKey) {
+                          // The first iterable in the group holds the real contents
+                          ctx.output(Iterables.get(groupedDummyAndContents, 0).getValue());
+                        } else {
+                          // The second iterable holds the real contents
+                          ctx.output(Iterables.get(groupedDummyAndContents, 1).getValue());
+                        }
+                      }
+                    }
+                  }));
+    }
+  }
+
+  /**
+   * A transform that applies an assertion-checking function over iterables of {@code ActualT} to
+   * the entirety of the contents of its input.
+   */
+  public static class GroupThenAssert<T> extends PTransform<PCollection<T>, PDone>
+      implements Serializable {
+    private final SerializableFunction<Iterable<T>, Void> checkerFn;
+
+    private GroupThenAssert(SerializableFunction<Iterable<T>, Void> checkerFn) {
+      this.checkerFn = checkerFn;
+    }
+
+    @Override
+    public PDone apply(PCollection<T> input) {
+      input
+          .apply("GroupGlobally", new GroupGlobally<T>())
+          .apply("RunChecks", ParDo.of(new GroupedValuesCheckerDoFn<>(checkerFn)));
+
+      return PDone.in(input.getPipeline());
+    }
+  }
+
+  /**
+   * A transform that applies an assertion-checking function to a single iterable contained as the
+   * sole element of a {@link PCollection}.
+   */
+  public static class GroupThenAssertForSingleton<T>
+      extends PTransform<PCollection<Iterable<T>>, PDone> implements Serializable {
+    private final SerializableFunction<Iterable<T>, Void> checkerFn;
+
+    private GroupThenAssertForSingleton(SerializableFunction<Iterable<T>, Void> checkerFn) {
+      this.checkerFn = checkerFn;
     }
 
     @Override
-    public PCollectionView<T> apply(PBegin input) {
-      return view;
+    public PDone apply(PCollection<Iterable<T>> input) {
+      input
+          .apply("GroupGlobally", new GroupGlobally<Iterable<T>>())
+          .apply("RunChecks", ParDo.of(new SingletonCheckerDoFn<>(checkerFn)));
+
+      return PDone.in(input.getPipeline());
     }
   }
 
   /**
-   * An assertion checker that takes a single
-   * {@link PCollectionView PCollectionView&lt;ActualT&gt;}
-   * and an assertion over {@code ActualT}, and checks it within a dataflow
-   * pipeline.
+   * An assertion checker that takes a single {@link PCollectionView
+   * PCollectionView&lt;ActualT&gt;} and an assertion over {@code ActualT}, and checks it within a
+   * Beam pipeline.
    *
-   * <p>Note that the entire assertion must be serializable. If
-   * you need to make assertions involving multiple inputs
-   * that are each not serializable, use TwoSideInputAssert.
+   * <p>Note that the entire assertion must be serializable.
    *
-   * <p>This is generally useful for assertion functions that
-   * are serializable but whose underlying data may not have a coder.
+   * <p>This is generally useful for assertion functions that are serializable but whose underlying
+   * data may not have a coder.
    */
-  public static class OneSideInputAssert<ActualT>
-      extends PTransform<PBegin, PDone> implements Serializable {
+  public static class OneSideInputAssert<ActualT> extends PTransform<PBegin, PDone>
+      implements Serializable {
     private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
     private final SerializableFunction<ActualT, Void> checkerFn;
 
-    public OneSideInputAssert(
+    private OneSideInputAssert(
         PTransform<PBegin, PCollectionView<ActualT>> createActual,
         SerializableFunction<ActualT, Void> checkerFn) {
       this.createActual = createActual;
@@ -594,21 +718,23 @@ public class PAssert {
 
       input
           .apply(Create.of(0).withCoder(VarIntCoder.of()))
-          .apply(ParDo.named("RunChecks").withSideInputs(actual)
-              .of(new CheckerDoFn<>(checkerFn, actual)));
+          .apply(
+              ParDo.named("RunChecks")
+                  .withSideInputs(actual)
+                  .of(new SideInputCheckerDoFn<>(checkerFn, actual)));
 
       return PDone.in(input.getPipeline());
     }
   }
 
   /**
-   * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of
-   * a {@link PCollectionView}, and adjusts counters and thrown exceptions for use in testing.
+   * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of a
+   * {@link PCollectionView}, and adjusts counters and thrown exceptions for use in testing.
    *
    * <p>The input is ignored, but is {@link Integer} to be usable on runners that do not support
    * null values.
    */
-  private static class CheckerDoFn<ActualT> extends DoFn<Integer, Void> {
+  private static class SideInputCheckerDoFn<ActualT> extends DoFn<Integer, Void> {
     private final SerializableFunction<ActualT, Void> checkerFn;
     private final Aggregator<Integer, Integer> success =
         createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
@@ -616,9 +742,8 @@ public class PAssert {
         createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
     private final PCollectionView<ActualT> actual;
 
-    private CheckerDoFn(
-        SerializableFunction<ActualT, Void> checkerFn,
-        PCollectionView<ActualT> actual) {
+    private SideInputCheckerDoFn(
+        SerializableFunction<ActualT, Void> checkerFn, PCollectionView<ActualT> actual) {
       this.checkerFn = checkerFn;
       this.actual = actual;
     }
@@ -627,12 +752,9 @@ public class PAssert {
     public void processElement(ProcessContext c) {
       try {
         ActualT actualContents = c.sideInput(actual);
-        checkerFn.apply(actualContents);
-        success.addValue(1);
+        doChecks(actualContents, checkerFn, success, failure);
       } catch (Throwable t) {
-        LOG.error("PAssert failed expectations.", t);
-        failure.addValue(1);
-        // TODO: allow for metrics to propagate on failure when running a streaming pipeline
+        // Suppress exception in streaming
         if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
           throw t;
         }
@@ -641,87 +763,89 @@ public class PAssert {
   }
 
   /**
-   * An assertion checker that takes a {@link PCollectionView PCollectionView&lt;ActualT&gt;},
-   * a {@link PCollectionView PCollectionView&lt;ExpectedT&gt;}, a relation
-   * over {@code A} and {@code B}, and checks that the relation holds
-   * within a dataflow pipeline.
+   * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of
+   * the single iterable element of the input {@link PCollection} and adjusts counters and
+   * thrown exceptions for use in testing.
    *
-   * <p>This is useful when either/both of {@code A} and {@code B}
-   * are not serializable, but have coders (provided
-   * by the underlying {@link PCollection}s).
+   * <p>The singleton property is presumed, not enforced.
    */
-  public static class TwoSideInputAssert<ActualT, ExpectedT>
-      extends PTransform<PBegin, PDone> implements Serializable {
-
-    private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
-    private final transient PTransform<PBegin, PCollectionView<ExpectedT>> createExpected;
-    private final AssertRelation<ActualT, ExpectedT> relation;
+  private static class GroupedValuesCheckerDoFn<ActualT> extends DoFn<ActualT, Void> {
+    private final SerializableFunction<ActualT, Void> checkerFn;
+    private final Aggregator<Integer, Integer> success =
+        createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
+    private final Aggregator<Integer, Integer> failure =
+        createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
 
-    protected TwoSideInputAssert(
-        PTransform<PBegin, PCollectionView<ActualT>> createActual,
-        PTransform<PBegin, PCollectionView<ExpectedT>> createExpected,
-        AssertRelation<ActualT, ExpectedT> relation) {
-      this.createActual = createActual;
-      this.createExpected = createExpected;
-      this.relation = relation;
+    private GroupedValuesCheckerDoFn(SerializableFunction<ActualT, Void> checkerFn) {
+      this.checkerFn = checkerFn;
     }
 
     @Override
-    public PDone apply(PBegin input) {
-      final PCollectionView<ActualT> actual = input.apply("CreateActual", createActual);
-      final PCollectionView<ExpectedT> expected = input.apply("CreateExpected", createExpected);
+    public void processElement(ProcessContext c) {
+      try {
+        doChecks(c.element(), checkerFn, success, failure);
+      } catch (Throwable t) {
+        // Suppress exception in streaming
+        if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
+          throw t;
+        }
+      }
+    }
+  }
 
-      input
-          .apply(Create.of(0).withCoder(VarIntCoder.of()))
-          .apply("RunChecks", ParDo.withSideInputs(actual, expected)
-              .of(new CheckerDoFn<>(relation, actual, expected)));
+  /**
+   * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of
+   * the single item contained within the single iterable on input and
+   * adjusts counters and thrown exceptions for use in testing.
+   *
+   * <p>The singleton property of the input {@link PCollection} is presumed, not enforced. However,
+   * each input element must be a singleton iterable, or this will fail.
+   */
+  private static class SingletonCheckerDoFn<ActualT> extends DoFn<Iterable<ActualT>, Void> {
+    private final SerializableFunction<ActualT, Void> checkerFn;
+    private final Aggregator<Integer, Integer> success =
+        createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
+    private final Aggregator<Integer, Integer> failure =
+        createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
 
-      return PDone.in(input.getPipeline());
+    private SingletonCheckerDoFn(SerializableFunction<ActualT, Void> checkerFn) {
+      this.checkerFn = checkerFn;
     }
 
-    /**
-     * Input is ignored, but is {@link Integer} for runners that do not support null values.
-     */
-    private static class CheckerDoFn<ActualT, ExpectedT> extends DoFn<Integer, Void> {
-      private final Aggregator<Integer, Integer> success =
-          createAggregator(SUCCESS_COUNTER, new Sum.SumIntegerFn());
-      private final Aggregator<Integer, Integer> failure =
-          createAggregator(FAILURE_COUNTER, new Sum.SumIntegerFn());
-      private final AssertRelation<ActualT, ExpectedT> relation;
-      private final PCollectionView<ActualT> actual;
-      private final PCollectionView<ExpectedT> expected;
-
-      private CheckerDoFn(AssertRelation<ActualT, ExpectedT> relation,
-          PCollectionView<ActualT> actual, PCollectionView<ExpectedT> expected) {
-        this.relation = relation;
-        this.actual = actual;
-        this.expected = expected;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) {
-        try {
-          ActualT actualContents = c.sideInput(actual);
-          ExpectedT expectedContents = c.sideInput(expected);
-          relation.assertFor(expectedContents).apply(actualContents);
-          success.addValue(1);
-        } catch (Throwable t) {
-          LOG.error("PAssert failed expectations.", t);
-          failure.addValue(1);
-          // TODO: allow for metrics to propagate on failure when running a streaming pipeline
-          if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
-            throw t;
-          }
+    @Override
+    public void processElement(ProcessContext c) {
+      try {
+        ActualT actualContents = Iterables.getOnlyElement(c.element());
+        doChecks(actualContents, checkerFn, success, failure);
+      } catch (Throwable t) {
+        // Suppress exception in streaming
+        if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
+          throw t;
         }
       }
     }
   }
 
+  private static <ActualT> void doChecks(
+      ActualT actualContents,
+      SerializableFunction<ActualT, Void> checkerFn,
+      Aggregator<Integer, Integer> successAggregator,
+      Aggregator<Integer, Integer> failureAggregator) {
+    try {
+      checkerFn.apply(actualContents);
+      successAggregator.addValue(1);
+    } catch (Throwable t) {
+      LOG.error("PAssert failed expectations.", t);
+      failureAggregator.addValue(1);
+      throw t;
+    }
+  }
+
   /////////////////////////////////////////////////////////////////////////////
 
   /**
-   * A {@link SerializableFunction} that verifies that an actual value is equal to an
-   * expected value.
+   * A {@link SerializableFunction} that verifies that an actual value is equal to an expected
+   * value.
    */
   private static class AssertIsEqualTo<T> implements SerializableFunction<T, Void> {
     private T expected;
@@ -738,8 +862,8 @@ public class PAssert {
   }
 
   /**
-   * A {@link SerializableFunction} that verifies that an actual value is not equal to an
-   * expected value.
+   * A {@link SerializableFunction} that verifies that an actual value is not equal to an expected
+   * value.
    */
   private static class AssertNotEqualTo<T> implements SerializableFunction<T, Void> {
     private T expected;
@@ -756,8 +880,8 @@ public class PAssert {
   }
 
   /**
-   * A {@link SerializableFunction} that verifies that an {@code Iterable} contains
-   * expected items in any order.
+   * A {@link SerializableFunction} that verifies that an {@code Iterable} contains expected items
+   * in any order.
    */
   private static class AssertContainsInAnyOrder<T>
       implements SerializableFunction<Iterable<T>, Void> {
@@ -787,10 +911,9 @@ public class PAssert {
   ////////////////////////////////////////////////////////////
 
   /**
-   * A binary predicate between types {@code Actual} and {@code Expected}.
-   * Implemented as a method {@code assertFor(Expected)} which returns
-   * a {@code SerializableFunction<Actual, Void>}
-   * that should verify the assertion..
+   * A binary predicate between types {@code Actual} and {@code Expected}. Implemented as a method
+   * {@code assertFor(Expected)} which returns a {@code SerializableFunction<Actual, Void>} that
+   * should verify the assertion..
    */
   private static interface AssertRelation<ActualT, ExpectedT> extends Serializable {
     public SerializableFunction<ActualT, Void> assertFor(ExpectedT input);
@@ -799,8 +922,7 @@ public class PAssert {
   /**
    * An {@link AssertRelation} implementing the binary predicate that two objects are equal.
    */
-  private static class AssertIsEqualToRelation<T>
-      implements AssertRelation<T, T> {
+  private static class AssertIsEqualToRelation<T> implements AssertRelation<T, T> {
     @Override
     public SerializableFunction<T, Void> assertFor(T expected) {
       return new AssertIsEqualTo<T>(expected);
@@ -810,8 +932,7 @@ public class PAssert {
   /**
    * An {@link AssertRelation} implementing the binary predicate that two objects are not equal.
    */
-  private static class AssertNotEqualToRelation<T>
-      implements AssertRelation<T, T> {
+  private static class AssertNotEqualToRelation<T> implements AssertRelation<T, T> {
     @Override
     public SerializableFunction<T, Void> assertFor(T expected) {
       return new AssertNotEqualTo<T>(expected);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e90a1b9d/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
index f540948..fdc8719 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
@@ -17,9 +17,6 @@
  */
 package org.apache.beam.sdk.testing;
 
-import static org.apache.beam.sdk.testing.SerializableMatchers.anything;
-import static org.apache.beam.sdk.testing.SerializableMatchers.not;
-
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -151,30 +148,6 @@ public class PAssertTest implements Serializable {
   }
 
   /**
-   * Basic test of succeeding {@link PAssert} using a {@link SerializableMatcher}.
-   */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testBasicMatcherSuccess() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-    PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
-    PAssert.that(pcollection).containsInAnyOrder(anything());
-    pipeline.run();
-  }
-
-  /**
-   * Basic test of failing {@link PAssert} using a {@link SerializableMatcher}.
-   */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testBasicMatcherFailure() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-    PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
-    PAssert.that(pcollection).containsInAnyOrder(not(anything()));
-    runExpectingAssertionFailure(pipeline);
-  }
-
-  /**
    * Test that we throw an error at pipeline construction time when the user mistakenly uses
    * {@code PAssert.thatSingleton().equals()} instead of the test method {@code .isEqualTo}.
    */


[14/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
new file mode 100644
index 0000000..c038910
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.WatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.coders.VarIntCoder;
+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 DirectTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class DirectTimerInternalsTest {
+  private MockClock clock;
+  @Mock private TransformWatermarks watermarks;
+
+  private TimerUpdateBuilder timerUpdateBuilder;
+
+  private DirectTimerInternals internals;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    clock = MockClock.fromInstant(new Instant(0));
+
+    timerUpdateBuilder = TimerUpdate.builder(StructuralKey.of(1234, VarIntCoder.of()));
+
+    internals = DirectTimerInternals.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/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
index e129489..b903ef1 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -51,7 +51,7 @@ import java.util.Collections;
 public class EncodabilityEnforcementFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   private EncodabilityEnforcementFactory factory = EncodabilityEnforcementFactory.create();
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void encodeFailsThrows() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
new file mode 100644
index 0000000..1726866
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -0,0 +1,545 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectExecutionContext.DirectStepContext;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+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 EvaluationContext}.
+ */
+@RunWith(JUnit4.class)
+public class EvaluationContextTest {
+  private TestPipeline p;
+  private EvaluationContext 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() {
+    DirectRunner runner =
+        DirectRunner.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 = ImmutableListBundleFactory.create();
+
+    context =
+        EvaluationContext.create(
+            runner.getPipelineOptions(),
+            ImmutableListBundleFactory.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(2));
+    // The cached value is served in the earlier reader
+    reader = context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
+    assertThat(reader.get(view, second), containsInAnyOrder(4444));
+  }
+
+  @Test
+  public void getExecutionContextSameStepSameKeyState() {
+    DirectExecutionContext fooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(),
+            StructuralKey.of("foo", StringUtf8Coder.of()));
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    DirectStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1");
+    stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
+
+    context.handleResult(ImmutableListBundleFactory.create()
+            .createKeyedBundle(null, StructuralKey.of("foo", StringUtf8Coder.of()), created)
+            .commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(created.getProducingTransformInternal())
+            .withState(stepContext.commitState())
+            .build());
+
+    DirectExecutionContext secondFooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(),
+            StructuralKey.of("foo", StringUtf8Coder.of()));
+    assertThat(
+        secondFooContext
+            .getOrCreateStepContext("s1", "s1")
+            .stateInternals()
+            .state(StateNamespaces.global(), intBag)
+            .read(),
+        contains(1));
+  }
+
+
+  @Test
+  public void getExecutionContextDifferentKeysIndependentState() {
+    DirectExecutionContext fooContext =
+        context.getExecutionContext(created.getProducingTransformInternal(),
+            StructuralKey.of("foo", StringUtf8Coder.of()));
+
+    StateTag<Object, BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
+
+    fooContext
+        .getOrCreateStepContext("s1", "s1")
+        .stateInternals()
+        .state(StateNamespaces.global(), intBag)
+        .add(1);
+
+    DirectExecutionContext barContext =
+        context.getExecutionContext(created.getProducingTransformInternal(),
+            StructuralKey.of("bar", StringUtf8Coder.of()));
+    assertThat(barContext, not(equalTo(fooContext)));
+    assertThat(
+        barContext
+            .getOrCreateStepContext("s1", "s1")
+            .stateInternals()
+            .state(StateNamespaces.global(), intBag)
+            .read(),
+        emptyIterable());
+  }
+
+  @Test
+  public void getExecutionContextDifferentStepsIndependentState() {
+    StructuralKey<?> myKey = StructuralKey.of("foo", StringUtf8Coder.of());
+    DirectExecutionContext 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);
+
+    DirectExecutionContext 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);
+    TransformResult 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);
+
+    TransformResult 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() {
+    StructuralKey<?> myKey = StructuralKey.of("foo".getBytes(), ByteArrayCoder.of());
+    DirectExecutionContext 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);
+
+    TransformResult stateResult =
+        StepTransformResult.withoutHold(downstream.getProducingTransformInternal())
+            .withState(state)
+            .build();
+
+    context.handleResult(
+        context.createKeyedBundle(null, myKey, created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        stateResult);
+
+    DirectExecutionContext 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);
+
+    TransformResult 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));
+
+    TransformResult finishedResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+    context.forceRefresh();
+    // Obtain the value via blocking call
+    assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
+  }
+
+  @Test
+  public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
+    TransformResult finishedResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), finishedResult);
+
+    final CountDownLatch callLatch = new CountDownLatch(1);
+    context.extractFiredTimers();
+    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() {
+    TransformResult holdResult =
+        StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0))
+            .build();
+    context.handleResult(null, ImmutableList.<TimerData>of(), holdResult);
+
+    StructuralKey<?> key = StructuralKey.of("foo".length(), VarIntCoder.of());
+    TimerData toFire =
+        TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
+    TransformResult 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());
+
+    TransformResult advanceResult =
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    // Should cause the downstream timer to fire
+    context.handleResult(null, ImmutableList.<TimerData>of(), advanceResult);
+
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> fired =
+        context.extractFiredTimers();
+    assertThat(
+        fired,
+        Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
+    Map<StructuralKey<?>, 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() {
+    StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
+    CommittedBundle<KV<String, Integer>> newBundle =
+        context
+            .createBundle(
+                bundleFactory.createKeyedBundle(
+                    null, key,
+                    created).commit(Instant.now()),
+                downstream).commit(Instant.now());
+    assertThat(newBundle.getKey(), Matchers.<StructuralKey<?>>equalTo(key));
+  }
+
+  @Test
+  public void createKeyedBundleKeyed() {
+    StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
+    CommittedBundle<KV<String, Integer>> keyedBundle =
+        context.createKeyedBundle(
+            bundleFactory.createRootBundle(created).commit(Instant.now()),
+            key,
+            downstream).commit(Instant.now());
+    assertThat(keyedBundle.getKey(),
+        Matchers.<StructuralKey<?>>equalTo(key));
+  }
+
+  @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());
+    context.extractFiredTimers();
+    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());
+    context.extractFiredTimers();
+    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());
+    }
+    context.extractFiredTimers();
+    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());
+    context.extractFiredTimers();
+    assertThat(context.isDone(), is(false));
+
+    context.handleResult(
+        context.createRootBundle(created).commit(Instant.now()),
+        ImmutableList.<TimerData>of(),
+        StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
+    context.extractFiredTimers();
+    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/babddbbc/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
index 5efb090..0bc3036 100644
--- 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
@@ -45,7 +45,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class FlattenEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
   @Test
   public void testFlattenInMemoryEvaluator() throws Exception {
     TestPipeline p = TestPipeline.create();
@@ -60,7 +60,7 @@ public class FlattenEvaluatorFactoryTest {
     CommittedBundle<Integer> rightBundle =
         bundleFactory.createRootBundle(right).commit(Instant.now());
 
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+    EvaluationContext context = mock(EvaluationContext.class);
 
     UncommittedBundle<Integer> flattenedLeftBundle = bundleFactory.createRootBundle(flattened);
     UncommittedBundle<Integer> flattenedRightBundle = bundleFactory.createRootBundle(flattened);
@@ -87,8 +87,8 @@ public class FlattenEvaluatorFactoryTest {
     rightSideEvaluator.processElement(
         WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
 
-    InProcessTransformResult rightSideResult = rightSideEvaluator.finishBundle();
-    InProcessTransformResult leftSideResult = leftSideEvaluator.finishBundle();
+    TransformResult rightSideResult = rightSideEvaluator.finishBundle();
+    TransformResult leftSideResult = leftSideEvaluator.finishBundle();
 
     assertThat(
         rightSideResult.getOutputBundles(),
@@ -124,13 +124,13 @@ public class FlattenEvaluatorFactoryTest {
 
     PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());
 
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+    EvaluationContext context = mock(EvaluationContext.class);
 
     FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory();
     TransformEvaluator<Integer> emptyEvaluator =
         factory.forApplication(flattened.getProducingTransformInternal(), null, context);
 
-    InProcessTransformResult leftSideResult = emptyEvaluator.finishBundle();
+    TransformResult leftSideResult = emptyEvaluator.finishBundle();
 
     assertThat(leftSideResult.getOutputBundles(), emptyIterable());
     assertThat(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index b589db0..a529043 100644
--- 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
@@ -52,7 +53,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class GroupByKeyEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void testInMemoryEvaluator() throws Exception {
@@ -68,11 +69,11 @@ public class GroupByKeyEvaluatorFactoryTest {
     PCollection<KV<String, WindowedValue<Integer>>> kvs =
         values.apply(new ReifyTimestampsAndWindows<String, Integer>());
     PCollection<KeyedWorkItem<String, Integer>> groupedKvs =
-        kvs.apply(new InProcessGroupByKey.InProcessGroupByKeyOnly<String, Integer>());
+        kvs.apply(new DirectGroupByKeyOnly<String, Integer>());
 
     CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
         bundleFactory.createRootBundle(kvs).commit(Instant.now());
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     StructuralKey<String> fooKey = StructuralKey.of("foo", StringUtf8Coder.of());
     UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle =
         bundleFactory.createKeyedBundle(null, fooKey, groupedKvs);
@@ -100,7 +101,7 @@ public class GroupByKeyEvaluatorFactoryTest {
     Coder<String> keyCoder =
         ((KvCoder<String, WindowedValue<Integer>>) kvs.getCoder()).getKeyCoder();
     TransformEvaluator<KV<String, WindowedValue<Integer>>> evaluator =
-        new InProcessGroupByKeyOnlyEvaluatorFactory()
+        new GroupByKeyOnlyEvaluatorFactory()
             .forApplication(
                 groupedKvs.getProducingTransformInternal(), inputBundle, evaluationContext);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
new file mode 100644
index 0000000..5fb5a76
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectGroupByKey.DirectGroupByKeyOnly;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.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.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 GroupByKeyOnlyEvaluatorFactory}.
+ */
+@RunWith(JUnit4.class)
+public class GroupByKeyOnlyEvaluatorFactoryTest {
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.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 DirectGroupByKeyOnly<String, Integer>());
+
+    CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
+        bundleFactory.createRootBundle(kvs).commit(Instant.now());
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
+
+    StructuralKey<String> fooKey = StructuralKey.of("foo", StringUtf8Coder.of());
+    UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle = bundleFactory.createKeyedBundle(
+        null, fooKey,
+        groupedKvs);
+    StructuralKey<String> barKey = StructuralKey.of("bar", StringUtf8Coder.of());
+    UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle = bundleFactory.createKeyedBundle(
+        null, barKey,
+        groupedKvs);
+    StructuralKey<String> bazKey = StructuralKey.of("baz", StringUtf8Coder.of());
+    UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle = bundleFactory.createKeyedBundle(
+        null, bazKey,
+        groupedKvs);
+
+    when(evaluationContext.createKeyedBundle(inputBundle,
+        fooKey,
+        groupedKvs)).thenReturn(fooBundle);
+    when(evaluationContext.createKeyedBundle(inputBundle,
+        barKey,
+        groupedKvs)).thenReturn(barBundle);
+    when(evaluationContext.createKeyedBundle(inputBundle,
+        bazKey,
+        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 GroupByKeyOnlyEvaluatorFactory()
+            .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/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index c4da86c..d40cf93 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -57,7 +57,7 @@ public class ImmutabilityCheckingBundleFactoryTest {
     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());
+    factory = ImmutabilityCheckingBundleFactory.create(ImmutableListBundleFactory.create());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
index ead9c9e..890e06d 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -53,7 +53,7 @@ public class ImmutabilityEnforcementFactoryTest implements Serializable {
   @Before
   public void setup() {
     factory = new ImmutabilityEnforcementFactory();
-    bundleFactory = InProcessBundleFactory.create();
+    bundleFactory = ImmutableListBundleFactory.create();
     TestPipeline p = TestPipeline.create();
     pcollection =
         p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
new file mode 100644
index 0000000..21e4bcb
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.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 ImmutableListBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class ImmutableListBundleFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private ImmutableListBundleFactory bundleFactory = ImmutableListBundleFactory.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 createRootBundleShouldCreateWithEmptyKey() {
+    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(),
+        Matchers.<StructuralKey<?>>equalTo(StructuralKey.of(null, VoidCoder.of())));
+  }
+
+  private <T> void createKeyedBundle(Coder<T> coder, T key) throws Exception {
+    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+    StructuralKey skey = StructuralKey.of(key, coder);
+
+    UncommittedBundle<Integer> inFlightBundle =
+        bundleFactory.createKeyedBundle(null, skey, pcollection);
+
+    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
+    assertThat(bundle.getKey(), equalTo(skey));
+  }
+
+  @Test
+  public void keyedWithNullKeyShouldCreateKeyedBundle() throws Exception {
+    createKeyedBundle(VoidCoder.of(), null);
+  }
+
+  @Test
+  public void keyedWithKeyShouldCreateKeyedBundle() throws Exception {
+    createKeyedBundle(StringUtf8Coder.of(), "foo");
+    createKeyedBundle(VarIntCoder.of(), 1234);
+    createKeyedBundle(ByteArrayCoder.of(), new byte[] {0, 2, 4, 99});
+  }
+
+  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(), Matchers.<StructuralKey<?>>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,
+                StructuralKey.of("foo", StringUtf8Coder.of()),
+                created).commit(Instant.now()),
+            downstream).commit(Instant.now());
+    assertThat(newBundle.getKey().getKey(), Matchers.<Object>equalTo("foo"));
+  }
+
+  @Test
+  public void createKeyedBundleKeyed() {
+    CommittedBundle<KV<String, Integer>> keyedBundle = bundleFactory.createKeyedBundle(
+        bundleFactory.createRootBundle(created).commit(Instant.now()),
+        StructuralKey.of("foo", StringUtf8Coder.of()),
+        downstream).commit(Instant.now());
+    assertThat(keyedBundle.getKey().getKey(), Matchers.<Object>equalTo("foo"));
+  }
+}


[11/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index 8b8d44f..c378cf4 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -25,9 +25,9 @@ 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.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
@@ -66,7 +66,7 @@ import java.io.Serializable;
  */
 @RunWith(JUnit4.class)
 public class ParDoSingleEvaluatorFactoryTest implements Serializable {
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void testParDoInMemoryTransformEvaluator() throws Exception {
@@ -85,11 +85,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CommittedBundle<String> inputBundle =
         bundleFactory.createRootBundle(input).commit(Instant.now());
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
     when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
+    DirectExecutionContext executionContext =
+        new DirectExecutionContext(null, null, null, null);
     when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(),
         inputBundle.getKey())).thenReturn(executionContext);
     CounterSet counters = new CounterSet();
@@ -106,7 +106,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
     assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
     assertThat(result.getCounters(), equalTo(counters));
@@ -137,11 +137,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CommittedBundle<String> inputBundle =
         bundleFactory.createRootBundle(input).commit(Instant.now());
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<Integer> outputBundle = bundleFactory.createRootBundle(collection);
     when(evaluationContext.createBundle(inputBundle, collection)).thenReturn(outputBundle);
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
+    DirectExecutionContext executionContext =
+        new DirectExecutionContext(null, null, null, null);
     when(evaluationContext.getExecutionContext(collection.getProducingTransformInternal(),
         inputBundle.getKey())).thenReturn(executionContext);
     CounterSet counters = new CounterSet();
@@ -158,7 +158,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getOutputBundles(), Matchers.<UncommittedBundle<?>>containsInAnyOrder(outputBundle));
     assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
@@ -198,13 +198,13 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CommittedBundle<String> inputBundle =
         bundleFactory.createRootBundle(input).commit(Instant.now());
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
 
     when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
 
-    InProcessExecutionContext executionContext = new InProcessExecutionContext(null,
+    DirectExecutionContext executionContext = new DirectExecutionContext(null,
         StructuralKey.of("myKey", StringUtf8Coder.of()),
         null, null);
     when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(),
@@ -224,7 +224,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
     assertThat(result.getState(), not(nullValue()));
     assertThat(
@@ -298,13 +298,13 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     CommittedBundle<String> inputBundle =
         bundleFactory.createRootBundle(input).commit(Instant.now());
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
 
     when(evaluationContext.createBundle(inputBundle, mainOutput)).thenReturn(mainOutputBundle);
 
-    InProcessExecutionContext executionContext = new InProcessExecutionContext(null,
+    DirectExecutionContext executionContext = new DirectExecutionContext(null,
         key,
         null,
         null);
@@ -321,7 +321,7 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     evaluator.processElement(WindowedValue.valueInGlobalWindow("foo"));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(result.getTimerUpdate(),
         equalTo(TimerUpdate.builder(StructuralKey.of("myKey", StringUtf8Coder.of()))
             .setTimer(addedTimer)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
new file mode 100644
index 0000000..c0242ed
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
@@ -0,0 +1,520 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.junit.Assert.fail;
+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.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.ReadyCheckingSideInputReader;
+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.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for {@link SideInputContainer}.
+ */
+@RunWith(JUnit4.class)
+public class SideInputContainerTest {
+  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 EvaluationContext context;
+
+  private TestPipeline pipeline;
+
+  private SideInputContainer 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 = SideInputContainer.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 getNotReadyThrows() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("not ready");
+
+    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
+        .get(mapView, GlobalWindow.INSTANCE);
+  }
+
+  @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)));
+    // Cached value is false
+    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
+
+    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(false));
+
+    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+
+    reader = container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
+    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
+    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
+  }
+
+  @Test
+  public void isReadyForEmptyWindowTrue() throws Exception {
+    CountDownLatch onComplete = new CountDownLatch(1);
+    immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
+    CountDownLatch latch = invokeLatchedCallback(singletonView, GlobalWindow.INSTANCE, onComplete);
+
+    ReadyCheckingSideInputReader reader =
+        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+
+    latch.countDown();
+    if (!onComplete.await(1500L, TimeUnit.MILLISECONDS)) {
+      fail("Callback to set empty values did not complete!");
+    }
+    // The cached value was false, so it continues to be true
+    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
+
+    // A new reader for the same container gets a fresh look
+    reader = container.createReaderForViews(ImmutableList.of(mapView, singletonView));
+    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));
+  }
+
+  /**
+   * When a callAfterWindowCloses with the specified view's producing transform, window, and
+   * windowing strategy is invoked, start a thread that will invoke the callback after the returned
+   * {@link CountDownLatch} is counted down once.
+   */
+  private CountDownLatch invokeLatchedCallback(
+      PCollectionView<?> view, BoundedWindow window, final CountDownLatch onComplete) {
+    final CountDownLatch runLatch = new CountDownLatch(1);
+    doAnswer(
+        new Answer<Void>() {
+          @Override
+          public Void answer(InvocationOnMock invocation) throws Throwable {
+            Object callback = invocation.getArguments()[3];
+            final Runnable callbackRunnable = (Runnable) callback;
+            Executors.newSingleThreadExecutor().submit(new Runnable() {
+              public void run() {
+                try {
+                  if (!runLatch.await(1500L, TimeUnit.MILLISECONDS)) {
+                    fail("Run latch didn't count down within timeout");
+                  }
+                  callbackRunnable.run();
+                  onComplete.countDown();
+                } catch (InterruptedException e) {
+                  fail("Unexpectedly interrupted while waiting for latch to be counted down");
+                }
+              }
+            });
+            return null;
+          }
+        })
+        .when(context)
+        .scheduleAfterOutputWouldBeProduced(
+            Mockito.eq(view),
+            Mockito.eq(window),
+            Mockito.eq(view.getWindowingStrategyInternal()),
+            Mockito.any(Runnable.class));
+    return runLatch;
+  }
+
+  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/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java
index b8d9a76..6e477d3 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java
@@ -115,7 +115,7 @@ public class ThreadLocalInvalidatingTransformEvaluatorTest {
     }
 
     @Override
-    public InProcessTransformResult finishBundle() throws Exception {
+    public TransformResult finishBundle() throws Exception {
       finishBundleCalled = true;
       return null;
     }
@@ -128,7 +128,7 @@ public class ThreadLocalInvalidatingTransformEvaluatorTest {
     }
 
     @Override
-    public InProcessTransformResult finishBundle() throws Exception {
+    public TransformResult finishBundle() throws Exception {
       throw new Exception();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index a5e6cee..cb5cd46 100644
--- 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
@@ -75,14 +75,14 @@ public class TransformExecutorTest {
   private RegisteringCompletionCallback completionCallback;
   private TransformExecutorService transformEvaluationState;
   private BundleFactory bundleFactory;
-  @Mock private InProcessEvaluationContext evaluationContext;
+  @Mock private EvaluationContext evaluationContext;
   @Mock private TransformEvaluatorRegistry registry;
 
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
 
-    bundleFactory = InProcessBundleFactory.create();
+    bundleFactory = ImmutableListBundleFactory.create();
 
     transformEvaluationState =
         TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService());
@@ -97,7 +97,7 @@ public class TransformExecutorTest {
 
   @Test
   public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
     final AtomicBoolean finishCalled = new AtomicBoolean(false);
     TransformEvaluator<Object> evaluator =
@@ -108,7 +108,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             finishCalled.set(true);
             return result;
           }
@@ -135,7 +135,7 @@ public class TransformExecutorTest {
 
   @Test
   public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
     final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
     TransformEvaluator<String> evaluator =
@@ -147,7 +147,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             return result;
           }
         };
@@ -183,7 +183,7 @@ public class TransformExecutorTest {
 
   @Test
   public void processElementThrowsExceptionCallsback() throws Exception {
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
     final Exception exception = new Exception();
     TransformEvaluator<String> evaluator =
@@ -194,7 +194,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             return result;
           }
         };
@@ -233,7 +233,7 @@ public class TransformExecutorTest {
           public void processElement(WindowedValue<String> element) throws Exception {}
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             throw exception;
           }
         };
@@ -264,7 +264,7 @@ public class TransformExecutorTest {
 
   @Test
   public void duringCallGetThreadIsNonNull() throws Exception {
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
     final CountDownLatch testLatch = new CountDownLatch(1);
     final CountDownLatch evaluatorLatch = new CountDownLatch(1);
@@ -276,7 +276,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             testLatch.countDown();
             evaluatorLatch.await();
             return result;
@@ -306,7 +306,7 @@ public class TransformExecutorTest {
 
   @Test
   public void callWithEnforcementAppliesEnforcement() throws Exception {
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
 
     TransformEvaluator<Object> evaluator =
@@ -316,7 +316,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             return result;
           }
         };
@@ -365,7 +365,7 @@ public class TransformExecutorTest {
               }
             });
 
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
     final CountDownLatch testLatch = new CountDownLatch(1);
     final CountDownLatch evaluatorLatch = new CountDownLatch(1);
@@ -376,7 +376,7 @@ public class TransformExecutorTest {
           public void processElement(WindowedValue<Object> element) throws Exception {}
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             testLatch.countDown();
             evaluatorLatch.await();
             return result;
@@ -423,7 +423,7 @@ public class TransformExecutorTest {
               }
             });
 
-    final InProcessTransformResult result =
+    final TransformResult result =
         StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
     final CountDownLatch testLatch = new CountDownLatch(1);
     final CountDownLatch evaluatorLatch = new CountDownLatch(1);
@@ -437,7 +437,7 @@ public class TransformExecutorTest {
           }
 
           @Override
-          public InProcessTransformResult finishBundle() throws Exception {
+          public TransformResult finishBundle() throws Exception {
             return result;
           }
         };
@@ -470,7 +470,7 @@ public class TransformExecutorTest {
   }
 
   private static class RegisteringCompletionCallback implements CompletionCallback {
-    private InProcessTransformResult handledResult = null;
+    private TransformResult handledResult = null;
     private Throwable handledThrowable = null;
     private final CountDownLatch onMethod;
 
@@ -480,7 +480,7 @@ public class TransformExecutorTest {
 
     @Override
     public CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+        CommittedBundle<?> inputBundle, TransformResult result) {
       handledResult = result;
       onMethod.countDown();
       @SuppressWarnings("rawtypes") Iterable unprocessedElements =
@@ -516,7 +516,7 @@ public class TransformExecutorTest {
   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<>();
+    private final List<TransformResult> finishedBundles = new ArrayList<>();
 
     @Override
     public void beforeElement(WindowedValue<T> element) {
@@ -531,7 +531,7 @@ public class TransformExecutorTest {
     @Override
     public void afterFinish(
         CommittedBundle<T> input,
-        InProcessTransformResult result,
+        TransformResult result,
         Iterable<? extends CommittedBundle<?>> outputs) {
       finishedBundles.add(result);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index be5c489..e182e8d 100644
--- 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
@@ -72,10 +72,10 @@ import javax.annotation.Nullable;
 public class UnboundedReadEvaluatorFactoryTest {
   private PCollection<Long> longs;
   private TransformEvaluatorFactory factory;
-  private InProcessEvaluationContext context;
+  private EvaluationContext context;
   private UncommittedBundle<Long> output;
 
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Before
   public void setup() {
@@ -85,7 +85,7 @@ public class UnboundedReadEvaluatorFactoryTest {
     longs = p.apply(Read.from(source));
 
     factory = new UnboundedReadEvaluatorFactory();
-    context = mock(InProcessEvaluationContext.class);
+    context = mock(EvaluationContext.class);
     output = bundleFactory.createRootBundle(longs);
     when(context.createRootBundle(longs)).thenReturn(output);
   }
@@ -95,7 +95,7 @@ public class UnboundedReadEvaluatorFactoryTest {
     TransformEvaluator<?> evaluator =
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
     assertThat(
@@ -114,7 +114,7 @@ public class UnboundedReadEvaluatorFactoryTest {
     TransformEvaluator<?> evaluator =
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
     assertThat(
@@ -127,7 +127,7 @@ public class UnboundedReadEvaluatorFactoryTest {
     when(context.createRootBundle(longs)).thenReturn(secondOutput);
     TransformEvaluator<?> secondEvaluator =
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
-    InProcessTransformResult secondResult = secondEvaluator.finishBundle();
+    TransformResult secondResult = secondEvaluator.finishBundle();
     assertThat(
         secondResult.getWatermarkHold(),
         Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));
@@ -220,7 +220,7 @@ public class UnboundedReadEvaluatorFactoryTest {
         factory.forApplication(longs.getProducingTransformInternal(), null, context);
 
     assertThat(secondEvaluator, nullValue());
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
 
     assertThat(
         result.getWatermarkHold(), Matchers.<ReadableInstant>lessThan(DateTime.now().toInstant()));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 714e9c9..6820792 100644
--- 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
@@ -51,7 +51,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ViewEvaluatorFactoryTest {
-  private BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void testInMemoryEvaluator() throws Exception {
@@ -69,7 +69,7 @@ public class ViewEvaluatorFactoryTest {
     PCollectionView<Iterable<String>> view =
         concat.apply(new ViewEvaluatorFactory.WriteView<>(createView));
 
-    InProcessEvaluationContext context = mock(InProcessEvaluationContext.class);
+    EvaluationContext context = mock(EvaluationContext.class);
     TestViewWriter<String, Iterable<String>> viewWriter = new TestViewWriter<>();
     when(context.createPCollectionViewWriter(concat, view)).thenReturn(viewWriter);
 



[35/50] [abbrv] incubator-beam git commit: Remove references to javax.servlet.

Posted by da...@apache.org.
Remove references to javax.servlet.


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

Branch: refs/heads/python-sdk
Commit: 09bf9b3720f08acc9e94784461f2482ab371cd90
Parents: 6a41da8
Author: Pei He <pe...@google.com>
Authored: Fri Jun 17 13:02:58 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                                          | 6 ------
 .../org/apache/beam/examples/common/DataflowExampleUtils.java  | 6 +++---
 2 files changed, 3 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09bf9b37/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 5211b80..9458962 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -285,12 +285,6 @@
       <scope>runtime</scope>
     </dependency>
 
-    <dependency>
-      <groupId>javax.servlet</groupId>
-      <artifactId>javax.servlet-api</artifactId>
-      <version>3.1.0</version>
-    </dependency>
-
     <!-- Hamcrest and JUnit are required dependencies of PAssert,
          which is used in the main code of DebuggingWordCount example. -->
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09bf9b37/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
index a0b7319..5b1af6d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -62,8 +62,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import javax.servlet.http.HttpServletResponse;
-
 /**
  * The utility class that sets up and tears down external resources, starts the Google Cloud Pub/Sub
  * injector, and cancels the streaming and the injector pipelines once the program terminates.
@@ -72,6 +70,8 @@ import javax.servlet.http.HttpServletResponse;
  */
 public class DataflowExampleUtils {
 
+  private static final int SC_NOT_FOUND = 404;
+
   private final DataflowPipelineOptions options;
   private Bigquery bigQueryClient = null;
   private Pubsub pubsubClient = null;
@@ -481,7 +481,7 @@ public class DataflowExampleUtils {
     try {
       return request.execute();
     } catch (GoogleJsonResponseException e) {
-      if (e.getStatusCode() == HttpServletResponse.SC_NOT_FOUND) {
+      if (e.getStatusCode() == SC_NOT_FOUND) {
         return null;
       } else {
         throw e;


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

Posted by da...@apache.org.
This closes #498


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

Branch: refs/heads/python-sdk
Commit: f166b16b811624aa66d54dfe3811a80306d12e9c
Parents: 0e5c662 4b41768
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jun 20 15:14:49 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:49 2016 -0700

----------------------------------------------------------------------
 README.md                                       |    4 +-
 examples/java/README.md                         |   42 +-
 examples/java/pom.xml                           |   22 +-
 .../beam/examples/DebuggingWordCount.java       |    2 +-
 .../apache/beam/examples/MinimalWordCount.java  |    4 +-
 .../apache/beam/examples/WindowedWordCount.java |   13 +-
 .../org/apache/beam/examples/WordCount.java     |   21 +-
 .../examples/common/DataflowExampleUtils.java   |   31 +-
 .../beam/examples/complete/AutoComplete.java    |   12 +-
 .../apache/beam/examples/complete/TfIdf.java    |    2 +-
 .../examples/complete/TopWikipediaSessions.java |    2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../examples/cookbook/DatastoreWordCount.java   |    8 +-
 .../beam/examples/cookbook/DeDupExample.java    |    2 +-
 .../beam/examples/cookbook/FilterExamples.java  |    2 +-
 .../beam/examples/cookbook/JoinExamples.java    |    2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |    2 +-
 .../org/apache/beam/examples/WordCountTest.java |    6 +-
 .../examples/complete/AutoCompleteTest.java     |   14 +-
 .../cookbook/BigQueryTornadoesTest.java         |    8 +-
 .../cookbook/CombinePerKeyExamplesTest.java     |    4 +-
 .../examples/cookbook/FilterExamplesTest.java   |    4 +-
 .../examples/cookbook/JoinExamplesTest.java     |    4 +-
 .../cookbook/MaxPerKeyExamplesTest.java         |    4 +-
 .../examples/cookbook/TriggerExampleTest.java   |    4 +-
 .../beam/examples/MinimalWordCountJava8.java    |   12 +-
 .../beam/examples/complete/game/GameStats.java  |    7 +-
 .../examples/complete/game/HourlyTeamScore.java |    8 +-
 .../examples/complete/game/LeaderBoard.java     |    2 +-
 .../beam/examples/complete/game/UserScore.java  |    7 +-
 .../examples/MinimalWordCountJava8Test.java     |    8 +-
 .../complete/game/HourlyTeamScoreTest.java      |    7 +-
 .../examples/complete/game/UserScoreTest.java   |    7 +-
 runners/core-java/pom.xml                       |   13 +
 runners/direct-java/pom.xml                     |   34 +-
 .../direct/AbstractModelEnforcement.java        |    4 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   16 +-
 .../beam/runners/direct/BundleFactory.java      |    8 +-
 .../beam/runners/direct/CommittedResult.java    |    6 +-
 .../beam/runners/direct/CompletionCallback.java |    4 +-
 .../runners/direct/DirectExecutionContext.java  |  106 +
 .../beam/runners/direct/DirectGroupByKey.java   |  132 +
 .../direct/DirectGroupByKeyOverrideFactory.java |   41 +
 .../beam/runners/direct/DirectOptions.java      |  101 +
 .../beam/runners/direct/DirectRegistrar.java    |   57 +
 .../beam/runners/direct/DirectRunner.java       |  372 ++
 .../runners/direct/DirectTimerInternals.java    |   84 +
 .../runners/direct/EmptyTransformEvaluator.java |    4 +-
 .../direct/EncodabilityEnforcementFactory.java  |    2 +-
 .../beam/runners/direct/EvaluationContext.java  |  429 +++
 .../beam/runners/direct/EvaluatorKey.java       |    4 +-
 .../direct/ExecutorServiceParallelExecutor.java |   32 +-
 .../runners/direct/FlattenEvaluatorFactory.java |   18 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  127 +
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |  186 +
 .../ImmutabilityCheckingBundleFactory.java      |    4 +-
 .../direct/ImmutabilityEnforcementFactory.java  |    4 +-
 .../direct/ImmutableListBundleFactory.java      |  163 +
 .../direct/InMemoryWatermarkManager.java        | 1420 --------
 .../runners/direct/InProcessBundleFactory.java  |  161 -
 .../direct/InProcessBundleOutputManager.java    |   51 -
 .../direct/InProcessEvaluationContext.java      |  428 ---
 .../direct/InProcessExecutionContext.java       |  105 -
 .../beam/runners/direct/InProcessExecutor.java  |   48 -
 ...rocessGroupAlsoByWindowEvaluatorFactory.java |  127 -
 .../runners/direct/InProcessGroupByKey.java     |  132 -
 ...InProcessGroupByKeyOnlyEvaluatorFactory.java |  185 -
 .../InProcessGroupByKeyOverrideFactory.java     |   41 -
 .../direct/InProcessPipelineOptions.java        |  101 -
 .../runners/direct/InProcessPipelineRunner.java |  370 --
 .../beam/runners/direct/InProcessRegistrar.java |   55 -
 .../direct/InProcessSideInputContainer.java     |  277 --
 .../runners/direct/InProcessTimerInternals.java |   84 -
 .../direct/InProcessTransformResult.java        |   84 -
 .../beam/runners/direct/ModelEnforcement.java   |    8 +-
 .../runners/direct/ModelEnforcementFactory.java |    2 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  186 +
 .../runners/direct/ParDoInProcessEvaluator.java |  186 -
 .../direct/ParDoMultiEvaluatorFactory.java      |   10 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   10 +-
 .../direct/PassthroughTransformEvaluator.java   |    4 +-
 .../beam/runners/direct/PipelineExecutor.java   |   48 +
 .../beam/runners/direct/SideInputContainer.java |  277 ++
 .../runners/direct/StepTransformResult.java     |    8 +-
 ...readLocalInvalidatingTransformEvaluator.java |    2 +-
 .../beam/runners/direct/TransformEvaluator.java |    6 +-
 .../direct/TransformEvaluatorFactory.java       |    4 +-
 .../direct/TransformEvaluatorRegistry.java      |   12 +-
 .../beam/runners/direct/TransformExecutor.java  |   16 +-
 .../beam/runners/direct/TransformResult.java    |   84 +
 .../direct/UnboundedReadEvaluatorFactory.java   |  161 +-
 .../direct/UncommittedBundleOutputManager.java  |   51 +
 .../runners/direct/ViewEvaluatorFactory.java    |   22 +-
 .../direct/WatermarkCallbackExecutor.java       |   10 +-
 .../beam/runners/direct/WatermarkManager.java   | 1420 ++++++++
 .../runners/direct/WindowEvaluatorFactory.java  |   18 +-
 .../direct/AvroIOShardedWriteFactoryTest.java   |   12 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |   14 +-
 .../runners/direct/CommittedResultTest.java     |   14 +-
 .../runners/direct/DirectRegistrarTest.java     |   74 +
 .../beam/runners/direct/DirectRunnerTest.java   |  339 ++
 .../direct/DirectTimerInternalsTest.java        |  134 +
 .../EncodabilityEnforcementFactoryTest.java     |    4 +-
 .../runners/direct/EvaluationContextTest.java   |  545 +++
 .../direct/FlattenEvaluatorFactoryTest.java     |   16 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   13 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  197 ++
 .../ImmutabilityCheckingBundleFactoryTest.java  |    6 +-
 .../ImmutabilityEnforcementFactoryTest.java     |    4 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  231 ++
 .../direct/InMemoryWatermarkManagerTest.java    | 1428 --------
 .../direct/InProcessBundleFactoryTest.java      |  231 --
 .../direct/InProcessEvaluationContextTest.java  |  545 ---
 ...ocessGroupByKeyOnlyEvaluatorFactoryTest.java |  196 --
 .../direct/InProcessPipelineRegistrarTest.java  |   74 -
 .../direct/InProcessPipelineRunnerTest.java     |  330 --
 .../direct/InProcessSideInputContainerTest.java |  520 ---
 .../direct/InProcessTimerInternalsTest.java     |  134 -
 .../direct/KeyedPValueTrackingVisitorTest.java  |    7 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  214 ++
 .../direct/ParDoInProcessEvaluatorTest.java     |  214 --
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |   36 +-
 .../direct/ParDoSingleEvaluatorFactoryTest.java |   36 +-
 .../runners/direct/SideInputContainerTest.java  |  520 +++
 .../direct/TextIOShardedWriteFactoryTest.java   |   12 +-
 ...LocalInvalidatingTransformEvaluatorTest.java |    4 +-
 .../runners/direct/TransformExecutorTest.java   |   44 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   72 +-
 .../direct/ViewEvaluatorFactoryTest.java        |    8 +-
 .../runners/direct/WatermarkManagerTest.java    | 1428 ++++++++
 .../direct/WindowEvaluatorFactoryTest.java      |   20 +-
 runners/flink/README.md                         |    4 +-
 .../beam/runners/flink/examples/TFIDF.java      |    6 +-
 .../beam/runners/flink/examples/WordCount.java  |    4 +-
 .../flink/examples/streaming/AutoComplete.java  |   16 +-
 .../flink/examples/streaming/JoinExamples.java  |    4 +-
 .../examples/streaming/KafkaIOExamples.java     |    4 +-
 .../KafkaWindowedWordCountExample.java          |    4 +-
 .../examples/streaming/WindowedWordCount.java   |    4 +-
 runners/flink/runner/pom.xml                    |   26 +-
 .../beam/runners/flink/FlinkPipelineRunner.java |  180 -
 .../apache/beam/runners/flink/FlinkRunner.java  |  179 +
 .../runners/flink/FlinkRunnerRegistrar.java     |    4 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |    8 +
 .../runners/flink/TestFlinkPipelineRunner.java  |   80 -
 .../beam/runners/flink/TestFlinkRunner.java     |   81 +
 .../FlinkBatchTransformTranslators.java         |   49 +-
 .../FlinkBatchTranslationContext.java           |   13 +-
 .../translation/types/CoderComparator.java      |  217 --
 .../translation/types/CoderTypeInformation.java |   14 +-
 .../translation/types/CoderTypeSerializer.java  |   37 +-
 .../types/EncodedValueComparator.java           |  197 ++
 .../types/EncodedValueSerializer.java           |  113 +
 .../types/EncodedValueTypeInformation.java      |  111 +
 .../translation/types/KvCoderComperator.java    |  259 --
 .../types/KvCoderTypeInformation.java           |  207 --
 .../flink/translation/types/KvKeySelector.java  |   51 +
 .../streaming/FlinkAbstractParDoWrapper.java    |    9 +-
 .../streaming/io/UnboundedFlinkSource.java      |    6 +-
 .../streaming/state/FlinkStateInternals.java    |    2 +-
 .../flink/EncodedValueComparatorTest.java       |   71 +
 .../runners/flink/FlinkRunnerRegistrarTest.java |    8 +-
 .../beam/runners/flink/FlinkTestPipeline.java   |    4 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   11 +-
 runners/google-cloud-dataflow-java/pom.xml      |   11 +
 .../BlockingDataflowPipelineRunner.java         |  186 -
 .../dataflow/BlockingDataflowRunner.java        |  186 +
 .../dataflow/DataflowJobCancelledException.java |    2 +-
 .../dataflow/DataflowJobExecutionException.java |    2 +-
 .../dataflow/DataflowJobUpdatedException.java   |    2 +-
 .../runners/dataflow/DataflowPipelineJob.java   |    2 +-
 .../dataflow/DataflowPipelineRegistrar.java     |    8 +-
 .../dataflow/DataflowPipelineRunner.java        | 3229 ------------------
 .../dataflow/DataflowPipelineRunnerHooks.java   |   39 -
 .../dataflow/DataflowPipelineTranslator.java    |   10 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 3229 ++++++++++++++++++
 .../runners/dataflow/DataflowRunnerHooks.java   |   39 +
 .../runners/dataflow/internal/package-info.java |    2 +-
 .../BlockingDataflowPipelineOptions.java        |    6 +-
 .../options/DataflowPipelineOptions.java        |    4 +-
 .../DataflowPipelineWorkerPoolOptions.java      |    6 +-
 .../options/DataflowWorkerLoggingOptions.java   |   79 +-
 .../testing/TestDataflowPipelineRunner.java     |  271 --
 .../dataflow/testing/TestDataflowRunner.java    |  271 ++
 .../dataflow/util/DataflowTransport.java        |   11 -
 .../BlockingDataflowPipelineRunnerTest.java     |  302 --
 .../dataflow/BlockingDataflowRunnerTest.java    |  304 ++
 .../dataflow/DataflowPipelineRegistrarTest.java |    4 +-
 .../dataflow/DataflowPipelineRunnerTest.java    | 1415 --------
 .../DataflowPipelineTranslatorTest.java         |   50 +-
 .../runners/dataflow/DataflowRunnerTest.java    | 1417 ++++++++
 .../runners/dataflow/io/DataflowAvroIOTest.java |    4 +-
 .../dataflow/io/DataflowPubsubIOTest.java       |    4 +-
 .../runners/dataflow/io/DataflowTextIOTest.java |   85 +-
 .../testing/TestDataflowPipelineRunnerTest.java |  601 ----
 .../testing/TestDataflowRunnerTest.java         |  601 ++++
 .../DataflowDisplayDataEvaluator.java           |   10 +-
 .../transforms/DataflowGroupByKeyTest.java      |    8 +-
 .../dataflow/transforms/DataflowViewTest.java   |    8 +-
 .../util/DataflowPathValidatorTest.java         |    8 +-
 runners/spark/pom.xml                           |  118 +-
 .../runners/spark/SparkRunnerRegistrar.java     |    3 +-
 .../runners/spark/TestSparkPipelineRunner.java  |   77 +
 .../spark/translation/TransformTranslator.java  |   17 +-
 .../beam/runners/spark/SimpleWordCountTest.java |    2 +-
 .../runners/spark/SparkRunnerRegistrarTest.java |    2 +-
 .../spark/translation/SerializationTest.java    |    2 +-
 .../translation/TransformTranslatorTest.java    |    6 +-
 .../translation/WindowedWordCountTest.java      |   19 +-
 sdks/java/core/pom.xml                          |    3 +
 .../apache/beam/sdk/coders/BigDecimalCoder.java |  102 +
 .../apache/beam/sdk/coders/BigIntegerCoder.java |   91 +
 .../java/org/apache/beam/sdk/io/BigQueryIO.java |    6 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |    3 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   44 -
 .../java/org/apache/beam/sdk/io/TextIO.java     |    3 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |    8 +
 .../beam/sdk/options/DirectPipelineOptions.java |   75 -
 .../beam/sdk/options/PipelineOptions.java       |   38 +-
 .../sdk/runners/DirectPipelineRegistrar.java    |   55 -
 .../beam/sdk/runners/DirectPipelineRunner.java  | 1298 -------
 .../apache/beam/sdk/runners/package-info.java   |   12 +-
 .../apache/beam/sdk/testing/CrashingRunner.java |   33 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  206 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |    5 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |    2 +-
 .../apache/beam/sdk/transforms/DoFnTester.java  |  159 +-
 .../org/apache/beam/sdk/transforms/Filter.java  |  128 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   32 -
 .../org/apache/beam/sdk/transforms/ParDo.java   |  302 +-
 .../org/apache/beam/sdk/transforms/View.java    |   24 -
 .../sdk/transforms/display/package-info.java    |   26 +
 .../transforms/windowing/AfterWatermark.java    |   14 +-
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |    9 +-
 .../beam/sdk/util/BigQueryTableRowIterator.java |    4 +-
 .../sdk/util/DirectModeExecutionContext.java    |  130 -
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |    3 +-
 .../apache/beam/sdk/util/ReduceFnRunner.java    |   19 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |    4 +-
 .../org/apache/beam/sdk/WindowMatchers.java     |    5 +
 .../beam/sdk/coders/BigDecimalCoderTest.java    |  144 +
 .../beam/sdk/coders/BigIntegerCoderTest.java    |  119 +
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |    4 +-
 .../io/BoundedReadFromUnboundedSourceTest.java  |    6 -
 .../beam/sdk/io/CompressedSourceTest.java       |    4 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |   75 +
 .../java/org/apache/beam/sdk/io/WriteTest.java  |    9 +-
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |   84 +-
 .../sdk/options/PipelineOptionsFactoryTest.java |   79 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |    8 -
 .../options/PipelineOptionsValidatorTest.java   |   15 +
 .../runners/DirectPipelineRegistrarTest.java    |   71 -
 .../sdk/runners/DirectPipelineRunnerTest.java   |  221 --
 .../beam/sdk/runners/PipelineRunnerTest.java    |   35 +-
 .../beam/sdk/testing/TestPipelineTest.java      |    5 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   21 -
 .../beam/sdk/transforms/DoFnTesterTest.java     |   54 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |   63 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |  219 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   29 +-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |    2 +-
 .../windowing/AfterWatermarkTest.java           |   19 +
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  293 +-
 .../beam/sdk/util/ReduceFnRunnerTest.java       |   68 +
 sdks/java/extensions/join-library/pom.xml       |    7 +
 sdks/java/io/google-cloud-platform/pom.xml      |    7 +
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |    5 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |    2 -
 sdks/java/io/hdfs/pom.xml                       |    7 +
 sdks/java/io/kafka/pom.xml                      |    7 +
 sdks/java/java8tests/pom.xml                    |    7 +
 .../beam/sdk/transforms/FilterJava8Test.java    |    8 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |    3 +-
 .../main/resources/archetype-resources/pom.xml  |   19 +
 .../src/main/java/DebuggingWordCount.java       |    2 +-
 .../src/main/java/MinimalWordCount.java         |    4 +-
 .../src/main/java/WindowedWordCount.java        |    2 +-
 .../src/main/java/WordCount.java                |    2 +-
 .../main/java/common/DataflowExampleUtils.java  |   25 +-
 .../src/test/java/WordCountTest.java            |    6 +-
 .../main/resources/archetype-resources/pom.xml  |   13 +
 .../src/main/java/StarterPipeline.java          |    4 +-
 .../resources/projects/basic/reference/pom.xml  |   13 +
 .../src/main/java/it/pkg/StarterPipeline.java   |    4 +-
 sdks/java/pom.xml                               |    2 +-
 testing/travis/test_wordcount.sh                |    4 +-
 287 files changed, 17190 insertions(+), 18220 deletions(-)
----------------------------------------------------------------------



[04/50] [abbrv] incubator-beam git commit: Run NeedsRunner tests in Runner Core on the DirectRunner

Posted by da...@apache.org.
Run NeedsRunner tests in Runner Core on the DirectRunner

This ensures that all runner tests in runners/core-java are executed in
the standard maven build.


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

Branch: refs/heads/python-sdk
Commit: 6ada1a635382fcddc42a7580e74e755839f7172e
Parents: d5e3dfa
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 15 12:01:56 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 runners/core-java/pom.xml   | 13 +++++++++++++
 runners/direct-java/pom.xml |  1 +
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ada1a63/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index cd8c2d1..8ede60b 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -41,6 +41,19 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <excludedGroups>
+            org.apache.beam.sdk.testing.NeedsRunner
+          </excludedGroups>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>true</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
         <executions>
           <execution>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ada1a63/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index b2cb607..8667aee 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -97,6 +97,7 @@
               <failIfNoTests>true</failIfNoTests>
               <dependenciesToScan>
                 <dependency>org.apache.beam:beam-sdks-java-core</dependency>
+                <dependency>org.apache.beam:beam-runners-java-core</dependency>
               </dependenciesToScan>
               <systemPropertyVariables>
                 <beamTestPipelineOptions>


[31/50] [abbrv] incubator-beam git commit: Remove last vestige of the words DirectPipeline

Posted by da...@apache.org.
Remove last vestige of the words DirectPipeline


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

Branch: refs/heads/python-sdk
Commit: 6a41da853537e152613fb17bed782bc16d767c57
Parents: e581244
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 10:25:44 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6a41da85/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index fa00e82..b34d933 100644
--- a/README.md
+++ b/README.md
@@ -66,7 +66,7 @@ Have ideas for new SDKs or DSLs? See the [Jira](https://issues.apache.org/jira/b
 
 Beam supports executing programs on multiple distributed processing backends through PipelineRunners. Currently, the following PipelineRunners are available:
 
-- The `DirectPipelineRunner` runs the pipeline on your local machine.
+- The `DirectRunner` runs the pipeline on your local machine.
 - The `DataflowPipelineRunner` submits the pipeline to the [Google Cloud Dataflow](http://cloud.google.com/dataflow/).
 - The `FlinkPipelineRunner` runs the pipeline on an Apache Flink cluster. The code has been donated from [dataArtisans/flink-dataflow](https://github.com/dataArtisans/flink-dataflow) and is now part of Beam.
 - The `SparkPipelineRunner` runs the pipeline on an Apache Spark cluster. The code has been donated from [cloudera/spark-dataflow](https://github.com/cloudera/spark-dataflow) and is now part of Beam.


[03/50] [abbrv] incubator-beam git commit: Update DataflowPipelineRunner worker container version

Posted by da...@apache.org.
Update DataflowPipelineRunner worker container version


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

Branch: refs/heads/python-sdk
Commit: d2ceaf5e5a778fad18472ab0d7c02a14259015d7
Parents: 99654ca
Author: Scott Wegner <sw...@google.com>
Authored: Tue Jun 14 09:00:49 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

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


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


[02/50] [abbrv] incubator-beam git commit: Remove the DirectPipelineRunner from the Core SDK

Posted by da...@apache.org.
Remove the DirectPipelineRunner from the Core SDK


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

Branch: refs/heads/python-sdk
Commit: 99654ca4bed6758d7128d0f0ad376e8b479d4eba
Parents: 45e57e0
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 14 17:52:49 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 .../examples/common/DataflowExampleUtils.java   |   11 +-
 runners/spark/pom.xml                           |    6 +
 .../translation/TransformTranslatorTest.java    |    4 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |    1 -
 .../main/java/org/apache/beam/sdk/io/Read.java  |   44 -
 .../java/org/apache/beam/sdk/io/TextIO.java     |    1 -
 .../beam/sdk/options/DirectPipelineOptions.java |    1 -
 .../sdk/runners/DirectPipelineRegistrar.java    |   55 -
 .../beam/sdk/runners/DirectPipelineRunner.java  | 1298 ------------------
 .../org/apache/beam/sdk/transforms/Flatten.java |   32 -
 .../org/apache/beam/sdk/transforms/ParDo.java   |  302 +---
 .../org/apache/beam/sdk/transforms/View.java    |   24 -
 .../sdk/util/DirectModeExecutionContext.java    |  130 --
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |    1 -
 .../java/org/apache/beam/sdk/PipelineTest.java  |    4 +-
 .../io/BoundedReadFromUnboundedSourceTest.java  |    6 -
 .../runners/DirectPipelineRegistrarTest.java    |   71 -
 .../sdk/runners/DirectPipelineRunnerTest.java   |  222 ---
 .../beam/sdk/runners/PipelineRunnerTest.java    |    9 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   21 -
 .../beam/sdk/transforms/GroupByKeyTest.java     |   13 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   29 +-
 .../main/java/common/DataflowExampleUtils.java  |   13 +-
 testing/travis/test_wordcount.sh                |    4 +-
 24 files changed, 40 insertions(+), 2262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
index fb4f3bf..a0b7319 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.examples.common;
 
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
 import org.apache.beam.runners.dataflow.DataflowPipelineJob;
 import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
@@ -25,7 +26,7 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.IntraBundleParallelization;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
@@ -315,11 +316,13 @@ public class DataflowExampleUtils {
 
   /**
    * Do some runner setup: check that the DirectPipelineRunner is not used in conjunction with
-   * streaming, and if streaming is specified, use the DataflowPipelineRunner. Return the streaming
-   * flag value.
+   * streaming, and if streaming is specified, use the DataflowPipelineRunner.
    */
   public void setupRunner() {
-    if (options.isStreaming() && options.getRunner() != DirectPipelineRunner.class) {
+    Class<? extends PipelineRunner<?>> runner = options.getRunner();
+    if (options.isStreaming()
+        && (runner.equals(DataflowPipelineRunner.class)
+            || runner.equals(BlockingDataflowPipelineRunner.class))) {
       // In order to cancel the pipelines automatically,
       // {@literal DataflowPipelineRunner} is forced to be used.
       options.setRunner(DataflowPipelineRunner.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 4110689..e7d0834 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -118,6 +118,12 @@
       <artifactId>hamcrest-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <version>0.2.0-incubating-SNAPSHOT</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
index 4ef26d3..01f3070 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
@@ -21,12 +21,12 @@ package org.apache.beam.runners.spark.translation;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 
+import org.apache.beam.runners.direct.InProcessPipelineRunner;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.values.PCollection;
 
@@ -58,7 +58,7 @@ public class TransformTranslatorTest {
    */
   @Test
   public void testTextIOReadAndWriteTransforms() throws IOException {
-    String directOut = runPipeline(DirectPipelineRunner.class);
+    String directOut = runPipeline(InProcessPipelineRunner.class);
     String sparkOut = runPipeline(SparkPipelineRunner.class);
 
     List<String> directOutput =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
index 7e24253..2a5698c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
@@ -23,7 +23,6 @@ import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
index fb40063..c0440f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
@@ -20,11 +20,9 @@ package org.apache.beam.sdk.io;
 import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
 
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
@@ -32,9 +30,6 @@ import org.apache.beam.sdk.values.PInput;
 
 import org.joda.time.Duration;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import javax.annotation.Nullable;
 
 /**
@@ -153,45 +148,6 @@ public class Read {
             .withLabel("Read Source"))
           .include(source);
     }
-
-    static {
-      registerDefaultTransformEvaluator();
-    }
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private static void registerDefaultTransformEvaluator() {
-      DirectPipelineRunner.registerDefaultTransformEvaluator(
-          Bounded.class,
-          new DirectPipelineRunner.TransformEvaluator<Bounded>() {
-            @Override
-            public void evaluate(
-                Bounded transform, DirectPipelineRunner.EvaluationContext context) {
-              evaluateReadHelper(transform, context);
-            }
-
-            private <T> void evaluateReadHelper(
-                Read.Bounded<T> transform, DirectPipelineRunner.EvaluationContext context) {
-              try {
-                List<DirectPipelineRunner.ValueWithMetadata<T>> output = new ArrayList<>();
-                BoundedSource<T> source = transform.getSource();
-                try (BoundedSource.BoundedReader<T> reader =
-                    source.createReader(context.getPipelineOptions())) {
-                  for (boolean available = reader.start();
-                      available;
-                      available = reader.advance()) {
-                    output.add(
-                        DirectPipelineRunner.ValueWithMetadata.of(
-                            WindowedValue.timestampedValueInGlobalWindow(
-                                reader.getCurrent(), reader.getCurrentTimestamp())));
-                  }
-                }
-                context.setPCollectionValuesWithMetadata(context.getOutput(transform), output);
-              } catch (Exception e) {
-                throw new RuntimeException(e);
-              }
-            }
-          });
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 13cb45e..bbef072 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -25,7 +25,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.io.Read.Bounded;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.IOChannelUtils;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
index 4cdc0ca..c2095e3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/DirectPipelineOptions.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.options;
 
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.values.PCollection;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.java
deleted file mode 100644
index 7dd0fdd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRegistrar.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;
-
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for
- * the {@link DirectPipeline}.
- */
-public class DirectPipelineRegistrar {
-  private DirectPipelineRegistrar() { }
-
-  /**
-   * Register the {@link DirectPipelineRunner}.
-   */
-  @AutoService(PipelineRunnerRegistrar.class)
-  public static class Runner implements PipelineRunnerRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(DirectPipelineRunner.class);
-    }
-  }
-
-  /**
-   * Register the {@link DirectPipelineOptions}.
-   */
-  @AutoService(PipelineOptionsRegistrar.class)
-  public static class Options implements PipelineOptionsRegistrar {
-    @Override
-    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(DirectPipelineOptions.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
deleted file mode 100644
index 1eb25c5..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/DirectPipelineRunner.java
+++ /dev/null
@@ -1,1298 +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;
-
-import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
-
-import static com.google.common.base.Preconditions.checkArgument;
-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.PipelineResult;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.FileBasedSink;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DirectPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
-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.Partition;
-import org.apache.beam.sdk.transforms.Partition.PartitionFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AppliedCombineFn;
-import org.apache.beam.sdk.util.AssignWindows;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.MapAggregatorValues;
-import org.apache.beam.sdk.util.PerKeyCombineFnRunner;
-import org.apache.beam.sdk.util.PerKeyCombineFnRunners;
-import org.apache.beam.sdk.util.SerializableUtils;
-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.CounterSet;
-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.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TypedPValue;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-/**
- * Executes the operations in the pipeline directly, in this process, without
- * any optimization.  Useful for small local execution and tests.
- *
- * <p>Throws an exception from {@link #run} if execution fails.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code DirectPipelineRunner}, the Cloud Platform account that you configured with the
- * <a href="https://cloud.google.com/sdk/gcloud">gcloud</a> executable will need access to the
- * corresponding source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class DirectPipelineRunner
-    extends PipelineRunner<DirectPipelineRunner.EvaluationResults> {
-  private static final Logger LOG = LoggerFactory.getLogger(DirectPipelineRunner.class);
-
-  /**
-   * A source of random data, which can be seeded if determinism is desired.
-   */
-  private Random rand;
-
-  /**
-   * A map from PTransform class to the corresponding
-   * TransformEvaluator to use to evaluate that transform.
-   *
-   * <p>A static map that contains system-wide defaults.
-   */
-  private static Map<Class, TransformEvaluator> defaultTransformEvaluators =
-      new HashMap<>();
-
-  /**
-   * A map from PTransform class to the corresponding
-   * TransformEvaluator to use to evaluate that transform.
-   *
-   * <p>An instance map that contains bindings for this DirectPipelineRunner.
-   * Bindings in this map override those in the default map.
-   */
-  private Map<Class, TransformEvaluator> localTransformEvaluators =
-      new HashMap<>();
-
-  /**
-   * Records that instances of the specified PTransform class
-   * should be evaluated by default by the corresponding
-   * TransformEvaluator.
-   */
-  public static <TransformT extends PTransform<?, ?>>
-  void registerDefaultTransformEvaluator(
-      Class<TransformT> transformClass,
-      TransformEvaluator<? super TransformT> transformEvaluator) {
-    if (defaultTransformEvaluators.put(transformClass, transformEvaluator)
-        != null) {
-      throw new IllegalArgumentException(
-          "defining multiple evaluators for " + transformClass);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Records that instances of the specified PTransform class
-   * should be evaluated by the corresponding TransformEvaluator.
-   * Overrides any bindings specified by
-   * {@link #registerDefaultTransformEvaluator}.
-   */
-  public <TransformT extends PTransform<?, ?>>
-  void registerTransformEvaluator(
-      Class<TransformT> transformClass,
-      TransformEvaluator<TransformT> transformEvaluator) {
-    if (localTransformEvaluators.put(transformClass, transformEvaluator)
-        != null) {
-      throw new IllegalArgumentException(
-          "defining multiple evaluators for " + transformClass);
-    }
-  }
-
-  /**
-   * Returns the TransformEvaluator to use for instances of the
-   * specified PTransform class, or null if none registered.
-   */
-  public <TransformT extends PTransform<?, ?>>
-      TransformEvaluator<TransformT> getTransformEvaluator(Class<TransformT> transformClass) {
-    TransformEvaluator<TransformT> transformEvaluator =
-        localTransformEvaluators.get(transformClass);
-    if (transformEvaluator == null) {
-      transformEvaluator = defaultTransformEvaluators.get(transformClass);
-    }
-    return transformEvaluator;
-  }
-
-  /**
-   * Constructs a DirectPipelineRunner from the given options.
-   */
-  public static DirectPipelineRunner fromOptions(PipelineOptions options) {
-    DirectPipelineOptions directOptions =
-        PipelineOptionsValidator.validate(DirectPipelineOptions.class, options);
-    LOG.debug("Creating DirectPipelineRunner");
-    return new DirectPipelineRunner(directOptions);
-  }
-
-  /**
-   * Enable runtime testing to verify that all functions and {@link Coder}
-   * instances can be serialized.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withSerializabilityTesting(boolean enable) {
-    this.testSerializability = enable;
-    return this;
-  }
-
-  /**
-   * Enable runtime testing to verify that all values can be encoded.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withEncodabilityTesting(boolean enable) {
-    this.testEncodability = enable;
-    return this;
-  }
-
-  /**
-   * Enable runtime testing to verify that functions do not depend on order
-   * of the elements.
-   *
-   * <p>This is accomplished by randomizing the order of elements.
-   *
-   * <p>Enabled by default.
-   *
-   * <p>This method modifies the {@code DirectPipelineRunner} instance and
-   * returns itself.
-   */
-  public DirectPipelineRunner withUnorderednessTesting(boolean enable) {
-    this.testUnorderedness = enable;
-    return this;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    if (transform instanceof Combine.GroupedValues) {
-      return (OutputT) applyTestCombine((Combine.GroupedValues) transform, (PCollection) input);
-    } else if (transform instanceof TextIO.Write.Bound) {
-      return (OutputT) applyTextIOWrite((TextIO.Write.Bound) transform, (PCollection<?>) input);
-    } else if (transform instanceof AvroIO.Write.Bound) {
-      return (OutputT) applyAvroIOWrite((AvroIO.Write.Bound) transform, (PCollection<?>) input);
-    } else if (transform instanceof GroupByKey) {
-      return (OutputT)
-          ((PCollection) input).apply(new GroupByKeyViaGroupByKeyOnly((GroupByKey) transform));
-    } else if (transform instanceof Window.Bound) {
-      return (OutputT)
-          ((PCollection) input).apply(new AssignWindowsAndSetStrategy((Window.Bound) transform));
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-  private <K, InputT, AccumT, OutputT> PCollection<KV<K, OutputT>> applyTestCombine(
-      Combine.GroupedValues<K, InputT, OutputT> transform,
-      PCollection<KV<K, Iterable<InputT>>> input) {
-
-    PCollection<KV<K, OutputT>> output = input
-        .apply(ParDo.of(TestCombineDoFn.create(transform, input, testSerializability, rand))
-            .withSideInputs(transform.getSideInputs()));
-
-    try {
-      output.setCoder(transform.getDefaultOutputCoder(input));
-    } catch (CannotProvideCoderException exc) {
-      // let coder inference occur later, if it can
-    }
-    return output;
-  }
-
-  private static class ElementProcessingOrderPartitionFn<T> implements PartitionFn<T> {
-    private int elementNumber;
-    @Override
-    public int partitionFor(T elem, int numPartitions) {
-      return elementNumber++ % numPartitions;
-    }
-  }
-
-  /**
-   * Applies TextIO.Write honoring user requested sharding controls (i.e. withNumShards)
-   * by applying a partition function based upon the number of shards the user requested.
-   */
-  private static class DirectTextIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final TextIO.Write.Bound<T> transform;
-
-    private DirectTextIOWrite(TextIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      checkState(transform.getNumShards() > 1,
-          "DirectTextIOWrite is expected to only be used when sharding controls are required.");
-
-      // Evenly distribute all the elements across the partitions.
-      PCollectionList<T> partitionedElements =
-          input.apply(Partition.of(transform.getNumShards(),
-                                   new ElementProcessingOrderPartitionFn<T>()));
-
-      // For each input PCollection partition, create a write transform that represents
-      // one of the specific shards.
-      for (int i = 0; i < transform.getNumShards(); ++i) {
-        /*
-         * This logic mirrors the file naming strategy within
-         * {@link FileBasedSink#generateDestinationFilenames()}
-         */
-        String outputFilename = IOChannelUtils.constructName(
-            transform.getFilenamePrefix(),
-            transform.getShardNameTemplate(),
-            getFileExtension(transform.getFilenameSuffix()),
-            i,
-            transform.getNumShards());
-
-        String transformName = String.format("%s(Shard:%s)", transform.getName(), i);
-        partitionedElements.get(i).apply(transformName,
-            transform.withNumShards(1).withShardNameTemplate("").withSuffix("").to(outputFilename));
-      }
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  /**
-   * Returns the file extension to be used. If the user did not request a file
-   * extension then this method returns the empty string. Otherwise this method
-   * adds a {@code "."} to the beginning of the users extension if one is not present.
-   *
-   * <p>This is copied from {@link FileBasedSink} to not expose it.
-   */
-  private static String getFileExtension(String usersExtension) {
-    if (usersExtension == null || usersExtension.isEmpty()) {
-      return "";
-    }
-    if (usersExtension.startsWith(".")) {
-      return usersExtension;
-    }
-    return "." + usersExtension;
-  }
-
-  /**
-   * Apply the override for TextIO.Write.Bound if the user requested sharding controls
-   * greater than one.
-   */
-  private <T> PDone applyTextIOWrite(TextIO.Write.Bound<T> transform, PCollection<T> input) {
-    if (transform.getNumShards() <= 1) {
-      // By default, the DirectPipelineRunner outputs to only 1 shard. Since the user never
-      // requested sharding controls greater than 1, we default to outputting to 1 file.
-      return super.apply(transform.withNumShards(1), input);
-    }
-    return input.apply(new DirectTextIOWrite<>(transform));
-  }
-
-  /**
-   * Applies AvroIO.Write honoring user requested sharding controls (i.e. withNumShards)
-   * by applying a partition function based upon the number of shards the user requested.
-   */
-  private static class DirectAvroIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final AvroIO.Write.Bound<T> transform;
-
-    private DirectAvroIOWrite(AvroIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      checkState(transform.getNumShards() > 1,
-          "DirectAvroIOWrite is expected to only be used when sharding controls are required.");
-
-      // Evenly distribute all the elements across the partitions.
-      PCollectionList<T> partitionedElements =
-          input.apply(Partition.of(transform.getNumShards(),
-                                   new ElementProcessingOrderPartitionFn<T>()));
-
-      // For each input PCollection partition, create a write transform that represents
-      // one of the specific shards.
-      for (int i = 0; i < transform.getNumShards(); ++i) {
-        /*
-         * This logic mirrors the file naming strategy within
-         * {@link FileBasedSink#generateDestinationFilenames()}
-         */
-        String outputFilename = IOChannelUtils.constructName(
-            transform.getFilenamePrefix(),
-            transform.getShardNameTemplate(),
-            getFileExtension(transform.getFilenameSuffix()),
-            i,
-            transform.getNumShards());
-
-        String transformName = String.format("%s(Shard:%s)", transform.getName(), i);
-        partitionedElements.get(i).apply(transformName,
-            transform.withNumShards(1).withShardNameTemplate("").withSuffix("").to(outputFilename));
-      }
-      return PDone.in(input.getPipeline());
-    }
-  }
-
-  private static class AssignWindowsAndSetStrategy<T, W extends BoundedWindow>
-      extends PTransform<PCollection<T>, PCollection<T>> {
-
-    private final Window.Bound<T> wrapped;
-
-    public AssignWindowsAndSetStrategy(Window.Bound<T> wrapped) {
-      this.wrapped = wrapped;
-    }
-
-    @Override
-    public PCollection<T> apply(PCollection<T> input) {
-      WindowingStrategy<?, ?> outputStrategy =
-          wrapped.getOutputStrategyInternal(input.getWindowingStrategy());
-
-      WindowFn<T, BoundedWindow> windowFn =
-          (WindowFn<T, BoundedWindow>) outputStrategy.getWindowFn();
-
-      // If the Window.Bound transform only changed parts other than the WindowFn, then
-      // we skip AssignWindows even though it should be harmless in a perfect world.
-      // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly
-      // crash if another GBK is performed without explicitly setting the WindowFn. So we skip
-      // AssignWindows in this case.
-      if (wrapped.getWindowFn() == null) {
-        return input.apply("Identity", ParDo.of(new IdentityFn<T>()))
-            .setWindowingStrategyInternal(outputStrategy);
-      } else {
-        return input
-            .apply("AssignWindows", new AssignWindows<T, BoundedWindow>(windowFn))
-            .setWindowingStrategyInternal(outputStrategy);
-      }
-    }
-  }
-
-  private static class IdentityFn<T> extends DoFn<T, T> {
-    @Override
-    public void processElement(ProcessContext c) {
-      c.output(c.element());
-    }
-  }
-
-  /**
-   * Apply the override for AvroIO.Write.Bound if the user requested sharding controls
-   * greater than one.
-   */
-  private <T> PDone applyAvroIOWrite(AvroIO.Write.Bound<T> transform, PCollection<T> input) {
-    if (transform.getNumShards() <= 1) {
-      // By default, the DirectPipelineRunner outputs to only 1 shard. Since the user never
-      // requested sharding controls greater than 1, we default to outputting to 1 file.
-      return super.apply(transform.withNumShards(1), input);
-    }
-    return input.apply(new DirectAvroIOWrite<>(transform));
-  }
-
-  /**
-   * The implementation may split the {@link KeyedCombineFn} into ADD, MERGE and EXTRACT phases (
-   * see {@code org.apache.beam.sdk.runners.worker.CombineValuesFn}). In order to emulate
-   * this for the {@link DirectPipelineRunner} and provide an experience closer to the service, go
-   * through heavy serializability checks for the equivalent of the results of the ADD phase, but
-   * after the {@link org.apache.beam.sdk.transforms.GroupByKey} shuffle, and the MERGE
-   * phase. Doing these checks ensure that not only is the accumulator coder serializable, but
-   * the accumulator coder can actually serialize the data in question.
-   */
-  public static class TestCombineDoFn<K, InputT, AccumT, OutputT>
-      extends DoFn<KV<K, Iterable<InputT>>, KV<K, OutputT>> {
-    private final PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, OutputT> fnRunner;
-    private final Coder<AccumT> accumCoder;
-    private final boolean testSerializability;
-    private final Random rand;
-
-    public static <K, InputT, AccumT, OutputT> TestCombineDoFn<K, InputT, AccumT, OutputT> create(
-        Combine.GroupedValues<K, InputT, OutputT> transform,
-        PCollection<KV<K, Iterable<InputT>>> input,
-        boolean testSerializability,
-        Random rand) {
-
-      AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn = transform.getAppliedFn(
-          input.getPipeline().getCoderRegistry(), input.getCoder(), input.getWindowingStrategy());
-
-      return new TestCombineDoFn(
-          PerKeyCombineFnRunners.create(fn.getFn()),
-          fn.getAccumulatorCoder(),
-          testSerializability,
-          rand);
-    }
-
-    public TestCombineDoFn(
-        PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, OutputT> fnRunner,
-        Coder<AccumT> accumCoder,
-        boolean testSerializability,
-        Random rand) {
-      this.fnRunner = fnRunner;
-      this.accumCoder = accumCoder;
-      this.testSerializability = testSerializability;
-      this.rand = rand;
-
-      // Check that this does not crash, specifically to catch anonymous CustomCoder subclasses.
-      this.accumCoder.getEncodingId();
-    }
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      K key = c.element().getKey();
-      Iterable<InputT> values = c.element().getValue();
-      List<AccumT> groupedPostShuffle =
-          ensureSerializableByCoder(ListCoder.of(accumCoder),
-              addInputsRandomly(fnRunner, key, values, rand, c),
-              "After addInputs of KeyedCombineFn " + fnRunner.fn().toString());
-      AccumT merged =
-          ensureSerializableByCoder(accumCoder,
-            fnRunner.mergeAccumulators(key, groupedPostShuffle, c),
-            "After mergeAccumulators of KeyedCombineFn " + fnRunner.fn().toString());
-      // Note: The serializability of KV<K, OutputT> is ensured by the
-      // runner itself, since it's a transform output.
-      c.output(KV.of(key, fnRunner.extractOutput(key, merged, c)));
-    }
-
-    /**
-     * Create a random list of accumulators from the given list of values.
-     *
-     * <p>Visible for testing purposes only.
-     */
-    public static <K, AccumT, InputT> List<AccumT> addInputsRandomly(
-        PerKeyCombineFnRunner<? super K, ? super InputT, AccumT, ?> fnRunner,
-        K key,
-        Iterable<InputT> values,
-        Random random,
-        DoFn<?, ?>.ProcessContext c) {
-      List<AccumT> out = new ArrayList<AccumT>();
-      int i = 0;
-      AccumT accumulator = fnRunner.createAccumulator(key, c);
-      boolean hasInput = false;
-
-      for (InputT value : values) {
-        accumulator = fnRunner.addInput(key, accumulator, value, c);
-        hasInput = true;
-
-        // For each index i, flip a 1/2^i weighted coin for whether to
-        // create a new accumulator after index i is added, i.e. [0]
-        // is guaranteed, [1] is an even 1/2, [2] is 1/4, etc. The
-        // goal is to partition the inputs into accumulators, and make
-        // the accumulators potentially lumpy.  Also compact about half
-        // of the accumulators.
-        if (i == 0 || random.nextInt(1 << Math.min(i, 30)) == 0) {
-          if (i % 2 == 0) {
-            accumulator = fnRunner.compact(key, accumulator, c);
-          }
-          out.add(accumulator);
-          accumulator = fnRunner.createAccumulator(key, c);
-          hasInput = false;
-        }
-        i++;
-      }
-      if (hasInput) {
-        out.add(accumulator);
-      }
-
-      Collections.shuffle(out, random);
-      return out;
-    }
-
-    public <T> T ensureSerializableByCoder(
-        Coder<T> coder, T value, String errorContext) {
-      if (testSerializability) {
-        return SerializableUtils.ensureSerializableByCoder(
-            coder, value, errorContext);
-      }
-      return value;
-    }
-  }
-
-  @Override
-  public EvaluationResults run(Pipeline pipeline) {
-    LOG.info("Executing pipeline using the DirectPipelineRunner.");
-
-    Evaluator evaluator = new Evaluator(rand);
-    evaluator.run(pipeline);
-
-    // Log all counter values for debugging purposes.
-    for (Counter counter : evaluator.getCounters()) {
-      LOG.info("Final aggregator value: {}", counter);
-    }
-
-    LOG.info("Pipeline execution complete.");
-
-    return evaluator;
-  }
-
-  /**
-   * An evaluator of a PTransform.
-   */
-  public interface TransformEvaluator<TransformT extends PTransform> {
-    public void evaluate(TransformT transform,
-                         EvaluationContext context);
-  }
-
-  /**
-   * The interface provided to registered callbacks for interacting
-   * with the {@code DirectPipelineRunner}, including reading and writing the
-   * values of {@link PCollection}s and {@link PCollectionView}s.
-   */
-  public interface EvaluationResults extends PipelineResult {
-    /**
-     * Retrieves the value of the given PCollection.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<T> getPCollection(PCollection<T> pc);
-
-    /**
-     * Retrieves the windowed value of the given PCollection.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<WindowedValue<T>> getPCollectionWindowedValues(PCollection<T> pc);
-
-    /**
-     * Retrieves the values of each PCollection in the given
-     * PCollectionList. Throws an exception if the PCollectionList's
-     * value hasn't already been set.
-     */
-    <T> List<List<T>> getPCollectionList(PCollectionList<T> pcs);
-
-    /**
-     * Retrieves the values indicated by the given {@link PCollectionView}.
-     * Note that within the {@link org.apache.beam.sdk.transforms.DoFn.Context}
-     * implementation a {@link PCollectionView} should convert from this representation to a
-     * suitable side input value.
-     */
-    <T, WindowedT> Iterable<WindowedValue<?>> getPCollectionView(PCollectionView<T> view);
-  }
-
-  /**
-   * An immutable (value, timestamp) pair, along with other metadata necessary
-   * for the implementation of {@code DirectPipelineRunner}.
-   */
-  public static class ValueWithMetadata<V> {
-    /**
-     * Returns a new {@code ValueWithMetadata} with the {@code WindowedValue}.
-     * Key is null.
-     */
-    public static <V> ValueWithMetadata<V> of(WindowedValue<V> windowedValue) {
-      return new ValueWithMetadata<>(windowedValue, null);
-    }
-
-    /**
-     * Returns a new {@code ValueWithMetadata} with the implicit key associated
-     * with this value set.  The key is the last key grouped by in the chain of
-     * productions that produced this element.
-     * These keys are used internally by {@link DirectPipelineRunner} for keeping
-     * persisted state separate across keys.
-     */
-    public ValueWithMetadata<V> withKey(Object key) {
-      return new ValueWithMetadata<>(windowedValue, key);
-    }
-
-    /**
-     * Returns a new {@code ValueWithMetadata} that is a copy of this one, but with
-     * a different value.
-     */
-    public <T> ValueWithMetadata<T> withValue(T value) {
-      return new ValueWithMetadata(windowedValue.withValue(value), getKey());
-    }
-
-    /**
-     * Returns the {@code WindowedValue} associated with this element.
-     */
-    public WindowedValue<V> getWindowedValue() {
-      return windowedValue;
-    }
-
-    /**
-     * Returns the value associated with this element.
-     *
-     * @see #withValue
-     */
-    public V getValue() {
-      return windowedValue.getValue();
-    }
-
-    /**
-     * Returns the timestamp associated with this element.
-     */
-    public Instant getTimestamp() {
-      return windowedValue.getTimestamp();
-    }
-
-    /**
-     * Returns the collection of windows this element has been placed into.  May
-     * be null if the {@code PCollection} this element is in has not yet been
-     * windowed.
-     *
-     * @see #getWindows()
-     */
-    public Collection<? extends BoundedWindow> getWindows() {
-      return windowedValue.getWindows();
-    }
-
-
-    /**
-     * Returns the key associated with this element.  May be null if the
-     * {@code PCollection} this element is in is not keyed.
-     *
-     * @see #withKey
-     */
-    public Object getKey() {
-      return key;
-    }
-
-    ////////////////////////////////////////////////////////////////////////////
-
-  private final Object key;
-    private final WindowedValue<V> windowedValue;
-
-    private ValueWithMetadata(WindowedValue<V> windowedValue,
-                              Object key) {
-      this.windowedValue = windowedValue;
-      this.key = key;
-    }
-  }
-
-  /**
-   * The interface provided to registered callbacks for interacting
-   * with the {@code DirectPipelineRunner}, including reading and writing the
-   * values of {@link PCollection}s and {@link PCollectionView}s.
-   */
-  public interface EvaluationContext extends EvaluationResults {
-    /**
-     * Returns the configured pipeline options.
-     */
-    DirectPipelineOptions getPipelineOptions();
-
-    /**
-     * Returns the input of the currently being processed transform.
-     */
-    <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
-
-    /**
-     * Returns the output of the currently being processed transform.
-     */
-    <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
-
-    /**
-     * Sets the value of the given PCollection, where each element also has a timestamp
-     * and collection of windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollectionValuesWithMetadata(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements);
-
-    /**
-     * Sets the value of the given PCollection, where each element also has a timestamp
-     * and collection of windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollectionWindowedValue(PCollection<T> pc, List<WindowedValue<T>> elements);
-
-    /**
-     * Shorthand for setting the value of a PCollection where the elements do not have
-     * timestamps or windows.
-     * Throws an exception if the PCollection's value has already been set.
-     */
-    <T> void setPCollection(PCollection<T> pc, List<T> elements);
-
-    /**
-     * Retrieves the value of the given PCollection, along with element metadata
-     * such as timestamps and windows.
-     * Throws an exception if the PCollection's value hasn't already been set.
-     */
-    <T> List<ValueWithMetadata<T>> getPCollectionValuesWithMetadata(PCollection<T> pc);
-
-    /**
-     * Sets the value associated with the given {@link PCollectionView}.
-     * Throws an exception if the {@link PCollectionView}'s value has already been set.
-     */
-    <ElemT, T, WindowedT> void setPCollectionView(
-        PCollectionView<T> pc,
-        Iterable<WindowedValue<ElemT>> value);
-
-    /**
-     * Ensures that the element is encodable and decodable using the
-     * TypePValue's coder, by encoding it and decoding it, and
-     * returning the result.
-     */
-    <T> T ensureElementEncodable(TypedPValue<T> pvalue, T element);
-
-    /**
-     * If the evaluation context is testing unorderedness,
-     * randomly permutes the order of the elements, in a
-     * copy if !inPlaceAllowed, and returns the permuted list,
-     * otherwise returns the argument unchanged.
-     */
-    <T> List<T> randomizeIfUnordered(List<T> elements,
-                                     boolean inPlaceAllowed);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the argument function is serializable and deserializable
-     * by encoding it and then decoding it, and returning the result.
-     * Otherwise returns the argument unchanged.
-     */
-    <FunctionT extends Serializable> FunctionT ensureSerializable(FunctionT fn);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the argument Coder is serializable and deserializable
-     * by encoding it and then decoding it, and returning the result.
-     * Otherwise returns the argument unchanged.
-     */
-    <T> Coder<T> ensureCoderSerializable(Coder<T> coder);
-
-    /**
-     * If the evaluation context is testing serializability, ensures
-     * that the given data is serializable and deserializable with the
-     * given Coder by encoding it and then decoding it, and returning
-     * the result. Otherwise returns the argument unchanged.
-     *
-     * <p>Error context is prefixed to any thrown exceptions.
-     */
-    <T> T ensureSerializableByCoder(Coder<T> coder,
-                                    T data, String errorContext);
-
-    /**
-     * Returns a mutator, which can be used to add additional counters to
-     * this EvaluationContext.
-     */
-    CounterSet.AddCounterMutator getAddCounterMutator();
-
-    /**
-     * Gets the step name for this transform.
-     */
-    public String getStepName(PTransform<?, ?> transform);
-  }
-
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  class Evaluator extends PipelineVisitor.Defaults implements EvaluationContext {
-    /**
-     * A map from PTransform to the step name of that transform. This is the internal name for the
-     * transform (e.g. "s2").
-     */
-    private final Map<PTransform<?, ?>, String> stepNames = new HashMap<>();
-    private final Map<PValue, Object> store = new HashMap<>();
-    private final CounterSet counters = new CounterSet();
-    private AppliedPTransform<?, ?, ?> currentTransform;
-
-    private Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps = null;
-
-    /**
-     * A map from PTransform to the full name of that transform. This is the user name of the
-     * transform (e.g. "RemoveDuplicates/Combine/GroupByKey").
-     */
-    private final Map<PTransform<?, ?>, String> fullNames = new HashMap<>();
-
-    private Random rand;
-
-    public Evaluator() {
-      this(new Random());
-    }
-
-    public Evaluator(Random rand) {
-      this.rand = rand;
-    }
-
-    public void run(Pipeline pipeline) {
-      pipeline.traverseTopologically(this);
-      aggregatorSteps = new AggregatorPipelineExtractor(pipeline).getAggregatorSteps();
-    }
-
-    @Override
-    public DirectPipelineOptions getPipelineOptions() {
-      return options;
-    }
-
-    @Override
-    public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
-      checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-          "can only be called with current transform");
-      return (InputT) currentTransform.getInput();
-    }
-
-    @Override
-    public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
-      checkArgument(currentTransform != null && currentTransform.getTransform() == transform,
-          "can only be called with current transform");
-      return (OutputT) currentTransform.getOutput();
-    }
-
-    @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
-      PTransform<?, ?> transform = node.getTransform();
-      fullNames.put(transform, node.getFullName());
-      TransformEvaluator evaluator =
-          getTransformEvaluator(transform.getClass());
-      if (evaluator == null) {
-        throw new IllegalStateException(
-            "no evaluator registered for " + transform);
-      }
-      LOG.debug("Evaluating {}", transform);
-      currentTransform = AppliedPTransform.of(
-          node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
-      evaluator.evaluate(transform, this);
-      currentTransform = null;
-    }
-
-    @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
-      LOG.debug("Checking evaluation of {}", value);
-      if (value.getProducingTransformInternal() == null) {
-        throw new RuntimeException(
-            "internal error: expecting a PValue to have a producingTransform");
-      }
-      if (!producer.isCompositeNode()) {
-        // Verify that primitive transform outputs are already computed.
-        getPValue(value);
-      }
-    }
-
-    /**
-     * Sets the value of the given PValue.
-     * Throws an exception if the PValue's value has already been set.
-     */
-    void setPValue(PValue pvalue, Object contents) {
-      if (store.containsKey(pvalue)) {
-        throw new IllegalStateException(
-            "internal error: setting the value of " + pvalue
-            + " more than once");
-      }
-      store.put(pvalue, contents);
-    }
-
-    /**
-     * Retrieves the value of the given PValue.
-     * Throws an exception if the PValue's value hasn't already been set.
-     */
-    Object getPValue(PValue pvalue) {
-      if (!store.containsKey(pvalue)) {
-        throw new IllegalStateException(
-            "internal error: getting the value of " + pvalue
-            + " before it has been computed");
-      }
-      return store.get(pvalue);
-    }
-
-    /**
-     * Convert a list of T to a list of {@code ValueWithMetadata<T>}, with a timestamp of 0
-     * and null windows.
-     */
-    <T> List<ValueWithMetadata<T>> toValueWithMetadata(List<T> values) {
-      List<ValueWithMetadata<T>> result = new ArrayList<>(values.size());
-      for (T value : values) {
-        result.add(ValueWithMetadata.of(WindowedValue.valueInGlobalWindow(value)));
-      }
-      return result;
-    }
-
-    /**
-     * Convert a list of {@code WindowedValue<T>} to a list of {@code ValueWithMetadata<T>}.
-     */
-    <T> List<ValueWithMetadata<T>> toValueWithMetadataFromWindowedValue(
-        List<WindowedValue<T>> values) {
-      List<ValueWithMetadata<T>> result = new ArrayList<>(values.size());
-      for (WindowedValue<T> value : values) {
-        result.add(ValueWithMetadata.of(value));
-      }
-      return result;
-    }
-
-    @Override
-    public <T> void setPCollection(PCollection<T> pc, List<T> elements) {
-      setPCollectionValuesWithMetadata(pc, toValueWithMetadata(elements));
-    }
-
-    @Override
-    public <T> void setPCollectionWindowedValue(
-        PCollection<T> pc, List<WindowedValue<T>> elements) {
-      setPCollectionValuesWithMetadata(pc, toValueWithMetadataFromWindowedValue(elements));
-    }
-
-    @Override
-    public <T> void setPCollectionValuesWithMetadata(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements) {
-      LOG.debug("Setting {} = {}", pc, elements);
-      ensurePCollectionEncodable(pc, elements);
-      setPValue(pc, elements);
-    }
-
-    @Override
-    public <ElemT, T, WindowedT> void setPCollectionView(
-        PCollectionView<T> view,
-        Iterable<WindowedValue<ElemT>> value) {
-      LOG.debug("Setting {} = {}", view, value);
-      setPValue(view, value);
-    }
-
-    /**
-     * Retrieves the value of the given {@link PCollection}.
-     * Throws an exception if the {@link PCollection}'s value hasn't already been set.
-     */
-    @Override
-    public <T> List<T> getPCollection(PCollection<T> pc) {
-      List<T> result = new ArrayList<>();
-      for (ValueWithMetadata<T> elem : getPCollectionValuesWithMetadata(pc)) {
-        result.add(elem.getValue());
-      }
-      return result;
-    }
-
-    @Override
-    public <T> List<WindowedValue<T>> getPCollectionWindowedValues(PCollection<T> pc) {
-      return Lists.transform(
-          getPCollectionValuesWithMetadata(pc),
-          new Function<ValueWithMetadata<T>, WindowedValue<T>>() {
-            @Override
-            public WindowedValue<T> apply(ValueWithMetadata<T> input) {
-              return input.getWindowedValue();
-            }});
-    }
-
-    @Override
-    public <T> List<ValueWithMetadata<T>> getPCollectionValuesWithMetadata(PCollection<T> pc) {
-      List<ValueWithMetadata<T>> elements = (List<ValueWithMetadata<T>>) getPValue(pc);
-      elements = randomizeIfUnordered(elements, false /* not inPlaceAllowed */);
-      LOG.debug("Getting {} = {}", pc, elements);
-      return elements;
-    }
-
-    @Override
-    public <T> List<List<T>> getPCollectionList(PCollectionList<T> pcs) {
-      List<List<T>> elementsList = new ArrayList<>();
-      for (PCollection<T> pc : pcs.getAll()) {
-        elementsList.add(getPCollection(pc));
-      }
-      return elementsList;
-    }
-
-    /**
-     * Retrieves the value indicated by the given {@link PCollectionView}.
-     * Note that within the {@link DoFnContext} a {@link PCollectionView}
-     * converts from this representation to a suitable side input value.
-     */
-    @Override
-    public <T, WindowedT> Iterable<WindowedValue<?>> getPCollectionView(PCollectionView<T> view) {
-      Iterable<WindowedValue<?>> value = (Iterable<WindowedValue<?>>) getPValue(view);
-      LOG.debug("Getting {} = {}", view, value);
-      return value;
-    }
-
-    /**
-     * If {@code testEncodability}, ensures that the {@link PCollection}'s coder and elements are
-     * encodable and decodable by encoding them and decoding them, and returning the result.
-     * Otherwise returns the argument elements.
-     */
-    <T> List<ValueWithMetadata<T>> ensurePCollectionEncodable(
-        PCollection<T> pc, List<ValueWithMetadata<T>> elements) {
-      ensureCoderSerializable(pc.getCoder());
-      if (!testEncodability) {
-        return elements;
-      }
-      List<ValueWithMetadata<T>> elementsCopy = new ArrayList<>(elements.size());
-      for (ValueWithMetadata<T> element : elements) {
-        elementsCopy.add(
-            element.withValue(ensureElementEncodable(pc, element.getValue())));
-      }
-      return elementsCopy;
-    }
-
-    @Override
-    public <T> T ensureElementEncodable(TypedPValue<T> pvalue, T element) {
-      return ensureSerializableByCoder(
-          pvalue.getCoder(), element, "Within " + pvalue.toString());
-    }
-
-    @Override
-    public <T> List<T> randomizeIfUnordered(List<T> elements,
-                                            boolean inPlaceAllowed) {
-      if (!testUnorderedness) {
-        return elements;
-      }
-      List<T> elementsCopy = new ArrayList<>(elements);
-      Collections.shuffle(elementsCopy, rand);
-      return elementsCopy;
-    }
-
-    @Override
-    public <FunctionT extends Serializable> FunctionT ensureSerializable(FunctionT fn) {
-      if (!testSerializability) {
-        return fn;
-      }
-      return SerializableUtils.ensureSerializable(fn);
-    }
-
-    @Override
-    public <T> Coder<T> ensureCoderSerializable(Coder<T> coder) {
-      if (testSerializability) {
-        SerializableUtils.ensureSerializable(coder);
-      }
-      return coder;
-    }
-
-    @Override
-    public <T> T ensureSerializableByCoder(
-        Coder<T> coder, T value, String errorContext) {
-      if (testSerializability) {
-        return SerializableUtils.ensureSerializableByCoder(
-            coder, value, errorContext);
-      }
-      return value;
-    }
-
-    @Override
-    public CounterSet.AddCounterMutator getAddCounterMutator() {
-      return counters.getAddCounterMutator();
-    }
-
-    @Override
-    public String getStepName(PTransform<?, ?> transform) {
-      String stepName = stepNames.get(transform);
-      if (stepName == null) {
-        stepName = "s" + (stepNames.size() + 1);
-        stepNames.put(transform, stepName);
-      }
-      return stepName;
-    }
-
-    /**
-     * Returns the CounterSet generated during evaluation, which includes
-     * user-defined Aggregators and may include system-defined counters.
-     */
-    public CounterSet getCounters() {
-      return counters;
-    }
-
-    /**
-     * Returns JobState.DONE in all situations. The Evaluator is not returned
-     * until the pipeline has been traversed, so it will either be returned
-     * after a successful run or the run call will terminate abnormally.
-     */
-    @Override
-    public State getState() {
-      return State.DONE;
-    }
-
-    @Override
-    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator) {
-      Map<String, T> stepValues = new HashMap<>();
-      for (PTransform<?, ?> step : aggregatorSteps.get(aggregator)) {
-        String stepName = String.format("user-%s-%s", stepNames.get(step), aggregator.getName());
-        String fullName = fullNames.get(step);
-        Counter<?> counter = counters.getExistingCounter(stepName);
-        if (counter == null) {
-          throw new IllegalArgumentException(
-              "Aggregator " + aggregator + " is not used in this pipeline");
-        }
-        stepValues.put(fullName, (T) counter.getAggregate());
-      }
-      return new MapAggregatorValues<>(stepValues);
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * The key by which GBK groups inputs - elements are grouped by the encoded form of the key,
-   * but the original key may be accessed as well.
-   */
-  private static class GroupingKey<K> {
-    private K key;
-    private byte[] encodedKey;
-
-    public GroupingKey(K key, byte[] encodedKey) {
-      this.key = key;
-      this.encodedKey = encodedKey;
-    }
-
-    public K getKey() {
-      return key;
-    }
-
-    @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);
-    }
-  }
-
-  private final DirectPipelineOptions options;
-  private boolean testSerializability;
-  private boolean testEncodability;
-  private boolean testUnorderedness;
-
-  /** Returns a new DirectPipelineRunner. */
-  private DirectPipelineRunner(DirectPipelineOptions options) {
-    this.options = options;
-    // (Re-)register standard IO factories. Clobbers any prior credentials.
-    IOChannelUtils.registerStandardIOFactories(options);
-    long randomSeed;
-    if (options.getDirectPipelineRunnerRandomSeed() != null) {
-      randomSeed = options.getDirectPipelineRunnerRandomSeed();
-    } else {
-      randomSeed = new Random().nextLong();
-    }
-
-    LOG.debug("DirectPipelineRunner using random seed {}.", randomSeed);
-    rand = new Random(randomSeed);
-
-    testSerializability = options.isTestSerializability();
-    testEncodability = options.isTestEncodability();
-    testUnorderedness = options.isTestUnorderedness();
-  }
-
-  /**
-   * Get the options used in this {@link Pipeline}.
-   */
-  public DirectPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public String toString() {
-    return "DirectPipelineRunner#" + hashCode();
-  }
-
-  public static <K, V> void evaluateGroupByKeyOnly(
-      GroupByKeyOnly<K, V> transform,
-      EvaluationContext context) {
-    PCollection<KV<K, V>> input = context.getInput(transform);
-
-    List<ValueWithMetadata<KV<K, V>>> inputElems =
-        context.getPCollectionValuesWithMetadata(input);
-
-    Coder<K> keyCoder = GroupByKey.getKeyCoder(input.getCoder());
-
-    Map<GroupingKey<K>, List<V>> groupingMap = new HashMap<>();
-
-    for (ValueWithMetadata<KV<K, V>> elem : inputElems) {
-      K key = elem.getValue().getKey();
-      V value = elem.getValue().getValue();
-      byte[] encodedKey;
-      try {
-        encodedKey = encodeToByteArray(keyCoder, key);
-      } catch (CoderException exn) {
-        // TODO: Put in better element printing:
-        // truncate if too long.
-        throw new IllegalArgumentException(
-            "unable to encode key " + key + " of input to " + transform
-            + " using " + keyCoder,
-            exn);
-      }
-      GroupingKey<K> groupingKey =
-          new GroupingKey<>(key, encodedKey);
-      List<V> values = groupingMap.get(groupingKey);
-      if (values == null) {
-        values = new ArrayList<V>();
-        groupingMap.put(groupingKey, values);
-      }
-      values.add(value);
-    }
-
-    List<ValueWithMetadata<KV<K, Iterable<V>>>> outputElems =
-        new ArrayList<>();
-    for (Map.Entry<GroupingKey<K>, List<V>> entry : groupingMap.entrySet()) {
-      GroupingKey<K> groupingKey = entry.getKey();
-      K key = groupingKey.getKey();
-      List<V> values = entry.getValue();
-      values = context.randomizeIfUnordered(values, true /* inPlaceAllowed */);
-      outputElems.add(ValueWithMetadata
-                      .of(WindowedValue.valueInEmptyWindows(KV.<K, Iterable<V>>of(key, values)))
-                      .withKey(key));
-    }
-
-    context.setPCollectionValuesWithMetadata(context.getOutput(transform),
-                                             outputElems);
-  }
-
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  public
-  static <K, V> void registerGroupByKeyOnly() {
-    registerDefaultTransformEvaluator(
-        GroupByKeyOnly.class,
-        new TransformEvaluator<GroupByKeyOnly>() {
-          @Override
-          public void evaluate(
-              GroupByKeyOnly transform,
-              EvaluationContext context) {
-            evaluateGroupByKeyOnly(transform, context);
-          }
-        });
-  }
-
-  static {
-    registerGroupByKeyOnly();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
index 7c6fed3..93917f3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java
@@ -20,16 +20,12 @@ package org.apache.beam.sdk.transforms;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableLikeCoder;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionList;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * {@code Flatten<T>} takes multiple {@code PCollection<T>}s bundled
  * into a {@code PCollectionList<T>} and returns a single
@@ -189,32 +185,4 @@ public class Flatten {
           .setCoder(elemCoder);
     }
   }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  static {
-    DirectPipelineRunner.registerDefaultTransformEvaluator(
-        FlattenPCollectionList.class,
-        new DirectPipelineRunner.TransformEvaluator<FlattenPCollectionList>() {
-          @Override
-          public void evaluate(
-              FlattenPCollectionList transform,
-              DirectPipelineRunner.EvaluationContext context) {
-            evaluateHelper(transform, context);
-          }
-        });
-  }
-
-  private static <T> void evaluateHelper(
-      FlattenPCollectionList<T> transform,
-      DirectPipelineRunner.EvaluationContext context) {
-    List<DirectPipelineRunner.ValueWithMetadata<T>> outputElems = new ArrayList<>();
-    PCollectionList<T> inputs = context.getInput(transform);
-
-    for (PCollection<T> input : inputs.getAll()) {
-      outputElems.addAll(context.getPCollectionValuesWithMetadata(input));
-    }
-
-    context.setPCollectionValuesWithMetadata(context.getOutput(transform), outputElems);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index 511f0d8..cb7d372 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -21,27 +21,12 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.DirectModeExecutionContext;
-import org.apache.beam.sdk.util.DirectSideInputReader;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunnerBase;
-import org.apache.beam.sdk.util.DoFnRunners;
-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.PTuple;
 import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.StringUtils;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
@@ -50,16 +35,10 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.TypedPValue;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
 
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nullable;
 
 /**
  * {@link ParDo} is the core element-wise transform in Google Cloud
@@ -84,7 +63,7 @@ import javax.annotation.Nullable;
  * <p>Conceptually, when a {@link ParDo} transform is executed, the
  * elements of the input {@link PCollection} are first divided up
  * into some number of "bundles". These are farmed off to distributed
- * worker machines (or run locally, if using the {@link DirectPipelineRunner}).
+ * worker machines (or run locally, if using the {@code DirectRunner}).
  * For each bundle of input elements processing proceeds as follows:
  *
  * <ol>
@@ -1072,288 +1051,11 @@ public class ParDo {
     }
   }
 
-  /////////////////////////////////////////////////////////////////////////////
-
-  static {
-    DirectPipelineRunner.registerDefaultTransformEvaluator(
-        Bound.class,
-        new DirectPipelineRunner.TransformEvaluator<Bound>() {
-          @Override
-          public void evaluate(
-              Bound transform,
-              DirectPipelineRunner.EvaluationContext context) {
-            evaluateSingleHelper(transform, context);
-          }
-        });
-  }
-
-  private static <InputT, OutputT> void evaluateSingleHelper(
-      Bound<InputT, OutputT> transform,
-      DirectPipelineRunner.EvaluationContext context) {
-    TupleTag<OutputT> mainOutputTag = new TupleTag<>("out");
-
-    DirectModeExecutionContext executionContext = DirectModeExecutionContext.create();
-
-    PCollectionTuple outputs = PCollectionTuple.of(mainOutputTag, context.getOutput(transform));
-
-    evaluateHelper(
-        transform.fn,
-        context.getStepName(transform),
-        context.getInput(transform),
-        transform.sideInputs,
-        mainOutputTag,
-        Collections.<TupleTag<?>>emptyList(),
-        outputs,
-        context,
-        executionContext);
-
-    context.setPCollectionValuesWithMetadata(
-        context.getOutput(transform),
-        executionContext.getOutput(mainOutputTag));
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  static {
-    DirectPipelineRunner.registerDefaultTransformEvaluator(
-        BoundMulti.class,
-        new DirectPipelineRunner.TransformEvaluator<BoundMulti>() {
-          @Override
-          public void evaluate(
-              BoundMulti transform,
-              DirectPipelineRunner.EvaluationContext context) {
-            evaluateMultiHelper(transform, context);
-          }
-        });
-  }
-
-  private static <InputT, OutputT> void evaluateMultiHelper(
-      BoundMulti<InputT, OutputT> transform,
-      DirectPipelineRunner.EvaluationContext context) {
-
-    DirectModeExecutionContext executionContext = DirectModeExecutionContext.create();
-
-    evaluateHelper(
-        transform.fn,
-        context.getStepName(transform),
-        context.getInput(transform),
-        transform.sideInputs,
-        transform.mainOutputTag,
-        transform.sideOutputTags.getAll(),
-        context.getOutput(transform),
-        context,
-        executionContext);
-
-    for (Map.Entry<TupleTag<?>, PCollection<?>> entry
-        : context.getOutput(transform).getAll().entrySet()) {
-      @SuppressWarnings("unchecked")
-      TupleTag<Object> tag = (TupleTag<Object>) entry.getKey();
-      @SuppressWarnings("unchecked")
-      PCollection<Object> pc = (PCollection<Object>) entry.getValue();
-
-      context.setPCollectionValuesWithMetadata(
-          pc,
-          (tag == transform.mainOutputTag
-              ? executionContext.getOutput(tag)
-              : executionContext.getSideOutput(tag)));
-    }
-  }
-
-  /**
-   * Evaluates a single-output or multi-output {@link ParDo} directly.
-   *
-   * <p>This evaluation method is intended for use in testing scenarios; it is designed for clarity
-   * and correctness-checking, not speed.
-   *
-   * <p>Of particular note, this performs best-effort checking that inputs and outputs are not
-   * mutated in violation of the requirements upon a {@link DoFn}.
-   */
-  private static <InputT, OutputT, ActualInputT extends InputT> void evaluateHelper(
-      DoFn<InputT, OutputT> doFn,
-      String stepName,
-      PCollection<ActualInputT> input,
-      List<PCollectionView<?>> sideInputs,
-      TupleTag<OutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      PCollectionTuple outputs,
-      DirectPipelineRunner.EvaluationContext context,
-      DirectModeExecutionContext executionContext) {
-    // TODO: Run multiple shards?
-    DoFn<InputT, OutputT> fn = context.ensureSerializable(doFn);
-
-    SideInputReader sideInputReader = makeSideInputReader(context, sideInputs);
-
-    // When evaluating via the DirectPipelineRunner, this output manager checks each output for
-    // illegal mutations when the next output comes along. We then verify again after finishBundle()
-    // The common case we expect this to catch is a user mutating an input in order to repeatedly
-    // emit "variations".
-    ImmutabilityCheckingOutputManager<ActualInputT> outputManager =
-        new ImmutabilityCheckingOutputManager<>(
-            fn.getClass().getSimpleName(),
-            new DoFnRunnerBase.ListOutputManager(),
-            outputs);
-
-    DoFnRunner<InputT, OutputT> fnRunner =
-        DoFnRunners.createDefault(
-            context.getPipelineOptions(),
-            fn,
-            sideInputReader,
-            outputManager,
-            mainOutputTag,
-            sideOutputTags,
-            executionContext.getOrCreateStepContext(stepName, stepName),
-            context.getAddCounterMutator(),
-            input.getWindowingStrategy());
-
-    fnRunner.startBundle();
-
-    for (DirectPipelineRunner.ValueWithMetadata<ActualInputT> elem
-             : context.getPCollectionValuesWithMetadata(input)) {
-      if (elem.getValue() instanceof KV) {
-        // In case the DoFn needs keyed state, set the implicit keys to the keys
-        // in the input elements.
-        @SuppressWarnings("unchecked")
-        KV<?, ?> kvElem = (KV<?, ?>) elem.getValue();
-        executionContext.setKey(kvElem.getKey());
-      } else {
-        executionContext.setKey(elem.getKey());
-      }
-
-      // We check the input for mutations only through the call span of processElement.
-      // This will miss some cases, but the check is ad hoc and best effort. The common case
-      // is that the input is mutated to be used for output.
-      try {
-        MutationDetector inputMutationDetector = MutationDetectors.forValueWithCoder(
-            elem.getWindowedValue().getValue(), input.getCoder());
-        @SuppressWarnings("unchecked")
-        WindowedValue<InputT> windowedElem = ((WindowedValue<InputT>) elem.getWindowedValue());
-        fnRunner.processElement(windowedElem);
-        inputMutationDetector.verifyUnmodified();
-      } catch (CoderException e) {
-        throw UserCodeException.wrap(e);
-      } catch (IllegalMutationException exn) {
-        throw new IllegalMutationException(
-            String.format("DoFn %s mutated input value %s of class %s (new value was %s)."
-                + " Input values must not be mutated in any way.",
-                fn.getClass().getSimpleName(),
-                exn.getSavedValue(), exn.getSavedValue().getClass(), exn.getNewValue()),
-            exn.getSavedValue(),
-            exn.getNewValue(),
-            exn);
-      }
-    }
-
-    // Note that the input could have been retained and mutated prior to this final output,
-    // but for now it degrades readability too much to be worth trying to catch that particular
-    // corner case.
-    fnRunner.finishBundle();
-    outputManager.verifyLatestOutputsUnmodified();
-  }
-
-  private static SideInputReader makeSideInputReader(
-      DirectPipelineRunner.EvaluationContext context, List<PCollectionView<?>> sideInputs) {
-    PTuple sideInputValues = PTuple.empty();
-    for (PCollectionView<?> view : sideInputs) {
-      sideInputValues = sideInputValues.and(
-          view.getTagInternal(),
-          context.getPCollectionView(view));
-    }
-    return DirectSideInputReader.of(sideInputValues);
-  }
-
   private static void populateDisplayData(
       DisplayData.Builder builder, DoFn<?, ?> fn, Class<?> fnClass) {
     builder
         .include(fn)
         .add(DisplayData.item("fn", fnClass)
-          .withLabel("Transform Function"));
-  }
-
-  /**
-   * A {@code DoFnRunner.OutputManager} that provides facilities for checking output values for
-   * illegal mutations.
-   *
-   * <p>When used via the try-with-resources pattern, it is guaranteed that every value passed
-   * to {@link #output} will have been checked for illegal mutation.
-   */
-  private static class ImmutabilityCheckingOutputManager<InputT>
-      implements DoFnRunners.OutputManager, AutoCloseable {
-
-    private final DoFnRunners.OutputManager underlyingOutputManager;
-    private final ConcurrentMap<TupleTag<?>, MutationDetector> mutationDetectorForTag;
-    private final PCollectionTuple outputs;
-    private String doFnName;
-
-    public ImmutabilityCheckingOutputManager(
-        String doFnName,
-        DoFnRunners.OutputManager underlyingOutputManager,
-        PCollectionTuple outputs) {
-      this.doFnName = doFnName;
-      this.underlyingOutputManager = underlyingOutputManager;
-      this.outputs = outputs;
-      this.mutationDetectorForTag = Maps.newConcurrentMap();
-    }
-
-    @Override
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-
-      // Skip verifying undeclared outputs, since we don't have coders for them.
-      if (outputs.has(tag)) {
-        try {
-          MutationDetector newDetector =
-              MutationDetectors.forValueWithCoder(
-                  output.getValue(), outputs.get(tag).getCoder());
-          MutationDetector priorDetector = mutationDetectorForTag.put(tag, newDetector);
-          verifyOutputUnmodified(priorDetector);
-        } catch (CoderException e) {
-          throw UserCodeException.wrap(e);
-        }
-      }
-
-      // Actually perform the output.
-      underlyingOutputManager.output(tag, output);
-    }
-
-    /**
-     * Throws {@link IllegalMutationException} if the prior output for any tag has been mutated
-     * since being output.
-     */
-    public void verifyLatestOutputsUnmodified() {
-      for (MutationDetector detector : mutationDetectorForTag.values()) {
-        verifyOutputUnmodified(detector);
-      }
-    }
-
-    /**
-     * Adapts the error message from the provided {@code detector}.
-     *
-     * <p>The {@code detector} may be null, in which case no check is performed. This is merely
-     * to consolidate null checking to this method.
-     */
-    private <T> void verifyOutputUnmodified(@Nullable MutationDetector detector) {
-      if (detector == null) {
-        return;
-      }
-
-      try {
-        detector.verifyUnmodified();
-      } catch (IllegalMutationException exn) {
-        throw new IllegalMutationException(String.format(
-            "DoFn %s mutated value %s after it was output (new value was %s)."
-                + " Values must not be mutated in any way after being output.",
-                doFnName, exn.getSavedValue(), exn.getNewValue()),
-            exn.getSavedValue(), exn.getNewValue(),
-            exn);
-      }
-    }
-
-    /**
-     * When used in a {@code try}-with-resources block, verifies all of the latest outputs upon
-     * {@link #close()}.
-     */
-    @Override
-    public void close() {
-      verifyLatestOutputsUnmodified();
-    }
+            .withLabel("Transform Function"));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
index 3df915b..7a97c13 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
@@ -17,10 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
@@ -445,27 +443,5 @@ public class View {
     public PCollectionView<ViewT> apply(PCollection<ElemT> input) {
       return view;
     }
-
-    static {
-      DirectPipelineRunner.registerDefaultTransformEvaluator(
-          CreatePCollectionView.class,
-          new DirectPipelineRunner.TransformEvaluator<CreatePCollectionView>() {
-            @SuppressWarnings("rawtypes")
-            @Override
-            public void evaluate(
-                CreatePCollectionView transform,
-                DirectPipelineRunner.EvaluationContext context) {
-              evaluateTyped(transform, context);
-            }
-
-            private <ElemT, ViewT> void evaluateTyped(
-                CreatePCollectionView<ElemT, ViewT> transform,
-                DirectPipelineRunner.EvaluationContext context) {
-              List<WindowedValue<ElemT>> elems =
-                  context.getPCollectionWindowedValues(context.getInput(transform));
-              context.setPCollectionView(context.getOutput(transform), elems);
-            }
-          });
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
deleted file mode 100644
index 85e36dd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.runners.DirectPipelineRunner.ValueWithMetadata;
-import org.apache.beam.sdk.util.state.InMemoryStateInternals;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * {@link ExecutionContext} for use in direct mode.
- */
-public class DirectModeExecutionContext
-    extends BaseExecutionContext<DirectModeExecutionContext.StepContext> {
-
-  private Object key;
-  private List<ValueWithMetadata<?>> output = Lists.newArrayList();
-  private Map<TupleTag<?>, List<ValueWithMetadata<?>>> sideOutputs = Maps.newHashMap();
-
-  protected DirectModeExecutionContext() {}
-
-  public static DirectModeExecutionContext create() {
-    return new DirectModeExecutionContext();
-  }
-
-  @Override
-  protected StepContext createStepContext(String stepName, String transformName) {
-    return new StepContext(this, stepName, transformName);
-  }
-
-  public Object getKey() {
-    return key;
-  }
-
-  public void setKey(Object newKey) {
-    // The direct mode runner may reorder elements, so we need to keep
-    // around the state used for each key.
-    for (ExecutionContext.StepContext stepContext : getAllStepContexts()) {
-      ((StepContext) stepContext).switchKey(newKey);
-    }
-    key = newKey;
-  }
-
-  @Override
-  public void noteOutput(WindowedValue<?> outputElem) {
-    output.add(ValueWithMetadata.of(outputElem).withKey(getKey()));
-  }
-
-  @Override
-  public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> outputElem) {
-    List<ValueWithMetadata<?>> output = sideOutputs.get(tag);
-    if (output == null) {
-      output = Lists.newArrayList();
-      sideOutputs.put(tag, output);
-    }
-    output.add(ValueWithMetadata.of(outputElem).withKey(getKey()));
-  }
-
-  public <T> List<ValueWithMetadata<T>> getOutput(@SuppressWarnings("unused") TupleTag<T> tag) {
-    @SuppressWarnings({"unchecked", "rawtypes"}) // Cast not expressible without rawtypes
-    List<ValueWithMetadata<T>> typedOutput = (List) output;
-    return typedOutput;
-  }
-
-  public <T> List<ValueWithMetadata<T>> getSideOutput(TupleTag<T> tag) {
-    if (sideOutputs.containsKey(tag)) {
-      @SuppressWarnings({"unchecked", "rawtypes"}) // Cast not expressible without rawtypes
-      List<ValueWithMetadata<T>> typedOutput = (List) sideOutputs.get(tag);
-      return typedOutput;
-    } else {
-      return Lists.newArrayList();
-    }
-  }
-
-  /**
-   * {@link ExecutionContext.StepContext} used in direct mode.
-   */
-  public static class StepContext extends BaseExecutionContext.StepContext {
-
-    /** A map from each key to the state associated with it. */
-    private final Map<Object, InMemoryStateInternals<Object>> stateInternals = Maps.newHashMap();
-    private InMemoryStateInternals<Object> currentStateInternals = null;
-
-    private StepContext(ExecutionContext executionContext, String stepName, String transformName) {
-      super(executionContext, stepName, transformName);
-      switchKey(null);
-    }
-
-    public void switchKey(Object newKey) {
-      currentStateInternals = stateInternals.get(newKey);
-      if (currentStateInternals == null) {
-        currentStateInternals = InMemoryStateInternals.forKey(newKey);
-        stateInternals.put(newKey, currentStateInternals);
-      }
-    }
-
-    @Override
-    public StateInternals<Object> stateInternals() {
-      return checkNotNull(currentStateInternals);
-    }
-
-    @Override
-    public TimerInternals timerInternals() {
-      throw new UnsupportedOperationException("Direct mode cannot return timerInternals");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/99654ca4/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
index 75861fe..58b10a7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.util;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.DirectPipelineRunner;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn;



[40/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java
new file mode 100644
index 0000000..05297ec
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.services.dataflow.model.Environment;
+
+/**
+ * An instance of this class can be passed to the
+ * {@link DataflowRunner} to add user defined hooks to be
+ * invoked at various times during pipeline execution.
+ */
+@Experimental
+public class DataflowRunnerHooks {
+  /**
+   * Allows the user to modify the environment of their job before their job is submitted
+   * to the service for execution.
+   *
+   * @param environment The environment of the job. Users can make change to this instance in order
+   *     to change the environment with which their job executes on the service.
+   */
+  public void modifyEnvironmentBeforeSubmission(Environment environment) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
index f2e8459..ea83bfb 100755
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Implementation of the {@link org.apache.beam.runners.dataflow.DataflowPipelineRunner}.
+ * Implementation of the {@link org.apache.beam.runners.dataflow.DataflowRunner}.
  */
 package org.apache.beam.runners.dataflow.internal;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
index 7fa5ad6..809df35 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.Description;
@@ -29,9 +29,9 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import java.io.PrintStream;
 
 /**
- * Options that are used to configure the {@link BlockingDataflowPipelineRunner}.
+ * Options that are used to configure the {@link BlockingDataflowRunner}.
  */
-@Description("Configure options on the BlockingDataflowPipelineRunner.")
+@Description("Configure options on the BlockingDataflowRunner.")
 public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions {
   /**
    * Output stream for job status messages.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index 6e6ad96..f665a08 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.BigQueryOptions;
 import org.apache.beam.sdk.options.Default;
@@ -38,7 +38,7 @@ import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 
 /**
- * Options that can be used to configure the {@link DataflowPipelineRunner}.
+ * Options that can be used to configure the {@link DataflowRunner}.
  */
 @Description("Options that configure the Dataflow pipeline.")
 public interface DataflowPipelineOptions

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
index a29b328..e3a1a0f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.DefaultValueFactory;
@@ -131,9 +131,9 @@ public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions {
     public String create(PipelineOptions options) {
       DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
       if (dataflowOptions.isStreaming()) {
-        return DataflowPipelineRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
+        return DataflowRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
       } else {
-        return DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
+        return DataflowRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
index c940e9a..f83a139 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
@@ -21,7 +21,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
 
 import org.apache.beam.runners.dataflow.DataflowJobExecutionException;
 import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
 import org.apache.beam.sdk.Pipeline;
@@ -54,7 +54,7 @@ import java.util.concurrent.TimeUnit;
 
 /**
  * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
- * {@link DataflowPipelineRunner} when running tests against the {@link TestPipeline}.
+ * {@link DataflowRunner} when running tests against the {@link TestPipeline}.
  *
  * @see TestPipeline
  */
@@ -63,12 +63,12 @@ public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJ
   private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
 
   private final TestDataflowPipelineOptions options;
-  private final DataflowPipelineRunner runner;
+  private final DataflowRunner runner;
   private int expectedNumberOfAssertions = 0;
 
   TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
     this.options = options;
-    this.runner = DataflowPipelineRunner.fromOptions(options);
+    this.runner = DataflowRunner.fromOptions(options);
   }
 
   /**
@@ -89,7 +89,7 @@ public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJ
     return run(pipeline, runner);
   }
 
-  DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
+  DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) {
 
     TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class);
     final DataflowPipelineJob job;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
deleted file mode 100644
index 55b4027..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.TestCredential;
-
-import org.hamcrest.Description;
-import org.hamcrest.Factory;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-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.concurrent.TimeUnit;
-
-/**
- * Tests for BlockingDataflowPipelineRunner.
- */
-@RunWith(JUnit4.class)
-public class BlockingDataflowPipelineRunnerTest {
-
-  @Rule
-  public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class);
-
-  @Rule
-  public ExpectedException expectedThrown = ExpectedException.none();
-
-  /**
-   * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
-   * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
-   */
-  private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
-      extends TypeSafeMatcher<T> {
-
-    private final Matcher<DataflowPipelineJob> matcher;
-
-    public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T ex) {
-      return matcher.matches(ex.getJob());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("job ");
-        matcher.describeMismatch(item.getMessage(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("exception with job matching ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowJobException> Matcher<T> expectJob(
-        Matcher<DataflowPipelineJob> matcher) {
-      return new DataflowJobExceptionMatcher<T>(matcher);
-    }
-  }
-
-  /**
-   * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
-   * to the return value of {@link DataflowPipelineJob#getJobId()}.
-   */
-  private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
-
-    private final Matcher<String> matcher;
-
-    public JobIdMatcher(Matcher<String> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T job) {
-      return matcher.matches(job.getJobId());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("jobId ");
-        matcher.describeMismatch(item.getJobId(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("job with jobId ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
-      return new JobIdMatcher<T>(equalTo(jobId));
-    }
-  }
-
-  /**
-   * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
-   * {@link Matcher} to the {@link DataflowPipelineJob} returned by
-   * {@link DataflowJobUpdatedException#getReplacedByJob()}.
-   */
-  private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
-      extends TypeSafeMatcher<T> {
-
-    private final Matcher<DataflowPipelineJob> matcher;
-
-    public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
-        this.matcher = matcher;
-    }
-
-    @Override
-    public boolean matchesSafely(T ex) {
-      return matcher.matches(ex.getReplacedByJob());
-    }
-
-    @Override
-    protected void describeMismatchSafely(T item, Description description) {
-        description.appendText("job ");
-        matcher.describeMismatch(item.getMessage(), description);
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("exception with replacedByJob() ");
-      description.appendDescriptionOf(matcher);
-    }
-
-    @Factory
-    public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
-        Matcher<DataflowPipelineJob> matcher) {
-      return new ReplacedByJobMatcher<T>(matcher);
-    }
-  }
-
-  /**
-   * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
-   * that will immediately terminate in the provided {@code terminalState}.
-   *
-   * <p>The return value may be further mocked.
-   */
-  private DataflowPipelineJob createMockJob(
-      String projectId, String jobId, State terminalState) throws Exception {
-    DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
-    when(mockJob.getProjectId()).thenReturn(projectId);
-    when(mockJob.getJobId()).thenReturn(jobId);
-    when(mockJob.waitToFinish(
-        anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
-        .thenReturn(terminalState);
-    return mockJob;
-  }
-
-  /**
-   * Returns a {@link BlockingDataflowPipelineRunner} that will return the provided a job to return.
-   * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
-   */
-  private BlockingDataflowPipelineRunner createMockRunner(DataflowPipelineJob job)
-      throws Exception {
-    DataflowPipelineRunner mockRunner = mock(DataflowPipelineRunner.class);
-    TestDataflowPipelineOptions options =
-        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
-    options.setRunner(BlockingDataflowPipelineRunner.class);
-    options.setProject(job.getProjectId());
-
-    when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
-
-    return new BlockingDataflowPipelineRunner(mockRunner, options);
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} returns normally when a job terminates in
-   * the {@link State#DONE DONE} state.
-   */
-  @Test
-  public void testJobDoneComplete() throws Exception {
-    createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
-        .run(TestPipeline.create());
-    expectedLogs.verifyInfo("Job finished with status DONE");
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#FAILED FAILED} state.
-   */
-  @Test
-  public void testFailedJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobExecutionException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testFailedJob-jobId")));
-    createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
-        .run(TestPipeline.create());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
-   */
-  @Test
-  public void testCancelledJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobCancelledException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testCancelledJob-jobId")));
-    createMockRunner(
-            createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
-        .run(TestPipeline.create());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#UPDATED UPDATED} state.
-   */
-  @Test
-  public void testUpdatedJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowJobUpdatedException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
-    expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
-        JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
-    DataflowPipelineJob job =
-        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
-    DataflowPipelineJob replacedByJob =
-        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
-    when(job.getReplacedByJob()).thenReturn(replacedByJob);
-    createMockRunner(job).run(TestPipeline.create());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
-   * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
-   * is an old SDK relative the service).
-   */
-  @Test
-  public void testUnknownJobThrowsException() throws Exception {
-    expectedThrown.expect(IllegalStateException.class);
-    createMockRunner(
-            createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
-        .run(TestPipeline.create());
-  }
-
-  /**
-   * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
-   * when a job returns a {@code null} state, indicating that it failed to contact the service,
-   * including all of its built-in resilience logic.
-   */
-  @Test
-  public void testNullJobThrowsException() throws Exception {
-    expectedThrown.expect(DataflowServiceException.class);
-    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
-        JobIdMatcher.expectJobId("testNullJob-jobId")));
-    createMockRunner(createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
-        .run(TestPipeline.create());
-  }
-
-  @Test
-  public void testToString() {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    options.setJobName("TestJobName");
-    options.setProject("test-project");
-    options.setTempLocation("gs://test/temp/location");
-    options.setGcpCredential(new TestCredential());
-    options.setPathValidatorClass(NoopPathValidator.class);
-    options.setRunner(BlockingDataflowPipelineRunner.class);
-    assertEquals("BlockingDataflowPipelineRunner#testjobname",
-        BlockingDataflowPipelineRunner.fromOptions(options).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java
new file mode 100644
index 0000000..7be074e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+
+import org.hamcrest.Description;
+import org.hamcrest.Factory;
+import org.hamcrest.Matcher;
+import org.hamcrest.TypeSafeMatcher;
+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.concurrent.TimeUnit;
+
+/**
+ * Tests for BlockingDataflowRunner.
+ */
+@RunWith(JUnit4.class)
+public class BlockingDataflowRunnerTest {
+
+  @Rule
+  public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowRunner.class);
+
+  @Rule
+  public ExpectedException expectedThrown = ExpectedException.none();
+
+  /**
+   * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
+   * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
+   */
+  private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
+      extends TypeSafeMatcher<T> {
+
+    private final Matcher<DataflowPipelineJob> matcher;
+
+    public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T ex) {
+      return matcher.matches(ex.getJob());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("job ");
+        matcher.describeMismatch(item.getMessage(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("exception with job matching ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowJobException> Matcher<T> expectJob(
+        Matcher<DataflowPipelineJob> matcher) {
+      return new DataflowJobExceptionMatcher<T>(matcher);
+    }
+  }
+
+  /**
+   * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
+   * to the return value of {@link DataflowPipelineJob#getJobId()}.
+   */
+  private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
+
+    private final Matcher<String> matcher;
+
+    public JobIdMatcher(Matcher<String> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T job) {
+      return matcher.matches(job.getJobId());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("jobId ");
+        matcher.describeMismatch(item.getJobId(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("job with jobId ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
+      return new JobIdMatcher<T>(equalTo(jobId));
+    }
+  }
+
+  /**
+   * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
+   * {@link Matcher} to the {@link DataflowPipelineJob} returned by
+   * {@link DataflowJobUpdatedException#getReplacedByJob()}.
+   */
+  private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
+      extends TypeSafeMatcher<T> {
+
+    private final Matcher<DataflowPipelineJob> matcher;
+
+    public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
+        this.matcher = matcher;
+    }
+
+    @Override
+    public boolean matchesSafely(T ex) {
+      return matcher.matches(ex.getReplacedByJob());
+    }
+
+    @Override
+    protected void describeMismatchSafely(T item, Description description) {
+        description.appendText("job ");
+        matcher.describeMismatch(item.getMessage(), description);
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("exception with replacedByJob() ");
+      description.appendDescriptionOf(matcher);
+    }
+
+    @Factory
+    public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
+        Matcher<DataflowPipelineJob> matcher) {
+      return new ReplacedByJobMatcher<T>(matcher);
+    }
+  }
+
+  /**
+   * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
+   * that will immediately terminate in the provided {@code terminalState}.
+   *
+   * <p>The return value may be further mocked.
+   */
+  private DataflowPipelineJob createMockJob(
+      String projectId, String jobId, State terminalState) throws Exception {
+    DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
+    when(mockJob.getProjectId()).thenReturn(projectId);
+    when(mockJob.getJobId()).thenReturn(jobId);
+    when(mockJob.waitToFinish(
+        anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
+        .thenReturn(terminalState);
+    return mockJob;
+  }
+
+  /**
+   * Returns a {@link BlockingDataflowRunner} that will return the provided a job to return.
+   * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
+   */
+  private BlockingDataflowRunner createMockRunner(DataflowPipelineJob job)
+      throws Exception {
+    DataflowRunner mockRunner = mock(DataflowRunner.class);
+    TestDataflowPipelineOptions options =
+        PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+    options.setRunner(BlockingDataflowRunner.class);
+    options.setProject(job.getProjectId());
+
+    when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
+
+    return new BlockingDataflowRunner(mockRunner, options);
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} returns normally when a job terminates in
+   * the {@link State#DONE DONE} state.
+   */
+  @Test
+  public void testJobDoneComplete() throws Exception {
+    createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
+        .run(TestPipeline.create());
+    expectedLogs.verifyInfo("Job finished with status DONE");
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#FAILED FAILED} state.
+   */
+  @Test
+  public void testFailedJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobExecutionException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testFailedJob-jobId")));
+    createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
+        .run(TestPipeline.create());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
+   */
+  @Test
+  public void testCancelledJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobCancelledException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testCancelledJob-jobId")));
+    createMockRunner(
+            createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
+        .run(TestPipeline.create());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#UPDATED UPDATED} state.
+   */
+  @Test
+  public void testUpdatedJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowJobUpdatedException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
+    expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
+        JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
+    DataflowPipelineJob job =
+        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
+    DataflowPipelineJob replacedByJob =
+        createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
+    when(job.getReplacedByJob()).thenReturn(replacedByJob);
+    createMockRunner(job).run(TestPipeline.create());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} throws the appropriate exception
+   * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
+   * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
+   * is an old SDK relative the service).
+   */
+  @Test
+  public void testUnknownJobThrowsException() throws Exception {
+    expectedThrown.expect(IllegalStateException.class);
+    createMockRunner(
+            createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
+        .run(TestPipeline.create());
+  }
+
+  /**
+   * Tests that the {@link BlockingDataflowRunner} throws the appropriate exception
+   * when a job returns a {@code null} state, indicating that it failed to contact the service,
+   * including all of its built-in resilience logic.
+   */
+  @Test
+  public void testNullJobThrowsException() throws Exception {
+    expectedThrown.expect(DataflowServiceException.class);
+    expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+        JobIdMatcher.expectJobId("testNullJob-jobId")));
+    createMockRunner(createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
+        .run(TestPipeline.create());
+  }
+
+  @Test
+  public void testToString() {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    options.setJobName("TestJobName");
+    options.setProject("test-project");
+    options.setTempLocation("gs://test/temp/location");
+    options.setGcpCredential(new TestCredential());
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setRunner(BlockingDataflowRunner.class);
+    assertEquals("BlockingDataflowRunner#testjobname",
+        BlockingDataflowRunner.fromOptions(options).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
index cf9a95a..388a85a 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
@@ -46,8 +46,8 @@ public class DataflowPipelineRegistrarTest {
 
   @Test
   public void testCorrectRunnersAreReturned() {
-    assertEquals(ImmutableList.of(DataflowPipelineRunner.class,
-                                  BlockingDataflowPipelineRunner.class),
+    assertEquals(ImmutableList.of(DataflowRunner.class,
+                                  BlockingDataflowRunner.class),
         new DataflowPipelineRegistrar.Runner().getPipelineRunners());
   }
 



[17/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index fb8eb7c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
+++ /dev/null
@@ -1,1420 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import org.apache.beam.runners.direct.DirectRunner.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.auto.value.AutoValue;
-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.ConcurrentLinkedQueue;
-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<StructuralKey<?>, 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<StructuralKey<?>, 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<StructuralKey<?>, NavigableSet<TimerData>> processingTimers;
-    private final Map<StructuralKey<?>, 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) {
-      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 completedTimer : update.completedTimers) {
-        pendingTimers.remove(completedTimer);
-      }
-      for (TimerData deletedTimer : update.deletedTimers) {
-        NavigableSet<TimerData> timerQueue = timerMap.get(deletedTimer.getDomain());
-        if (timerQueue != null) {
-          timerQueue.remove(deletedTimer);
-        }
-      }
-    }
-
-    private synchronized Map<StructuralKey<?>, List<TimerData>> extractFiredDomainTimers(
-        TimeDomain domain, Instant firingTime) {
-      Map<StructuralKey<?>, 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<StructuralKey<?>, ? extends Collection<TimerData>> firedTimer :
-          firedTimers.entrySet()) {
-        pendingTimers.addAll(firedTimer.getValue());
-      }
-      return firedTimers;
-    }
-
-    private Map<TimeDomain, NavigableSet<TimerData>> timerMap(StructuralKey<?> 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<StructuralKey<?>, List<TimerData>> extractFiredTimers(
-      Instant latestTime, Map<StructuralKey<?>, NavigableSet<TimerData>> objectTimers) {
-    Map<StructuralKey<?>, List<TimerData>> result = new HashMap<>();
-    Set<StructuralKey<?>> emptyKeys = new HashSet<>();
-    for (Map.Entry<StructuralKey<?>, 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;
-
-  /**
-   * A queue of pending updates to the state of this {@link InMemoryWatermarkManager}.
-   */
-  private final ConcurrentLinkedQueue<PendingWatermarkUpdate> pendingUpdates;
-
-  /**
-   * A queue of pending {@link AppliedPTransform AppliedPTransforms} that have potentially
-   * stale data.
-   */
-  private final ConcurrentLinkedQueue<AppliedPTransform<?, ?, ?>> pendingRefreshes;
-
-  /**
-   * 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;
-    this.pendingUpdates = new ConcurrentLinkedQueue<>();
-    this.pendingRefreshes = new ConcurrentLinkedQueue<>();
-
-    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 timerUpdate the timers that were added, removed, and completed as part of producing
-   *                    this update
-   * @param result the result that was produced by processing the input
-   * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there
-   *                     is no hold
-   */
-  public void updateWatermarks(
-      @Nullable CommittedBundle<?> completed,
-      TimerUpdate timerUpdate,
-      CommittedResult result,
-      Instant earliestHold) {
-    pendingUpdates.offer(PendingWatermarkUpdate.create(completed,
-        timerUpdate,
-        result,
-        earliestHold));
-  }
-
-  /**
-   * Applies all pending updates to this {@link InMemoryWatermarkManager}, causing the pending state
-   * of all {@link TransformWatermarks} to be advanced as far as possible.
-   */
-  private void applyPendingUpdates() {
-    Set<AppliedPTransform<?, ?, ?>> updatedTransforms = new HashSet<>();
-    PendingWatermarkUpdate pending = pendingUpdates.poll();
-    while (pending != null) {
-      applyPendingUpdate(pending);
-      updatedTransforms.add(pending.getTransform());
-      pending = pendingUpdates.poll();
-    }
-    pendingRefreshes.addAll(updatedTransforms);
-  }
-
-  private void applyPendingUpdate(PendingWatermarkUpdate pending) {
-    CommittedResult result = pending.getResult();
-    AppliedPTransform transform = result.getTransform();
-    CommittedBundle<?> inputBundle = pending.getInputBundle();
-
-    updatePending(inputBundle, pending.getTimerUpdate(), result);
-
-    TransformWatermarks transformWms = transformToWatermarks.get(transform);
-    transformWms.setEventTimeHold(inputBundle == null ? null : inputBundle.getKey(),
-        pending.getEarliestHold());
-  }
-
-  /**
-   * First adds all produced elements to the queue of pending elements for each consumer, then adds
-   * all pending timers to the collection of pending timers, then removes all completed and deleted
-   * timers from the collection of pending timers, then removes all completed elements from the
-   * pending queue of the transform.
-   *
-   * <p>It is required that all newly pending elements are added to the queue of pending elements
-   * for each consumer prior to the completed elements being removed, as doing otherwise could cause
-   * a Watermark to appear in a state in which the upstream (completed) element does not hold the
-   * watermark but the element it produced is not yet pending. This can cause the watermark to
-   * erroneously advance.
-   */
-  private void updatePending(
-      CommittedBundle<?> input,
-      TimerUpdate timerUpdate,
-      CommittedResult result) {
-    // Newly pending elements must be added before completed elements are removed, as the two
-    // do not share a Mutex within this call and thus can be interleaved with external calls to
-    // refresh.
-    for (CommittedBundle<?> bundle : result.getOutputs()) {
-      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
-        TransformWatermarks watermarks = transformToWatermarks.get(consumer);
-        watermarks.addPending(bundle);
-      }
-    }
-
-    TransformWatermarks completedTransform = transformToWatermarks.get(result.getTransform());
-    if (input != null) {
-      // Add the unprocessed inputs
-      completedTransform.addPending(result.getUnprocessedInputs());
-    }
-    completedTransform.updateTimers(timerUpdate);
-    if (input != null) {
-      completedTransform.removePending(input);
-    }
-  }
-
-  /**
-   * Refresh the watermarks contained within this {@link InMemoryWatermarkManager}, causing all
-   * watermarks to be advanced as far as possible.
-   */
-  synchronized void refreshAll() {
-    applyPendingUpdates();
-    while (!pendingRefreshes.isEmpty()) {
-      refreshWatermarks(pendingRefreshes.poll());
-    }
-  }
-
-  private void refreshWatermarks(AppliedPTransform<?, ?, ?> toRefresh) {
-    TransformWatermarks myWatermarks = transformToWatermarks.get(toRefresh);
-    WatermarkUpdate updateResult = myWatermarks.refresh();
-    Set<AppliedPTransform<?, ?, ?>> additionalRefreshes = new HashSet<>();
-    if (updateResult.isAdvanced()) {
-      for (PValue outputPValue : toRefresh.getOutput().expand()) {
-        additionalRefreshes.addAll(consumers.get(outputPValue));
-      }
-    }
-    pendingRefreshes.addAll(additionalRefreshes);
-  }
-
-  /**
-   * 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<StructuralKey<?>, FiredTimers>> extractFiredTimers() {
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> allTimers = new HashMap<>();
-    for (Map.Entry<AppliedPTransform<?, ?, ?>, TransformWatermarks> watermarksEntry :
-        transformToWatermarks.entrySet()) {
-      Map<StructuralKey<?>, 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<StructuralKey<?>, FiredTimers> extractFiredTimers() {
-      Map<StructuralKey<?>, List<TimerData>> eventTimeTimers =
-          inputWatermark.extractFiredEventTimeTimers();
-      Map<StructuralKey<?>, List<TimerData>> processingTimers;
-      Map<StructuralKey<?>, 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<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> groupedTimers = new HashMap<>();
-      groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers);
-
-      Map<StructuralKey<?>, FiredTimers> keyFiredTimers = new HashMap<>();
-      for (Map.Entry<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> firedTimers :
-          groupedTimers.entrySet()) {
-        keyFiredTimers.put(firedTimers.getKey(), new FiredTimers(firedTimers.getValue()));
-      }
-      return keyFiredTimers;
-    }
-
-    @SafeVarargs
-    private final void groupFiredTimers(
-        Map<StructuralKey<?>, Map<TimeDomain, List<TimerData>>> groupedToMutate,
-        Map<StructuralKey<?>, List<TimerData>>... timersToGroup) {
-      for (Map<StructuralKey<?>, List<TimerData>> subGroup : timersToGroup) {
-        for (Map.Entry<StructuralKey<?>, 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 StructuralKey<?> 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(StructuralKey<?> 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 StructuralKey<?> key;
-      private final Collection<TimerData> completedTimers;
-      private final Collection<TimerData> setTimers;
-      private final Collection<TimerData> deletedTimers;
-
-      private TimerUpdateBuilder(StructuralKey<?> 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(
-        StructuralKey<?> 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
-    StructuralKey<?> 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;
-  }
-
-  @AutoValue
-  abstract static class PendingWatermarkUpdate {
-    @Nullable
-    public abstract CommittedBundle<?> getInputBundle();
-    public abstract TimerUpdate getTimerUpdate();
-    public abstract CommittedResult getResult();
-    public abstract Instant getEarliestHold();
-
-    /**
-     * Gets the {@link AppliedPTransform} that generated this result.
-     */
-    public AppliedPTransform<?, ?, ?> getTransform() {
-      return getResult().getTransform();
-    }
-
-    public static PendingWatermarkUpdate create(
-        CommittedBundle<?> inputBundle,
-        TimerUpdate timerUpdate,
-        CommittedResult result, Instant earliestHold) {
-      return new AutoValue_InMemoryWatermarkManager_PendingWatermarkUpdate(inputBundle,
-          timerUpdate,
-          result,
-          earliestHold);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index 0c7449c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessBundleFactory.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.coders.VoidCoder;
-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;
-
-/**
- * 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, StructuralKey.of(null, VoidCoder.of()));
-  }
-
-  @Override
-  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
-    return InProcessBundle.create(output, input.getKey());
-  }
-
-  @Override
-  public <K, T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, StructuralKey<K> 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;
-    private final StructuralKey<?> 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, StructuralKey<?> key) {
-      return new InProcessBundle<>(pcollection, key);
-    }
-
-    private InProcessBundle(PCollection<T> pcollection, StructuralKey<?> 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,
-        StructuralKey<?> key,
-        Iterable<WindowedValue<T>> committedElements,
-        Instant synchronizedCompletionTime) {
-      this.pcollection = pcollection;
-      this.key = key;
-      this.committedElements = committedElements;
-      this.synchronizedCompletionTime = synchronizedCompletionTime;
-    }
-
-    private final PCollection<T> pcollection;
-    /** The structural value key of the Bundle, as specified by the coder that created it. */
-    private final StructuralKey<?> key;
-    private final Iterable<WindowedValue<T>> committedElements;
-    private final Instant synchronizedCompletionTime;
-
-    @Override
-    public StructuralKey<?> 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/babddbbc/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
deleted file mode 100644
index bd07040..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.runners.direct;
-
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.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 DirectRunner}.
- */
-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/babddbbc/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
deleted file mode 100644
index 220ff83..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java
+++ /dev/null
@@ -1,429 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
-import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
-import org.apache.beam.runners.direct.DirectRunner.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.ReadyCheckingSideInputReader;
-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.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.MoreExecutors;
-
-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 DirectRunner}. Contains state shared within the execution across all
- * transforms.
- *
- * <p>{@link InProcessEvaluationContext} contains shared state for an execution of the
- * {@link DirectRunner} 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 DirectOptions 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(
-      DirectOptions 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(
-      DirectOptions 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(MoreExecutors.directExecutor());
-  }
-
-  /**
-   * 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 CommittedResult handleResult(
-      @Nullable CommittedBundle<?> completedBundle,
-      Iterable<TimerData> completedTimers,
-      InProcessTransformResult result) {
-    Iterable<? extends CommittedBundle<?>> committedBundles =
-        commitBundles(result.getOutputBundles());
-    // Update watermarks and timers
-    CommittedResult committedResult = CommittedResult.create(result,
-        completedBundle == null
-            ? null
-            : completedBundle.withElements((Iterable) result.getUnprocessedElements()),
-        committedBundles);
-    watermarkManager.updateWatermarks(
-        completedBundle,
-        result.getTimerUpdate().withCompletedTimers(completedTimers),
-        committedResult,
-        result.getWatermarkHold());
-    // 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;
-  }
-
-  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 <K, T> UncommittedBundle<T> createKeyedBundle(
-      CommittedBundle<?> input, StructuralKey<K> 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 DirectOptions getPipelineOptions() {
-    return options;
-  }
-
-  /**
-   * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
-   */
-  public InProcessExecutionContext getExecutionContext(
-      AppliedPTransform<?, ?, ?> application, StructuralKey<?> 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);
-  }
-
-
-  /**
-   * 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;
-  }
-
-  @VisibleForTesting
-  void forceRefresh() {
-    watermarkManager.refreshAll();
-    fireAllAvailableCallbacks();
-  }
-
-  /**
-   * 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<StructuralKey<?>, FiredTimers>> extractFiredTimers() {
-    forceRefresh();
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, 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 DirectOptions#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/babddbbc/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
deleted file mode 100644
index d2558ce..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.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 DirectRunner}.
- *
- * 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 StructuralKey<?> key;
-  private final CopyOnAccessInMemoryStateInternals<Object> existingState;
-  private final TransformWatermarks watermarks;
-
-  public InProcessExecutionContext(Clock clock, StructuralKey<?> 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 DirectRunner}.
-   */
-  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();
-    }
-  }
-}



[27/50] [abbrv] incubator-beam git commit: Add BigIntegerCoder and tests

Posted by da...@apache.org.
Add BigIntegerCoder and tests


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

Branch: refs/heads/python-sdk
Commit: 8268f1d7ffdd1205a1904037f7dd1e1887a52f8d
Parents: 6460df1
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 9 13:24:49 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/coders/BigIntegerCoder.java |  91 ++++++++++++++
 .../beam/sdk/coders/BigIntegerCoderTest.java    | 119 +++++++++++++++++++
 2 files changed, 210 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8268f1d7/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java
new file mode 100644
index 0000000..a41defe
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.coders;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigInteger;
+
+/**
+ * A {@link BigIntegerCoder} encodes a {@link BigInteger} as a byte array containing the big endian
+ * two's-complement representation, encoded via {@link ByteArrayCoder}.
+ */
+public class BigIntegerCoder extends AtomicCoder<BigInteger> {
+
+  @JsonCreator
+  public static BigIntegerCoder of() {
+    return INSTANCE;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  private static final BigIntegerCoder INSTANCE = new BigIntegerCoder();
+
+  private BigIntegerCoder() {}
+
+  private final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of();
+
+  @Override
+  public void encode(BigInteger value, OutputStream outStream, Context context)
+      throws IOException, CoderException {
+    checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName()));
+    byteArrayCoder.encode(value.toByteArray(), outStream, context);
+  }
+
+  @Override
+  public BigInteger decode(InputStream inStream, Context context)
+      throws IOException, CoderException {
+    return new BigInteger(byteArrayCoder.decode(inStream, context));
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code true}. This coder is injective.
+   */
+  @Override
+  public boolean consistentWithEquals() {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code true}, because {@link #getEncodedElementByteSize} runs in constant time.
+   */
+  @Override
+  public boolean isRegisterByteSizeObserverCheap(BigInteger value, Context context) {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return the size of the encoding as a byte array according to {@link ByteArrayCoder}
+   */
+  @Override
+  protected long getEncodedElementByteSize(BigInteger value, Context context) throws Exception {
+    checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName()));
+    return byteArrayCoder.getEncodedElementByteSize(value.toByteArray(), context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8268f1d7/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
new file mode 100644
index 0000000..3934b03
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.coders;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterName;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
+import com.google.common.collect.ImmutableList;
+
+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.math.BigInteger;
+import java.util.List;
+
+/**
+ * Test case for {@link BigIntegerCoder}.
+ */
+@RunWith(JUnit4.class)
+public class BigIntegerCoderTest {
+
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private static final Coder<BigInteger> TEST_CODER = BigIntegerCoder.of();
+
+  private static final List<BigInteger> TEST_VALUES =
+      ImmutableList.of(
+          BigInteger.valueOf(Integer.MIN_VALUE).subtract(BigInteger.valueOf(Integer.MAX_VALUE)),
+          BigInteger.valueOf(Integer.MIN_VALUE).subtract(BigInteger.ONE),
+          BigInteger.valueOf(-1),
+          BigInteger.ZERO,
+          BigInteger.valueOf(1),
+          BigInteger.valueOf(Integer.MAX_VALUE).add(BigInteger.ONE),
+          BigInteger.valueOf(Integer.MAX_VALUE).multiply(BigInteger.TEN));
+
+  @Test
+  public void testDecodeEncodeEqual() throws Exception {
+    for (BigInteger value : TEST_VALUES) {
+      CoderProperties.coderDecodeEncodeEqual(TEST_CODER, value);
+    }
+  }
+
+  // This should never change. The definition of big endian encoding is fixed.
+  private static final String EXPECTED_ENCODING_ID = "";
+
+  @Test
+  public void testEncodingId() throws Exception {
+    CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID);
+  }
+
+  /**
+   * Generated data to check that the wire format has not changed. To regenerate, see
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
+   */
+  private static final List<String> TEST_ENCODINGS =
+      ImmutableList.of("_wAAAAE", "_3____8", "_w", "AA", "AQ", "AIAAAAA", "BP____Y");
+
+  @Test
+  public void testWireFormatEncode() throws Exception {
+    CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS);
+  }
+
+  @Test
+  public void testGetEncodedElementByteSize() throws Exception {
+    Counter<Long> counter = Counter.longs(CounterName.named("dummy"), AggregationKind.SUM);
+    ElementByteSizeObserver observer = new ElementByteSizeObserver(counter);
+    for (BigInteger value : TEST_VALUES) {
+      TEST_CODER.registerByteSizeObserver(value, observer, Coder.Context.OUTER);
+      observer.advance();
+      assertThat(
+          counter.getAggregate(),
+          equalTo((long) CoderUtils.encodeToByteArray(TEST_CODER, value).length));
+      counter.resetToValue(0L);
+    }
+  }
+
+  @Test
+  public void encodeNullThrowsCoderException() throws Exception {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("cannot encode a null BigInteger");
+
+    CoderUtils.encodeToBase64(TEST_CODER, null);
+  }
+
+  /**
+   * This is a change-detector test. If this test fails, then the encoding id of
+   * {@link BigIntegerCoder} must change.
+   */
+  @Test
+  public void testCoderIdDependencies() {
+    assertThat(VarIntCoder.of().getEncodingId(), equalTo(""));
+    assertThat(ByteArrayCoder.of().getEncodingId(), equalTo(""));
+  }
+}


[49/50] [abbrv] incubator-beam git commit: Remove Pipeline from TestDataflowPipelineRunner

Posted by da...@apache.org.
Remove Pipeline from TestDataflowPipelineRunner


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

Branch: refs/heads/python-sdk
Commit: a24e557090c7fb59846b332c0c9df9d49565c808
Parents: 90d0bcf
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 16:36:22 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 examples/java/README.md                         |   4 +-
 .../beam/runners/flink/examples/TFIDF.java      |   2 +-
 .../testing/TestDataflowPipelineRunner.java     | 271 -------------------
 .../dataflow/testing/TestDataflowRunner.java    | 271 +++++++++++++++++++
 .../testing/TestDataflowRunnerTest.java         |  40 +--
 5 files changed, 294 insertions(+), 294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a24e5570/examples/java/README.md
----------------------------------------------------------------------
diff --git a/examples/java/README.md b/examples/java/README.md
index ef3cf07..2b5edf5 100644
--- a/examples/java/README.md
+++ b/examples/java/README.md
@@ -64,7 +64,7 @@ the same pipeline on fully managed resources in Google Cloud Platform:
     -Dexec.mainClass=org.apache.beam.examples.WordCount \
     -Dexec.args="--project=<YOUR CLOUD PLATFORM PROJECT ID> \
     --tempLocation=<YOUR CLOUD STORAGE LOCATION> \
-    --runner=BlockingDataflowPipelineRunner"
+    --runner=BlockingDataflowRunner"
 
 Make sure to use your project id, not the project number or the descriptive name.
 The Cloud Storage location should be entered in the form of
@@ -86,7 +86,7 @@ Platform:
     org.apache.beam.examples.WordCount \
     --project=<YOUR CLOUD PLATFORM PROJECT ID> \
     --tempLocation=<YOUR CLOUD STORAGE LOCATION> \
-    --runner=BlockingDataflowPipelineRunner
+    --runner=BlockingDataflowRunner
 
 Other examples can be run similarly by replacing the `WordCount` class path with the example classpath, e.g.
 `org.apache.beam.examples.cookbook.BigQueryTornadoes`,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a24e5570/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
index 0afde0a..876ecde 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
@@ -82,7 +82,7 @@ import java.util.Set;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * and an output prefix on GCS:
  *   --output=gs://YOUR_OUTPUT_PREFIX
  * }</pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a24e5570/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
deleted file mode 100644
index f83a139..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.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.runners.dataflow.testing;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import org.apache.beam.runners.dataflow.DataflowJobExecutionException;
-import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.dataflow.DataflowRunner;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.TestPipelineOptions;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
-import com.google.common.base.Throwables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-/**
- * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
- * {@link DataflowRunner} when running tests against the {@link TestPipeline}.
- *
- * @see TestPipeline
- */
-public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
-  private static final String TENTATIVE_COUNTER = "tentative";
-  private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
-
-  private final TestDataflowPipelineOptions options;
-  private final DataflowRunner runner;
-  private int expectedNumberOfAssertions = 0;
-
-  TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
-    this.options = options;
-    this.runner = DataflowRunner.fromOptions(options);
-  }
-
-  /**
-   * Constructs a runner from the provided options.
-   */
-  public static TestDataflowPipelineRunner fromOptions(
-      PipelineOptions options) {
-    TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
-    dataflowOptions.setStagingLocation(Joiner.on("/").join(
-        new String[]{dataflowOptions.getTempRoot(),
-          dataflowOptions.getJobName(), "output", "results"}));
-
-    return new TestDataflowPipelineRunner(dataflowOptions);
-  }
-
-  @Override
-  public DataflowPipelineJob run(Pipeline pipeline) {
-    return run(pipeline, runner);
-  }
-
-  DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) {
-
-    TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class);
-    final DataflowPipelineJob job;
-    try {
-      job = runner.run(pipeline);
-    } catch (DataflowJobExecutionException ex) {
-      throw new IllegalStateException("The dataflow failed.");
-    }
-
-    LOG.info("Running Dataflow job {} with {} expected assertions.",
-        job.getJobId(), expectedNumberOfAssertions);
-
-    assertThat(job, testPipelineOptions.getOnCreateMatcher());
-
-    CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
-        job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
-
-    try {
-      final Optional<Boolean> result;
-
-      if (options.isStreaming()) {
-        Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
-            new Callable<Optional<Boolean>>() {
-          @Override
-          public Optional<Boolean> call() throws Exception {
-            try {
-              for (;;) {
-                Optional<Boolean> result = checkForSuccess(job);
-                if (result.isPresent()) {
-                  return result;
-                }
-                Thread.sleep(10000L);
-              }
-            } finally {
-              LOG.info("Cancelling Dataflow job {}", job.getJobId());
-              job.cancel();
-            }
-          }
-        });
-        State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
-        if (finalState == null || finalState == State.RUNNING) {
-          LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
-              job.getJobId());
-          job.cancel();
-        }
-        result = resultFuture.get();
-      } else {
-        job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
-        result = checkForSuccess(job);
-      }
-      if (!result.isPresent()) {
-        throw new IllegalStateException(
-            "The dataflow did not output a success or failure metric.");
-      } else if (!result.get()) {
-        throw new AssertionError(messageHandler.getErrorMessage() == null
-            ? "The dataflow did not return a failure reason."
-            : messageHandler.getErrorMessage());
-      } else {
-        assertThat(job, testPipelineOptions.getOnSuccessMatcher());
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      throw new RuntimeException(e);
-    } catch (ExecutionException e) {
-      Throwables.propagateIfPossible(e.getCause());
-      throw new RuntimeException(e.getCause());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return job;
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    if (transform instanceof PAssert.OneSideInputAssert
-        || transform instanceof PAssert.GroupThenAssert
-        || transform instanceof PAssert.GroupThenAssertForSingleton) {
-      expectedNumberOfAssertions += 1;
-    }
-
-    return runner.apply(transform, input);
-  }
-
-  Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
-      throws IOException {
-    State state = job.getState();
-    if (state == State.FAILED || state == State.CANCELLED) {
-      LOG.info("The pipeline failed");
-      return Optional.of(false);
-    }
-
-    JobMetrics metrics = job.getDataflowClient().projects().jobs()
-        .getMetrics(job.getProjectId(), job.getJobId()).execute();
-
-    if (metrics == null || metrics.getMetrics() == null) {
-      LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
-    } else {
-      int successes = 0;
-      int failures = 0;
-      for (MetricUpdate metric : metrics.getMetrics()) {
-        if (metric.getName() == null || metric.getName().getContext() == null
-            || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
-          // Don't double count using the non-tentative version of the metric.
-          continue;
-        }
-        if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
-          successes += ((BigDecimal) metric.getScalar()).intValue();
-        } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
-          failures += ((BigDecimal) metric.getScalar()).intValue();
-        }
-      }
-
-      if (failures > 0) {
-        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
-            + "{} expected assertions.", job.getJobId(), successes, failures,
-            expectedNumberOfAssertions);
-        return Optional.of(false);
-      } else if (successes >= expectedNumberOfAssertions) {
-        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
-            + "{} expected assertions.", job.getJobId(), successes, failures,
-            expectedNumberOfAssertions);
-        return Optional.of(true);
-      }
-
-      LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
-          + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
-    }
-
-    return Optional.<Boolean>absent();
-  }
-
-  @Override
-  public String toString() {
-    return "TestDataflowPipelineRunner#" + options.getAppName();
-  }
-
-  /**
-   * Cancels the workflow on the first error message it sees.
-   *
-   * <p>Creates an error message representing the concatenation of all error messages seen.
-   */
-  private static class CancelWorkflowOnError implements JobMessagesHandler {
-    private final DataflowPipelineJob job;
-    private final JobMessagesHandler messageHandler;
-    private final StringBuffer errorMessage;
-    private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
-      this.job = job;
-      this.messageHandler = messageHandler;
-      this.errorMessage = new StringBuffer();
-    }
-
-    @Override
-    public void process(List<JobMessage> messages) {
-      messageHandler.process(messages);
-      for (JobMessage message : messages) {
-        if (message.getMessageImportance() != null
-            && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
-          LOG.info("Dataflow job {} threw exception. Failure message was: {}",
-              job.getJobId(), message.getMessageText());
-          errorMessage.append(message.getMessageText());
-        }
-      }
-      if (errorMessage.length() > 0) {
-        LOG.info("Cancelling Dataflow job {}", job.getJobId());
-        try {
-          job.cancel();
-        } catch (Exception ignore) {
-          // The TestDataflowPipelineRunner will thrown an AssertionError with the job failure
-          // messages.
-        }
-      }
-    }
-
-    private String getErrorMessage() {
-      return errorMessage.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a24e5570/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
new file mode 100644
index 0000000..19a2178
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.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.dataflow.testing;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.beam.runners.dataflow.DataflowJobExecutionException;
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.base.Throwables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * {@link TestDataflowRunner} is a pipeline runner that wraps a
+ * {@link DataflowRunner} when running tests against the {@link TestPipeline}.
+ *
+ * @see TestPipeline
+ */
+public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
+  private static final String TENTATIVE_COUNTER = "tentative";
+  private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class);
+
+  private final TestDataflowPipelineOptions options;
+  private final DataflowRunner runner;
+  private int expectedNumberOfAssertions = 0;
+
+  TestDataflowRunner(TestDataflowPipelineOptions options) {
+    this.options = options;
+    this.runner = DataflowRunner.fromOptions(options);
+  }
+
+  /**
+   * Constructs a runner from the provided options.
+   */
+  public static TestDataflowRunner fromOptions(
+      PipelineOptions options) {
+    TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
+    dataflowOptions.setStagingLocation(Joiner.on("/").join(
+        new String[]{dataflowOptions.getTempRoot(),
+          dataflowOptions.getJobName(), "output", "results"}));
+
+    return new TestDataflowRunner(dataflowOptions);
+  }
+
+  @Override
+  public DataflowPipelineJob run(Pipeline pipeline) {
+    return run(pipeline, runner);
+  }
+
+  DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) {
+
+    TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class);
+    final DataflowPipelineJob job;
+    try {
+      job = runner.run(pipeline);
+    } catch (DataflowJobExecutionException ex) {
+      throw new IllegalStateException("The dataflow failed.");
+    }
+
+    LOG.info("Running Dataflow job {} with {} expected assertions.",
+        job.getJobId(), expectedNumberOfAssertions);
+
+    assertThat(job, testPipelineOptions.getOnCreateMatcher());
+
+    CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
+        job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+
+    try {
+      final Optional<Boolean> result;
+
+      if (options.isStreaming()) {
+        Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
+            new Callable<Optional<Boolean>>() {
+          @Override
+          public Optional<Boolean> call() throws Exception {
+            try {
+              for (;;) {
+                Optional<Boolean> result = checkForSuccess(job);
+                if (result.isPresent()) {
+                  return result;
+                }
+                Thread.sleep(10000L);
+              }
+            } finally {
+              LOG.info("Cancelling Dataflow job {}", job.getJobId());
+              job.cancel();
+            }
+          }
+        });
+        State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
+        if (finalState == null || finalState == State.RUNNING) {
+          LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
+              job.getJobId());
+          job.cancel();
+        }
+        result = resultFuture.get();
+      } else {
+        job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
+        result = checkForSuccess(job);
+      }
+      if (!result.isPresent()) {
+        throw new IllegalStateException(
+            "The dataflow did not output a success or failure metric.");
+      } else if (!result.get()) {
+        throw new AssertionError(messageHandler.getErrorMessage() == null
+            ? "The dataflow did not return a failure reason."
+            : messageHandler.getErrorMessage());
+      } else {
+        assertThat(job, testPipelineOptions.getOnSuccessMatcher());
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause());
+      throw new RuntimeException(e.getCause());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return job;
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    if (transform instanceof PAssert.OneSideInputAssert
+        || transform instanceof PAssert.GroupThenAssert
+        || transform instanceof PAssert.GroupThenAssertForSingleton) {
+      expectedNumberOfAssertions += 1;
+    }
+
+    return runner.apply(transform, input);
+  }
+
+  Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
+      throws IOException {
+    State state = job.getState();
+    if (state == State.FAILED || state == State.CANCELLED) {
+      LOG.info("The pipeline failed");
+      return Optional.of(false);
+    }
+
+    JobMetrics metrics = job.getDataflowClient().projects().jobs()
+        .getMetrics(job.getProjectId(), job.getJobId()).execute();
+
+    if (metrics == null || metrics.getMetrics() == null) {
+      LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
+    } else {
+      int successes = 0;
+      int failures = 0;
+      for (MetricUpdate metric : metrics.getMetrics()) {
+        if (metric.getName() == null || metric.getName().getContext() == null
+            || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
+          // Don't double count using the non-tentative version of the metric.
+          continue;
+        }
+        if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
+          successes += ((BigDecimal) metric.getScalar()).intValue();
+        } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
+          failures += ((BigDecimal) metric.getScalar()).intValue();
+        }
+      }
+
+      if (failures > 0) {
+        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+            + "{} expected assertions.", job.getJobId(), successes, failures,
+            expectedNumberOfAssertions);
+        return Optional.of(false);
+      } else if (successes >= expectedNumberOfAssertions) {
+        LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+            + "{} expected assertions.", job.getJobId(), successes, failures,
+            expectedNumberOfAssertions);
+        return Optional.of(true);
+      }
+
+      LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
+          + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
+    }
+
+    return Optional.<Boolean>absent();
+  }
+
+  @Override
+  public String toString() {
+    return "TestDataflowRunner#" + options.getAppName();
+  }
+
+  /**
+   * Cancels the workflow on the first error message it sees.
+   *
+   * <p>Creates an error message representing the concatenation of all error messages seen.
+   */
+  private static class CancelWorkflowOnError implements JobMessagesHandler {
+    private final DataflowPipelineJob job;
+    private final JobMessagesHandler messageHandler;
+    private final StringBuffer errorMessage;
+    private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
+      this.job = job;
+      this.messageHandler = messageHandler;
+      this.errorMessage = new StringBuffer();
+    }
+
+    @Override
+    public void process(List<JobMessage> messages) {
+      messageHandler.process(messages);
+      for (JobMessage message : messages) {
+        if (message.getMessageImportance() != null
+            && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
+          LOG.info("Dataflow job {} threw exception. Failure message was: {}",
+              job.getJobId(), message.getMessageText());
+          errorMessage.append(message.getMessageText());
+        }
+      }
+      if (errorMessage.length() > 0) {
+        LOG.info("Cancelling Dataflow job {}", job.getJobId());
+        try {
+          job.cancel();
+        } catch (Exception ignore) {
+          // The TestDataflowRunner will thrown an AssertionError with the job failure
+          // messages.
+        }
+      }
+    }
+
+    private String getErrorMessage() {
+      return errorMessage.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a24e5570/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
index 4067f08..cd99643 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
@@ -85,7 +85,7 @@ import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
-/** Tests for {@link TestDataflowPipelineRunner}. */
+/** Tests for {@link TestDataflowRunner}. */
 @RunWith(JUnit4.class)
 public class TestDataflowRunnerTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
@@ -110,14 +110,14 @@ public class TestDataflowRunnerTest {
     options.setTempRoot("gs://test");
     options.setGcpCredential(new TestCredential());
     options.setDataflowClient(service);
-    options.setRunner(TestDataflowPipelineRunner.class);
+    options.setRunner(TestDataflowRunner.class);
     options.setPathValidatorClass(NoopPathValidator.class);
   }
 
   @Test
   public void testToString() {
-    assertEquals("TestDataflowPipelineRunner#TestAppName",
-        new TestDataflowPipelineRunner(options).toString());
+    assertEquals("TestDataflowRunner#TestAppName",
+        new TestDataflowRunner(options).toString());
   }
 
   @Test
@@ -135,7 +135,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     when(request.execute()).thenReturn(
         generateMockMetricResponse(true /* success */, true /* tentative */));
     assertEquals(mockJob, runner.run(p, mockRunner));
@@ -156,7 +156,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     try {
       runner.run(p, mockRunner);
     } catch (AssertionError expected) {
@@ -197,7 +197,7 @@ public class TestDataflowRunnerTest {
 
     when(request.execute()).thenReturn(
         generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     try {
       runner.run(p, mockRunner);
     } catch (AssertionError expected) {
@@ -228,7 +228,7 @@ public class TestDataflowRunnerTest {
 
     when(request.execute()).thenReturn(
         generateMockMetricResponse(true /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     runner.run(p, mockRunner);
   }
 
@@ -250,7 +250,7 @@ public class TestDataflowRunnerTest {
 
     when(request.execute()).thenReturn(
         generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     try {
       runner.run(p, mockRunner);
     } catch (AssertionError expected) {
@@ -269,7 +269,7 @@ public class TestDataflowRunnerTest {
     PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
     PAssert.that(pc).containsInAnyOrder(1, 2, 3);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     when(request.execute()).thenReturn(
         generateMockMetricResponse(true /* success */, true /* tentative */));
     doReturn(State.DONE).when(job).getState();
@@ -284,7 +284,7 @@ public class TestDataflowRunnerTest {
     PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
     PAssert.that(pc).containsInAnyOrder(1, 2, 3);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     when(request.execute()).thenReturn(
         generateMockMetricResponse(false /* success */, true /* tentative */));
     doReturn(State.DONE).when(job).getState();
@@ -299,7 +299,7 @@ public class TestDataflowRunnerTest {
     PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
     PAssert.that(pc).containsInAnyOrder(1, 2, 3);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     when(request.execute()).thenReturn(
         generateMockMetricResponse(true /* success */, false /* tentative */));
     doReturn(State.RUNNING).when(job).getState();
@@ -335,7 +335,7 @@ public class TestDataflowRunnerTest {
     PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
     PAssert.that(pc).containsInAnyOrder(1, 2, 3);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     when(request.execute()).thenReturn(
         generateMockMetricResponse(true /* success */, false /* tentative */));
     doReturn(State.FAILED).when(job).getState();
@@ -373,7 +373,7 @@ public class TestDataflowRunnerTest {
 
     when(request.execute()).thenReturn(
         generateMockMetricResponse(false /* success */, true /* tentative */));
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     try {
       runner.run(p, mockRunner);
     } catch (AssertionError expected) {
@@ -401,7 +401,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
 
@@ -426,7 +426,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0));
 
@@ -453,7 +453,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
 
@@ -478,7 +478,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1));
 
@@ -505,7 +505,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnSuccessMatcher(new TestFailureMatcher());
 
@@ -537,7 +537,7 @@ public class TestDataflowRunnerTest {
     DataflowRunner mockRunner = Mockito.mock(DataflowRunner.class);
     when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
 
-    TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
     p.getOptions().as(TestPipelineOptions.class)
         .setOnSuccessMatcher(new TestFailureMatcher());
 


[26/50] [abbrv] incubator-beam git commit: Touch up BigDecimalCoder and tests

Posted by da...@apache.org.
Touch up BigDecimalCoder and tests


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

Branch: refs/heads/python-sdk
Commit: 6491100a5d655cb9f6c702767d6354269208f650
Parents: 8268f1d
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 9 13:24:28 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/coders/BigDecimalCoder.java |  56 ++++------
 .../beam/sdk/coders/BigDecimalCoderTest.java    | 105 +++++++++++++------
 2 files changed, 95 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6491100a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
index 3b723b9..c4b7cd1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
@@ -17,20 +17,22 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.fasterxml.jackson.annotation.JsonCreator;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.math.MathContext;
 
 /**
- * A {@link BigDecimalCoder} encodes {@link BigDecimal} in an integer and
- * a byte array. The integer represents the scale and the byte array
- * represents a {@link BigInteger}. The integer is in 4 bytes, big-endian.
+ * A {@link BigDecimalCoder} encodes a {@link BigDecimal} as an integer scale encoded with
+ * {@link VarIntCoder} and a {@link BigInteger} encoded using {@link BigIntegerCoder}. The
+ * {@link BigInteger}, when scaled (with unlimited precision, aka {@link MathContext#UNLIMITED}),
+ * yields the expected {@link BigDecimal}.
  */
 public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
 
@@ -43,37 +45,25 @@ public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
 
   private static final BigDecimalCoder INSTANCE = new BigDecimalCoder();
 
+  private final VarIntCoder integerCoder = VarIntCoder.of();
+  private final BigIntegerCoder bigIntegerCoder = BigIntegerCoder.of();
+
   private BigDecimalCoder() {}
 
   @Override
   public void encode(BigDecimal value, OutputStream outStream, Context context)
       throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null BigDecimal");
-    }
-
-    byte[] bigIntBytes = value.unscaledValue().toByteArray();
-
-    DataOutputStream dataOutputStream = new DataOutputStream(outStream);
-    dataOutputStream.writeInt(value.scale());
-    dataOutputStream.writeInt(bigIntBytes.length);
-    dataOutputStream.write(bigIntBytes);
+    checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName()));
+    integerCoder.encode(value.scale(), outStream, context.nested());
+    bigIntegerCoder.encode(value.unscaledValue(), outStream, context.nested());
   }
 
   @Override
   public BigDecimal decode(InputStream inStream, Context context)
       throws IOException, CoderException {
-    DataInputStream dataInputStream = new DataInputStream(inStream);
-    int scale = dataInputStream.readInt();
-    int bigIntBytesSize = dataInputStream.readInt();
-
-    byte[] bigIntBytes = new byte[bigIntBytesSize];
-    dataInputStream.readFully(bigIntBytes);
-
-    BigInteger bigInteger = new BigInteger(bigIntBytes);
-    BigDecimal bigDecimal = new BigDecimal(bigInteger, scale);
-
-    return bigDecimal;
+    int scale = integerCoder.decode(inStream, context.nested());
+    BigInteger bigInteger = bigIntegerCoder.decode(inStream, context.nested());
+    return new BigDecimal(bigInteger, scale);
   }
 
   /**
@@ -99,14 +89,14 @@ public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
   /**
    * {@inheritDoc}
    *
-   * @return {@code 8} plus the size of the {@link BigInteger} bytes.
+   * @return {@code 4} (the size of an integer denoting the scale) plus {@code 4} (the size of an
+   * integer length prefix for the following bytes) plus the size of the two's-complement
+   * representation of the {@link BigInteger} that, when scaled, equals the given value.
    */
   @Override
-  protected long getEncodedElementByteSize(BigDecimal value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null BigDecimal");
-    }
-    return 8 + value.unscaledValue().toByteArray().length;
+  protected long getEncodedElementByteSize(BigDecimal value, Context context) throws Exception {
+    checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName()));
+    return integerCoder.getEncodedElementByteSize(value.scale(), context.nested())
+        + bigIntegerCoder.getEncodedElementByteSize(value.unscaledValue(), context.nested());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6491100a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
index 5911535..033f076 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
@@ -17,8 +17,17 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+
 import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterName;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
+import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -27,7 +36,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 import java.math.BigDecimal;
-import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -36,19 +44,21 @@ import java.util.List;
 @RunWith(JUnit4.class)
 public class BigDecimalCoderTest {
 
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
   private static final Coder<BigDecimal> TEST_CODER = BigDecimalCoder.of();
 
-  private static final List<BigDecimal> TEST_VALUES = Arrays.asList(
-      new BigDecimal(Double.MIN_VALUE),
-      new BigDecimal(-11),
-      new BigDecimal(-3),
-      new BigDecimal(-1),
-      new BigDecimal(0),
-      new BigDecimal(1),
-      new BigDecimal(5),
-      new BigDecimal(13),
-      new BigDecimal(29),
-      new BigDecimal(Double.MAX_VALUE));
+  private static final List<BigDecimal> TEST_VALUES =
+      ImmutableList.of(
+          new BigDecimal(Double.MIN_VALUE).divide(BigDecimal.TEN),
+          new BigDecimal(Double.MIN_VALUE),
+          new BigDecimal(-10.5),
+          new BigDecimal(-1),
+          new BigDecimal(0),
+          new BigDecimal(1),
+          new BigDecimal(13.258),
+          new BigDecimal(Double.MAX_VALUE),
+          new BigDecimal(Double.MAX_VALUE).multiply(BigDecimal.TEN));
 
   @Test
   public void testDecodeEncodeEqual() throws Exception {
@@ -69,37 +79,66 @@ public class BigDecimalCoderTest {
    * Generated data to check that the wire format has not changed. To regenerate, see
    * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
-  private static final List<String> TEST_ENCODINGS = Arrays.asList(
-      "AAAEMgAAATg12KOw51bHBNnjNkPn-wPiaWQ_AsohTe-mXyOGWcybUGt9TKi2FHqY2OH-gV0_GWqRbjNAGsSskI7K3" +
-          "xf9JmTjf1ySZXuvF9S9PsgV3kT-sgypaRw_i1MK_orzcJVg_s3cEGTjTY1_Xor3JM9UBVKiQy3Vpulf7aN9LM" +
-          "kiQEfO28mXQibyUtXL4yoLIwujoo8ArC9SayfbH5HmUxX9G0e506_cefoYIGByfq3M8GLp1_METj97ViU38je" +
-          "xsXkggqxXrMG8PO6pCYNB8P_jcf9i5OagpPafem18giZ8-v3fWJPN63vkbuOtaHb9u9yGQfrN25aLpNW9ooU9" +
-          "eYbL-1ewSBwENptcIT5SMhkulcVY6e9LyAqamGWdvnbevpwW84rTQpkeJePOkIt6G1_slfkQn6VBw7Jz3Vk",
-      "AAAAAAAAAAH1",
-      "AAAAAAAAAAH9",
-      "AAAAAAAAAAH_",
-      "AAAAAAAAAAEA",
-      "AAAAAAAAAAEB",
-      "AAAAAAAAAAEF",
-      "AAAAAAAAAAEN",
-      "AAAAAAAAAAEd",
-      "AAAAAAAAAIEA________-AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
-          "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
-          "AAAAAAAAA");
+  private static final List<String> TEST_ENCODINGS =
+      ImmutableList.of(
+          "swi4AjXYo7DnVscE2eM2Q-f7A-JpZD8CyiFN76ZfI4ZZzJtQa31MqLYUepjY4f6BXT8ZapFuM"
+              + "0AaxKyQjsrfF_0mZON_XJJle68X1L0-yBXeRP6yDKlpHD-LUwr-ivNwlWD-zdwQZONNjX9"
+              + "eivckz1QFUqJDLdWm6V_to30sySJAR87byZdCJvJS1cvjKgsjC6OijwCsL1JrJ9sfkeZTF"
+              + "f0bR7nTr9x5-hggYHJ-rczwYunX8wROP3tWJTfyN7GxeSCCrFeswbw87qkJg0Hw_-Nx_2L"
+              + "k5qCk9p96bXyCJnz6_d9Yk83re-Ru461odv273IZB-s3blouk1b2ihT15hsv7V7BIHAQ2m"
+              + "1whPlIyGS6VxVjp70vICpqYZZ2-dt6-nBbzitNCmR4l486Qi3obX-yV-RCfpUHDsnPdWQ",
+          "sgi4AjXYo7DnVscE2eM2Q-f7A-JpZD8CyiFN76ZfI4ZZzJtQa31MqLYUepjY4f6BXT8ZapFu"
+              + "M0AaxKyQjsrfF_0mZON_XJJle68X1L0-yBXeRP6yDKlpHD-LUwr-ivNwlWD-zdwQZONNj"
+              + "X9eivckz1QFUqJDLdWm6V_to30sySJAR87byZdCJvJS1cvjKgsjC6OijwCsL1JrJ9sfke"
+              + "ZTFf0bR7nTr9x5-hggYHJ-rczwYunX8wROP3tWJTfyN7GxeSCCrFeswbw87qkJg0Hw_-Nx"
+              + "_2Lk5qCk9p96bXyCJnz6_d9Yk83re-Ru461odv273IZB-s3blouk1b2ihT15hsv7V7BIHA"
+              + "Q2m1whPlIyGS6VxVjp70vICpqYZZ2-dt6-nBbzitNCmR4l486Qi3obX-yV-RCfpUHDsnPdWQ",
+          "AQGX",
+          "AAH_",
+          "AAEA",
+          "AAEB",
+          "MBUJEk1IAgE1H9Gsru39PDZgUqT1NnU",
+          "AIEBAP________gAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA",
+          "AIEBCf_______7AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA");
 
   @Test
   public void testWireFormatEncode() throws Exception {
     CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS);
   }
 
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
+  @Test
+  public void testGetEncodedElementByteSize() throws Exception {
+    Counter<Long> counter = Counter.longs(CounterName.named("dummy"), AggregationKind.SUM);
+    ElementByteSizeObserver observer = new ElementByteSizeObserver(counter);
+    for (BigDecimal value : TEST_VALUES) {
+      TEST_CODER.registerByteSizeObserver(value, observer, Coder.Context.OUTER);
+      observer.advance();
+      assertThat(
+          counter.getAggregate(),
+          equalTo((long) CoderUtils.encodeToByteArray(TEST_CODER, value).length));
+      counter.resetToValue(0L);
+    }
+  }
 
   @Test
-  public void encodeNullThrowsCoderException() throws Exception {
-    thrown.expect(CoderException.class);
+  public void encodeNullThrowsException() throws Exception {
+    thrown.expect(NullPointerException.class);
     thrown.expectMessage("cannot encode a null BigDecimal");
 
     CoderUtils.encodeToBase64(TEST_CODER, null);
   }
+
+  /**
+   * This is a change-detector test. If this test fails, then the encoding id of
+   * {@link BigDecimalCoder} must change.
+   */
+  @Test
+  public void testCoderIdDependencies() {
+    assertThat(VarIntCoder.of().getEncodingId(), equalTo(""));
+    assertThat(BigIntegerCoder.of().getEncodingId(), equalTo(""));
+  }
 }


[45/50] [abbrv] incubator-beam git commit: DataflowPipelineJob: Retry messages, metrics, and status polls

Posted by da...@apache.org.
DataflowPipelineJob: Retry messages, metrics, and status polls

At some point in the past, we decided to use a rawDataflowClient that
does not do retries when checking job status, because it was best-effort
reporting to users. The purported goal was to not clutter the log with
networking errors.

However, since that time, we have:
* Added the ability to suppress logs (emit only at DEBUG level or not at
  all) when retrying.
* Increased reliability of the job checking status so that these errors
  are less frequent and more indicative of quota or other issues.
* Started using the metrics in tests, where we do need to retry
  transient issues (BEAM-350).

So let's drop the raw transport client and just use the one that
retries.


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

Branch: refs/heads/python-sdk
Commit: d285e675920cd790c68053291c9bf843c21fc493
Parents: 30d226a
Author: Dan Halperin <dh...@google.com>
Authored: Thu Jun 16 08:57:18 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/runners/dataflow/DataflowRunner.java |  4 ++--
 .../beam/runners/dataflow/util/DataflowTransport.java    | 11 -----------
 2 files changed, 2 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d285e675/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 91e34ac..5818ba5 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -633,12 +633,12 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     // regularly and need not be retried automatically.
     DataflowPipelineJob dataflowPipelineJob =
         new DataflowPipelineJob(options.getProject(), jobResult.getId(),
-            DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
+            options.getDataflowClient(), aggregatorTransforms);
 
     // If the service returned client request id, the SDK needs to compare it
     // with the original id generated in the request, if they are not the same
     // (i.e., the returned job is not created by this request), throw
-    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
+    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedException
     // depending on whether this is a reload or not.
     if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
         && !jobResult.getClientRequestId().equals(requestId)) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d285e675/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
index 09fca94..f988749 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
@@ -92,17 +92,6 @@ public class DataflowTransport {
         .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
   }
 
-  /**
-   * Returns a Dataflow client that does not automatically retry failed
-   * requests.
-   */
-  public static Dataflow.Builder
-      newRawDataflowClient(DataflowPipelineOptions options) {
-    return newDataflowClient(options)
-        .setHttpRequestInitializer(options.getGcpCredential())
-        .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
-  }
-
   private static HttpRequestInitializer chainHttpRequestInitializer(
       Credential credential, HttpRequestInitializer httpRequestInitializer) {
     if (credential == null) {


[22/50] [abbrv] incubator-beam git commit: Fix type error in Eclipse

Posted by da...@apache.org.
Fix type error in Eclipse

This type error occurs in my Eclipse installation. It apparently
does not bother the various JDKs we test with. But this is an
accurate typing, so it may help other Eclipse-using contributors,
too.


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

Branch: refs/heads/python-sdk
Commit: cfa217a894575f392f1dfe1612e10e393df5c7ab
Parents: babddbb
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 14 16:12:11 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/ExecutorServiceParallelExecutor.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cfa217a8/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
index 4bb5021..78f3fe4 100644
--- 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
@@ -415,7 +415,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
               if (delivery.isEmpty()) {
                 continue;
               }
-              KeyedWorkItem<Object, Object> work =
+              KeyedWorkItem<?, Object> work =
                   KeyedWorkItems.timersWorkItem(keyTimers.getKey().getKey(), delivery);
               @SuppressWarnings({"unchecked", "rawtypes"})
               CommittedBundle<?> bundle =


[13/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index 21c941a..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java
+++ /dev/null
@@ -1,1428 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.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.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.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;
-
-import javax.annotation.Nullable;
-
-/**
- * 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(output)),
-        new Instant(8000L));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(intsToFlatten.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    // 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,
-        TimerUpdate.empty(),
-        result(flattened.getProducingTransformInternal(),
-            secondPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks transformAfterProcessing =
-        manager.getWatermarks(flattened.getProducingTransformInternal());
-    manager.updateWatermarks(secondPcollectionBundle,
-        TimerUpdate.empty(),
-        result(flattened.getProducingTransformInternal(),
-            secondPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(flattened.getProducingTransformInternal(),
-            firstPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(completedFlattenBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createdBundle)),
-        new Instant(Long.MAX_VALUE));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(keyBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            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,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(keyBundle)),
-        new Instant(500L));
-    manager.refreshAll();
-    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,
-        StructuralKey.of("Odd", StringUtf8Coder.of()),
-        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,
-        StructuralKey.of("Even", StringUtf8Coder.of()),
-        createdInts)
-        .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
-        .commit(clock.now());
-
-    manager.updateWatermarks(null,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            ImmutableList.of(firstKeyBundle, secondKeyBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    manager.updateWatermarks(firstKeyBundle,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            firstKeyBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(-1000L));
-    manager.updateWatermarks(secondKeyBundle,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            secondKeyBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(1234L));
-    manager.refreshAll();
-
-    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,
-        StructuralKey.of("Odd", StringUtf8Coder.of()),
-        createdInts).commit(clock.now());
-    manager.updateWatermarks(fauxFirstKeyTimerBundle,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            fauxFirstKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
-
-    CommittedBundle<Integer> fauxSecondKeyTimerBundle = bundleFactory.createKeyedBundle(null,
-        StructuralKey.of("Even", StringUtf8Coder.of()),
-        createdInts).commit(clock.now());
-    manager.updateWatermarks(fauxSecondKeyTimerBundle,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            fauxSecondKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(5678L));
-    manager.refreshAll();
-    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
-
-    manager.updateWatermarks(fauxSecondKeyTimerBundle,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            fauxSecondKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,  TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(firstInput)),
-        new Instant(0L));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(secondInput)),
-        new Instant(-250L));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            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,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(keyBundle)),
-        new Instant(500L));
-    manager.refreshAll();
-    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));
-  }
-
-  @Test
-  public void updateWatermarkWithUnprocessedElements() {
-    WindowedValue<Integer> first = WindowedValue.valueInGlobalWindow(1);
-    WindowedValue<Integer> second =
-        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-1000L));
-    WindowedValue<Integer> third =
-        WindowedValue.timestampedValueInGlobalWindow(3, new Instant(1234L));
-    CommittedBundle<Integer> createdBundle = bundleFactory.createRootBundle(createdInts)
-        .add(first)
-        .add(second)
-        .add(third)
-        .commit(clock.now());
-    manager.updateWatermarks(null,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createdBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    CommittedBundle<KV<String, Integer>> keyBundle = timestampedBundle(keyed,
-        TimestampedValue.of(KV.of("MyKey", 1), BoundedWindow.TIMESTAMP_MIN_VALUE));
-    manager.updateWatermarks(createdBundle,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(ImmutableList.of(second, third)),
-            Collections.<CommittedBundle<?>>singleton(keyBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    TransformWatermarks keyedWatermarks =
-        manager.getWatermarks(keyed.getProducingTransformInternal());
-    // the unprocessed second and third are readded to pending
-    assertThat(
-        keyedWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
-  }
-
-  /**
-   * 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            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,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(keyBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(lateDataBundle)),
-        new Instant(2_000_000L));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            lateDataBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(lateKeyedBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-  }
-
-  public void updateWatermarkWithDifferentWindowedValueInstances() {
-    manager.updateWatermarks(
-        null,
-        TimerUpdate.empty(), result(createdInts.getProducingTransformInternal(), null,
-        Collections.<CommittedBundle<?>>singleton(
-            bundleFactory
-                .createRootBundle(createdInts)
-                .add(WindowedValue.valueInGlobalWindow(1))
-                .commit(Instant.now()))),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    CommittedBundle<Integer> createdBundle = bundleFactory.createRootBundle(createdInts)
-        .add(WindowedValue.valueInGlobalWindow(1))
-        .commit(Instant.now());
-    manager.updateWatermarks(createdBundle,
-        TimerUpdate.empty(),
-        result(keyed.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        null);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(emptyCreateOutput)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(firstCreateOutput)),
-        new Instant(12_000L));
-
-    CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
-    manager.updateWatermarks(firstCreateOutput,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            firstCreateOutput.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(firstFilterOutput)),
-        new Instant(10_000L));
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(emptyCreateOutput)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createOutput)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            createOutput.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filterOutputBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createdBundle)),
-        new Instant(1248L));
-    manager.refreshAll();
-
-    TransformWatermarks filteredWms =
-        manager.getWatermarks(filtered.getProducingTransformInternal());
-    TransformWatermarks filteredDoubledWms =
-        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
-    Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
-    Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
-
-    StructuralKey<String> key = StructuralKey.of("key", StringUtf8Coder.of());
-    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,
-        timers,
-        result(filtered.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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<StructuralKey<?>, 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,
-        TimerUpdate.builder(key)
-            .withCompletedTimers(Collections.<TimerData>singleton(pastTimer)).build(),
-        result(filtered.getProducingTransformInternal(),
-            filteredTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filteredTimerResult)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    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,
-        TimerUpdate.empty(),
-        result(filteredTimesTwo.getProducingTransformInternal(),
-            filteredTimerResult.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createOutput)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-    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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(createSecondOutput)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
-  }
-
-  @Test
-  public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
-    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
-    manager.updateWatermarks(null,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(created)),
-        new Instant(40_900L));
-    manager.refreshAll();
-
-    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,
-        TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of()))
-            .setTimer(upstreamProcessingTimer)
-            .build(),
-        result(filtered.getProducingTransformInternal(),
-            created.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    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,
-        TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of()))
-            .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)).build(),
-        result(filtered.getProducingTransformInternal(),
-            otherCreated.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>emptyList()),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
-  }
-
-  @Test
-  public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
-    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
-    manager.updateWatermarks(
-        null,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>singleton(created)),
-        new Instant(29_919_235L));
-
-    Instant upstreamHold = new Instant(2048L);
-    CommittedBundle<Integer> filteredBundle = bundleFactory.createKeyedBundle(created,
-        StructuralKey.of("key", StringUtf8Coder.of()),
-        filtered).commit(upstreamHold);
-    manager.updateWatermarks(
-        created,
-        TimerUpdate.empty(),
-        result(filtered.getProducingTransformInternal(),
-            created.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
-        BoundedWindow.TIMESTAMP_MAX_VALUE);
-    manager.refreshAll();
-
-    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<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.singleton(createdBundle)),
-        new Instant(1500L));
-    manager.refreshAll();
-
-    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);
-    StructuralKey<byte[]> key = StructuralKey.of(new byte[] {1, 4, 9}, ByteArrayCoder.of());
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .setTimer(lastTimer)
-            .build();
-
-    manager.updateWatermarks(createdBundle,
-        update,
-        result(filtered.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
-        new Instant(1000L));
-    manager.refreshAll();
-
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(50_000L));
-    manager.refreshAll();
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            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);
-    StructuralKey<?> key = StructuralKey.of(-12L, VarLongCoder.of());
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(lastTimer)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .build();
-
-    manager.updateWatermarks(
-        createdBundle,
-        update,
-        result(filtered.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
-        new Instant(1000L));
-    manager.refreshAll();
-
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(50_000L));
-    manager.refreshAll();
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            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);
-    StructuralKey<byte[]> key = StructuralKey.of(new byte[] {2, -2, 22}, ByteArrayCoder.of());
-    TimerUpdate update =
-        TimerUpdate.builder(key)
-            .setTimer(lastTimer)
-            .setTimer(earliestTimer)
-            .setTimer(middleTimer)
-            .build();
-
-    manager.updateWatermarks(
-        createdBundle,
-        update,
-        result(filtered.getProducingTransformInternal(),
-            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
-            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
-        new Instant(1000L));
-    manager.refreshAll();
-
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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,
-        TimerUpdate.empty(),
-        result(createdInts.getProducingTransformInternal(),
-            null,
-            Collections.<CommittedBundle<?>>emptyList()),
-        new Instant(50_000L));
-    manager.refreshAll();
-    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
-        manager.extractFiredTimers();
-    assertThat(
-        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
-    Map<StructuralKey<?>, 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(StructuralKey.of("foo", StringUtf8Coder.of()))
-            .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);
-  }
-
-  private final CommittedResult result(
-      AppliedPTransform<?, ?, ?> transform,
-      @Nullable CommittedBundle<?> unprocessedBundle,
-      Iterable<? extends CommittedBundle<?>> bundles) {
-    return CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
-        unprocessedBundle,
-        bundles);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index 3a3ac8c..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessBundleFactoryTest.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.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 createRootBundleShouldCreateWithEmptyKey() {
-    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(),
-        Matchers.<StructuralKey<?>>equalTo(StructuralKey.of(null, VoidCoder.of())));
-  }
-
-  private <T> void createKeyedBundle(Coder<T> coder, T key) throws Exception {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
-    StructuralKey skey = StructuralKey.of(key, coder);
-
-    UncommittedBundle<Integer> inFlightBundle =
-        bundleFactory.createKeyedBundle(null, skey, pcollection);
-
-    CommittedBundle<Integer> bundle = inFlightBundle.commit(Instant.now());
-    assertThat(bundle.getKey(), equalTo(skey));
-  }
-
-  @Test
-  public void keyedWithNullKeyShouldCreateKeyedBundle() throws Exception {
-    createKeyedBundle(VoidCoder.of(), null);
-  }
-
-  @Test
-  public void keyedWithKeyShouldCreateKeyedBundle() throws Exception {
-    createKeyedBundle(StringUtf8Coder.of(), "foo");
-    createKeyedBundle(VarIntCoder.of(), 1234);
-    createKeyedBundle(ByteArrayCoder.of(), new byte[] {0, 2, 4, 99});
-  }
-
-  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(), Matchers.<StructuralKey<?>>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,
-                StructuralKey.of("foo", StringUtf8Coder.of()),
-                created).commit(Instant.now()),
-            downstream).commit(Instant.now());
-    assertThat(newBundle.getKey().getKey(), Matchers.<Object>equalTo("foo"));
-  }
-
-  @Test
-  public void createKeyedBundleKeyed() {
-    CommittedBundle<KV<String, Integer>> keyedBundle = bundleFactory.createKeyedBundle(
-        bundleFactory.createRootBundle(created).commit(Instant.now()),
-        StructuralKey.of("foo", StringUtf8Coder.of()),
-        downstream).commit(Instant.now());
-    assertThat(keyedBundle.getKey().getKey(), Matchers.<Object>equalTo("foo"));
-  }
-}


[43/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
deleted file mode 100644
index 1eb39ad..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
+++ /dev/null
@@ -1,3229 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow;
-
-import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName;
-import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
-import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
-import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
-import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
-import org.apache.beam.runners.dataflow.internal.AssignWindows;
-import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
-import org.apache.beam.runners.dataflow.internal.IsmFormat;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
-import org.apache.beam.runners.dataflow.internal.IsmFormat.MetadataKeyCoder;
-import org.apache.beam.runners.dataflow.internal.ReadTranslator;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions;
-import org.apache.beam.runners.dataflow.util.DataflowTransport;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.coders.AvroCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.MapCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.StandardCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.FileBasedSink;
-import org.apache.beam.sdk.io.PubsubIO;
-import org.apache.beam.sdk.io.PubsubUnboundedSink;
-import org.apache.beam.sdk.io.PubsubUnboundedSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.ShardNameTemplate;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.Write;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.options.StreamingOptions;
-import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-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.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.PathValidator;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.ReleaseInfo;
-import org.apache.beam.sdk.util.Reshuffle;
-import org.apache.beam.sdk.util.SystemDoFnInternal;
-import org.apache.beam.sdk.util.ValueWithRecordId;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PCollectionTuple;
-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.POutput;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.services.clouddebugger.v2.Clouddebugger;
-import com.google.api.services.clouddebugger.v2.model.Debuggee;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.base.Utf8;
-import com.google.common.collect.ForwardingMap;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.joda.time.DateTimeUtils;
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.format.DateTimeFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.Serializable;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import javax.annotation.Nullable;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to the Dataflow representation
- * using the {@link DataflowPipelineTranslator} and then submitting
- * them to a Dataflow service for execution.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code DataflowPipelineRunner}, the Google cloudservices account and the Google compute engine
- * service account of the GCP project running the Dataflow Job will need access to the corresponding
- * source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
-  private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class);
-
-  /** Provided configuration options. */
-  private final DataflowPipelineOptions options;
-
-  /** Client for the Dataflow service. This is used to actually submit jobs. */
-  private final Dataflow dataflowClient;
-
-  /** Translator for this DataflowPipelineRunner, based on options. */
-  private final DataflowPipelineTranslator translator;
-
-  /** Custom transforms implementations. */
-  private final Map<Class<?>, Class<?>> overrides;
-
-  /** A set of user defined functions to invoke at different points in execution. */
-  private DataflowPipelineRunnerHooks hooks;
-
-  // Environment version information.
-  private static final String ENVIRONMENT_MAJOR_VERSION = "5";
-
-  // Default Docker container images that execute Dataflow worker harness, residing in Google
-  // Container Registry, separately for Batch and Streaming.
-  public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
-      = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160613";
-  public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
-      = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160613";
-
-  // The limit of CreateJob request size.
-  private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
-
-  private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
-
-  /**
-   * Project IDs must contain lowercase letters, digits, or dashes.
-   * IDs must start with a letter and may not end with a dash.
-   * This regex isn't exact - this allows for patterns that would be rejected by
-   * the service, but this is sufficient for basic validation of project IDs.
-   */
-  public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
-
-  /**
-   * Construct a runner from the provided options.
-   *
-   * @param options Properties that configure the runner.
-   * @return The newly created runner.
-   */
-  public static DataflowPipelineRunner fromOptions(PipelineOptions options) {
-    // (Re-)register standard IO factories. Clobbers any prior credentials.
-    IOChannelUtils.registerStandardIOFactories(options);
-
-    DataflowPipelineOptions dataflowOptions =
-        PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
-    ArrayList<String> missing = new ArrayList<>();
-
-    if (dataflowOptions.getAppName() == null) {
-      missing.add("appName");
-    }
-    if (missing.size() > 0) {
-      throw new IllegalArgumentException(
-          "Missing required values: " + Joiner.on(',').join(missing));
-    }
-
-    PathValidator validator = dataflowOptions.getPathValidator();
-    Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
-        && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
-        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
-    if (dataflowOptions.getStagingLocation() != null) {
-      validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
-    }
-    if (dataflowOptions.getTempLocation() != null) {
-      validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
-    }
-    if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
-      dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
-    } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
-      try {
-        dataflowOptions.setStagingLocation(
-            IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
-      } catch (IOException e) {
-        throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
-            + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
-      }
-    }
-
-    if (dataflowOptions.getFilesToStage() == null) {
-      dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
-          DataflowPipelineRunner.class.getClassLoader()));
-      LOG.info("PipelineOptions.filesToStage was not specified. "
-          + "Defaulting to files from the classpath: will stage {} files. "
-          + "Enable logging at DEBUG level to see which files will be staged.",
-          dataflowOptions.getFilesToStage().size());
-      LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
-    }
-
-    // Verify jobName according to service requirements, truncating converting to lowercase if
-    // necessary.
-    String jobName =
-        dataflowOptions
-            .getJobName()
-            .toLowerCase();
-    checkArgument(
-        jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
-        "JobName invalid; the name must consist of only the characters "
-            + "[-a-z0-9], starting with a letter and ending with a letter "
-            + "or number");
-    if (!jobName.equals(dataflowOptions.getJobName())) {
-      LOG.info(
-          "PipelineOptions.jobName did not match the service requirements. "
-              + "Using {} instead of {}.",
-          jobName,
-          dataflowOptions.getJobName());
-    }
-    dataflowOptions.setJobName(jobName);
-
-    // Verify project
-    String project = dataflowOptions.getProject();
-    if (project.matches("[0-9]*")) {
-      throw new IllegalArgumentException("Project ID '" + project
-          + "' invalid. Please make sure you specified the Project ID, not project number.");
-    } else if (!project.matches(PROJECT_ID_REGEXP)) {
-      throw new IllegalArgumentException("Project ID '" + project
-          + "' invalid. Please make sure you specified the Project ID, not project description.");
-    }
-
-    DataflowPipelineDebugOptions debugOptions =
-        dataflowOptions.as(DataflowPipelineDebugOptions.class);
-    // Verify the number of worker threads is a valid value
-    if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
-      throw new IllegalArgumentException("Number of worker harness threads '"
-          + debugOptions.getNumberOfWorkerHarnessThreads()
-          + "' invalid. Please make sure the value is non-negative.");
-    }
-
-    return new DataflowPipelineRunner(dataflowOptions);
-  }
-
-  @VisibleForTesting protected DataflowPipelineRunner(DataflowPipelineOptions options) {
-    this.options = options;
-    this.dataflowClient = options.getDataflowClient();
-    this.translator = DataflowPipelineTranslator.fromOptions(options);
-    this.pcollectionsRequiringIndexedFormat = new HashSet<>();
-    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
-
-    ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
-    if (options.isStreaming()) {
-      builder.put(Combine.GloballyAsSingletonView.class,
-                  StreamingCombineGloballyAsSingletonView.class);
-      builder.put(Create.Values.class, StreamingCreate.class);
-      builder.put(View.AsMap.class, StreamingViewAsMap.class);
-      builder.put(View.AsMultimap.class, StreamingViewAsMultimap.class);
-      builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class);
-      builder.put(View.AsList.class, StreamingViewAsList.class);
-      builder.put(View.AsIterable.class, StreamingViewAsIterable.class);
-      builder.put(Write.Bound.class, StreamingWrite.class);
-      builder.put(Read.Unbounded.class, StreamingUnboundedRead.class);
-      builder.put(Read.Bounded.class, UnsupportedIO.class);
-      builder.put(AvroIO.Read.Bound.class, UnsupportedIO.class);
-      builder.put(AvroIO.Write.Bound.class, UnsupportedIO.class);
-      builder.put(BigQueryIO.Read.Bound.class, UnsupportedIO.class);
-      builder.put(TextIO.Read.Bound.class, UnsupportedIO.class);
-      builder.put(TextIO.Write.Bound.class, UnsupportedIO.class);
-      builder.put(Window.Bound.class, AssignWindows.class);
-      // In streaming mode must use either the custom Pubsub unbounded source/sink or
-      // defer to Windmill's built-in implementation.
-      builder.put(PubsubIO.Read.Bound.PubsubBoundedReader.class, UnsupportedIO.class);
-      builder.put(PubsubIO.Write.Bound.PubsubBoundedWriter.class, UnsupportedIO.class);
-      if (options.getExperiments() == null
-          || !options.getExperiments().contains("enable_custom_pubsub_source")) {
-        builder.put(PubsubUnboundedSource.class, StreamingPubsubIORead.class);
-      }
-      if (options.getExperiments() == null
-          || !options.getExperiments().contains("enable_custom_pubsub_sink")) {
-        builder.put(PubsubUnboundedSink.class, StreamingPubsubIOWrite.class);
-      }
-    } else {
-      builder.put(Read.Unbounded.class, UnsupportedIO.class);
-      builder.put(Window.Bound.class, AssignWindows.class);
-      builder.put(Write.Bound.class, BatchWrite.class);
-      builder.put(AvroIO.Write.Bound.class, BatchAvroIOWrite.class);
-      builder.put(TextIO.Write.Bound.class, BatchTextIOWrite.class);
-      // In batch mode must use the custom Pubsub bounded source/sink.
-      builder.put(PubsubUnboundedSource.class, UnsupportedIO.class);
-      builder.put(PubsubUnboundedSink.class, UnsupportedIO.class);
-      if (options.getExperiments() == null
-          || !options.getExperiments().contains("disable_ism_side_input")) {
-        builder.put(View.AsMap.class, BatchViewAsMap.class);
-        builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
-        builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
-        builder.put(View.AsList.class, BatchViewAsList.class);
-        builder.put(View.AsIterable.class, BatchViewAsIterable.class);
-      }
-    }
-    overrides = builder.build();
-  }
-
-  /**
-   * Applies the given transform to the input. For transforms with customized definitions
-   * for the Dataflow pipeline runner, the application is intercepted and modified here.
-   */
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-
-    if (Combine.GroupedValues.class.equals(transform.getClass())
-        || GroupByKey.class.equals(transform.getClass())) {
-
-      // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
-      // primitives. Returning a primitive output instead of the expanded definition
-      // signals to the translator that translation is necessary.
-      @SuppressWarnings("unchecked")
-      PCollection<?> pc = (PCollection<?>) input;
-      @SuppressWarnings("unchecked")
-      OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
-          pc.getPipeline(),
-          transform instanceof GroupByKey
-              ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
-              : pc.getWindowingStrategy(),
-          pc.isBounded());
-      return outputT;
-    } else if (Window.Bound.class.equals(transform.getClass())) {
-      /*
-       * TODO: make this the generic way overrides are applied (using super.apply() rather than
-       * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
-       * additional nodes into the graph.
-       */
-      // casting to wildcard
-      @SuppressWarnings("unchecked")
-      OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
-      return windowed;
-    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
-        && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform(input, Create.of());
-    } else if (overrides.containsKey(transform.getClass())) {
-      // It is the responsibility of whoever constructs overrides to ensure this is type safe.
-      @SuppressWarnings("unchecked")
-      Class<PTransform<InputT, OutputT>> transformClass =
-          (Class<PTransform<InputT, OutputT>>) transform.getClass();
-
-      @SuppressWarnings("unchecked")
-      Class<PTransform<InputT, OutputT>> customTransformClass =
-          (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
-
-      PTransform<InputT, OutputT> customTransform =
-          InstanceBuilder.ofType(customTransformClass)
-          .withArg(DataflowPipelineRunner.class, this)
-          .withArg(transformClass, transform)
-          .build();
-
-      return Pipeline.applyTransform(input, customTransform);
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-  private <T> PCollection<T> applyWindow(
-      Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
-    // types are matched at compile time
-    @SuppressWarnings("unchecked")
-    Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
-    @SuppressWarnings("unchecked")
-    PCollection<T> input = (PCollection<T>) initialInput;
-    return super.apply(new AssignWindows<>(transform), input);
-  }
-
-  private String debuggerMessage(String projectId, String uniquifier) {
-    return String.format("To debug your job, visit Google Cloud Debugger at: "
-        + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
-        projectId, uniquifier);
-  }
-
-  private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
-    if (!options.getEnableCloudDebugger()) {
-      return;
-    }
-
-    if (options.getDebuggee() != null) {
-      throw new RuntimeException("Should not specify the debuggee");
-    }
-
-    Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
-    Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
-    options.setDebuggee(debuggee);
-
-    System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
-  }
-
-  private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
-    RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
-    registerReq.setDebuggee(new Debuggee()
-        .setProject(options.getProject())
-        .setUniquifier(uniquifier)
-        .setDescription(uniquifier)
-        .setAgentVersion("google.com/cloud-dataflow-java/v1"));
-
-    try {
-      RegisterDebuggeeResponse registerResponse =
-          debuggerClient.controller().debuggees().register(registerReq).execute();
-      Debuggee debuggee = registerResponse.getDebuggee();
-      if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
-        throw new RuntimeException("Unable to register with the debugger: "
-            + debuggee.getStatus().getDescription().getFormat());
-      }
-
-      return debuggee;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to register with the debugger: ", e);
-    }
-  }
-
-  @Override
-  public DataflowPipelineJob run(Pipeline pipeline) {
-    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
-
-    LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
-        + "related to Google Compute Engine usage and other Google Cloud Services.");
-
-    List<DataflowPackage> packages = options.getStager().stageFiles();
-
-
-    // Set a unique client_request_id in the CreateJob request.
-    // This is used to ensure idempotence of job creation across retried
-    // attempts to create a job. Specifically, if the service returns a job with
-    // a different client_request_id, it means the returned one is a different
-    // job previously created with the same job name, and that the job creation
-    // has been effectively rejected. The SDK should return
-    // Error::Already_Exists to user in that case.
-    int randomNum = new Random().nextInt(9000) + 1000;
-    String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
-        .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
-
-    // Try to create a debuggee ID. This must happen before the job is translated since it may
-    // update the options.
-    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-    maybeRegisterDebuggee(dataflowOptions, requestId);
-
-    JobSpecification jobSpecification =
-        translator.translate(pipeline, this, packages);
-    Job newJob = jobSpecification.getJob();
-    newJob.setClientRequestId(requestId);
-
-    String version = ReleaseInfo.getReleaseInfo().getVersion();
-    System.out.println("Dataflow SDK version: " + version);
-
-    newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
-    // The Dataflow Service may write to the temporary directory directly, so
-    // must be verified.
-    if (!Strings.isNullOrEmpty(options.getTempLocation())) {
-      newJob.getEnvironment().setTempStoragePrefix(
-          dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
-    }
-    newJob.getEnvironment().setDataset(options.getTempDatasetId());
-    newJob.getEnvironment().setExperiments(options.getExperiments());
-
-    // Set the Docker container image that executes Dataflow worker harness, residing in Google
-    // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
-    String workerHarnessContainerImage =
-        options.as(DataflowPipelineWorkerPoolOptions.class)
-        .getWorkerHarnessContainerImage();
-    for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
-      workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
-    }
-
-    // Requirements about the service.
-    Map<String, Object> environmentVersion = new HashMap<>();
-    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
-    newJob.getEnvironment().setVersion(environmentVersion);
-    // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
-    // autoscale if specified.
-    String jobType = "JAVA_BATCH_AUTOSCALING";
-
-    if (options.isStreaming()) {
-      jobType = "STREAMING";
-    }
-    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
-
-    if (hooks != null) {
-      hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
-    }
-
-    if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
-      try (PrintWriter printWriter = new PrintWriter(
-          new File(options.getDataflowJobFile()))) {
-        String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
-        printWriter.print(workSpecJson);
-        LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
-      } catch (IllegalStateException ex) {
-        LOG.warn("Cannot translate workflow spec to json for debug.");
-      } catch (FileNotFoundException ex) {
-        LOG.warn("Cannot create workflow spec output file.");
-      }
-    }
-
-    String jobIdToUpdate = null;
-    if (options.isUpdate()) {
-      jobIdToUpdate = getJobIdFromName(options.getJobName());
-      newJob.setTransformNameMapping(options.getTransformNameMapping());
-      newJob.setReplaceJobId(jobIdToUpdate);
-    }
-    Job jobResult;
-    try {
-      jobResult = dataflowClient
-              .projects()
-              .jobs()
-              .create(options.getProject(), newJob)
-              .execute();
-    } catch (GoogleJsonResponseException e) {
-      String errorMessages = "Unexpected errors";
-      if (e.getDetails() != null) {
-        if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
-          errorMessages = "The size of the serialized JSON representation of the pipeline "
-              + "exceeds the allowable limit. "
-              + "For more information, please check the FAQ link below:\n"
-              + "https://cloud.google.com/dataflow/faq";
-        } else {
-          errorMessages = e.getDetails().getMessage();
-        }
-      }
-      throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to create a workflow job", e);
-    }
-
-    // Obtain all of the extractors from the PTransforms used in the pipeline so the
-    // DataflowPipelineJob has access to them.
-    AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
-    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
-        aggregatorExtractor.getAggregatorSteps();
-
-    DataflowAggregatorTransforms aggregatorTransforms =
-        new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
-
-    // Use a raw client for post-launch monitoring, as status calls may fail
-    // regularly and need not be retried automatically.
-    DataflowPipelineJob dataflowPipelineJob =
-        new DataflowPipelineJob(options.getProject(), jobResult.getId(),
-            DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
-
-    // If the service returned client request id, the SDK needs to compare it
-    // with the original id generated in the request, if they are not the same
-    // (i.e., the returned job is not created by this request), throw
-    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
-    // depending on whether this is a reload or not.
-    if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
-        && !jobResult.getClientRequestId().equals(requestId)) {
-      // If updating a job.
-      if (options.isUpdate()) {
-        throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
-            String.format("The job named %s with id: %s has already been updated into job id: %s "
-                + "and cannot be updated again.",
-                newJob.getName(), jobIdToUpdate, jobResult.getId()));
-      } else {
-        throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
-            String.format("There is already an active job named %s with id: %s. If you want "
-                + "to submit a second job, try again by setting a different name using --jobName.",
-                newJob.getName(), jobResult.getId()));
-      }
-    }
-
-    LOG.info("To access the Dataflow monitoring console, please navigate to {}",
-        MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
-    System.out.println("Submitted job: " + jobResult.getId());
-
-    LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
-        MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
-
-    return dataflowPipelineJob;
-  }
-
-  /**
-   * Returns the DataflowPipelineTranslator associated with this object.
-   */
-  public DataflowPipelineTranslator getTranslator() {
-    return translator;
-  }
-
-  /**
-   * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}.
-   */
-  @Experimental
-  public void setHooks(DataflowPipelineRunnerHooks hooks) {
-    this.hooks = hooks;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  /** Outputs a warning about PCollection views without deterministic key coders. */
-  private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
-    // We need to wait till this point to determine the names of the transforms since only
-    // at this time do we know the hierarchy of the transforms otherwise we could
-    // have just recorded the full names during apply time.
-    if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
-      final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
-      pipeline.traverseTopologically(new PipelineVisitor() {
-        @Override
-        public void visitValue(PValue value, TransformTreeNode producer) {
-        }
-
-        @Override
-        public void visitPrimitiveTransform(TransformTreeNode node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-        }
-
-        @Override
-        public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
-          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
-            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
-          }
-          return CompositeBehavior.ENTER_TRANSFORM;
-        }
-
-        @Override
-        public void leaveCompositeTransform(TransformTreeNode node) {
-        }
-      });
-
-      LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
-          + "because the key coder is not deterministic. Falling back to singleton implementation "
-          + "which may cause memory and/or performance problems. Future major versions of "
-          + "Dataflow will require deterministic key coders.",
-          ptransformViewNamesWithNonDeterministicKeyCoders);
-    }
-  }
-
-  /**
-   * Returns true if the passed in {@link PCollection} needs to be materialiazed using
-   * an indexed format.
-   */
-  boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
-    return pcollectionsRequiringIndexedFormat.contains(pcol);
-  }
-
-  /**
-   * Marks the passed in {@link PCollection} as requiring to be materialized using
-   * an indexed format.
-   */
-  private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
-    pcollectionsRequiringIndexedFormat.add(pcol);
-  }
-
-  /** A set of {@link View}s with non-deterministic key coders. */
-  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
-
-  /**
-   * Records that the {@link PTransform} requires a deterministic key coder.
-   */
-  private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
-    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
-  }
-
-  /**
-   * A {@link GroupByKey} transform for the {@link DataflowPipelineRunner} which sorts
-   * values using the secondary key {@code K2}.
-   *
-   * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
-   * the empty window. Care must be taken *afterwards* to either re-window
-   * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
-   * values being within a window.
-   */
-  static class GroupByKeyAndSortValuesOnly<K1, K2, V>
-      extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
-    private GroupByKeyAndSortValuesOnly() {
-    }
-
-    @Override
-    public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
-      PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
-          PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
-          input.getPipeline(),
-          WindowingStrategy.globalDefault(),
-          IsBounded.BOUNDED);
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
-      rval.setCoder(
-          KvCoder.of(inputCoder.getKeyCoder(),
-          IterableCoder.of(inputCoder.getValueCoder())));
-      return rval;
-    }
-  }
-
-  /**
-   * A {@link PTransform} that groups the values by a hash of the window's byte representation
-   * and sorts the values using the windows byte representation.
-   */
-  private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
-      PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
-
-    /**
-     * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
-     * grouping by the hash of the window's byte representation and sorting the grouped values
-     * using the window's byte representation.
-     */
-    @SystemDoFnInternal
-    private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
-        extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
-
-      private final IsmRecordCoder<?> ismCoderForHash;
-      private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
-        this.ismCoderForHash = ismCoderForHash;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        @SuppressWarnings("unchecked")
-        W window = (W) c.window();
-        c.output(
-            KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
-                KV.of(window,
-                    WindowedValue.of(
-                        c.element(),
-                        c.timestamp(),
-                        c.window(),
-                        c.pane()))));
-      }
-    }
-
-    private final IsmRecordCoder<?> ismCoderForHash;
-    private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
-      this.ismCoderForHash = ismCoderForHash;
-    }
-
-    @Override
-    public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-      PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
-          input.apply(ParDo.of(
-              new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
-      rval.setCoder(
-          KvCoder.of(
-              VarIntCoder.of(),
-              KvCoder.of(windowCoder,
-                  FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
-      return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
-   * byte representation and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsSingleton<T>
-      extends PTransform<PCollection<T>, PCollectionView<T>> {
-
-    /**
-     * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
-     * <ul>
-     *   <li>Key 1: Window
-     *   <li>Value: Windowed value
-     * </ul>
-     */
-    static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                     IsmRecord<WindowedValue<T>>> {
-
-      private final Coder<W> windowCoder;
-      IsmRecordForSingularValuePerWindowDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        T previousValue = null;
-
-        Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
-        while (iterator.hasNext()) {
-          KV<W, WindowedValue<T>> next = iterator.next();
-          Object currentWindowStructuralValue = windowCoder.structuralValue(next.getKey());
-
-          // Verify that the user isn't trying to have more than one element per window as
-          // a singleton.
-          checkState(!previousWindowStructuralValue.isPresent()
-              || !previousWindowStructuralValue.get().equals(currentWindowStructuralValue),
-              "Multiple values [%s, %s] found for singleton within window [%s].",
-              previousValue,
-              next.getValue().getValue(),
-              next.getKey());
-
-          c.output(
-              IsmRecord.of(
-                  ImmutableList.of(next.getKey()), next.getValue()));
-
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          previousValue = next.getValue().getValue();
-        }
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    private final View.AsSingleton<T> transform;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsSingleton(DataflowPipelineRunner runner, View.AsSingleton<T> transform) {
-      this.runner = runner;
-      this.transform = transform;
-    }
-
-    @Override
-    public PCollectionView<T> apply(PCollection<T> input) {
-      @SuppressWarnings("unchecked")
-      Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
-          runner,
-          input,
-          new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(windowCoder),
-          transform.hasDefaultValue(),
-          transform.defaultValue(),
-          input.getCoder());
-    }
-
-    static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
-        applyForSingleton(
-            DataflowPipelineRunner runner,
-            PCollection<T> input,
-            DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                 IsmRecord<WindowedValue<FinalT>>> doFn,
-            boolean hasDefault,
-            FinalT defaultValue,
-            Coder<FinalT> defaultValueCoder) {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      PCollectionView<ViewT> view =
-          (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
-              input.getPipeline(),
-              (WindowingStrategy) input.getWindowingStrategy(),
-              hasDefault,
-              defaultValue,
-              defaultValueCoder);
-
-      IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
-          coderForSingleton(windowCoder, defaultValueCoder);
-
-      PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
-              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
-              .apply(ParDo.of(doFn));
-      reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-      return reifiedPerWindowAndSorted.apply(
-          CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsSingleton";
-    }
-
-    static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
-        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
-      return IsmRecordCoder.of(
-          1, // We hash using only the window
-          0, // There are no metadata records
-          ImmutableList.<Coder<?>>of(windowCoder),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
-   * and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Key 2: Index offset within window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsIterable<T>
-      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsIterable(DataflowPipelineRunner runner, View.AsIterable<T> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
-      PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
-          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
-      return BatchViewAsList.applyForIterableLike(runner, input, view);
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
-   * and with records having:
-   * <ul>
-   *   <li>Key 1: Window</li>
-   *   <li>Key 2: Index offset within window</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   */
-  static class BatchViewAsList<T>
-      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
-     * global window. Each {@link IsmRecord} has
-     * <ul>
-     *   <li>Key 1: Global window</li>
-     *   <li>Key 2: Index offset within window</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     */
-    @SystemDoFnInternal
-    static class ToIsmRecordForGlobalWindowDoFn<T>
-        extends DoFn<T, IsmRecord<WindowedValue<T>>> {
-
-      long indexInBundle;
-      @Override
-      public void startBundle(Context c) throws Exception {
-        indexInBundle = 0;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        c.output(IsmRecord.of(
-            ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
-            WindowedValue.of(
-                c.element(),
-                c.timestamp(),
-                GlobalWindow.INSTANCE,
-                c.pane())));
-        indexInBundle += 1;
-      }
-    }
-
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
-     * to locate the window boundaries. The {@link IsmRecord} has:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Key 2: Index offset within window</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     */
-    @SystemDoFnInternal
-    static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
-                     IsmRecord<WindowedValue<T>>> {
-
-      private final Coder<W> windowCoder;
-      ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        long elementsInWindow = 0;
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
-          // Compare to see if this is a new window so we can reset the index counter i
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Reset i since we have a new window.
-            elementsInWindow = 0;
-          }
-          c.output(IsmRecord.of(
-              ImmutableList.of(value.getKey(), elementsInWindow),
-              value.getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          elementsInWindow += 1;
-        }
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsList(DataflowPipelineRunner runner, View.AsList<T> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<List<T>> apply(PCollection<T> input) {
-      PCollectionView<List<T>> view = PCollectionViews.listView(
-          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
-      return applyForIterableLike(runner, input, view);
-    }
-
-    static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
-        DataflowPipelineRunner runner,
-        PCollection<T> input,
-        PCollectionView<ViewT> view) {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
-
-      // If we are working in the global window, we do not need to do a GBK using the window
-      // as the key since all the elements of the input PCollection are already such.
-      // We just reify the windowed value while converting them to IsmRecords and generating
-      // an index based upon where we are within the bundle. Each bundle
-      // maps to one file exactly.
-      if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
-        PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
-            input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
-        reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-        runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-        return reifiedPerWindowAndSorted.apply(
-            CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
-      }
-
-      PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
-              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
-              .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
-      reifiedPerWindowAndSorted.setCoder(ismCoder);
-
-      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
-      return reifiedPerWindowAndSorted.apply(
-          CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsList";
-    }
-
-    static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
-        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
-      // TODO: swap to use a variable length long coder which has values which compare
-      // the same as their byte representation compare lexicographically within the key coder
-      return IsmRecordCoder.of(
-          1, // We hash using only the window
-          0, // There are no metadata records
-          ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
-   * representation. Each record is structured as follows:
-   * <ul>
-   *   <li>Key 1: User key K</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: 0L (constant)</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   *
-   * <p>Alongside the data records, there are the following metadata records:
-   * <ul>
-   *   <li>Key 1: Metadata Key</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index [0, size of map]</li>
-   *   <li>Value: variable length long byte representation of size of map if index is 0,
-   *              otherwise the byte representation of a key</li>
-   * </ul>
-   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
-   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
-   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
-   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
-   * {@code [1, size of map]}.
-   *
-   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
-   * a warning to users to specify a deterministic key coder.
-   */
-  static class BatchViewAsMap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
-
-    /**
-     * A {@link DoFn} which groups elements by window boundaries. For each group,
-     * the group of elements is transformed into a {@link TransformedMap}.
-     * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
-     * and contains a function {@code WindowedValue<V> -> V}.
-     *
-     * <p>Outputs {@link IsmRecord}s having:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Value: Transformed map containing a transform that removes the encapsulation
-     *              of the window around each value,
-     *              {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
-     * </ul>
-     */
-    static class ToMapDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
-                     IsmRecord<WindowedValue<TransformedMap<K,
-                                             WindowedValue<V>,
-                                             V>>>> {
-
-      private final Coder<W> windowCoder;
-      ToMapDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c)
-          throws Exception {
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        Optional<W> previousWindow = Optional.absent();
-        Map<K, WindowedValue<V>> map = new HashMap<>();
-        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Construct the transformed map containing all the elements since we
-            // are at a window boundary.
-            c.output(IsmRecord.of(
-                ImmutableList.of(previousWindow.get()),
-                valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
-            map = new HashMap<>();
-          }
-
-          // Verify that the user isn't trying to insert the same key multiple times.
-          checkState(!map.containsKey(kv.getValue().getValue().getKey()),
-              "Multiple values [%s, %s] found for single key [%s] within window [%s].",
-              map.get(kv.getValue().getValue().getKey()),
-              kv.getValue().getValue().getValue(),
-              kv.getKey());
-          map.put(kv.getValue().getValue().getKey(),
-                  kv.getValue().withValue(kv.getValue().getValue().getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          previousWindow = Optional.of(kv.getKey());
-        }
-
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a transformed map containing all the elements since the last
-        // window boundary.
-        c.output(IsmRecord.of(
-            ImmutableList.of(previousWindow.get()),
-            valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsMap(DataflowPipelineRunner runner, View.AsMap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
-      return this.<BoundedWindow>applyInternal(input);
-    }
-
-    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
-        applyInternal(PCollection<KV<K, V>> input) {
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
-            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
-        return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
-      } catch (NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-
-        // Since the key coder is not deterministic, we convert the map into a singleton
-        // and return a singleton view equivalent.
-        return applyForSingletonFallback(input);
-      }
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsMap";
-    }
-
-    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
-        applyForSingletonFallback(PCollection<KV<K, V>> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      Coder<Function<WindowedValue<V>, V>> transformCoder =
-          (Coder) SerializableCoder.of(WindowedValueToValue.class);
-
-      Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
-          TransformedMapCoder.of(
-          transformCoder,
-          MapCoder.of(
-              inputCoder.getKeyCoder(),
-              FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
-
-      TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
-          WindowedValueToValue.<V>of(),
-          ImmutableMap.<K, WindowedValue<V>>of());
-
-      return BatchViewAsSingleton.<KV<K, V>,
-                                   TransformedMap<K, WindowedValue<V>, V>,
-                                   Map<K, V>,
-                                   W> applyForSingleton(
-          runner,
-          input,
-          new ToMapDoFn<K, V, W>(windowCoder),
-          true,
-          defaultValue,
-          finalValueCoder);
-    }
-  }
-
-  /**
-   * Specialized implementation for
-   * {@link org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
-   * Dataflow runner in batch mode.
-   *
-   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
-   * representation. Each record is structured as follows:
-   * <ul>
-   *   <li>Key 1: User key K</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index offset for a given key and window.</li>
-   *   <li>Value: Windowed value</li>
-   * </ul>
-   *
-   * <p>Alongside the data records, there are the following metadata records:
-   * <ul>
-   *   <li>Key 1: Metadata Key</li>
-   *   <li>Key 2: Window</li>
-   *   <li>Key 3: Index [0, size of map]</li>
-   *   <li>Value: variable length long byte representation of size of map if index is 0,
-   *              otherwise the byte representation of a key</li>
-   * </ul>
-   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
-   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
-   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
-   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
-   * {@code [1, size of map]}.
-   *
-   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
-   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
-   * a warning to users to specify a deterministic key coder.
-   */
-  static class BatchViewAsMultimap<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
-    /**
-     * A {@link PTransform} that groups elements by the hash of window's byte representation
-     * if the input {@link PCollection} is not within the global window. Otherwise by the hash
-     * of the window and key's byte representation. This {@link PTransform} also sorts
-     * the values by the combination of the window and key's byte representations.
-     */
-    private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
-        extends PTransform<PCollection<KV<K, V>>,
-                           PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
-
-      @SystemDoFnInternal
-      private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
-          extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
-          implements DoFn.RequiresWindowAccess {
-
-        private final IsmRecordCoder<?> coder;
-        private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
-          this.coder = coder;
-        }
-
-        @Override
-        public void processElement(ProcessContext c) throws Exception {
-          @SuppressWarnings("unchecked")
-          W window = (W) c.window();
-
-          c.output(
-              KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
-                  KV.of(KV.of(c.element().getKey(), window),
-                      WindowedValue.of(
-                          c.element().getValue(),
-                          c.timestamp(),
-                          (BoundedWindow) window,
-                          c.pane()))));
-        }
-      }
-
-      private final IsmRecordCoder<?> coder;
-      public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
-        this.coder = coder;
-      }
-
-      @Override
-      public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
-          apply(PCollection<KV<K, V>> input) {
-
-        @SuppressWarnings("unchecked")
-        Coder<W> windowCoder = (Coder<W>)
-            input.getWindowingStrategy().getWindowFn().windowCoder();
-        @SuppressWarnings("unchecked")
-        KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
-        PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
-        keyedByHash = input.apply(
-            ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
-        keyedByHash.setCoder(
-            KvCoder.of(
-                VarIntCoder.of(),
-                KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
-                    FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
-        return keyedByHash.apply(
-            new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
-      }
-    }
-
-    /**
-     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
-     * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Key 2: User key K</li>
-     *   <li>Key 3: Index offset for a given key and window.</li>
-     *   <li>Value: Windowed value</li>
-     * </ul>
-     *
-     * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
-     * and the unique key count per window to {@code outputForSize}.
-     *
-     * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
-     * throw an {@link IllegalStateException} if more than one key per window is found.
-     */
-    static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
-                     IsmRecord<WindowedValue<V>>> {
-
-      private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
-      private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
-      private final Coder<W> windowCoder;
-      private final Coder<K> keyCoder;
-      private final IsmRecordCoder<WindowedValue<V>> ismCoder;
-      private final boolean uniqueKeysExpected;
-      ToIsmRecordForMapLikeDoFn(
-          TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
-          TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
-          Coder<W> windowCoder,
-          Coder<K> keyCoder,
-          IsmRecordCoder<WindowedValue<V>> ismCoder,
-          boolean uniqueKeysExpected) {
-        this.outputForSize = outputForSize;
-        this.outputForEntrySet = outputForEntrySet;
-        this.windowCoder = windowCoder;
-        this.keyCoder = keyCoder;
-        this.ismCoder = ismCoder;
-        this.uniqueKeysExpected = uniqueKeysExpected;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        long currentKeyIndex = 0;
-        // We use one based indexing while counting
-        long currentUniqueKeyCounter = 1;
-        Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
-
-        KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
-        Object currentKeyStructuralValue =
-            keyCoder.structuralValue(currentValue.getKey().getKey());
-        Object currentWindowStructuralValue =
-            windowCoder.structuralValue(currentValue.getKey().getValue());
-
-        while (iterator.hasNext()) {
-          KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
-          Object nextKeyStructuralValue =
-              keyCoder.structuralValue(nextValue.getKey().getKey());
-          Object nextWindowStructuralValue =
-              windowCoder.structuralValue(nextValue.getKey().getValue());
-
-          outputDataRecord(c, currentValue, currentKeyIndex);
-
-          final long nextKeyIndex;
-          final long nextUniqueKeyCounter;
-
-          // Check to see if its a new window
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            // The next value is a new window, so we output for size the number of unique keys
-            // seen and the last key of the window. We also reset the next key index the unique
-            // key counter.
-            outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
-            outputMetadataRecordForEntrySet(c, currentValue);
-
-            nextKeyIndex = 0;
-            nextUniqueKeyCounter = 1;
-          } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
-            // It is a new key within the same window so output the key for the entry set,
-            // reset the key index and increase the count of unique keys seen within this window.
-            outputMetadataRecordForEntrySet(c, currentValue);
-
-            nextKeyIndex = 0;
-            nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
-          } else if (!uniqueKeysExpected) {
-            // It is not a new key so we don't have to output the number of elements in this
-            // window or increase the unique key counter. All we do is increase the key index.
-
-            nextKeyIndex = currentKeyIndex + 1;
-            nextUniqueKeyCounter = currentUniqueKeyCounter;
-          } else {
-            throw new IllegalStateException(String.format(
-                "Unique keys are expected but found key %s with values %s and %s in window %s.",
-                currentValue.getKey().getKey(),
-                currentValue.getValue().getValue(),
-                nextValue.getValue().getValue(),
-                currentValue.getKey().getValue()));
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-          currentKeyStructuralValue = nextKeyStructuralValue;
-          currentKeyIndex = nextKeyIndex;
-          currentUniqueKeyCounter = nextUniqueKeyCounter;
-        }
-
-        outputDataRecord(c, currentValue, currentKeyIndex);
-        outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a record with the number of unique keys seen.
-        outputMetadataRecordForEntrySet(c, currentValue);
-      }
-
-      /** This outputs the data record. */
-      private void outputDataRecord(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
-        IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
-            ImmutableList.of(
-                value.getKey().getKey(),
-                value.getKey().getValue(),
-                keyIndex),
-            value.getValue());
-        c.output(ismRecord);
-      }
-
-      /**
-       * This outputs records which will be used to compute the number of keys for a given window.
-       */
-      private void outputMetadataRecordForSize(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
-        c.sideOutput(outputForSize,
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
-                                                 value.getKey().getValue())),
-                KV.of(value.getKey().getValue(), uniqueKeyCount)));
-      }
-
-      /** This outputs records which will be used to construct the entry set. */
-      private void outputMetadataRecordForEntrySet(
-          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
-        c.sideOutput(outputForEntrySet,
-            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
-                                                 value.getKey().getValue())),
-                KV.of(value.getKey().getValue(), value.getKey().getKey())));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
-       * <ul>
-       *   <li>Key 1: META key</li>
-       *   <li>Key 2: window</li>
-       *   <li>Key 3: 0L (constant)</li>
-       *   <li>Value: sum of values for window</li>
-       * </ul>
-       *
-       * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
-       * per window for map and multimap side inputs.
-       */
-    static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
-      private final Coder<W> windowCoder;
-      ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
-        KV<W, Long> currentValue = iterator.next();
-        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
-        long size = 0;
-        while (iterator.hasNext()) {
-          KV<W, Long> nextValue = iterator.next();
-          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
-          size += currentValue.getValue();
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            c.output(IsmRecord.<WindowedValue<V>>meta(
-                ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
-                CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
-            size = 0;
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-        }
-
-        size += currentValue.getValue();
-        // Output the final value since it is guaranteed to be on a window boundary.
-        c.output(IsmRecord.<WindowedValue<V>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
-            CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
-       * <ul>
-       *   <li>Key 1: META key</li>
-       *   <li>Key 2: window</li>
-       *   <li>Key 3: index offset (1-based index)</li>
-       *   <li>Value: key</li>
-       * </ul>
-       *
-       * <p>This {@link DoFn} is meant to be used to output index to key records
-       * per window for map and multimap side inputs.
-       */
-    static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
-
-      private final Coder<K> keyCoder;
-      private final Coder<W> windowCoder;
-      ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
-        this.keyCoder = keyCoder;
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c) throws Exception {
-        Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
-        KV<W, K> currentValue = iterator.next();
-        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
-        long elementsInWindow = 1;
-        while (iterator.hasNext()) {
-          KV<W, K> nextValue = iterator.next();
-          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
-          c.output(IsmRecord.<WindowedValue<V>>meta(
-              ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
-              CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
-          elementsInWindow += 1;
-
-          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
-            elementsInWindow = 1;
-          }
-
-          currentValue = nextValue;
-          currentWindowStructuralValue = nextWindowStructuralValue;
-        }
-
-        // Output the final value since it is guaranteed to be on a window boundary.
-        c.output(IsmRecord.<WindowedValue<V>>meta(
-            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
-            CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
-      }
-    }
-
-    /**
-     * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
-     * partition, the set of elements is transformed into a {@link TransformedMap}.
-     * The transformed {@code Map<K, Iterable<V>>} is backed by a
-     * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
-     * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
-     *
-     * <p>Outputs {@link IsmRecord}s having:
-     * <ul>
-     *   <li>Key 1: Window</li>
-     *   <li>Value: Transformed map containing a transform that removes the encapsulation
-     *              of the window around each value,
-     *              {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
-     * </ul>
-     */
-    static class ToMultimapDoFn<K, V, W extends BoundedWindow>
-        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
-                     IsmRecord<WindowedValue<TransformedMap<K,
-                                                            Iterable<WindowedValue<V>>,
-                                                            Iterable<V>>>>> {
-
-      private final Coder<W> windowCoder;
-      ToMultimapDoFn(Coder<W> windowCoder) {
-        this.windowCoder = windowCoder;
-      }
-
-      @Override
-      public void processElement(ProcessContext c)
-          throws Exception {
-        Optional<Object> previousWindowStructuralValue = Optional.absent();
-        Optional<W> previousWindow = Optional.absent();
-        Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
-        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
-          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
-          if (previousWindowStructuralValue.isPresent()
-              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
-            // Construct the transformed map containing all the elements since we
-            // are at a window boundary.
-            @SuppressWarnings({"unchecked", "rawtypes"})
-            Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
-            c.output(IsmRecord.<WindowedValue<TransformedMap<K,
-                                                             Iterable<WindowedValue<V>>,
-                                                             Iterable<V>>>>of(
-                ImmutableList.of(previousWindow.get()),
-                valueInEmptyWindows(
-                    new TransformedMap<>(
-                        IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
-            multimap = HashMultimap.create();
-          }
-
-          multimap.put(kv.getValue().getValue().getKey(),
-                       kv.getValue().withValue(kv.getValue().getValue().getValue()));
-          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
-          previousWindow = Optional.of(kv.getKey());
-        }
-
-        // The last value for this hash is guaranteed to be at a window boundary
-        // so we output a transformed map containing all the elements since the last
-        // window boundary.
-        @SuppressWarnings({"unchecked", "rawtypes"})
-        Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
-        c.output(IsmRecord.<WindowedValue<TransformedMap<K,
-                                                         Iterable<WindowedValue<V>>,
-                                                         Iterable<V>>>>of(
-            ImmutableList.of(previousWindow.get()),
-            valueInEmptyWindows(
-                new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
-      }
-    }
-
-    private final DataflowPipelineRunner runner;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
-    public BatchViewAsMultimap(DataflowPipelineRunner runner, View.AsMultimap<K, V> transform) {
-      this.runner = runner;
-    }
-
-    @Override
-    public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-      return this.<BoundedWindow>applyInternal(input);
-    }
-
-    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
-        applyInternal(PCollection<KV<K, V>> input) {
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-      try {
-        PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
-            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
-
-        return applyForMapLike(runner, input, view, false /* unique keys not expected */);
-      } catch (NonDeterministicException e) {
-        runner.recordViewUsesNonDeterministicKeyCoder(this);
-
-        // Since the key coder is not deterministic, we convert the map into a singleton
-        // and return a singleton view equivalent.
-        return applyForSingletonFallback(input);
-      }
-    }
-
-    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
-        applyForSingletonFallback(PCollection<KV<K, V>> input) {
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
-          (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
-
-      Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
-          TransformedMapCoder.of(
-          transformCoder,
-          MapCoder.of(
-              inputCoder.getKeyCoder(),
-              IterableCoder.of(
-                  FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
-      TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
-          new TransformedMap<>(
-              IterableWithWindowedValuesToIterable.<V>of(),
-              ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
-
-      return BatchViewAsSingleton.<KV<K, V>,
-                                   TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
-                                   Map<K, Iterable<V>>,
-                                   W> applyForSingleton(
-          runner,
-          input,
-          new ToMultimapDoFn<K, V, W>(windowCoder),
-          true,
-          defaultValue,
-          finalValueCoder);
-    }
-
-    private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
-        DataflowPipelineRunner runner,
-        PCollection<KV<K, V>> input,
-        PCollectionView<ViewT> view,
-        boolean uniqueKeysExpected) throws NonDeterministicException {
-
-      @SuppressWarnings("unchecked")
-      Coder<W> windowCoder = (Coder<W>)
-          input.getWindowingStrategy().getWindowFn().windowCoder();
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
-      // If our key coder is deterministic, we can use the key portion of each KV
-      // part of a composite key containing the window , key and index.
-      inputCoder.getKeyCoder().verifyDeterministic();
-
-      IsmRecordCoder<WindowedValue<V>> ismCoder =
-          coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
-
-      // Create the various output tags representing the main output containing the data stream
-      // and the side outputs containing the metadata about the size and entry set.
-      TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
-      TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
-      TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
-
-      // Process all the elements grouped by key hash, and sorted by key and then window
-      // outputting to all the outputs defined above.
-      PCollectionTuple outputTuple = input
-           .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
-           .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
-                   outputForSizeTag, outputForEntrySetTag,
-                   windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
-                       .withOutputTags(mainOutputTag,
-                                       TupleTagList.of(
-                                           ImmutableList.<TupleTag<?>>of(outputForSizeTag,
-                                                                         outputForEntrySetTag))));
-
-      // Set the coder on the main data output.
-      PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
-          outputTuple.get(mainOutputTag);
-      perHashWithReifiedWindows.setCoder(ismCoder);
-
-      // Set the coder on the metadata output for size and process the entries
-      // producing a [META, Window, 0L] record per window storing the number of unique keys
-      // for each window.
-      PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
-      outputForSize.setCoder(
-          KvCoder.of(VarIntCoder.of(),
-                     KvCoder.of(windowCoder, VarLongCoder.of())));
-      PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
-          .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
-          .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
-      windowMapSizeMetadata.setCoder(ismCoder);
-
-      // Set the coder on the metadata output destined to build the entry set and process the
-      // entries producing a [META, Window, Index] record per window key pair storing the key.
-      PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
-          outputTuple.get(outputForEntrySetTag);
-      outputForEntrySet.setCoder(
-          KvCoder.of(VarIntCoder.of(),
-                     KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
-      PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
-          .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
-          .apply(ParDo.of(
-              new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
-      windowMapKeysMetadata.setCoder(ismCoder);
-
-      // Set that all these outputs should be materialized using an indexed format.
-      runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
-      runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
-      runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
-
-      PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
-          PCollectionList.of(ImmutableList.of(
-              perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
-
-      return Pipeline.applyTransform(outputs,
-                                     Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
-          .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
-                                        ViewT>of(view));
-    }
-
-    @Override
-    protected String getKindString() {
-      return "BatchViewAsMultimap";
-    }
-
-    static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
-        Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
-      // TODO: swap to use a variable length long coder which has values which compare
-      // the same as their byte representation compare lexicographically within the key coder
-      return IsmRecordCoder.of(
-          1, // We use only the key for hashing when producing value records
-          2, // Since the key is not present, we add the window to the hash when
-             // producing metadata records
-          ImmutableList.of(
-              MetadataKeyCoder.of(keyCoder),
-              windowCoder,
-              BigEndianLongCoder.of()),
-          FullWindowedValueCoder.of(valueCoder, windowCoder));
-    }
-  }
-
-  /**
-   * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
-   * {@code V1 -> V2}.
-   */
-  static class TransformedMap<K, V1, V2>
-      extends ForwardingMap<K, V2> {
-    private final Fu

<TRUNCATED>


[05/50] [abbrv] incubator-beam git commit: Fix NullPointerException in AfterWatermark display data

Posted by da...@apache.org.
Fix NullPointerException in AfterWatermark display data

Window transforms register display data for the associated trigger
function by calling its .toString() method. The AfterWatermark
trigger .toString() method was not properly handling cases where
there is no late firings registered.


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

Branch: refs/heads/python-sdk
Commit: d5e3dfaa864744ec9a011c51707d15f1ab68a734
Parents: 340fe3e
Author: Scott Wegner <sw...@google.com>
Authored: Wed Jun 15 09:51:59 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 .../sdk/transforms/windowing/AfterWatermark.java | 14 +++++++++-----
 .../transforms/windowing/AfterWatermarkTest.java | 19 +++++++++++++++++++
 2 files changed, 28 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d5e3dfaa/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
index e48cc44..019a68d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -31,6 +31,8 @@ import org.joda.time.Instant;
 import java.util.List;
 import java.util.Objects;
 
+import javax.annotation.Nullable;
+
 /**
  * <p>{@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a
  * lower-bound, sometimes heuristically established, on event times that have been fully processed
@@ -106,6 +108,7 @@ public class AfterWatermark {
     private static final int LATE_INDEX = 1;
 
     private final OnceTrigger earlyTrigger;
+    @Nullable
     private final OnceTrigger lateTrigger;
 
     @SuppressWarnings("unchecked")
@@ -226,7 +229,6 @@ public class AfterWatermark {
     public String toString() {
       StringBuilder builder = new StringBuilder(TO_STRING);
 
-      Trigger earlyTrigger = subTriggers.get(EARLY_INDEX);
       if (!(earlyTrigger instanceof Never.NeverTrigger)) {
         builder
             .append(".withEarlyFirings(")
@@ -234,10 +236,12 @@ public class AfterWatermark {
             .append(")");
       }
 
-      builder
-          .append(".withLateFirings(")
-          .append(subTriggers.get(LATE_INDEX))
-          .append(")");
+      if (lateTrigger != null && !(lateTrigger instanceof Never.NeverTrigger)) {
+        builder
+            .append(".withLateFirings(")
+            .append(lateTrigger)
+            .append(")");
+      }
 
       return builder.toString();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d5e3dfaa/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
index be0ec1c..d692cbf 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
@@ -345,6 +345,15 @@ public class AfterWatermarkTest {
   }
 
   @Test
+  public void testEarlyFiringsToString() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow()
+        .withEarlyFirings(StubTrigger.named("t1"))
+        .buildTrigger();
+
+    assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
+  }
+
+  @Test
   public void testLateFiringsToString() {
     Trigger trigger = AfterWatermark.pastEndOfWindow()
         .withLateFirings(StubTrigger.named("t1"))
@@ -363,4 +372,14 @@ public class AfterWatermarkTest {
     assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
         trigger.toString());
   }
+
+  @Test
+  public void testToStringExcludesNeverTrigger() {
+    Trigger trigger = AfterWatermark.pastEndOfWindow()
+        .withEarlyFirings(Never.ever())
+        .withLateFirings(Never.ever())
+        .buildTrigger();
+
+    assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
+  }
 }


[48/50] [abbrv] incubator-beam git commit: Configure RunnableOnService tests for Spark runner, batch mode

Posted by da...@apache.org.
Configure RunnableOnService tests for Spark runner, batch mode


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

Branch: refs/heads/python-sdk
Commit: 30d226a3ae547c4a2d890d1d42487862323a4ae3
Parents: a24e557
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu May 5 15:11:07 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 runners/spark/pom.xml                           | 112 +++++++++++++------
 .../runners/spark/SparkRunnerRegistrar.java     |   3 +-
 .../runners/spark/TestSparkPipelineRunner.java  |  77 +++++++++++++
 .../runners/spark/SparkRunnerRegistrarTest.java |   2 +-
 4 files changed, 155 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30d226a3/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index e7d0834..747464e 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -37,6 +37,62 @@
     <spark.version>1.6.1</spark.version>
   </properties>
 
+  <profiles>
+    <profile>
+      <id>jacoco</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.jacoco</groupId>
+            <artifactId>jacoco-maven-plugin</artifactId>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <profile>
+      <!-- This profile adds execution of RunnableOnService integration tests 
+           against a local Spark endpoint. -->
+      <id>runnable-on-service-tests</id>
+      <activation><activeByDefault>false</activeByDefault></activation>
+      <build>
+        <pluginManagement>
+          <plugins>
+            <plugin>
+              <groupId>org.apache.maven.plugins</groupId>
+              <artifactId>maven-surefire-plugin</artifactId>
+              <executions>
+                <execution>
+                  <id>runnable-on-service-tests</id>
+                  <configuration>
+                    <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
+                    <parallel>none</parallel>
+                    <failIfNoTests>true</failIfNoTests>
+                    <dependenciesToScan>
+                      <dependency>org.apache.beam:java-sdk-all</dependency>
+                    </dependenciesToScan>
+                    <excludes>
+                      org.apache.beam.sdk.io.BoundedReadFromUnboundedSourceTest
+                    </excludes>
+                    <systemPropertyVariables>
+                      <beamTestPipelineOptions>
+                        [
+                          "--runner=org.apache.beam.runners.spark.TestSparkPipelineRunner",
+                          "--streaming=false"
+                        ]
+                      </beamTestPipelineOptions>
+                      <dataflow.spark.test.reuseSparkContext>true</dataflow.spark.test.reuseSparkContext>
+                    </systemPropertyVariables>
+                  </configuration>
+                </execution>
+              </executions>
+            </plugin>
+          </plugins>
+        </pluginManagement>
+      </build>
+    </profile>
+  </profiles>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
@@ -122,6 +178,25 @@
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-direct-java</artifactId>
       <version>0.2.0-incubating-SNAPSHOT</version>
+    </dependency>
+
+    <!-- Depend on test jar to scan for RunnableOnService tests -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
@@ -237,41 +312,4 @@
     </plugins>
   </build>
 
-  <profiles>
-    <profile>
-      <id>jacoco</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.jacoco</groupId>
-            <artifactId>jacoco-maven-plugin</artifactId>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-
-    <profile>
-      <id>disable-runnable-on-service-tests</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>runnable-on-service-tests</id>
-                <configuration>
-                  <skip>true</skip>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30d226a3/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
index 9537ec6..baa2241 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java
@@ -43,7 +43,8 @@ public final class SparkRunnerRegistrar {
   public static class Runner implements PipelineRunnerRegistrar {
     @Override
     public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
+          SparkPipelineRunner.class, TestSparkPipelineRunner.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30d226a3/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkPipelineRunner.java
new file mode 100644
index 0000000..d11d1c1
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkPipelineRunner.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.spark;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+/**
+ * The SparkPipelineRunner translate operations defined on a pipeline to a representation executable
+ * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a dataflow
+ * pipeline with the default options of a single threaded spark instance in local mode, we would do
+ * the following:
+ *
+ * {@code
+ * Pipeline p = [logic for pipeline creation]
+ * EvaluationResult result = SparkPipelineRunner.create().run(p);
+ * }
+ *
+ * To create a pipeline runner to run against a different spark cluster, with a custom master url we
+ * would do the following:
+ *
+ * {@code
+ * Pipeline p = [logic for pipeline creation]
+ * SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
+ * options.setSparkMaster("spark://host:port");
+ * EvaluationResult result = SparkPipelineRunner.create(options).run(p);
+ * }
+ *
+ * To create a Spark streaming pipeline runner use {@link SparkStreamingPipelineOptions}
+ */
+public final class TestSparkPipelineRunner extends PipelineRunner<EvaluationResult> {
+
+  private SparkPipelineRunner delegate;
+
+  private TestSparkPipelineRunner(SparkPipelineOptions options) {
+    this.delegate = SparkPipelineRunner.fromOptions(options);
+  }
+
+  public static TestSparkPipelineRunner fromOptions(PipelineOptions options) {
+    // Default options suffice to set it up as a test runner
+    SparkPipelineOptions sparkOptions =
+        PipelineOptionsValidator.validate(SparkPipelineOptions.class, options);
+    return new TestSparkPipelineRunner(sparkOptions);
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput>
+      OutputT apply(PTransform<InputT, OutputT> transform, InputT input) {
+    return delegate.apply(transform, input);
+  };
+
+  @Override
+  public EvaluationResult run(Pipeline pipeline) {
+    return delegate.run(pipeline);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30d226a3/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java
index 88f4a06..d2e57aa 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java
@@ -47,7 +47,7 @@ public class SparkRunnerRegistrarTest {
 
   @Test
   public void testRunners() {
-    assertEquals(ImmutableList.of(SparkPipelineRunner.class),
+    assertEquals(ImmutableList.of(SparkPipelineRunner.class, TestSparkPipelineRunner.class),
         new SparkRunnerRegistrar.Runner().getPipelineRunners());
   }
 


[29/50] [abbrv] incubator-beam git commit: Remove the beam.examples dependency from flink.

Posted by da...@apache.org.
Remove the beam.examples dependency from flink.


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

Branch: refs/heads/python-sdk
Commit: e5812440ef985a44316e0dde7c5fa19d38f91aa0
Parents: 93f9ef9
Author: Pei He <pe...@google.com>
Authored: Thu Jun 16 11:38:51 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 runners/flink/runner/pom.xml | 12 ------------
 1 file changed, 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5812440/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 2fdf61d..8933457 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -106,18 +106,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-examples-java</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
       <groupId>org.apache.flink</groupId>
       <artifactId>flink-streaming-java_2.10</artifactId>
       <version>${flink.version}</version>


[24/50] [abbrv] incubator-beam git commit: [Spark] Elide assigning windows when WindowFn is null

Posted by da...@apache.org.
[Spark] Elide assigning windows when WindowFn is null

Previously, when translating a Window.Bound transform, the case
where the WindowFn was null was missed, resulting in a
NullPointerException.


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

Branch: refs/heads/python-sdk
Commit: 8278e5f78f36fb48fae994ee7abcc1485db84189
Parents: 0a7246d
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Jun 15 10:42:59 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../spark/translation/TransformTranslator.java     | 17 ++++++++++-------
 .../apache/beam/sdk/util/AssignWindowsDoFn.java    |  9 ++++++++-
 2 files changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8278e5f7/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
index ebceb6b..34a0ede 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
@@ -689,8 +689,6 @@ public final class TransformTranslator {
     rdd.saveAsNewAPIHadoopFile(outputDir, keyClass, valueClass, formatClass, conf);
   }
 
-  private static final FieldGetter WINDOW_FG = new FieldGetter(Window.Bound.class);
-
   private static <T, W extends BoundedWindow> TransformEvaluator<Window.Bound<T>> window() {
     return new TransformEvaluator<Window.Bound<T>>() {
       @Override
@@ -698,14 +696,19 @@ public final class TransformTranslator {
         @SuppressWarnings("unchecked")
         JavaRDDLike<WindowedValue<T>, ?> inRDD =
             (JavaRDDLike<WindowedValue<T>, ?>) context.getInputRDD(transform);
-        WindowFn<? super T, W> windowFn = WINDOW_FG.get("windowFn", transform);
+
+        @SuppressWarnings("unchecked")
+        WindowFn<? super T, W> windowFn = (WindowFn<? super T, W>) transform.getWindowFn();
+
         // Avoid running assign windows if both source and destination are global window
-        if (context.getInput(transform).getWindowingStrategy().getWindowFn()
-                instanceof GlobalWindows
-            && windowFn instanceof GlobalWindows) {
+        // or if the user has not specified the WindowFn (meaning they are just messing
+        // with triggering or allowed lateness)
+        if (windowFn == null
+            || (context.getInput(transform).getWindowingStrategy().getWindowFn()
+                    instanceof GlobalWindows
+                && windowFn instanceof GlobalWindows)) {
           context.setOutputRDD(transform, inRDD);
         } else {
-          @SuppressWarnings("unchecked")
           DoFn<T, T> addWindowsDoFn = new AssignWindowsDoFn<>(windowFn);
           DoFnFunction<T, T> dofn =
               new DoFnFunction<>(addWindowsDoFn, context.getRuntimeContext(), null);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8278e5f7/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
index e71a47e..caec40e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
@@ -37,7 +39,12 @@ public class AssignWindowsDoFn<T, W extends BoundedWindow> extends DoFn<T, T> {
   private WindowFn<? super T, W> fn;
 
   public AssignWindowsDoFn(WindowFn<? super T, W> fn) {
-    this.fn = fn;
+    this.fn =
+        checkNotNull(
+            fn,
+            "%s provided to %s cannot be null",
+            WindowFn.class.getSimpleName(),
+            AssignWindowsDoFn.class.getSimpleName());
   }
 
   @Override


[28/50] [abbrv] incubator-beam git commit: [BEAM-342] Implement Filter#greaterThan, etc with Filter#byPredicate

Posted by da...@apache.org.
[BEAM-342] Implement Filter#greaterThan,etc with Filter#byPredicate


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

Branch: refs/heads/python-sdk
Commit: 3d87f8b987e243c6b3d99ab67142301af7b65743
Parents: 6491100
Author: manuzhang <ow...@gmail.com>
Authored: Wed Jun 15 16:02:35 2016 +0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/AutoComplete.java    |   2 +-
 .../examples/complete/AutoCompleteTest.java     |  14 +-
 .../beam/examples/MinimalWordCountJava8.java    |   2 +-
 .../examples/complete/game/HourlyTeamScore.java |   6 +-
 .../examples/MinimalWordCountJava8Test.java     |   2 +-
 .../complete/game/HourlyTeamScoreTest.java      |   2 +-
 .../flink/examples/streaming/AutoComplete.java  |  12 +-
 .../org/apache/beam/sdk/transforms/Filter.java  | 128 +++++++------------
 .../apache/beam/sdk/transforms/FilterTest.java  |  63 +++------
 .../beam/sdk/transforms/FilterJava8Test.java    |   8 +-
 10 files changed, 89 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index d725e0a..3e4440c 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -235,7 +235,7 @@ public class AutoComplete {
             .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
             // ...together with those (previously excluded) candidates of length
             // exactly minPrefix...
-            .and(input.apply(Filter.byPredicate(
+            .and(input.apply(Filter.by(
                 new SerializableFunction<CompletionCandidate, Boolean>() {
                   @Override
                   public Boolean apply(CompletionCandidate c) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
index 93dd0be..b2ed9a2 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
@@ -85,13 +85,13 @@ public class AutoCompleteTest implements Serializable {
 
     PCollection<KV<String, List<CompletionCandidate>>> output =
       input.apply(new ComputeTopCompletions(2, recursive))
-           .apply(Filter.byPredicate(
-                        new SerializableFunction<KV<String, List<CompletionCandidate>>, Boolean>() {
-                          @Override
-                          public Boolean apply(KV<String, List<CompletionCandidate>> element) {
-                            return element.getKey().length() <= 2;
-                          }
-                      }));
+           .apply(Filter.by(
+               new SerializableFunction<KV<String, List<CompletionCandidate>>, Boolean>() {
+                 @Override
+                 public Boolean apply(KV<String, List<CompletionCandidate>> element) {
+                   return element.getKey().length() <= 2;
+                 }
+               }));
 
     PAssert.that(output).containsInAnyOrder(
         KV.of("a", parseList("apple:2", "apricot:1")),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
index d491741..0ad1a04 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -55,7 +55,7 @@ public class MinimalWordCountJava8 {
     p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
      .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
          .withOutputType(TypeDescriptors.strings()))
-     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
+     .apply(Filter.by((String word) -> !word.isEmpty()))
      .apply(Count.<String>perElement())
      .apply(MapElements
          .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index 845c56f..ba3983d 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -44,7 +44,7 @@ import java.util.TimeZone;
 /**
  * This class is the second in a series of four pipelines that tell a story in a 'gaming'
  * domain, following {@link UserScore}. In addition to the concepts introduced in {@link UserScore},
- * new concepts include: windowing and element timestamps; use of {@code Filter.byPredicate()}.
+ * new concepts include: windowing and element timestamps; use of {@code Filter.by()}.
  *
  * <p> This pipeline processes data collected from gaming events in batch, building on {@link
  * UserScore} but using fixed windows. It calculates the sum of scores per team, for each window,
@@ -164,10 +164,10 @@ public class HourlyTeamScore extends UserScore {
       // (to scoop up late-arriving events from the day we're analyzing), we need to weed out events
       // that fall after the time period we want to analyze.
       // [START DocInclude_HTSFilters]
-      .apply("FilterStartTime", Filter.byPredicate(
+      .apply("FilterStartTime", Filter.by(
           (GameActionInfo gInfo)
               -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
-      .apply("FilterEndTime", Filter.byPredicate(
+      .apply("FilterEndTime", Filter.by(
           (GameActionInfo gInfo)
               -> gInfo.getTimestamp() < stopMinTimestamp.getMillis()))
       // [END DocInclude_HTSFilters]

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
index f73250f..4dfa474 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java
@@ -66,7 +66,7 @@ public class MinimalWordCountJava8Test implements Serializable {
     p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*"))
      .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+")))
          .withOutputType(TypeDescriptors.strings()))
-     .apply(Filter.byPredicate((String word) -> !word.isEmpty()))
+     .apply(Filter.by((String word) -> !word.isEmpty()))
      .apply(Count.<String>perElement())
      .apply(MapElements
          .via((KV<String, Long> wordCount) -> wordCount.getKey() + ": " + wordCount.getValue())

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
index 5ff615a..4254902 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
@@ -96,7 +96,7 @@ public class HourlyTeamScoreTest implements Serializable {
     PCollection<KV<String, Integer>> output = input
       .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
 
-      .apply("FilterStartTime", Filter.byPredicate(
+      .apply("FilterStartTime", Filter.by(
           (GameActionInfo gInfo)
               -> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
       // run a map to access the fields in the result.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
index 9d1168b..d83e662 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
@@ -215,13 +215,13 @@ public class AutoComplete {
             // ...together with those (previously excluded) candidates of length
             // exactly minPrefix...
             .and(input.apply(Filter.by(new SerializableFunction<CompletionCandidate, Boolean>() {
-                    private static final long serialVersionUID = 0;
+              private static final long serialVersionUID = 0;
 
-                    @Override
-                    public Boolean apply(CompletionCandidate c) {
-                      return c.getValue().length() == minPrefix;
-                    }
-                  })))
+              @Override
+              public Boolean apply(CompletionCandidate c) {
+                return c.getValue().length() == minPrefix;
+              }
+            })))
             .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
             // ...set the key to be the minPrefix-length prefix...
             .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
index 57796b8..a31799e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java
@@ -41,7 +41,7 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * <pre> {@code
    * PCollection<String> wordList = ...;
    * PCollection<String> longWords =
-   *     wordList.apply(Filter.byPredicate(new MatchIfWordLengthGT(6)));
+   *     wordList.apply(Filter.by(new MatchIfWordLengthGT(6)));
    * } </pre>
    *
    * <p>See also {@link #lessThan}, {@link #lessThanEq},
@@ -50,25 +50,8 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * the elements' natural ordering.
    */
   public static <T, PredicateT extends SerializableFunction<T, Boolean>> Filter<T>
-  byPredicate(PredicateT predicate) {
-    return new Filter<T>("Filter", predicate);
-  }
-
-  /**
-   * @deprecated use {@link #byPredicate}, which returns a {@link Filter} transform instead of
-   * a {@link ParDo.Bound}.
-   */
-  @Deprecated
-  public static <T, PredicateT extends SerializableFunction<T, Boolean>> ParDo.Bound<T, T>
-  by(final PredicateT filterPred) {
-    return ParDo.named("Filter").of(new DoFn<T, T>() {
-      @Override
-      public void processElement(ProcessContext c) {
-        if (filterPred.apply(c.element()) == true) {
-          c.output(c.element());
-        }
-      }
-    });
+  by(PredicateT predicate) {
+    return new Filter<>(predicate);
   }
 
   /**
@@ -89,24 +72,16 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * inequalities with the specified value based on the elements'
    * natural ordering.
    *
-   * <p>See also {@link #byPredicate}, which returns elements
+   * <p>See also {@link #by}, which returns elements
    * that satisfy the given predicate.
    */
-  public static <T extends Comparable<T>> ParDo.Bound<T, T> lessThan(final T value) {
-    return ParDo.named("Filter.lessThan").of(new DoFn<T, T>() {
+  public static <T extends Comparable<T>> Filter<T> lessThan(final T value) {
+    return by(new SerializableFunction<T, Boolean>() {
       @Override
-      public void processElement(ProcessContext c) {
-        if (c.element().compareTo(value) < 0) {
-          c.output(c.element());
-        }
-      }
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        Filter.populateDisplayData(builder, String.format("x < %s", value));
+      public Boolean apply(T input) {
+        return input.compareTo(value) < 0;
       }
-    });
+    }).described(String.format("x < %s", value));
   }
 
 
@@ -128,24 +103,16 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * inequalities with the specified value based on the elements'
    * natural ordering.
    *
-   * <p>See also {@link #byPredicate}, which returns elements
+   * <p>See also {@link #by}, which returns elements
    * that satisfy the given predicate.
    */
-  public static <T extends Comparable<T>> ParDo.Bound<T, T> greaterThan(final T value) {
-    return ParDo.named("Filter.greaterThan").of(new DoFn<T, T>() {
-      @Override
-      public void processElement(ProcessContext c) {
-        if (c.element().compareTo(value) > 0) {
-          c.output(c.element());
-        }
-      }
-
+  public static <T extends Comparable<T>> Filter<T> greaterThan(final T value) {
+    return by(new SerializableFunction<T, Boolean>() {
       @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        Filter.populateDisplayData(builder, String.format("x > %s", value));
+      public Boolean apply(T input) {
+        return input.compareTo(value) > 0;
       }
-    });
+    }).described(String.format("x > %s", value));
   }
 
   /**
@@ -166,24 +133,16 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * inequalities with the specified value based on the elements'
    * natural ordering.
    *
-   * <p>See also {@link #byPredicate}, which returns elements
+   * <p>See also {@link #by}, which returns elements
    * that satisfy the given predicate.
    */
-  public static <T extends Comparable<T>> ParDo.Bound<T, T> lessThanEq(final T value) {
-    return ParDo.named("Filter.lessThanEq").of(new DoFn<T, T>() {
-      @Override
-      public void processElement(ProcessContext c) {
-        if (c.element().compareTo(value) <= 0) {
-          c.output(c.element());
-        }
-      }
-
+  public static <T extends Comparable<T>> Filter<T> lessThanEq(final T value) {
+    return by(new SerializableFunction<T, Boolean>() {
       @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        Filter.populateDisplayData(builder, String.format("x \u2264 %s", value));
+      public Boolean apply(T input) {
+        return input.compareTo(value) <= 0;
       }
-    });
+    }).described(String.format("x \u2264 %s", value));
   }
 
   /**
@@ -204,46 +163,46 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
    * inequalities with the specified value based on the elements'
    * natural ordering.
    *
-   * <p>See also {@link #byPredicate}, which returns elements
+   * <p>See also {@link #by}, which returns elements
    * that satisfy the given predicate.
    */
-  public static <T extends Comparable<T>> ParDo.Bound<T, T> greaterThanEq(final T value) {
-    return ParDo.named("Filter.greaterThanEq").of(new DoFn<T, T>() {
+  public static <T extends Comparable<T>> Filter<T> greaterThanEq(final T value) {
+    return by(new SerializableFunction<T, Boolean>() {
       @Override
-      public void processElement(ProcessContext c) {
-        if (c.element().compareTo(value) >= 0) {
-          c.output(c.element());
-        }
+      public Boolean apply(T input) {
+        return input.compareTo(value) >= 0;
       }
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        Filter.populateDisplayData(builder, String.format("x \u2265 %s", value));
-      }
-    });
+    }).described(String.format("x \u2265 %s", value));
   }
 
   ///////////////////////////////////////////////////////////////////////////////
 
   private SerializableFunction<T, Boolean> predicate;
+  private String predicateDescription;
 
   private Filter(SerializableFunction<T, Boolean> predicate) {
-    this.predicate = predicate;
+    this(predicate, "Filter.predicate");
   }
 
-  private Filter(String name, SerializableFunction<T, Boolean> predicate) {
-    super(name);
+  private Filter(SerializableFunction<T, Boolean> predicate,
+      String predicateDescription) {
     this.predicate = predicate;
+    this.predicateDescription = predicateDescription;
   }
 
-  public Filter<T> named(String name) {
-    return new Filter<>(name, predicate);
+  /**
+   * Returns a new {@link Filter} {@link PTransform} that's like this
+   * {@link PTransform} but with the specified description for {@link DisplayData}. Does not
+   * modify this {@link PTransform}.
+   */
+  Filter<T> described(String description) {
+    return new Filter<>(predicate, description);
+
   }
 
   @Override
   public PCollection<T> apply(PCollection<T> input) {
-    PCollection<T> output = input.apply(ParDo.named("Filter").of(new DoFn<T, T>() {
+    PCollection<T> output = input.apply(ParDo.of(new DoFn<T, T>() {
       @Override
       public void processElement(ProcessContext c) {
         if (predicate.apply(c.element()) == true) {
@@ -259,8 +218,9 @@ public class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> {
     return input.getCoder();
   }
 
-  private static void populateDisplayData(
-      DisplayData.Builder builder, String predicateDescription) {
+  @Override
+  public void populateDisplayData(DisplayData.Builder builder) {
+    super.populateDisplayData(builder);
     builder.add(DisplayData.item("predicate", predicateDescription)
       .withLabel("Filter Predicate"));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
index 367bbc0..2edab05 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
@@ -21,7 +21,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 
 import static org.hamcrest.MatcherAssert.assertThat;
 
-import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -61,10 +60,9 @@ public class FilterTest implements Serializable {
     }
   }
 
-  @Deprecated
   @Test
   @Category(RunnableOnService.class)
-  public void testIdentityFilterBy() {
+  public void testIdentityFilterByPredicate() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
@@ -75,10 +73,9 @@ public class FilterTest implements Serializable {
     p.run();
   }
 
-  @Deprecated
   @Test
-  @Category(NeedsRunner.class)
-  public void testNoFilter() {
+  @Category(RunnableOnService.class)
+  public void testNoFilterByPredicate() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
@@ -89,10 +86,9 @@ public class FilterTest implements Serializable {
     p.run();
   }
 
-  @Deprecated
   @Test
   @Category(RunnableOnService.class)
-  public void testFilterBy() {
+  public void testFilterByPredicate() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
@@ -105,81 +101,64 @@ public class FilterTest implements Serializable {
 
   @Test
   @Category(RunnableOnService.class)
-  public void testIdentityFilterByPredicate() {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<Integer> output = p
-        .apply(Create.of(591, 11789, 1257, 24578, 24799, 307))
-        .apply(Filter.byPredicate(new TrivialFn(true)));
-
-    PAssert.that(output).containsInAnyOrder(591, 11789, 1257, 24578, 24799, 307);
-    p.run();
-  }
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testNoFilterByPredicate() {
+  public void testFilterLessThan() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
-        .apply(Create.of(1, 2, 4, 5))
-        .apply(Filter.byPredicate(new TrivialFn(false)));
+        .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
+        .apply(Filter.lessThan(4));
 
-    PAssert.that(output).empty();
+    PAssert.that(output).containsInAnyOrder(1, 2, 3);
     p.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
-  public void testFilterByPredicate() {
+  public void testFilterGreaterThan() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
-        .apply(Filter.byPredicate(new EvenFn()));
+        .apply(Filter.greaterThan(4));
 
-    PAssert.that(output).containsInAnyOrder(2, 4, 6);
+    PAssert.that(output).containsInAnyOrder(5, 6, 7);
     p.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
-  public void testFilterLessThan() {
+  public void testFilterLessThanEq() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
-        .apply(Filter.lessThan(4));
+        .apply(Filter.lessThanEq(4));
 
-    PAssert.that(output).containsInAnyOrder(1, 2, 3);
+    PAssert.that(output).containsInAnyOrder(1, 2, 3, 4);
     p.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
-  public void testFilterGreaterThan() {
+  public void testFilterGreaterThanEq() {
     TestPipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
-        .apply(Filter.greaterThan(4));
+        .apply(Filter.greaterThanEq(4));
 
-    PAssert.that(output).containsInAnyOrder(5, 6, 7);
+    PAssert.that(output).containsInAnyOrder(4, 5, 6, 7);
     p.run();
   }
 
   @Test
   public void testDisplayData() {
-    ParDo.Bound<Integer, Integer> lessThan = Filter.lessThan(123);
-    assertThat(DisplayData.from(lessThan), hasDisplayItem("predicate", "x < 123"));
+    assertThat(DisplayData.from(Filter.lessThan(123)), hasDisplayItem("predicate", "x < 123"));
 
-    ParDo.Bound<Integer, Integer> lessThanOrEqual = Filter.lessThanEq(234);
-    assertThat(DisplayData.from(lessThanOrEqual), hasDisplayItem("predicate", "x \u2264 234"));
+    assertThat(DisplayData.from(Filter.lessThanEq(234)), hasDisplayItem("predicate", "x \u2264 234"));
 
-    ParDo.Bound<Integer, Integer> greaterThan = Filter.greaterThan(345);
-    assertThat(DisplayData.from(greaterThan), hasDisplayItem("predicate", "x > 345"));
+    assertThat(DisplayData.from(Filter.greaterThan(345)), hasDisplayItem("predicate", "x > 345"));
 
-    ParDo.Bound<Integer, Integer> greaterThanOrEqual = Filter.greaterThanEq(456);
-    assertThat(DisplayData.from(greaterThanOrEqual), hasDisplayItem("predicate", "x \u2265 456"));
+    assertThat(DisplayData.from(Filter.greaterThanEq(456)), hasDisplayItem("predicate", "x \u2265 456"));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d87f8b9/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
index 170071b..3c83be2 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
@@ -50,7 +50,7 @@ public class FilterJava8Test implements Serializable {
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(591, 11789, 1257, 24578, 24799, 307))
-        .apply(Filter.byPredicate(i -> true));
+        .apply(Filter.by(i -> true));
 
     PAssert.that(output).containsInAnyOrder(591, 11789, 1257, 24578, 24799, 307);
     pipeline.run();
@@ -62,7 +62,7 @@ public class FilterJava8Test implements Serializable {
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 4, 5))
-        .apply(Filter.byPredicate(i -> false));
+        .apply(Filter.by(i -> false));
 
     PAssert.that(output).empty();
     pipeline.run();
@@ -75,7 +75,7 @@ public class FilterJava8Test implements Serializable {
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
-        .apply(Filter.byPredicate(i -> i % 2 == 0));
+        .apply(Filter.by(i -> i % 2 == 0));
 
     PAssert.that(output).containsInAnyOrder(2, 4, 6);
     pipeline.run();
@@ -105,7 +105,7 @@ public class FilterJava8Test implements Serializable {
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
-        .apply(Filter.byPredicate(new EvenFilter()::isEven));
+        .apply(Filter.by(new EvenFilter()::isEven));
 
     PAssert.that(output).containsInAnyOrder(2, 4, 6);
     pipeline.run();


[44/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
Rename DataflowPipelineRunner to DataflowRunner


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

Branch: refs/heads/python-sdk
Commit: 6d028ac6118e31ee2415f77cf744a99c59b31d41
Parents: 340d098
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 11:42:19 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/DebuggingWordCount.java       |    2 +-
 .../apache/beam/examples/MinimalWordCount.java  |    4 +-
 .../apache/beam/examples/WindowedWordCount.java |    2 +-
 .../org/apache/beam/examples/WordCount.java     |    2 +-
 .../examples/common/DataflowExampleUtils.java   |   18 +-
 .../beam/examples/complete/AutoComplete.java    |    4 +-
 .../apache/beam/examples/complete/TfIdf.java    |    2 +-
 .../examples/complete/TopWikipediaSessions.java |    2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../examples/cookbook/DatastoreWordCount.java   |    4 +-
 .../beam/examples/cookbook/DeDupExample.java    |    2 +-
 .../beam/examples/cookbook/FilterExamples.java  |    2 +-
 .../beam/examples/cookbook/JoinExamples.java    |    2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |    2 +-
 .../beam/examples/MinimalWordCountJava8.java    |    4 +-
 .../beam/examples/complete/game/GameStats.java  |    2 +-
 .../examples/complete/game/HourlyTeamScore.java |    2 +-
 .../examples/complete/game/LeaderBoard.java     |    2 +-
 .../beam/examples/complete/game/UserScore.java  |    2 +-
 .../beam/runners/flink/FlinkPipelineRunner.java |    2 +-
 .../BlockingDataflowPipelineRunner.java         |  186 -
 .../dataflow/BlockingDataflowRunner.java        |  186 +
 .../dataflow/DataflowJobCancelledException.java |    2 +-
 .../dataflow/DataflowJobExecutionException.java |    2 +-
 .../dataflow/DataflowJobUpdatedException.java   |    2 +-
 .../runners/dataflow/DataflowPipelineJob.java   |    2 +-
 .../dataflow/DataflowPipelineRegistrar.java     |    8 +-
 .../dataflow/DataflowPipelineRunner.java        | 3229 ------------------
 .../dataflow/DataflowPipelineRunnerHooks.java   |   39 -
 .../dataflow/DataflowPipelineTranslator.java    |   10 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 3229 ++++++++++++++++++
 .../runners/dataflow/DataflowRunnerHooks.java   |   39 +
 .../runners/dataflow/internal/package-info.java |    2 +-
 .../BlockingDataflowPipelineOptions.java        |    6 +-
 .../options/DataflowPipelineOptions.java        |    4 +-
 .../DataflowPipelineWorkerPoolOptions.java      |    6 +-
 .../testing/TestDataflowPipelineRunner.java     |   10 +-
 .../BlockingDataflowPipelineRunnerTest.java     |  304 --
 .../dataflow/BlockingDataflowRunnerTest.java    |  304 ++
 .../dataflow/DataflowPipelineRegistrarTest.java |    4 +-
 .../dataflow/DataflowPipelineRunnerTest.java    | 1417 --------
 .../DataflowPipelineTranslatorTest.java         |   50 +-
 .../runners/dataflow/DataflowRunnerTest.java    | 1417 ++++++++
 .../runners/dataflow/io/DataflowAvroIOTest.java |    4 +-
 .../dataflow/io/DataflowPubsubIOTest.java       |    4 +-
 .../runners/dataflow/io/DataflowTextIOTest.java |    4 +-
 .../testing/TestDataflowPipelineRunnerTest.java |  601 ----
 .../testing/TestDataflowRunnerTest.java         |  601 ++++
 .../DataflowDisplayDataEvaluator.java           |   10 +-
 .../transforms/DataflowGroupByKeyTest.java      |    8 +-
 .../dataflow/transforms/DataflowViewTest.java   |    8 +-
 .../util/DataflowPathValidatorTest.java         |    8 +-
 .../beam/runners/spark/SimpleWordCountTest.java |    2 +-
 .../spark/translation/SerializationTest.java    |    2 +-
 .../apache/beam/sdk/runners/package-info.java   |   12 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |    2 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |    2 +-
 .../beam/sdk/util/BigQueryTableRowIterator.java |    2 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |    2 +-
 .../src/main/java/DebuggingWordCount.java       |    2 +-
 .../src/main/java/MinimalWordCount.java         |    4 +-
 .../src/main/java/WindowedWordCount.java        |    2 +-
 .../src/main/java/WordCount.java                |    2 +-
 .../main/java/common/DataflowExampleUtils.java  |   14 +-
 .../src/main/java/StarterPipeline.java          |    2 +-
 .../src/main/java/it/pkg/StarterPipeline.java   |    2 +-
 sdks/java/pom.xml                               |    2 +-
 68 files changed, 5913 insertions(+), 5913 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
index 11c56d1..85823c2 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
@@ -69,7 +69,7 @@ import java.util.regex.Pattern;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
  * </pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
index 65d450d..6d4bfd4 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.examples;
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
@@ -65,7 +65,7 @@ public class MinimalWordCount {
     // in Google Cloud Storage to stage files.
     DataflowPipelineOptions options = PipelineOptionsFactory.create()
         .as(DataflowPipelineOptions.class);
-    options.setRunner(BlockingDataflowPipelineRunner.class);
+    options.setRunner(BlockingDataflowRunner.class);
     // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud.
     options.setProject("SET_YOUR_PROJECT_ID_HERE");
     // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
index 9ca26bf..d4f1280 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java
@@ -84,7 +84,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
index 03ed5d7..cf6c45a 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
@@ -82,7 +82,7 @@ import java.io.IOException;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and an output prefix on GCS:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
index 46b8af3..b436ed1 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.examples.common;
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil;
 import org.apache.beam.sdk.Pipeline;
@@ -316,16 +316,16 @@ public class DataflowExampleUtils {
 
   /**
    * Do some runner setup: check that the DirectRunner is not used in conjunction with
-   * streaming, and if streaming is specified, use the DataflowPipelineRunner.
+   * streaming, and if streaming is specified, use the DataflowRunner.
    */
   public void setupRunner() {
     Class<? extends PipelineRunner<?>> runner = options.getRunner();
     if (options.isStreaming()
-        && (runner.equals(DataflowPipelineRunner.class)
-            || runner.equals(BlockingDataflowPipelineRunner.class))) {
+        && (runner.equals(DataflowRunner.class)
+            || runner.equals(BlockingDataflowRunner.class))) {
       // In order to cancel the pipelines automatically,
-      // {@literal DataflowPipelineRunner} is forced to be used.
-      options.setRunner(DataflowPipelineRunner.class);
+      // {@literal DataflowRunner} is forced to be used.
+      options.setRunner(DataflowRunner.class);
     }
   }
 
@@ -363,7 +363,7 @@ public class DataflowExampleUtils {
     }
     copiedOptions.setStreaming(false);
     copiedOptions.setWorkerHarnessContainerImage(
-        DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
+        DataflowRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE);
     copiedOptions.setNumWorkers(options.as(DataflowExampleOptions.class).getInjectorNumWorkers());
     copiedOptions.setJobName(options.getJobName() + "-injector");
     Pipeline injectorPipeline = Pipeline.create(copiedOptions);
@@ -396,7 +396,7 @@ public class DataflowExampleUtils {
   }
 
   /**
-   * If {@literal DataflowPipelineRunner} or {@literal BlockingDataflowPipelineRunner} is used,
+   * If {@literal DataflowRunner} or {@literal BlockingDataflowRunner} is used,
    * waits for the pipeline to finish and cancels it (and the injector) before the program exists.
    */
   public void waitToFinish(PipelineResult result) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index 3e4440c..ef47762 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -82,7 +82,7 @@ import java.util.regex.Pattern;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=DataflowPipelineRunner
+ *   --runner=DataflowRunner
  *   --inputFile=gs://path/to/input*.txt
  * }</pre>
  *
@@ -91,7 +91,7 @@ import java.util.regex.Pattern;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=DataflowPipelineRunner
+ *   --runner=DataflowRunner
  *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
  *   --streaming
  * }</pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
index 1f1885f..73f3323 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -80,7 +80,7 @@ import java.util.Set;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * and an output prefix on GCS:
  *   --output=gs://YOUR_OUTPUT_PREFIX
  * }</pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
index c483608..2c0d0eb 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -60,7 +60,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and an output prefix on GCS:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
index 4c69efb..a0ac6c5 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
@@ -61,7 +61,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and the BigQuery table for the output:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
index 505989c..dbaa1f8 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
@@ -68,7 +68,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://<STAGING DIRECTORY>
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and the BigQuery table for the output:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
index 2d1f88c..b0192c9 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
@@ -71,7 +71,7 @@ import javax.annotation.Nullable;
  * <p>To run this example using Dataflow service, you must additionally
  * provide either {@literal --tempLocation} or {@literal --tempLocation}, and
  * select one of the Dataflow pipeline runners, eg
- * {@literal --runner=BlockingDataflowPipelineRunner}.
+ * {@literal --runner=BlockingDataflowRunner}.
  *
  * <p><b>Note:</b> this example creates entities with <i>Ancestor keys</i> to ensure that all
  * entities created are in the same entity group. Similarly, the query used to read from the Cloud
@@ -248,7 +248,7 @@ public class DatastoreWordCount {
   /**
    * An example to demo how to use {@link DatastoreIO}.  The runner here is
    * customizable, which means users could pass either {@code DirectRunner}
-   * or {@code DataflowPipelineRunner} in the pipeline options.
+   * or {@code DataflowRunner} in the pipeline options.
    */
   public static void main(String args[]) {
     // The options are used in two places, for Dataflow service, and

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
index 37ddb4b..db65543 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
@@ -43,7 +43,7 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath;
  * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * and an output prefix on GCS:
  *   --output=gs://YOUR_OUTPUT_PREFIX
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
index 1c6a6e1..a669b0c 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
@@ -73,7 +73,7 @@ import java.util.logging.Logger;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and the BigQuery table for the output:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
index 8e642f8..e8f1f01 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
@@ -56,7 +56,7 @@ import com.google.api.services.bigquery.model.TableRow;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and an output prefix on GCS:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
index 2480967..6630f3d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
@@ -62,7 +62,7 @@ import java.util.List;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  * }
  * </pre>
  * and the BigQuery table for the output:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
index 0ad1a04..cf3bbf9 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.examples;
 
-import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.BlockingDataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
@@ -42,7 +42,7 @@ public class MinimalWordCountJava8 {
     DataflowPipelineOptions options = PipelineOptionsFactory.create()
         .as(DataflowPipelineOptions.class);
 
-    options.setRunner(BlockingDataflowPipelineRunner.class);
+    options.setRunner(BlockingDataflowRunner.class);
 
     // CHANGE 1 of 3: Your project ID is required in order to run your pipeline on the Google Cloud.
     options.setProject("SET_YOUR_PROJECT_ID_HERE");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
index c557913..c8bcc8c 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
@@ -83,7 +83,7 @@ import java.util.TimeZone;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
  * }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index ba3983d..c5c2fb5 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -60,7 +60,7 @@ import java.util.TimeZone;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --dataset=YOUR-DATASET
  * }
  * </pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index d79403a..12d2729 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -80,7 +80,7 @@ import java.util.TimeZone;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
  * }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index de049e8..97b6929 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -63,7 +63,7 @@ import java.util.Map;
  * <pre>{@code
  *   --project=YOUR_PROJECT_ID
  *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowPipelineRunner
+ *   --runner=BlockingDataflowRunner
  *   --dataset=YOUR-DATASET
  * }
  * </pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
index 220e8e8..46a4fce 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
@@ -145,7 +145,7 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
 
   @Override
   public String toString() {
-    return "DataflowPipelineRunner#" + hashCode();
+    return "DataflowRunner#" + hashCode();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
deleted file mode 100644
index d8ee16a..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.dataflow;
-
-import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
-import org.apache.beam.runners.dataflow.util.MonitoringUtil;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link PipelineRunner} that's like {@link DataflowPipelineRunner}
- * but that waits for the launched job to finish.
- *
- * <p>Prints out job status updates and console messages while it waits.
- *
- * <p>Returns the final job state, or throws an exception if the job
- * fails or cannot be monitored.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code BlockingDataflowPipelineRunner}, the Google cloud services account and the Google compute
- * engine service account of the GCP project running the Dataflow Job will need access to the
- * corresponding source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class BlockingDataflowPipelineRunner extends
-    PipelineRunner<DataflowPipelineJob> {
-  private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class);
-
-  // Defaults to an infinite wait period.
-  // TODO: make this configurable after removal of option map.
-  private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
-
-  private final DataflowPipelineRunner dataflowPipelineRunner;
-  private final BlockingDataflowPipelineOptions options;
-
-  protected BlockingDataflowPipelineRunner(
-      DataflowPipelineRunner internalRunner,
-      BlockingDataflowPipelineOptions options) {
-    this.dataflowPipelineRunner = internalRunner;
-    this.options = options;
-  }
-
-  /**
-   * Constructs a runner from the provided options.
-   */
-  public static BlockingDataflowPipelineRunner fromOptions(
-      PipelineOptions options) {
-    BlockingDataflowPipelineOptions dataflowOptions =
-        PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
-    DataflowPipelineRunner dataflowPipelineRunner =
-        DataflowPipelineRunner.fromOptions(dataflowOptions);
-
-    return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, dataflowOptions);
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws DataflowJobExecutionException if there is an exception during job execution.
-   * @throws DataflowServiceException if there is an exception retrieving information about the job.
-   */
-  @Override
-  public DataflowPipelineJob run(Pipeline p) {
-    final DataflowPipelineJob job = dataflowPipelineRunner.run(p);
-
-    // We ignore the potential race condition here (Ctrl-C after job submission but before the
-    // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
-    // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
-    // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
-    // etc. If the user wants to verify the job was cancelled they should look at the job status.
-    Thread shutdownHook = new Thread() {
-      @Override
-      public void run() {
-        LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
-            + "To cancel the job in the cloud, run:\n> {}",
-            MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
-      }
-    };
-
-    try {
-      Runtime.getRuntime().addShutdownHook(shutdownHook);
-
-      @Nullable
-      State result;
-      try {
-        result = job.waitToFinish(
-            BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
-            new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
-      } catch (IOException | InterruptedException ex) {
-        if (ex instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
-        throw new DataflowServiceException(
-            job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
-      }
-
-      if (result == null) {
-        throw new DataflowServiceException(
-            job, "Timed out while retrieving status for job " + job.getJobId());
-      }
-
-      LOG.info("Job finished with status {}", result);
-      if (!result.isTerminal()) {
-        throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
-            + ", got " + result);
-      }
-
-      if (result == State.DONE) {
-        return job;
-      } else if (result == State.UPDATED) {
-        DataflowPipelineJob newJob = job.getReplacedByJob();
-        LOG.info("Job {} has been updated and is running as the new job with id {}."
-            + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
-            job.getJobId(),
-            newJob.getJobId(),
-            MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
-        throw new DataflowJobUpdatedException(
-            job,
-            String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
-            newJob);
-      } else if (result == State.CANCELLED) {
-        String message = String.format("Job %s cancelled by user", job.getJobId());
-        LOG.info(message);
-        throw new DataflowJobCancelledException(job, message);
-      } else {
-        throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
-            + " failed with status " + result);
-      }
-    } finally {
-      Runtime.getRuntime().removeShutdownHook(shutdownHook);
-    }
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-    return dataflowPipelineRunner.apply(transform, input);
-  }
-
-  /**
-   * Sets callbacks to invoke during execution. See {@link DataflowPipelineRunnerHooks}.
-   */
-  @Experimental
-  public void setHooks(DataflowPipelineRunnerHooks hooks) {
-    this.dataflowPipelineRunner.setHooks(hooks);
-  }
-
-  @Override
-  public String toString() {
-    return "BlockingDataflowPipelineRunner#" + options.getJobName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
new file mode 100644
index 0000000..5c59bc2
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow;
+
+import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link PipelineRunner} that's like {@link DataflowRunner}
+ * but that waits for the launched job to finish.
+ *
+ * <p>Prints out job status updates and console messages while it waits.
+ *
+ * <p>Returns the final job state, or throws an exception if the job
+ * fails or cannot be monitored.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code BlockingDataflowRunner}, the Google cloud services account and the Google compute
+ * engine service account of the GCP project running the Dataflow Job will need access to the
+ * corresponding source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class BlockingDataflowRunner extends
+    PipelineRunner<DataflowPipelineJob> {
+  private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowRunner.class);
+
+  // Defaults to an infinite wait period.
+  // TODO: make this configurable after removal of option map.
+  private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
+
+  private final DataflowRunner dataflowRunner;
+  private final BlockingDataflowPipelineOptions options;
+
+  protected BlockingDataflowRunner(
+      DataflowRunner internalRunner,
+      BlockingDataflowPipelineOptions options) {
+    this.dataflowRunner = internalRunner;
+    this.options = options;
+  }
+
+  /**
+   * Constructs a runner from the provided options.
+   */
+  public static BlockingDataflowRunner fromOptions(
+      PipelineOptions options) {
+    BlockingDataflowPipelineOptions dataflowOptions =
+        PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
+    DataflowRunner dataflowRunner =
+        DataflowRunner.fromOptions(dataflowOptions);
+
+    return new BlockingDataflowRunner(dataflowRunner, dataflowOptions);
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws DataflowJobExecutionException if there is an exception during job execution.
+   * @throws DataflowServiceException if there is an exception retrieving information about the job.
+   */
+  @Override
+  public DataflowPipelineJob run(Pipeline p) {
+    final DataflowPipelineJob job = dataflowRunner.run(p);
+
+    // We ignore the potential race condition here (Ctrl-C after job submission but before the
+    // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
+    // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
+    // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
+    // etc. If the user wants to verify the job was cancelled they should look at the job status.
+    Thread shutdownHook = new Thread() {
+      @Override
+      public void run() {
+        LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
+            + "To cancel the job in the cloud, run:\n> {}",
+            MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
+      }
+    };
+
+    try {
+      Runtime.getRuntime().addShutdownHook(shutdownHook);
+
+      @Nullable
+      State result;
+      try {
+        result = job.waitToFinish(
+            BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
+            new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+      } catch (IOException | InterruptedException ex) {
+        if (ex instanceof InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
+        throw new DataflowServiceException(
+            job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
+      }
+
+      if (result == null) {
+        throw new DataflowServiceException(
+            job, "Timed out while retrieving status for job " + job.getJobId());
+      }
+
+      LOG.info("Job finished with status {}", result);
+      if (!result.isTerminal()) {
+        throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
+            + ", got " + result);
+      }
+
+      if (result == State.DONE) {
+        return job;
+      } else if (result == State.UPDATED) {
+        DataflowPipelineJob newJob = job.getReplacedByJob();
+        LOG.info("Job {} has been updated and is running as the new job with id {}."
+            + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
+            job.getJobId(),
+            newJob.getJobId(),
+            MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
+        throw new DataflowJobUpdatedException(
+            job,
+            String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
+            newJob);
+      } else if (result == State.CANCELLED) {
+        String message = String.format("Job %s cancelled by user", job.getJobId());
+        LOG.info(message);
+        throw new DataflowJobCancelledException(job, message);
+      } else {
+        throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
+            + " failed with status " + result);
+      }
+    } finally {
+      Runtime.getRuntime().removeShutdownHook(shutdownHook);
+    }
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+    return dataflowRunner.apply(transform, input);
+  }
+
+  /**
+   * Sets callbacks to invoke during execution. See {@link DataflowRunnerHooks}.
+   */
+  @Experimental
+  public void setHooks(DataflowRunnerHooks hooks) {
+    this.dataflowRunner.setHooks(hooks);
+  }
+
+  @Override
+  public String toString() {
+    return "BlockingDataflowRunner#" + options.getJobName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
index de44a19..e2edb6a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.dataflow;
 
 /**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ * Signals that a job run by a {@link BlockingDataflowRunner} was updated during execution.
  */
 public class DataflowJobCancelledException extends DataflowJobException {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
index 11b8723..ccf8057 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
@@ -20,7 +20,7 @@ package org.apache.beam.runners.dataflow;
 import javax.annotation.Nullable;
 
 /**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} fails during execution, and
+ * Signals that a job run by a {@link BlockingDataflowRunner} fails during execution, and
  * provides access to the failed job.
  */
 public class DataflowJobExecutionException extends DataflowJobException {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
index e30a0e7..39d1d47 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.dataflow;
 
 /**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ * Signals that a job run by a {@link BlockingDataflowRunner} was updated during execution.
  */
 public class DataflowJobUpdatedException extends DataflowJobException {
   private DataflowPipelineJob replacedByJob;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
index 0f42148..548d2b9 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
@@ -55,7 +55,7 @@ import javax.annotation.Nullable;
 
 /**
  * A DataflowPipelineJob represents a job submitted to Dataflow using
- * {@link DataflowPipelineRunner}.
+ * {@link DataflowRunner}.
  */
 public class DataflowPipelineJob implements PipelineResult {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
index d7d243f..4c88c4e 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
@@ -29,7 +29,7 @@ import com.google.common.collect.ImmutableList;
 
 /**
  * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for the
- * {@link DataflowPipelineRunner}.
+ * {@link DataflowRunner}.
  */
 public class DataflowPipelineRegistrar {
   private DataflowPipelineRegistrar() { }
@@ -48,15 +48,15 @@ public class DataflowPipelineRegistrar {
   }
 
   /**
-   * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}.
+   * Register the {@link DataflowRunner} and {@link BlockingDataflowRunner}.
    */
   @AutoService(PipelineRunnerRegistrar.class)
   public static class Runner implements PipelineRunnerRegistrar {
     @Override
     public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
       return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
-          DataflowPipelineRunner.class,
-          BlockingDataflowPipelineRunner.class);
+          DataflowRunner.class,
+          BlockingDataflowRunner.class);
     }
   }
 }


[47/50] [abbrv] incubator-beam git commit: Rename FlinkPipelineRunner to FlinkRunner

Posted by da...@apache.org.
Rename FlinkPipelineRunner to FlinkRunner


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

Branch: refs/heads/python-sdk
Commit: 4b417680d09da9f9f1c990a4f235572689efdf93
Parents: d285e67
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 09:57:25 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 README.md                                       |   2 +-
 runners/flink/README.md                         |   4 +-
 .../beam/runners/flink/examples/TFIDF.java      |   4 +-
 .../beam/runners/flink/examples/WordCount.java  |   4 +-
 .../flink/examples/streaming/AutoComplete.java  |   4 +-
 .../flink/examples/streaming/JoinExamples.java  |   4 +-
 .../examples/streaming/KafkaIOExamples.java     |   4 +-
 .../KafkaWindowedWordCountExample.java          |   4 +-
 .../examples/streaming/WindowedWordCount.java   |   4 +-
 runners/flink/runner/pom.xml                    |   4 +-
 .../beam/runners/flink/FlinkPipelineRunner.java | 180 -------------------
 .../apache/beam/runners/flink/FlinkRunner.java  | 179 ++++++++++++++++++
 .../runners/flink/FlinkRunnerRegistrar.java     |   4 +-
 .../runners/flink/TestFlinkPipelineRunner.java  |  81 ---------
 .../beam/runners/flink/TestFlinkRunner.java     |  81 +++++++++
 .../streaming/io/UnboundedFlinkSource.java      |   6 +-
 .../streaming/state/FlinkStateInternals.java    |   2 +-
 .../runners/flink/FlinkRunnerRegistrarTest.java |   8 +-
 .../beam/runners/flink/FlinkTestPipeline.java   |   4 +-
 19 files changed, 291 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index b34d933..c4a9155 100644
--- a/README.md
+++ b/README.md
@@ -68,7 +68,7 @@ Beam supports executing programs on multiple distributed processing backends thr
 
 - The `DirectRunner` runs the pipeline on your local machine.
 - The `DataflowPipelineRunner` submits the pipeline to the [Google Cloud Dataflow](http://cloud.google.com/dataflow/).
-- The `FlinkPipelineRunner` runs the pipeline on an Apache Flink cluster. The code has been donated from [dataArtisans/flink-dataflow](https://github.com/dataArtisans/flink-dataflow) and is now part of Beam.
+- The `FlinkRunner` runs the pipeline on an Apache Flink cluster. The code has been donated from [dataArtisans/flink-dataflow](https://github.com/dataArtisans/flink-dataflow) and is now part of Beam.
 - The `SparkPipelineRunner` runs the pipeline on an Apache Spark cluster. The code has been donated from [cloudera/spark-dataflow](https://github.com/cloudera/spark-dataflow) and is now part of Beam.
 
 Have ideas for new Runners? See the [Jira](https://issues.apache.org/jira/browse/BEAM/component/12328916/).

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index 69e2abb..3348119 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -107,14 +107,14 @@ Flink-Runner is now installed in your local maven repository.
 
 Next, let's run the classic WordCount example. It's semantically identically to
 the example provided with Apache Beam. Only this time, we chose the
-`FlinkPipelineRunner` to execute the WordCount on top of Flink.
+`FlinkRunner` to execute the WordCount on top of Flink.
 
 Here's an excerpt from the WordCount class file:
 
 ```java
 Options options = PipelineOptionsFactory.fromArgs(args).as(Options.class);
 // yes, we want to run WordCount with Flink
-options.setRunner(FlinkPipelineRunner.class);
+options.setRunner(FlinkRunner.class);
 
 Pipeline p = Pipeline.create(options);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
index 876ecde..af920aa 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.flink.examples;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
@@ -439,7 +439,7 @@ public class TFIDF {
   public static void main(String[] args) throws Exception {
     Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
 
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     Pipeline pipeline = Pipeline.create(options);
     pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
index 702fb63..2817622 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.flink.examples;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.Default;
@@ -105,7 +105,7 @@ public class WordCount {
 
     Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
         .as(Options.class);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     Pipeline p = Pipeline.create(options);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
index d83e662..9299955 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.flink.examples.streaming;
 
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AvroCoder;
@@ -378,7 +378,7 @@ public class AutoComplete {
     options.setCheckpointingInterval(1000L);
     options.setNumberOfExecutionRetries(5);
     options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     PTransform<? super PBegin, PCollection<String>> readSource =
             Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
index d3e963d..b447a20 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.flink.examples.streaming;
 
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
@@ -133,7 +133,7 @@ public class JoinExamples {
     options.setCheckpointingInterval(1000L);
     options.setNumberOfExecutionRetries(5);
     options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     PTransform<? super PBegin, PCollection<String>> readSourceA =
         Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java
index af6bb35..8756abe 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.flink.examples.streaming;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSink;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
 import org.apache.beam.sdk.Pipeline;
@@ -289,7 +289,7 @@ public class KafkaIOExamples {
         PipelineOptionsFactory.fromArgs(args).as(KafkaOptions.class);
 
     options.setStreaming(true);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     options.setCheckpointingInterval(1000L);
     options.setNumberOfExecutionRetries(5);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
index abb9fea..b14c5ae 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.flink.examples.streaming;
 
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
@@ -115,7 +115,7 @@ public class KafkaWindowedWordCountExample {
     options.setCheckpointingInterval(1000L);
     options.setNumberOfExecutionRetries(5);
     options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
     Pipeline pipeline = Pipeline.create(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
index e803e6e..f72b705 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.flink.examples.streaming;
 
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
@@ -111,7 +111,7 @@ public class WindowedWordCount {
     options.setCheckpointingInterval(1000L);
     options.setNumberOfExecutionRetries(5);
     options.setExecutionRetryDelay(3000L);
-    options.setRunner(FlinkPipelineRunner.class);
+    options.setRunner(FlinkRunner.class);
 
     LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
         " sec. and a slide of " + options.getSlide());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 33c13bf..dd32063 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -194,7 +194,7 @@
               <systemPropertyVariables>
                 <beamTestPipelineOptions>
                   [
-                    "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner",
+                    "--runner=TestFlinkRunner",
                     "--streaming=false"
                   ]
                 </beamTestPipelineOptions>
@@ -218,7 +218,7 @@
               <systemPropertyVariables>
                 <beamTestPipelineOptions>
                   [
-                    "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner",
+                    "--runner=TestFlinkRunner",
                     "--streaming=true"
                   ]
                 </beamTestPipelineOptions>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
deleted file mode 100644
index 46a4fce..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-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.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.common.base.Joiner;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to a Flink Plan and then executing them either locally
- * or on a Flink cluster, depending on the configuration.
- * <p>
- */
-public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
-
-  /**
-   * Provided options.
-   */
-  private final FlinkPipelineOptions options;
-
-  /**
-   * Construct a runner from the provided options.
-   *
-   * @param options Properties which configure the runner.
-   * @return The newly created runner.
-   */
-  public static FlinkPipelineRunner fromOptions(PipelineOptions options) {
-    FlinkPipelineOptions flinkOptions =
-        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
-    ArrayList<String> missing = new ArrayList<>();
-
-    if (flinkOptions.getAppName() == null) {
-      missing.add("appName");
-    }
-    if (missing.size() > 0) {
-      throw new IllegalArgumentException(
-          "Missing required values: " + Joiner.on(',').join(missing));
-    }
-
-    if (flinkOptions.getFilesToStage() == null) {
-      flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
-          FlinkPipelineRunner.class.getClassLoader()));
-      LOG.info("PipelineOptions.filesToStage was not specified. "
-              + "Defaulting to files from the classpath: will stage {} files. "
-              + "Enable logging at DEBUG level to see which files will be staged.",
-          flinkOptions.getFilesToStage().size());
-      LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
-    }
-
-    // Set Flink Master to [auto] if no option was specified.
-    if (flinkOptions.getFlinkMaster() == null) {
-      flinkOptions.setFlinkMaster("[auto]");
-    }
-
-    return new FlinkPipelineRunner(flinkOptions);
-  }
-
-  private FlinkPipelineRunner(FlinkPipelineOptions options) {
-    this.options = options;
-  }
-
-  @Override
-  public FlinkRunnerResult run(Pipeline pipeline) {
-    LOG.info("Executing pipeline using FlinkPipelineRunner.");
-
-    FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options);
-
-    LOG.info("Translating pipeline to Flink program.");
-    env.translate(pipeline);
-
-    JobExecutionResult result;
-    try {
-      LOG.info("Starting execution of Flink program.");
-      result = env.executePipeline();
-    } catch (Exception e) {
-      LOG.error("Pipeline execution failed", e);
-      throw new RuntimeException("Pipeline execution failed", e);
-    }
-
-    LOG.info("Execution finished in {} msecs", result.getNetRuntime());
-
-    Map<String, Object> accumulators = result.getAllAccumulatorResults();
-    if (accumulators != null && !accumulators.isEmpty()) {
-      LOG.info("Final aggregator values:");
-
-      for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
-        LOG.info("{} : {}", entry.getKey(), entry.getValue());
-      }
-    }
-
-    return new FlinkRunnerResult(accumulators, result.getNetRuntime());
-  }
-
-  /**
-   * For testing.
-   */
-  public FlinkPipelineOptions getPipelineOptions() {
-    return options;
-  }
-
-  @Override
-  public <Output extends POutput, Input extends PInput> Output apply(
-      PTransform<Input, Output> transform, Input input) {
-    return super.apply(transform, input);
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-
-  @Override
-  public String toString() {
-    return "DataflowRunner#" + hashCode();
-  }
-
-  /**
-   * Attempts to detect all the resources the class loader has access to. This does not recurse
-   * to class loader parents stopping it from pulling in resources from the system class loader.
-   *
-   * @param classLoader The URLClassLoader to use to detect resources to stage.
-   * @return A list of absolute paths to the resources the class loader uses.
-   * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
-   *                                  of the resources the class loader exposes is not a file resource.
-   */
-  protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
-    if (!(classLoader instanceof URLClassLoader)) {
-      String message = String.format("Unable to use ClassLoader to detect classpath elements. "
-          + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
-      LOG.error(message);
-      throw new IllegalArgumentException(message);
-    }
-
-    List<String> files = new ArrayList<>();
-    for (URL url : ((URLClassLoader) classLoader).getURLs()) {
-      try {
-        files.add(new File(url.toURI()).getAbsolutePath());
-      } catch (IllegalArgumentException | URISyntaxException e) {
-        String message = String.format("Unable to convert url (%s) to file.", url);
-        LOG.error(message);
-        throw new IllegalArgumentException(message, e);
-      }
-    }
-    return files;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
new file mode 100644
index 0000000..d8c5c12
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.flink;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.common.base.Joiner;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to a Flink Plan and then executing them either locally
+ * or on a Flink cluster, depending on the configuration.
+ * <p>
+ */
+public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FlinkRunner.class);
+
+  /**
+   * Provided options.
+   */
+  private final FlinkPipelineOptions options;
+
+  /**
+   * Construct a runner from the provided options.
+   *
+   * @param options Properties which configure the runner.
+   * @return The newly created runner.
+   */
+  public static FlinkRunner fromOptions(PipelineOptions options) {
+    FlinkPipelineOptions flinkOptions =
+        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+    ArrayList<String> missing = new ArrayList<>();
+
+    if (flinkOptions.getAppName() == null) {
+      missing.add("appName");
+    }
+    if (missing.size() > 0) {
+      throw new IllegalArgumentException(
+          "Missing required values: " + Joiner.on(',').join(missing));
+    }
+
+    if (flinkOptions.getFilesToStage() == null) {
+      flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
+          FlinkRunner.class.getClassLoader()));
+      LOG.info("PipelineOptions.filesToStage was not specified. "
+              + "Defaulting to files from the classpath: will stage {} files. "
+              + "Enable logging at DEBUG level to see which files will be staged.",
+          flinkOptions.getFilesToStage().size());
+      LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
+    }
+
+    // Set Flink Master to [auto] if no option was specified.
+    if (flinkOptions.getFlinkMaster() == null) {
+      flinkOptions.setFlinkMaster("[auto]");
+    }
+
+    return new FlinkRunner(flinkOptions);
+  }
+
+  private FlinkRunner(FlinkPipelineOptions options) {
+    this.options = options;
+  }
+
+  @Override
+  public FlinkRunnerResult run(Pipeline pipeline) {
+    LOG.info("Executing pipeline using FlinkRunner.");
+
+    FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options);
+
+    LOG.info("Translating pipeline to Flink program.");
+    env.translate(pipeline);
+
+    JobExecutionResult result;
+    try {
+      LOG.info("Starting execution of Flink program.");
+      result = env.executePipeline();
+    } catch (Exception e) {
+      LOG.error("Pipeline execution failed", e);
+      throw new RuntimeException("Pipeline execution failed", e);
+    }
+
+    LOG.info("Execution finished in {} msecs", result.getNetRuntime());
+
+    Map<String, Object> accumulators = result.getAllAccumulatorResults();
+    if (accumulators != null && !accumulators.isEmpty()) {
+      LOG.info("Final aggregator values:");
+
+      for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
+        LOG.info("{} : {}", entry.getKey(), entry.getValue());
+      }
+    }
+
+    return new FlinkRunnerResult(accumulators, result.getNetRuntime());
+  }
+
+  /**
+   * For testing.
+   */
+  public FlinkPipelineOptions getPipelineOptions() {
+    return options;
+  }
+
+  @Override
+  public <Output extends POutput, Input extends PInput> Output apply(
+      PTransform<Input, Output> transform, Input input) {
+    return super.apply(transform, input);
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public String toString() {
+    return "DataflowRunner#" + hashCode();
+  }
+
+  /**
+   * Attempts to detect all the resources the class loader has access to. This does not recurse
+   * to class loader parents stopping it from pulling in resources from the system class loader.
+   *
+   * @param classLoader The URLClassLoader to use to detect resources to stage.
+   * @return A list of absolute paths to the resources the class loader uses.
+   * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
+   *                                  of the resources the class loader exposes is not a file resource.
+   */
+  protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
+    if (!(classLoader instanceof URLClassLoader)) {
+      String message = String.format("Unable to use ClassLoader to detect classpath elements. "
+          + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
+      LOG.error(message);
+      throw new IllegalArgumentException(message);
+    }
+
+    List<String> files = new ArrayList<>();
+    for (URL url : ((URLClassLoader) classLoader).getURLs()) {
+      try {
+        files.add(new File(url.toURI()).getAbsolutePath());
+      } catch (IllegalArgumentException | URISyntaxException e) {
+        String message = String.format("Unable to convert url (%s) to file.", url);
+        LOG.error(message);
+        throw new IllegalArgumentException(message, e);
+      }
+    }
+    return files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
index ec61805..52b2a8d 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java
@@ -42,8 +42,8 @@ public class FlinkRunnerRegistrar {
     @Override
     public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
       return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
-          FlinkPipelineRunner.class,
-          TestFlinkPipelineRunner.class);
+          FlinkRunner.class,
+          TestFlinkRunner.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
deleted file mode 100644
index a55acb7..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.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.runners.flink;
-
-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.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import org.apache.flink.runtime.client.JobExecutionException;
-
-public class TestFlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
-
-  private FlinkPipelineRunner delegate;
-
-  private TestFlinkPipelineRunner(FlinkPipelineOptions options) {
-    // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment
-    options.setFlinkMaster("[auto]");
-    this.delegate = FlinkPipelineRunner.fromOptions(options);
-  }
-
-  public static TestFlinkPipelineRunner fromOptions(PipelineOptions options) {
-    FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
-    return new TestFlinkPipelineRunner(flinkOptions);
-  }
-
-  public static TestFlinkPipelineRunner create(boolean streaming) {
-    FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
-    flinkOptions.setRunner(TestFlinkPipelineRunner.class);
-    flinkOptions.setStreaming(streaming);
-    return TestFlinkPipelineRunner.fromOptions(flinkOptions);
-  }
-
-  @Override
-  public <OutputT extends POutput, InputT extends PInput>
-      OutputT apply(PTransform<InputT,OutputT> transform, InputT input) {
-    return delegate.apply(transform, input);
-  }
-
-  @Override
-  public FlinkRunnerResult run(Pipeline pipeline) {
-    try {
-      return delegate.run(pipeline);
-    } catch (RuntimeException e) {
-      // Special case hack to pull out assertion errors from PAssert; instead there should
-      // probably be a better story along the lines of UserCodeException.
-      if (e.getCause() != null
-          && e.getCause() instanceof JobExecutionException
-          && e.getCause().getCause() instanceof AssertionError) {
-          throw (AssertionError) e.getCause().getCause();
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  public PipelineOptions getPipelineOptions() {
-    return delegate.getPipelineOptions();
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
new file mode 100644
index 0000000..460933f
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.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.flink;
+
+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.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import org.apache.flink.runtime.client.JobExecutionException;
+
+public class TestFlinkRunner extends PipelineRunner<FlinkRunnerResult> {
+
+  private FlinkRunner delegate;
+
+  private TestFlinkRunner(FlinkPipelineOptions options) {
+    // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment
+    options.setFlinkMaster("[auto]");
+    this.delegate = FlinkRunner.fromOptions(options);
+  }
+
+  public static TestFlinkRunner fromOptions(PipelineOptions options) {
+    FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+    return new TestFlinkRunner(flinkOptions);
+  }
+
+  public static TestFlinkRunner create(boolean streaming) {
+    FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+    flinkOptions.setRunner(TestFlinkRunner.class);
+    flinkOptions.setStreaming(streaming);
+    return TestFlinkRunner.fromOptions(flinkOptions);
+  }
+
+  @Override
+  public <OutputT extends POutput, InputT extends PInput>
+      OutputT apply(PTransform<InputT,OutputT> transform, InputT input) {
+    return delegate.apply(transform, input);
+  }
+
+  @Override
+  public FlinkRunnerResult run(Pipeline pipeline) {
+    try {
+      return delegate.run(pipeline);
+    } catch (RuntimeException e) {
+      // Special case hack to pull out assertion errors from PAssert; instead there should
+      // probably be a better story along the lines of UserCodeException.
+      if (e.getCause() != null
+          && e.getCause() instanceof JobExecutionException
+          && e.getCause().getCause() instanceof AssertionError) {
+          throw (AssertionError) e.getCause().getCause();
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  public PipelineOptions getPipelineOptions() {
+    return delegate.getPipelineOptions();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
index a157b46..b636036 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -51,18 +51,18 @@ public class UnboundedFlinkSource<T> extends UnboundedSource<T, UnboundedSource.
 
   @Override
   public List<? extends UnboundedSource<T, UnboundedSource.CheckpointMark>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
-    throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+    throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner.");
   }
 
   @Override
   public UnboundedReader<T> createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) {
-    throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+    throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner.");
   }
 
   @Nullable
   @Override
   public Coder<UnboundedSource.CheckpointMark> getCheckpointMarkCoder() {
-    throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+    throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner.");
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
index 352c550..18d4c3c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -86,7 +86,7 @@ public class FlinkStateInternals<K> implements StateInternals<K> {
 
   /**
    * This is the interface state has to implement in order for it to be fault tolerant when
-   * executed by the FlinkPipelineRunner.
+   * executed by the FlinkRunner.
    */
   private interface CheckpointableIF {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
index bd149c7..ff1025f 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java
@@ -33,17 +33,17 @@ public class FlinkRunnerRegistrarTest {
   @Test
   public void testFullName() {
     String[] args =
-        new String[] {String.format("--runner=%s", FlinkPipelineRunner.class.getName())};
+        new String[] {String.format("--runner=%s", FlinkRunner.class.getName())};
     PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
-    assertEquals(opts.getRunner(), FlinkPipelineRunner.class);
+    assertEquals(opts.getRunner(), FlinkRunner.class);
   }
 
   @Test
   public void testClassName() {
     String[] args =
-        new String[] {String.format("--runner=%s", FlinkPipelineRunner.class.getSimpleName())};
+        new String[] {String.format("--runner=%s", FlinkRunner.class.getSimpleName())};
     PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create();
-    assertEquals(opts.getRunner(), FlinkPipelineRunner.class);
+    assertEquals(opts.getRunner(), FlinkRunner.class);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b417680/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
index edde925..9f7bc00 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
@@ -24,7 +24,7 @@ import org.apache.beam.sdk.runners.PipelineRunner;
 
 /**
  * {@link org.apache.beam.sdk.Pipeline} for testing Dataflow programs on the
- * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
+ * {@link FlinkRunner}.
  */
 public class FlinkTestPipeline extends Pipeline {
 
@@ -60,7 +60,7 @@ public class FlinkTestPipeline extends Pipeline {
    * @return The Test Pipeline.
    */
   private static FlinkTestPipeline create(boolean streaming) {
-    TestFlinkPipelineRunner flinkRunner = TestFlinkPipelineRunner.create(streaming);
+    TestFlinkRunner flinkRunner = TestFlinkRunner.create(streaming);
     return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
   }
 



[34/50] [abbrv] incubator-beam git commit: Added BigDecimal coder and tests.

Posted by da...@apache.org.
Added BigDecimal coder and tests.


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

Branch: refs/heads/python-sdk
Commit: 6460df195240dac4d488fcf111642e8706008690
Parents: 4f7a2ab
Author: Jesse Anderson <je...@smokinghand.com>
Authored: Mon May 9 10:05:15 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/coders/BigDecimalCoder.java | 112 +++++++++++++++++++
 .../beam/sdk/coders/BigDecimalCoderTest.java    | 105 +++++++++++++++++
 2 files changed, 217 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6460df19/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
new file mode 100644
index 0000000..3b723b9
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.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.sdk.coders;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+/**
+ * A {@link BigDecimalCoder} encodes {@link BigDecimal} in an integer and
+ * a byte array. The integer represents the scale and the byte array
+ * represents a {@link BigInteger}. The integer is in 4 bytes, big-endian.
+ */
+public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
+
+  @JsonCreator
+  public static BigDecimalCoder of() {
+    return INSTANCE;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  private static final BigDecimalCoder INSTANCE = new BigDecimalCoder();
+
+  private BigDecimalCoder() {}
+
+  @Override
+  public void encode(BigDecimal value, OutputStream outStream, Context context)
+      throws IOException, CoderException {
+    if (value == null) {
+      throw new CoderException("cannot encode a null BigDecimal");
+    }
+
+    byte[] bigIntBytes = value.unscaledValue().toByteArray();
+
+    DataOutputStream dataOutputStream = new DataOutputStream(outStream);
+    dataOutputStream.writeInt(value.scale());
+    dataOutputStream.writeInt(bigIntBytes.length);
+    dataOutputStream.write(bigIntBytes);
+  }
+
+  @Override
+  public BigDecimal decode(InputStream inStream, Context context)
+      throws IOException, CoderException {
+    DataInputStream dataInputStream = new DataInputStream(inStream);
+    int scale = dataInputStream.readInt();
+    int bigIntBytesSize = dataInputStream.readInt();
+
+    byte[] bigIntBytes = new byte[bigIntBytesSize];
+    dataInputStream.readFully(bigIntBytes);
+
+    BigInteger bigInteger = new BigInteger(bigIntBytes);
+    BigDecimal bigDecimal = new BigDecimal(bigInteger, scale);
+
+    return bigDecimal;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code true}. This coder is injective.
+   */
+  @Override
+  public boolean consistentWithEquals() {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code true}, because {@link #getEncodedElementByteSize} runs in constant time.
+   */
+  @Override
+  public boolean isRegisterByteSizeObserverCheap(BigDecimal value, Context context) {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@code 8} plus the size of the {@link BigInteger} bytes.
+   */
+  @Override
+  protected long getEncodedElementByteSize(BigDecimal value, Context context)
+      throws Exception {
+    if (value == null) {
+      throw new CoderException("cannot encode a null BigDecimal");
+    }
+    return 8 + value.unscaledValue().toByteArray().length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6460df19/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
new file mode 100644
index 0000000..5911535
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.coders;
+
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.util.CoderUtils;
+
+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.math.BigDecimal;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Test case for {@link BigDecimalCoder}.
+ */
+@RunWith(JUnit4.class)
+public class BigDecimalCoderTest {
+
+  private static final Coder<BigDecimal> TEST_CODER = BigDecimalCoder.of();
+
+  private static final List<BigDecimal> TEST_VALUES = Arrays.asList(
+      new BigDecimal(Double.MIN_VALUE),
+      new BigDecimal(-11),
+      new BigDecimal(-3),
+      new BigDecimal(-1),
+      new BigDecimal(0),
+      new BigDecimal(1),
+      new BigDecimal(5),
+      new BigDecimal(13),
+      new BigDecimal(29),
+      new BigDecimal(Double.MAX_VALUE));
+
+  @Test
+  public void testDecodeEncodeEqual() throws Exception {
+    for (BigDecimal value : TEST_VALUES) {
+      CoderProperties.coderDecodeEncodeEqual(TEST_CODER, value);
+    }
+  }
+
+  // This should never change. The definition of big endian encoding is fixed.
+  private static final String EXPECTED_ENCODING_ID = "";
+
+  @Test
+  public void testEncodingId() throws Exception {
+    CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID);
+  }
+
+  /**
+   * Generated data to check that the wire format has not changed. To regenerate, see
+   * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
+   */
+  private static final List<String> TEST_ENCODINGS = Arrays.asList(
+      "AAAEMgAAATg12KOw51bHBNnjNkPn-wPiaWQ_AsohTe-mXyOGWcybUGt9TKi2FHqY2OH-gV0_GWqRbjNAGsSskI7K3" +
+          "xf9JmTjf1ySZXuvF9S9PsgV3kT-sgypaRw_i1MK_orzcJVg_s3cEGTjTY1_Xor3JM9UBVKiQy3Vpulf7aN9LM" +
+          "kiQEfO28mXQibyUtXL4yoLIwujoo8ArC9SayfbH5HmUxX9G0e506_cefoYIGByfq3M8GLp1_METj97ViU38je" +
+          "xsXkggqxXrMG8PO6pCYNB8P_jcf9i5OagpPafem18giZ8-v3fWJPN63vkbuOtaHb9u9yGQfrN25aLpNW9ooU9" +
+          "eYbL-1ewSBwENptcIT5SMhkulcVY6e9LyAqamGWdvnbevpwW84rTQpkeJePOkIt6G1_slfkQn6VBw7Jz3Vk",
+      "AAAAAAAAAAH1",
+      "AAAAAAAAAAH9",
+      "AAAAAAAAAAH_",
+      "AAAAAAAAAAEA",
+      "AAAAAAAAAAEB",
+      "AAAAAAAAAAEF",
+      "AAAAAAAAAAEN",
+      "AAAAAAAAAAEd",
+      "AAAAAAAAAIEA________-AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
+          "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
+          "AAAAAAAAA");
+
+  @Test
+  public void testWireFormatEncode() throws Exception {
+    CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS);
+  }
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void encodeNullThrowsCoderException() throws Exception {
+    thrown.expect(CoderException.class);
+    thrown.expectMessage("cannot encode a null BigDecimal");
+
+    CoderUtils.encodeToBase64(TEST_CODER, null);
+  }
+}


[30/50] [abbrv] incubator-beam git commit: CrashingRunner: cleanup some code

Posted by da...@apache.org.
CrashingRunner: cleanup some code

make it final, fix an error message, remove unused code


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

Branch: refs/heads/python-sdk
Commit: 93f9ef92dcdcdec4f481e996b02f256cb18dc628
Parents: 3d87f8b
Author: Dan Halperin <dh...@google.com>
Authored: Thu Jun 16 10:15:58 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/CrashingRunner.java | 33 +++-----------------
 .../apache/beam/sdk/testing/TestPipeline.java   |  3 +-
 2 files changed, 6 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/93f9ef92/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
index 975facc..b3360ae 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java
@@ -21,17 +21,15 @@ package org.apache.beam.sdk.testing;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.PipelineOptions;
-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;
 
 /**
  * A {@link PipelineRunner} that applies no overrides and throws an exception on calls to
  * {@link Pipeline#run()}. For use in {@link TestPipeline} to construct but not execute pipelines.
  */
-public class CrashingRunner extends PipelineRunner<PipelineResult>{
+public final class CrashingRunner extends PipelineRunner<PipelineResult>{
 
+  @SuppressWarnings("unused") // used by reflection
   public static CrashingRunner fromOptions(PipelineOptions opts) {
     return new CrashingRunner();
   }
@@ -41,32 +39,11 @@ public class CrashingRunner extends PipelineRunner<PipelineResult>{
     throw new IllegalArgumentException(String.format("Cannot call #run(Pipeline) on an instance "
             + "of %s. %s should only be used as the default to construct a Pipeline "
             + "using %s, and cannot execute Pipelines. Instead, specify a %s "
-            + "by providing PipelineOptions in the environment variable '%s'.",
-        getClass().getSimpleName(),
-        getClass().getSimpleName(),
+            + "by providing PipelineOptions in the system property '%s'.",
+        CrashingRunner.class.getSimpleName(),
+        CrashingRunner.class.getSimpleName(),
         TestPipeline.class.getSimpleName(),
         PipelineRunner.class.getSimpleName(),
         TestPipeline.PROPERTY_BEAM_TEST_PIPELINE_OPTIONS));
   }
-
-  private static class TestPipelineResult implements PipelineResult {
-    private TestPipelineResult() {
-      // Should never be instantiated by the enclosing class
-      throw new UnsupportedOperationException(String.format("Forbidden to instantiate %s",
-          getClass().getSimpleName()));
-    }
-
-    @Override
-    public State getState() {
-      throw new UnsupportedOperationException(String.format("Forbidden to instantiate %s",
-          getClass().getSimpleName()));
-    }
-
-    @Override
-    public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
-        throws AggregatorRetrievalException {
-      throw new AssertionError(String.format("Forbidden to instantiate %s",
-          getClass().getSimpleName()));
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/93f9ef92/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 4618e33..0dba043 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -141,8 +141,7 @@ public class TestPipeline extends Pipeline {
           Strings.isNullOrEmpty(beamTestPipelineOptions)
               ? PipelineOptionsFactory.create()
               : PipelineOptionsFactory.fromArgs(
-                      MAPPER.readValue(
-                          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), String[].class))
+                      MAPPER.readValue(beamTestPipelineOptions, String[].class))
                   .as(TestPipelineOptions.class);
 
       options.as(ApplicationNameOptions.class).setAppName(getAppName());


[12/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
deleted file mode 100644
index b1cbeb1..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessEvaluationContextTest.java
+++ /dev/null
@@ -1,545 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.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.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
-import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-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() {
-    DirectRunner runner =
-        DirectRunner.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(2));
-    // The cached value is served in the earlier reader
-    reader = context.createSideInputReader(ImmutableList.<PCollectionView<?>>of(view));
-    assertThat(reader.get(view, second), containsInAnyOrder(4444));
-  }
-
-  @Test
-  public void getExecutionContextSameStepSameKeyState() {
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(),
-            StructuralKey.of("foo", StringUtf8Coder.of()));
-
-    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, StructuralKey.of("foo", StringUtf8Coder.of()), created)
-            .commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(created.getProducingTransformInternal())
-            .withState(stepContext.commitState())
-            .build());
-
-    InProcessExecutionContext secondFooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(),
-            StructuralKey.of("foo", StringUtf8Coder.of()));
-    assertThat(
-        secondFooContext
-            .getOrCreateStepContext("s1", "s1")
-            .stateInternals()
-            .state(StateNamespaces.global(), intBag)
-            .read(),
-        contains(1));
-  }
-
-
-  @Test
-  public void getExecutionContextDifferentKeysIndependentState() {
-    InProcessExecutionContext fooContext =
-        context.getExecutionContext(created.getProducingTransformInternal(),
-            StructuralKey.of("foo", StringUtf8Coder.of()));
-
-    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(),
-            StructuralKey.of("bar", StringUtf8Coder.of()));
-    assertThat(barContext, not(equalTo(fooContext)));
-    assertThat(
-        barContext
-            .getOrCreateStepContext("s1", "s1")
-            .stateInternals()
-            .state(StateNamespaces.global(), intBag)
-            .read(),
-        emptyIterable());
-  }
-
-  @Test
-  public void getExecutionContextDifferentStepsIndependentState() {
-    StructuralKey<?> myKey = StructuralKey.of("foo", StringUtf8Coder.of());
-    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() {
-    StructuralKey<?> myKey = StructuralKey.of("foo".getBytes(), ByteArrayCoder.of());
-    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);
-    context.forceRefresh();
-    // 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);
-    context.extractFiredTimers();
-    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);
-
-    StructuralKey<?> key = StructuralKey.of("foo".length(), VarIntCoder.of());
-    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<StructuralKey<?>, FiredTimers>> fired =
-        context.extractFiredTimers();
-    assertThat(
-        fired,
-        Matchers.<AppliedPTransform<?, ?, ?>>hasKey(downstream.getProducingTransformInternal()));
-    Map<StructuralKey<?>, 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() {
-    StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
-    CommittedBundle<KV<String, Integer>> newBundle =
-        context
-            .createBundle(
-                bundleFactory.createKeyedBundle(
-                    null, key,
-                    created).commit(Instant.now()),
-                downstream).commit(Instant.now());
-    assertThat(newBundle.getKey(), Matchers.<StructuralKey<?>>equalTo(key));
-  }
-
-  @Test
-  public void createKeyedBundleKeyed() {
-    StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
-    CommittedBundle<KV<String, Integer>> keyedBundle =
-        context.createKeyedBundle(
-            bundleFactory.createRootBundle(created).commit(Instant.now()),
-            key,
-            downstream).commit(Instant.now());
-    assertThat(keyedBundle.getKey(),
-        Matchers.<StructuralKey<?>>equalTo(key));
-  }
-
-  @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());
-    context.extractFiredTimers();
-    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());
-    context.extractFiredTimers();
-    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());
-    }
-    context.extractFiredTimers();
-    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());
-    context.extractFiredTimers();
-    assertThat(context.isDone(), is(false));
-
-    context.handleResult(
-        context.createRootBundle(created).commit(Instant.now()),
-        ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(view.getProducingTransformInternal()).build());
-    context.extractFiredTimers();
-    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/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
deleted file mode 100644
index e8d4711..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.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.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.coders.Coder;
-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.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 InProcessGroupByKeyOnlyEvaluatorFactory}.
- */
-@RunWith(JUnit4.class)
-public class InProcessGroupByKeyOnlyEvaluatorFactoryTest {
-  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 InProcessGroupByKey.InProcessGroupByKeyOnly<String, Integer>());
-
-    CommittedBundle<KV<String, WindowedValue<Integer>>> inputBundle =
-        bundleFactory.createRootBundle(kvs).commit(Instant.now());
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
-
-    StructuralKey<String> fooKey = StructuralKey.of("foo", StringUtf8Coder.of());
-    UncommittedBundle<KeyedWorkItem<String, Integer>> fooBundle = bundleFactory.createKeyedBundle(
-        null, fooKey,
-        groupedKvs);
-    StructuralKey<String> barKey = StructuralKey.of("bar", StringUtf8Coder.of());
-    UncommittedBundle<KeyedWorkItem<String, Integer>> barBundle = bundleFactory.createKeyedBundle(
-        null, barKey,
-        groupedKvs);
-    StructuralKey<String> bazKey = StructuralKey.of("baz", StringUtf8Coder.of());
-    UncommittedBundle<KeyedWorkItem<String, Integer>> bazBundle = bundleFactory.createKeyedBundle(
-        null, bazKey,
-        groupedKvs);
-
-    when(evaluationContext.createKeyedBundle(inputBundle,
-        fooKey,
-        groupedKvs)).thenReturn(fooBundle);
-    when(evaluationContext.createKeyedBundle(inputBundle,
-        barKey,
-        groupedKvs)).thenReturn(barBundle);
-    when(evaluationContext.createKeyedBundle(inputBundle,
-        bazKey,
-        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 InProcessGroupByKeyOnlyEvaluatorFactory()
-            .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/babddbbc/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
deleted file mode 100644
index 746c0f8..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java
+++ /dev/null
@@ -1,520 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.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.junit.Assert.fail;
-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.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.ReadyCheckingSideInputReader;
-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.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-/**
- * 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 getNotReadyThrows() throws Exception {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("not ready");
-
-    container.createReaderForViews(ImmutableList.<PCollectionView<?>>of(mapView))
-        .get(mapView, GlobalWindow.INSTANCE);
-  }
-
-  @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)));
-    // Cached value is false
-    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(false));
-
-    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(false));
-
-    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(false));
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-
-    reader = container.createReaderForViews(ImmutableList.of(mapView, singletonView));
-    assertThat(reader.isReady(mapView, SECOND_WINDOW), is(true));
-    assertThat(reader.isReady(singletonView, SECOND_WINDOW), is(true));
-    assertThat(reader.isReady(mapView, FIRST_WINDOW), is(true));
-  }
-
-  @Test
-  public void isReadyForEmptyWindowTrue() throws Exception {
-    CountDownLatch onComplete = new CountDownLatch(1);
-    immediatelyInvokeCallback(mapView, GlobalWindow.INSTANCE);
-    CountDownLatch latch = invokeLatchedCallback(singletonView, GlobalWindow.INSTANCE, onComplete);
-
-    ReadyCheckingSideInputReader reader =
-        container.createReaderForViews(ImmutableList.of(mapView, singletonView));
-    assertThat(reader.isReady(mapView, GlobalWindow.INSTANCE), is(true));
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-
-    latch.countDown();
-    if (!onComplete.await(1500L, TimeUnit.MILLISECONDS)) {
-      fail("Callback to set empty values did not complete!");
-    }
-    // The cached value was false, so it continues to be true
-    assertThat(reader.isReady(singletonView, GlobalWindow.INSTANCE), is(false));
-
-    // A new reader for the same container gets a fresh look
-    reader = container.createReaderForViews(ImmutableList.of(mapView, singletonView));
-    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));
-  }
-
-  /**
-   * When a callAfterWindowCloses with the specified view's producing transform, window, and
-   * windowing strategy is invoked, start a thread that will invoke the callback after the returned
-   * {@link CountDownLatch} is counted down once.
-   */
-  private CountDownLatch invokeLatchedCallback(
-      PCollectionView<?> view, BoundedWindow window, final CountDownLatch onComplete) {
-    final CountDownLatch runLatch = new CountDownLatch(1);
-    doAnswer(
-        new Answer<Void>() {
-          @Override
-          public Void answer(InvocationOnMock invocation) throws Throwable {
-            Object callback = invocation.getArguments()[3];
-            final Runnable callbackRunnable = (Runnable) callback;
-            Executors.newSingleThreadExecutor().submit(new Runnable() {
-              public void run() {
-                try {
-                  if (!runLatch.await(1500L, TimeUnit.MILLISECONDS)) {
-                    fail("Run latch didn't count down within timeout");
-                  }
-                  callbackRunnable.run();
-                  onComplete.countDown();
-                } catch (InterruptedException e) {
-                  fail("Unexpectedly interrupted while waiting for latch to be counted down");
-                }
-              }
-            });
-            return null;
-          }
-        })
-        .when(context)
-        .scheduleAfterOutputWouldBeProduced(
-            Mockito.eq(view),
-            Mockito.eq(window),
-            Mockito.eq(view.getWindowingStrategyInternal()),
-            Mockito.any(Runnable.class));
-    return runLatch;
-  }
-
-  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/babddbbc/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
deleted file mode 100644
index 3e01f44..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessTimerInternalsTest.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.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.coders.VarIntCoder;
-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(StructuralKey.of(1234, VarIntCoder.of()));
-
-    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/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
new file mode 100644
index 0000000..3c9c9ee
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+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.transforms.windowing.Window;
+import org.apache.beam.sdk.util.IdentitySideInputWindowFn;
+import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+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 org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * Tests for {@link ParDoEvaluator}.
+ */
+@RunWith(JUnit4.class)
+public class ParDoEvaluatorTest {
+  @Mock private EvaluationContext evaluationContext;
+  private PCollection<Integer> inputPc;
+  private TupleTag<Integer> mainOutputTag;
+  private List<TupleTag<?>> sideOutputTags;
+  private BundleFactory bundleFactory;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    TestPipeline p = TestPipeline.create();
+    inputPc = p.apply(Create.of(1, 2, 3));
+    mainOutputTag = new TupleTag<Integer>() {};
+    sideOutputTags = TupleTagList.empty().getAll();
+
+    bundleFactory = ImmutableListBundleFactory.create();
+  }
+
+  @Test
+  public void sideInputsNotReadyResultHasUnprocessedElements() {
+    PCollectionView<Integer> singletonView =
+        inputPc
+            .apply(Window.into(new IdentitySideInputWindowFn()))
+            .apply(View.<Integer>asSingleton().withDefaultValue(0));
+    RecorderFn fn = new RecorderFn(singletonView);
+    PCollection<Integer> output = inputPc.apply(ParDo.of(fn).withSideInputs(singletonView));
+
+    CommittedBundle<Integer> inputBundle =
+        bundleFactory.createRootBundle(inputPc).commit(Instant.now());
+    UncommittedBundle<Integer> outputBundle = bundleFactory.createBundle(inputBundle, output);
+    when(evaluationContext.createBundle(inputBundle, output))
+        .thenReturn(outputBundle);
+
+    ParDoEvaluator<Integer> evaluator =
+        createEvaluator(singletonView, fn, inputBundle, output);
+
+    IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L));
+    WindowedValue<Integer> first = WindowedValue.valueInGlobalWindow(3);
+    WindowedValue<Integer> second =
+        WindowedValue.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING);
+    WindowedValue<Integer> third =
+        WindowedValue.of(
+            1,
+            new Instant(2468L),
+            ImmutableList.of(nonGlobalWindow, GlobalWindow.INSTANCE),
+            PaneInfo.NO_FIRING);
+
+    evaluator.processElement(first);
+    evaluator.processElement(second);
+    evaluator.processElement(third);
+    TransformResult result = evaluator.finishBundle();
+
+    assertThat(
+        result.getUnprocessedElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(
+            second, WindowedValue.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING)));
+    assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
+    assertThat(fn.processed, containsInAnyOrder(1, 3));
+    assertThat(
+        Iterables.getOnlyElement(result.getOutputBundles()).commit(Instant.now()).getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(
+            first.withValue(8),
+            WindowedValue.timestampedValueInGlobalWindow(6, new Instant(2468L))));
+  }
+
+  private ParDoEvaluator<Integer> createEvaluator(
+      PCollectionView<Integer> singletonView,
+      RecorderFn fn,
+      DirectRunner.CommittedBundle<Integer> inputBundle,
+      PCollection<Integer> output) {
+    when(
+            evaluationContext.createSideInputReader(
+                ImmutableList.<PCollectionView<?>>of(singletonView)))
+        .thenReturn(new ReadyInGlobalWindowReader());
+    DirectExecutionContext executionContext = mock(DirectExecutionContext.class);
+    DirectStepContext stepContext = mock(DirectStepContext.class);
+    when(
+            executionContext.getOrCreateStepContext(
+                Mockito.any(String.class), Mockito.any(String.class)))
+        .thenReturn(stepContext);
+    when(stepContext.getTimerUpdate()).thenReturn(TimerUpdate.empty());
+    when(
+            evaluationContext.getExecutionContext(
+                Mockito.any(AppliedPTransform.class), Mockito.any(StructuralKey.class)))
+        .thenReturn(executionContext);
+    when(evaluationContext.createCounterSet()).thenReturn(new CounterSet());
+
+    return ParDoEvaluator.create(
+        evaluationContext,
+        inputBundle,
+        (AppliedPTransform<PCollection<Integer>, ?, ?>) output.getProducingTransformInternal(),
+        fn,
+        ImmutableList.<PCollectionView<?>>of(singletonView),
+        mainOutputTag,
+        sideOutputTags,
+        ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, output));
+  }
+
+  private static class RecorderFn extends DoFn<Integer, Integer> {
+    private Collection<Integer> processed;
+    private final PCollectionView<Integer> view;
+
+    public RecorderFn(PCollectionView<Integer> view) {
+      processed = new ArrayList<>();
+      this.view = view;
+    }
+
+    @Override
+    public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
+      processed.add(c.element());
+      c.output(c.element() + c.sideInput(view));
+    }
+  }
+
+  private static class ReadyInGlobalWindowReader implements ReadyCheckingSideInputReader {
+    @Override
+    @Nullable
+    public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+      if (window.equals(GlobalWindow.INSTANCE)) {
+        return (T) (Integer) 5;
+      }
+      fail("Should only call get in the Global Window, others are not ready");
+      throw new AssertionError("Unreachable");
+    }
+
+    @Override
+    public <T> boolean contains(PCollectionView<T> view) {
+      return true;
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return false;
+    }
+
+    @Override
+    public boolean isReady(PCollectionView<?> view, BoundedWindow window) {
+      return window.equals(GlobalWindow.INSTANCE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
deleted file mode 100644
index b78eb40..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-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.InProcessExecutionContext.InProcessStepContext;
-import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.runners.direct.DirectRunner.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.ParDo;
-import org.apache.beam.sdk.transforms.View;
-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.transforms.windowing.Window;
-import org.apache.beam.sdk.util.IdentitySideInputWindowFn;
-import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-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 org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import javax.annotation.Nullable;
-
-/**
- * Tests for {@link ParDoInProcessEvaluator}.
- */
-@RunWith(JUnit4.class)
-public class ParDoInProcessEvaluatorTest {
-  @Mock private InProcessEvaluationContext evaluationContext;
-  private PCollection<Integer> inputPc;
-  private TupleTag<Integer> mainOutputTag;
-  private List<TupleTag<?>> sideOutputTags;
-  private BundleFactory bundleFactory;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    TestPipeline p = TestPipeline.create();
-    inputPc = p.apply(Create.of(1, 2, 3));
-    mainOutputTag = new TupleTag<Integer>() {};
-    sideOutputTags = TupleTagList.empty().getAll();
-
-    bundleFactory = InProcessBundleFactory.create();
-  }
-
-  @Test
-  public void sideInputsNotReadyResultHasUnprocessedElements() {
-    PCollectionView<Integer> singletonView =
-        inputPc
-            .apply(Window.into(new IdentitySideInputWindowFn()))
-            .apply(View.<Integer>asSingleton().withDefaultValue(0));
-    RecorderFn fn = new RecorderFn(singletonView);
-    PCollection<Integer> output = inputPc.apply(ParDo.of(fn).withSideInputs(singletonView));
-
-    CommittedBundle<Integer> inputBundle =
-        bundleFactory.createRootBundle(inputPc).commit(Instant.now());
-    UncommittedBundle<Integer> outputBundle = bundleFactory.createBundle(inputBundle, output);
-    when(evaluationContext.createBundle(inputBundle, output))
-        .thenReturn(outputBundle);
-
-    ParDoInProcessEvaluator<Integer> evaluator =
-        createEvaluator(singletonView, fn, inputBundle, output);
-
-    IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L));
-    WindowedValue<Integer> first = WindowedValue.valueInGlobalWindow(3);
-    WindowedValue<Integer> second =
-        WindowedValue.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING);
-    WindowedValue<Integer> third =
-        WindowedValue.of(
-            1,
-            new Instant(2468L),
-            ImmutableList.of(nonGlobalWindow, GlobalWindow.INSTANCE),
-            PaneInfo.NO_FIRING);
-
-    evaluator.processElement(first);
-    evaluator.processElement(second);
-    evaluator.processElement(third);
-    InProcessTransformResult result = evaluator.finishBundle();
-
-    assertThat(
-        result.getUnprocessedElements(),
-        Matchers.<WindowedValue<?>>containsInAnyOrder(
-            second, WindowedValue.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING)));
-    assertThat(result.getOutputBundles(), Matchers.<UncommittedBundle<?>>contains(outputBundle));
-    assertThat(fn.processed, containsInAnyOrder(1, 3));
-    assertThat(
-        Iterables.getOnlyElement(result.getOutputBundles()).commit(Instant.now()).getElements(),
-        Matchers.<WindowedValue<?>>containsInAnyOrder(
-            first.withValue(8),
-            WindowedValue.timestampedValueInGlobalWindow(6, new Instant(2468L))));
-  }
-
-  private ParDoInProcessEvaluator<Integer> createEvaluator(
-      PCollectionView<Integer> singletonView,
-      RecorderFn fn,
-      DirectRunner.CommittedBundle<Integer> inputBundle,
-      PCollection<Integer> output) {
-    when(
-            evaluationContext.createSideInputReader(
-                ImmutableList.<PCollectionView<?>>of(singletonView)))
-        .thenReturn(new ReadyInGlobalWindowReader());
-    InProcessExecutionContext executionContext = mock(InProcessExecutionContext.class);
-    InProcessStepContext stepContext = mock(InProcessStepContext.class);
-    when(
-            executionContext.getOrCreateStepContext(
-                Mockito.any(String.class), Mockito.any(String.class)))
-        .thenReturn(stepContext);
-    when(stepContext.getTimerUpdate()).thenReturn(TimerUpdate.empty());
-    when(
-            evaluationContext.getExecutionContext(
-                Mockito.any(AppliedPTransform.class), Mockito.any(StructuralKey.class)))
-        .thenReturn(executionContext);
-    when(evaluationContext.createCounterSet()).thenReturn(new CounterSet());
-
-    return ParDoInProcessEvaluator.create(
-        evaluationContext,
-        inputBundle,
-        (AppliedPTransform<PCollection<Integer>, ?, ?>) output.getProducingTransformInternal(),
-        fn,
-        ImmutableList.<PCollectionView<?>>of(singletonView),
-        mainOutputTag,
-        sideOutputTags,
-        ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, output));
-  }
-
-  private static class RecorderFn extends DoFn<Integer, Integer> {
-    private Collection<Integer> processed;
-    private final PCollectionView<Integer> view;
-
-    public RecorderFn(PCollectionView<Integer> view) {
-      processed = new ArrayList<>();
-      this.view = view;
-    }
-
-    @Override
-    public void processElement(DoFn<Integer, Integer>.ProcessContext c) throws Exception {
-      processed.add(c.element());
-      c.output(c.element() + c.sideInput(view));
-    }
-  }
-
-  private static class ReadyInGlobalWindowReader implements ReadyCheckingSideInputReader {
-    @Override
-    @Nullable
-    public <T> T get(PCollectionView<T> view, BoundedWindow window) {
-      if (window.equals(GlobalWindow.INSTANCE)) {
-        return (T) (Integer) 5;
-      }
-      fail("Should only call get in the Global Window, others are not ready");
-      throw new AssertionError("Unreachable");
-    }
-
-    @Override
-    public <T> boolean contains(PCollectionView<T> view) {
-      return true;
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return false;
-    }
-
-    @Override
-    public boolean isReady(PCollectionView<?> view, BoundedWindow window) {
-      return window.equals(GlobalWindow.INSTANCE);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index e61881e..6206c22 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -25,9 +25,9 @@ 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.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
@@ -69,7 +69,7 @@ import java.io.Serializable;
  */
 @RunWith(JUnit4.class)
 public class ParDoMultiEvaluatorFactoryTest implements Serializable {
-  private transient BundleFactory bundleFactory = InProcessBundleFactory.create();
+  private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
   @Test
   public void testParDoMultiInMemoryTransformEvaluator() throws Exception {
@@ -101,7 +101,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     PCollection<String> elementOutput = outputTuple.get(elementTag);
     PCollection<Integer> lengthOutput = outputTuple.get(lengthTag);
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
     UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
@@ -112,8 +112,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
         .thenReturn(elementOutputBundle);
     when(evaluationContext.createBundle(inputBundle, lengthOutput)).thenReturn(lengthOutputBundle);
 
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
+    DirectExecutionContext executionContext =
+        new DirectExecutionContext(null, null, null, null);
     when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(),
         inputBundle.getKey())).thenReturn(executionContext);
     CounterSet counters = new CounterSet();
@@ -130,7 +130,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getOutputBundles(),
         Matchers.<UncommittedBundle<?>>containsInAnyOrder(
@@ -188,7 +188,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
     PCollection<String> elementOutput = outputTuple.get(elementTag);
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
     UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
@@ -197,8 +197,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     when(evaluationContext.createBundle(inputBundle, elementOutput))
         .thenReturn(elementOutputBundle);
 
-    InProcessExecutionContext executionContext =
-        new InProcessExecutionContext(null, null, null, null);
+    DirectExecutionContext executionContext =
+        new DirectExecutionContext(null, null, null, null);
     when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(),
         inputBundle.getKey())).thenReturn(executionContext);
     CounterSet counters = new CounterSet();
@@ -215,7 +215,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getOutputBundles(),
         Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
@@ -278,7 +278,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
     PCollection<String> elementOutput = outputTuple.get(elementTag);
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
     UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
@@ -287,7 +287,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     when(evaluationContext.createBundle(inputBundle, elementOutput))
         .thenReturn(elementOutputBundle);
 
-    InProcessExecutionContext executionContext = new InProcessExecutionContext(null,
+    DirectExecutionContext executionContext = new DirectExecutionContext(null,
         StructuralKey.of("myKey", StringUtf8Coder.of()),
         null,
         null);
@@ -307,7 +307,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getOutputBundles(),
         Matchers.<UncommittedBundle<?>>containsInAnyOrder(mainOutputBundle, elementOutputBundle));
@@ -390,7 +390,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     PCollection<KV<String, Integer>> mainOutput = outputTuple.get(mainOutputTag);
     PCollection<String> elementOutput = outputTuple.get(elementTag);
 
-    InProcessEvaluationContext evaluationContext = mock(InProcessEvaluationContext.class);
+    EvaluationContext evaluationContext = mock(EvaluationContext.class);
     UncommittedBundle<KV<String, Integer>> mainOutputBundle =
         bundleFactory.createRootBundle(mainOutput);
     UncommittedBundle<String> elementOutputBundle = bundleFactory.createRootBundle(elementOutput);
@@ -399,7 +399,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     when(evaluationContext.createBundle(inputBundle, elementOutput))
         .thenReturn(elementOutputBundle);
 
-    InProcessExecutionContext executionContext = new InProcessExecutionContext(null,
+    DirectExecutionContext executionContext = new DirectExecutionContext(null,
         StructuralKey.of("myKey", StringUtf8Coder.of()),
         null, null);
     when(evaluationContext.getExecutionContext(mainOutput.getProducingTransformInternal(),
@@ -418,7 +418,7 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
     evaluator.processElement(
         WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     assertThat(
         result.getTimerUpdate(),
         equalTo(


[46/50] [abbrv] incubator-beam git commit: [BEAM-321] Fix Flink Comparators

Posted by da...@apache.org.
[BEAM-321] Fix Flink Comparators

KvCoderComparator and CoderComparator were hashing the key directly
while doing comparisons on the encoded form. This lead to
inconsistencies in GroupByKey results with large numbers of elements per
key.

This changes the comparators to hash on the encoded form and also adds
tests to verify the correct behavior.


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

Branch: refs/heads/python-sdk
Commit: 90d0bcfa74a0e99acb6721cc9c7623cf55e6626b
Parents: c955010
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Jun 1 11:56:18 2016 +0200
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 runners/flink/runner/pom.xml                    |  10 +
 .../beam/runners/flink/FlinkRunnerResult.java   |   8 +
 .../FlinkBatchTransformTranslators.java         |  49 +---
 .../FlinkBatchTranslationContext.java           |  13 +-
 .../translation/types/CoderComparator.java      | 217 ----------------
 .../translation/types/CoderTypeInformation.java |  14 +-
 .../translation/types/CoderTypeSerializer.java  |  37 +--
 .../types/EncodedValueComparator.java           | 197 ++++++++++++++
 .../types/EncodedValueSerializer.java           | 113 ++++++++
 .../types/EncodedValueTypeInformation.java      | 111 ++++++++
 .../translation/types/KvCoderComperator.java    | 259 -------------------
 .../types/KvCoderTypeInformation.java           | 207 ---------------
 .../flink/translation/types/KvKeySelector.java  |  51 ++++
 .../flink/EncodedValueComparatorTest.java       |  71 +++++
 .../beam/sdk/transforms/GroupByKeyTest.java     | 206 ++++++++++++++-
 15 files changed, 799 insertions(+), 764 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 8933457..33c13bf 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -50,6 +50,16 @@
       <version>${flink.version}</version>
     </dependency>
 
+    <!-- For testing -->
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-core</artifactId>
+      <version>${flink.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+
     <!-- Beam -->
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
index 481d867..a8f4cac 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
@@ -65,4 +65,12 @@ public class FlinkRunnerResult implements PipelineResult {
           new RuntimeException("Accumulator does not exist."));
     }
   }
+
+  @Override
+  public String toString() {
+    return "FlinkRunnerResult{" +
+        "aggregators=" + aggregators +
+        ", runtime=" + runtime +
+        '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
index 200e4af..ac058b2 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
@@ -27,7 +27,7 @@ import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruni
 import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
 import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.KvKeySelector;
 import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
@@ -65,7 +65,6 @@ import com.google.common.collect.Maps;
 
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.operators.Keys;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DataSource;
@@ -75,7 +74,6 @@ import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.operators.Grouping;
 import org.apache.flink.api.java.operators.MapPartitionOperator;
 import org.apache.flink.api.java.operators.SingleInputUdfOperator;
-import org.apache.flink.api.java.operators.UnsortedGrouping;
 import org.apache.flink.util.Collector;
 
 import java.util.ArrayList;
@@ -207,23 +205,15 @@ class FlinkBatchTransformTranslators {
       WindowingStrategy<?, ?> windowingStrategy =
           context.getInput(transform).getWindowingStrategy();
 
-      TypeInformation<WindowedValue<KV<K, InputT>>> kvCoderTypeInformation =
-          new KvCoderTypeInformation<>(
-              WindowedValue.getFullCoder(
-                  inputCoder,
-                  windowingStrategy.getWindowFn().windowCoder()));
-
       TypeInformation<WindowedValue<KV<K, List<InputT>>>> partialReduceTypeInfo =
-          new KvCoderTypeInformation<>(
+          new CoderTypeInformation<>(
               WindowedValue.getFullCoder(
                   KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
                   windowingStrategy.getWindowFn().windowCoder()));
 
+
       Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
-          new UnsortedGrouping<>(
-              inputDataSet,
-              new Keys.ExpressionKeys<>(new String[]{"key"},
-                  kvCoderTypeInformation));
+          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
 
       FlinkPartialReduceFunction<K, InputT, List<InputT>, ?> partialReduceFunction;
       FlinkReduceFunction<K, List<InputT>, List<InputT>, ?> reduceFunction;
@@ -279,8 +269,7 @@ class FlinkBatchTransformTranslators {
               "GroupCombine: " + transform.getName());
 
       Grouping<WindowedValue<KV<K, List<InputT>>>> intermediateGrouping =
-          new UnsortedGrouping<>(
-              groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
+          groupCombine.groupBy(new KvKeySelector<List<InputT>, K>(inputCoder.getKeyCoder()));
 
       // Fully reduce the values and create output format VO
       GroupReduceOperator<
@@ -291,6 +280,7 @@ class FlinkBatchTransformTranslators {
       context.setOutputDataSet(context.getOutput(transform), outputDataSet);
 
     }
+
   }
 
   /**
@@ -373,23 +363,13 @@ class FlinkBatchTransformTranslators {
       WindowingStrategy<?, ?> windowingStrategy =
           context.getInput(transform).getWindowingStrategy();
 
-      TypeInformation<WindowedValue<KV<K, InputT>>> kvCoderTypeInformation =
-          new KvCoderTypeInformation<>(
-              WindowedValue.getFullCoder(
-                  inputCoder,
-                  windowingStrategy.getWindowFn().windowCoder()));
-
       TypeInformation<WindowedValue<KV<K, AccumT>>> partialReduceTypeInfo =
-          new KvCoderTypeInformation<>(
-              WindowedValue.getFullCoder(
-                  KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
-                  windowingStrategy.getWindowFn().windowCoder()));
+          context.getTypeInfo(
+              KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
+              windowingStrategy);
 
       Grouping<WindowedValue<KV<K, InputT>>> inputGrouping =
-          new UnsortedGrouping<>(
-              inputDataSet,
-              new Keys.ExpressionKeys<>(new String[]{"key"},
-                  kvCoderTypeInformation));
+          inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
 
       // construct a map from side input to WindowingStrategy so that
       // the DoFn runner can map main-input windows to side input windows
@@ -432,9 +412,7 @@ class FlinkBatchTransformTranslators {
             context.getTypeInfo(context.getOutput(transform));
 
         Grouping<WindowedValue<KV<K, AccumT>>> intermediateGrouping =
-            new UnsortedGrouping<>(
-                groupCombine,
-                new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
+            groupCombine.groupBy(new KvKeySelector<AccumT, K>(inputCoder.getKeyCoder()));
 
         // Fully reduce the values and create output format OutputT
         GroupReduceOperator<
@@ -469,9 +447,7 @@ class FlinkBatchTransformTranslators {
             context.getTypeInfo(context.getOutput(transform));
 
         Grouping<WindowedValue<KV<K, InputT>>> grouping =
-            new UnsortedGrouping<>(
-                inputDataSet,
-                new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
+            inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder()));
 
         // Fully reduce the values and create output format OutputT
         GroupReduceOperator<
@@ -720,4 +696,5 @@ class FlinkBatchTransformTranslators {
   }
 
   private FlinkBatchTransformTranslators() {}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
index ecc3a65..a73bf13 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
@@ -23,6 +23,7 @@ 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.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
@@ -119,11 +120,17 @@ public class FlinkBatchTranslationContext {
 
   @SuppressWarnings("unchecked")
   public <T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
-    Coder<T> valueCoder = collection.getCoder();
+    return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy());
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> TypeInformation<WindowedValue<T>> getTypeInfo(
+      Coder<T> coder,
+      WindowingStrategy<?, ?> windowingStrategy) {
     WindowedValue.FullWindowedValueCoder<T> windowedValueCoder =
         WindowedValue.getFullCoder(
-            valueCoder,
-            collection.getWindowingStrategy().getWindowFn().windowCoder());
+            coder,
+            windowingStrategy.getWindowFn().windowCoder());
 
     return new CoderTypeInformation<>(windowedValueCoder);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
deleted file mode 100644
index e06741c..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import org.apache.beam.sdk.coders.Coder;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link org.apache.beam.sdk.coders.Coder}.
- */
-public class CoderComparator<T> extends TypeComparator<T> {
-
-  private Coder<T> coder;
-
-  // We use these for internal encoding/decoding for creating copies and comparing
-  // serialized forms using a Coder
-  private transient InspectableByteArrayOutputStream buffer1;
-  private transient InspectableByteArrayOutputStream buffer2;
-
-  // For storing the Reference in encoded form
-  private transient InspectableByteArrayOutputStream referenceBuffer;
-
-  public CoderComparator(Coder<T> coder) {
-    this.coder = coder;
-    buffer1 = new InspectableByteArrayOutputStream();
-    buffer2 = new InspectableByteArrayOutputStream();
-    referenceBuffer = new InspectableByteArrayOutputStream();
-  }
-
-  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-    in.defaultReadObject();
-    buffer1 = new InspectableByteArrayOutputStream();
-    buffer2 = new InspectableByteArrayOutputStream();
-    referenceBuffer = new InspectableByteArrayOutputStream();
-  }
-
-  @Override
-  public int hash(T record) {
-    return record.hashCode();
-  }
-
-  @Override
-  public void setReference(T toCompare) {
-    referenceBuffer.reset();
-    try {
-      coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
-    } catch (IOException e) {
-      throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
-    }
-  }
-
-  @Override
-  public boolean equalToReference(T candidate) {
-    try {
-      buffer2.reset();
-      coder.encode(candidate, buffer2, Coder.Context.OUTER);
-      byte[] arr = referenceBuffer.getBuffer();
-      byte[] arrOther = buffer2.getBuffer();
-      if (referenceBuffer.size() != buffer2.size()) {
-        return false;
-      }
-      int len = buffer2.size();
-      for(int i = 0; i < len; i++ ) {
-        if (arr[i] != arrOther[i]) {
-          return false;
-        }
-      }
-      return true;
-    } catch (IOException e) {
-      throw new RuntimeException("Could not compare reference.", e);
-    }
-  }
-
-  @Override
-  public int compareToReference(TypeComparator<T> other) {
-    InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
-
-    byte[] arr = referenceBuffer.getBuffer();
-    byte[] arrOther = otherReferenceBuffer.getBuffer();
-    if (referenceBuffer.size() != otherReferenceBuffer.size()) {
-      return referenceBuffer.size() - otherReferenceBuffer.size();
-    }
-    int len = referenceBuffer.size();
-    for (int i = 0; i < len; i++) {
-      if (arr[i] != arrOther[i]) {
-        return arr[i] - arrOther[i];
-      }
-    }
-    return 0;
-  }
-
-  @Override
-  public int compare(T first, T second) {
-    try {
-      buffer1.reset();
-      buffer2.reset();
-      coder.encode(first, buffer1, Coder.Context.OUTER);
-      coder.encode(second, buffer2, Coder.Context.OUTER);
-      byte[] arr = buffer1.getBuffer();
-      byte[] arrOther = buffer2.getBuffer();
-      if (buffer1.size() != buffer2.size()) {
-        return buffer1.size() - buffer2.size();
-      }
-      int len = buffer1.size();
-      for(int i = 0; i < len; i++ ) {
-        if (arr[i] != arrOther[i]) {
-          return arr[i] - arrOther[i];
-        }
-      }
-      return 0;
-    } catch (IOException e) {
-      throw new RuntimeException("Could not compare: ", e);
-    }
-  }
-
-  @Override
-  public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
-    CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
-    T first = serializer.deserialize(firstSource);
-    T second = serializer.deserialize(secondSource);
-    return compare(first, second);
-  }
-
-  @Override
-  public boolean supportsNormalizedKey() {
-    return true;
-  }
-
-  @Override
-  public boolean supportsSerializationWithKeyNormalization() {
-    return false;
-  }
-
-  @Override
-  public int getNormalizeKeyLen() {
-    return Integer.MAX_VALUE;
-  }
-
-  @Override
-  public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-    return true;
-  }
-
-  @Override
-  public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
-    buffer1.reset();
-    try {
-      coder.encode(record, buffer1, Coder.Context.OUTER);
-    } catch (IOException e) {
-      throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
-    }
-    final byte[] data = buffer1.getBuffer();
-    final int limit = offset + numBytes;
-
-    target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
-
-    offset += buffer1.size();
-
-    while (offset < limit) {
-      target.put(offset++, (byte) 0);
-    }
-  }
-
-  @Override
-  public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean invertNormalizedKey() {
-    return false;
-  }
-
-  @Override
-  public TypeComparator<T> duplicate() {
-    return new CoderComparator<>(coder);
-  }
-
-  @Override
-  public int extractKeys(Object record, Object[] target, int index) {
-    target[index] = record;
-    return 1;
-  }
-
-  @Override
-  public TypeComparator[] getFlatComparators() {
-    return new TypeComparator[] { this.duplicate() };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
index 4434cf8..0e85486 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -18,8 +18,6 @@
 package org.apache.beam.runners.flink.translation.types;
 
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.util.WindowedValue;
 
 import com.google.common.base.Preconditions;
 
@@ -107,19 +105,13 @@ public class CoderTypeInformation<T> extends TypeInformation<T> implements Atomi
 
   @Override
   public String toString() {
-    return "CoderTypeInformation{" +
-        "coder=" + coder +
-        '}';
+    return "CoderTypeInformation{coder=" + coder + '}';
   }
 
   @Override
   public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
       executionConfig) {
-    WindowedValue.WindowedValueCoder windowCoder = (WindowedValue.WindowedValueCoder) coder;
-    if (windowCoder.getValueCoder() instanceof KvCoder) {
-      return new KvCoderComperator(windowCoder);
-    } else {
-      return new CoderComparator<>(coder);
-    }
+    throw new UnsupportedOperationException(
+        "Non-encoded values cannot be compared directly.");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index 097316b..4621951 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -21,6 +21,7 @@ import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
 import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.util.CoderUtils;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
@@ -38,28 +39,11 @@ import java.io.ObjectInputStream;
 public class CoderTypeSerializer<T> extends TypeSerializer<T> {
   
   private Coder<T> coder;
-  private transient DataInputViewWrapper inputWrapper;
-  private transient DataOutputViewWrapper outputWrapper;
-
-  // We use this for internal encoding/decoding for creating copies using the Coder.
-  private transient InspectableByteArrayOutputStream buffer;
 
   public CoderTypeSerializer(Coder<T> coder) {
     this.coder = coder;
-    this.inputWrapper = new DataInputViewWrapper(null);
-    this.outputWrapper = new DataOutputViewWrapper(null);
-
-    buffer = new InspectableByteArrayOutputStream();
   }
-  
-  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-    in.defaultReadObject();
-    this.inputWrapper = new DataInputViewWrapper(null);
-    this.outputWrapper = new DataOutputViewWrapper(null);
 
-    buffer = new InspectableByteArrayOutputStream();
-  }
-  
   @Override
   public boolean isImmutableType() {
     return false;
@@ -77,17 +61,10 @@ public class CoderTypeSerializer<T> extends TypeSerializer<T> {
 
   @Override
   public T copy(T t) {
-    buffer.reset();
-    try {
-      coder.encode(t, buffer, Coder.Context.OUTER);
-    } catch (IOException e) {
-      throw new RuntimeException("Could not copy.", e);
-    }
     try {
-      return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer
-          .size()), Coder.Context.OUTER);
-    } catch (IOException e) {
-      throw new RuntimeException("Could not copy.", e);
+      return CoderUtils.clone(coder, t);
+    } catch (CoderException e) {
+      throw new RuntimeException("Could not clone.", e);
     }
   }
 
@@ -98,19 +75,19 @@ public class CoderTypeSerializer<T> extends TypeSerializer<T> {
 
   @Override
   public int getLength() {
-    return 0;
+    return -1;
   }
 
   @Override
   public void serialize(T t, DataOutputView dataOutputView) throws IOException {
-    outputWrapper.setOutputView(dataOutputView);
+    DataOutputViewWrapper outputWrapper = new DataOutputViewWrapper(dataOutputView);
     coder.encode(t, outputWrapper, Coder.Context.NESTED);
   }
 
   @Override
   public T deserialize(DataInputView dataInputView) throws IOException {
     try {
-      inputWrapper.setInputView(dataInputView);
+      DataInputViewWrapper inputWrapper = new DataInputViewWrapper(dataInputView);
       return coder.decode(inputWrapper, Coder.Context.NESTED);
     } catch (CoderException e) {
       Throwable cause = e.getCause();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
new file mode 100644
index 0000000..69bcb41
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have
+ * been encoded to byte data by a {@link Coder}.
+ */
+public class EncodedValueComparator extends TypeComparator<byte[]> {
+
+  /** For storing the Reference in encoded form. */
+  private transient byte[] encodedReferenceKey;
+
+  private final boolean ascending;
+
+  public EncodedValueComparator(boolean ascending) {
+    this.ascending = ascending;
+  }
+
+  @Override
+  public int hash(byte[] record) {
+    return Arrays.hashCode(record);
+  }
+
+  @Override
+  public void setReference(byte[] toCompare) {
+    this.encodedReferenceKey = toCompare;
+  }
+
+  @Override
+  public boolean equalToReference(byte[] candidate) {
+    if (encodedReferenceKey.length != candidate.length) {
+      return false;
+    }
+    int len = candidate.length;
+    for (int i = 0; i < len; i++) {
+      if (encodedReferenceKey[i] != candidate[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int compareToReference(TypeComparator<byte[]> other) {
+    // VERY IMPORTANT: compareToReference does not behave like Comparable.compare
+    // the meaning of the return value is inverted.
+
+    EncodedValueComparator otherEncodedValueComparator = (EncodedValueComparator) other;
+
+    int len = Math.min(
+        encodedReferenceKey.length,
+        otherEncodedValueComparator.encodedReferenceKey.length);
+
+    for (int i = 0; i < len; i++) {
+      byte b1 = encodedReferenceKey[i];
+      byte b2 = otherEncodedValueComparator.encodedReferenceKey[i];
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? -result : result;
+      }
+    }
+    int result =
+        encodedReferenceKey.length - otherEncodedValueComparator.encodedReferenceKey.length;
+    return ascending ? -result : result;
+  }
+
+
+  @Override
+  public int compare(byte[] first, byte[] second) {
+    int len = Math.min(first.length, second.length);
+    for (int i = 0; i < len; i++) {
+      byte b1 = first[i];
+      byte b2 = second[i];
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? result : -result;
+      }
+    }
+    int result = first.length - second.length;
+    return ascending ? result : -result;
+  }
+
+  @Override
+  public int compareSerialized(
+      DataInputView firstSource,
+      DataInputView secondSource) throws IOException {
+    int lengthFirst = firstSource.readInt();
+    int lengthSecond = secondSource.readInt();
+
+    int len = Math.min(lengthFirst, lengthSecond);
+    for (int i = 0; i < len; i++) {
+      byte b1 = firstSource.readByte();
+      byte b2 = secondSource.readByte();
+      int result = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1));
+      if (result != 0) {
+        return ascending ? result : -result;
+      }
+    }
+
+    int result = lengthFirst - lengthSecond;
+    return ascending ? result : -result;
+  }
+
+
+
+  @Override
+  public boolean supportsNormalizedKey() {
+    // disabled because this seems to not work with some coders,
+    // such as the AvroCoder
+    return false;
+  }
+
+  @Override
+  public boolean supportsSerializationWithKeyNormalization() {
+    return false;
+  }
+
+  @Override
+  public int getNormalizeKeyLen() {
+    return Integer.MAX_VALUE;
+  }
+
+  @Override
+  public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+    return true;
+  }
+
+  @Override
+  public void putNormalizedKey(byte[] record, MemorySegment target, int offset, int numBytes) {
+    final int limit = offset + numBytes;
+
+    target.put(offset, record, 0, Math.min(numBytes, record.length));
+
+    offset += record.length;
+
+    while (offset < limit) {
+      target.put(offset++, (byte) 0);
+    }
+  }
+
+  @Override
+  public void writeWithKeyNormalization(byte[] record, DataOutputView target) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte[] readWithKeyDenormalization(byte[] reuse, DataInputView source) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean invertNormalizedKey() {
+    return !ascending;
+  }
+
+  @Override
+  public TypeComparator<byte[]> duplicate() {
+    return new EncodedValueComparator(ascending);
+  }
+
+  @Override
+  public int extractKeys(Object record, Object[] target, int index) {
+    target[index] = record;
+    return 1;
+  }
+
+  @Override
+  public TypeComparator[] getFlatComparators() {
+    return new TypeComparator[] { this.duplicate() };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
new file mode 100644
index 0000000..33af8d9
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * {@link TypeSerializer} for values that were encoded using a {@link Coder}.
+ */
+public final class EncodedValueSerializer extends TypeSerializer<byte[]> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final byte[] EMPTY = new byte[0];
+
+	@Override
+	public boolean isImmutableType() {
+		return true;
+	}
+
+	@Override
+	public byte[] createInstance() {
+		return EMPTY;
+	}
+
+	@Override
+	public byte[] copy(byte[] from) {
+		return from;
+	}
+	
+	@Override
+	public byte[] copy(byte[] from, byte[] reuse) {
+		return copy(from);
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+
+	@Override
+	public void serialize(byte[] record, DataOutputView target) throws IOException {
+		if (record == null) {
+			throw new IllegalArgumentException("The record must not be null.");
+		}
+		
+		final int len = record.length;
+		target.writeInt(len);
+		target.write(record);
+	}
+
+	@Override
+	public byte[] deserialize(DataInputView source) throws IOException {
+		final int len = source.readInt();
+		byte[] result = new byte[len];
+		source.readFully(result);
+		return result;
+	}
+	
+	@Override
+	public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException {
+		return deserialize(source);
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		final int len = source.readInt();
+		target.writeInt(len);
+		target.write(source, len);
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof EncodedValueSerializer;
+	}
+
+	@Override
+	public int hashCode() {
+		return this.getClass().hashCode();
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return obj instanceof EncodedValueSerializer;
+	}
+
+	@Override
+	public TypeSerializer<byte[]> duplicate() {
+		return this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
new file mode 100644
index 0000000..46c854f
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+
+import java.util.Objects;
+
+/**
+ * Flink {@link TypeInformation} for Beam values that have been encoded to byte data
+ * by a {@link Coder}.
+ */
+public class EncodedValueTypeInformation
+    extends TypeInformation<byte[]>
+    implements AtomicType<byte[]> {
+
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  @PublicEvolving
+  public boolean isBasicType() {
+    return false;
+  }
+
+  @Override
+  @PublicEvolving
+  public boolean isTupleType() {
+    return false;
+  }
+
+  @Override
+  @PublicEvolving
+  public int getArity() {
+    return 0;
+  }
+
+  @Override
+  @PublicEvolving
+  public int getTotalFields() {
+    return 0;
+  }
+
+  @Override
+  @PublicEvolving
+  public Class<byte[]> getTypeClass() {
+    return byte[].class;
+  }
+
+  @Override
+  @PublicEvolving
+  public boolean isKeyType() {
+    return true;
+  }
+
+  @Override
+  @PublicEvolving
+  public TypeSerializer<byte[]> createSerializer(ExecutionConfig executionConfig) {
+    return new EncodedValueSerializer();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return other instanceof EncodedValueTypeInformation;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.getClass().hashCode();
+  }
+
+  @Override
+  public boolean canEqual(Object obj) {
+    return obj instanceof EncodedValueTypeInformation;
+  }
+
+  @Override
+  public String toString() {
+    return "EncodedValueTypeInformation";
+  }
+
+  @Override
+  @PublicEvolving
+  public TypeComparator<byte[]> createComparator(
+      boolean sortOrderAscending,
+      ExecutionConfig executionConfig) {
+    return new EncodedValueComparator(sortOrderAscending);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
deleted file mode 100644
index 79b127d..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link TypeComparator} for {@link KvCoder}. We have a special comparator
- * for {@link KV} that always compares on the key only.
- */
-public class KvCoderComperator <K, V> extends TypeComparator<WindowedValue<KV<K, V>>> {
-  
-  private final WindowedValue.WindowedValueCoder<KV<K, V>> coder;
-  private final Coder<K> keyCoder;
-
-  // We use these for internal encoding/decoding for creating copies and comparing
-  // serialized forms using a Coder
-  private transient InspectableByteArrayOutputStream buffer1;
-  private transient InspectableByteArrayOutputStream buffer2;
-
-  // For storing the Reference in encoded form
-  private transient InspectableByteArrayOutputStream referenceBuffer;
-
-
-  // For deserializing the key
-  private transient DataInputViewWrapper inputWrapper;
-
-  public KvCoderComperator(WindowedValue.WindowedValueCoder<KV<K, V>> coder) {
-    this.coder = coder;
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) coder.getValueCoder();
-    this.keyCoder = kvCoder.getKeyCoder();
-
-    buffer1 = new InspectableByteArrayOutputStream();
-    buffer2 = new InspectableByteArrayOutputStream();
-    referenceBuffer = new InspectableByteArrayOutputStream();
-
-    inputWrapper = new DataInputViewWrapper(null);
-  }
-
-  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-    in.defaultReadObject();
-
-    buffer1 = new InspectableByteArrayOutputStream();
-    buffer2 = new InspectableByteArrayOutputStream();
-    referenceBuffer = new InspectableByteArrayOutputStream();
-
-    inputWrapper = new DataInputViewWrapper(null);
-  }
-
-  @Override
-  public int hash(WindowedValue<KV<K, V>> record) {
-    K key = record.getValue().getKey();
-    if (key != null) {
-      return key.hashCode();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public void setReference(WindowedValue<KV<K, V>> toCompare) {
-    referenceBuffer.reset();
-    try {
-      keyCoder.encode(toCompare.getValue().getKey(), referenceBuffer, Coder.Context.OUTER);
-    } catch (IOException e) {
-      throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
-    }
-  }
-
-  @Override
-  public boolean equalToReference(WindowedValue<KV<K, V>> candidate) {
-    try {
-      buffer2.reset();
-      keyCoder.encode(candidate.getValue().getKey(), buffer2, Coder.Context.OUTER);
-      byte[] arr = referenceBuffer.getBuffer();
-      byte[] arrOther = buffer2.getBuffer();
-      if (referenceBuffer.size() != buffer2.size()) {
-        return false;
-      }
-      int len = buffer2.size();
-      for (int i = 0; i < len; i++) {
-        if (arr[i] != arrOther[i]) {
-          return false;
-        }
-      }
-      return true;
-    } catch (IOException e) {
-      throw new RuntimeException("Could not compare reference.", e);
-    }
-  }
-
-  @Override
-  public int compareToReference(TypeComparator<WindowedValue<KV<K, V>>> other) {
-    InspectableByteArrayOutputStream otherReferenceBuffer =
-        ((KvCoderComperator<K, V>) other).referenceBuffer;
-
-    byte[] arr = referenceBuffer.getBuffer();
-    byte[] arrOther = otherReferenceBuffer.getBuffer();
-    if (referenceBuffer.size() != otherReferenceBuffer.size()) {
-      return referenceBuffer.size() - otherReferenceBuffer.size();
-    }
-    int len = referenceBuffer.size();
-    for (int i = 0; i < len; i++) {
-      if (arr[i] != arrOther[i]) {
-        return arr[i] - arrOther[i];
-      }
-    }
-    return 0;
-  }
-
-
-  @Override
-  public int compare(WindowedValue<KV<K, V>> first, WindowedValue<KV<K, V>> second) {
-    try {
-      buffer1.reset();
-      buffer2.reset();
-      keyCoder.encode(first.getValue().getKey(), buffer1, Coder.Context.OUTER);
-      keyCoder.encode(second.getValue().getKey(), buffer2, Coder.Context.OUTER);
-      byte[] arr = buffer1.getBuffer();
-      byte[] arrOther = buffer2.getBuffer();
-      if (buffer1.size() != buffer2.size()) {
-        return buffer1.size() - buffer2.size();
-      }
-      int len = buffer1.size();
-      for (int i = 0; i < len; i++) {
-        if (arr[i] != arrOther[i]) {
-          return arr[i] - arrOther[i];
-        }
-      }
-      return 0;
-    } catch (IOException e) {
-      throw new RuntimeException("Could not compare reference.", e);
-    }
-  }
-
-  @Override
-  public int compareSerialized(
-      DataInputView firstSource,
-      DataInputView secondSource) throws IOException {
-    inputWrapper.setInputView(firstSource);
-    WindowedValue<KV<K, V>> first = coder.decode(inputWrapper, Coder.Context.NESTED);
-    inputWrapper.setInputView(secondSource);
-    WindowedValue<KV<K, V>> second = coder.decode(inputWrapper, Coder.Context.NESTED);
-    return compare(first, second);
-  }
-
-  @Override
-  public boolean supportsNormalizedKey() {
-    return false;
-  }
-
-  @Override
-  public boolean supportsSerializationWithKeyNormalization() {
-    return false;
-  }
-
-  @Override
-  public int getNormalizeKeyLen() {
-    return Integer.MAX_VALUE;
-  }
-
-  @Override
-  public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-    return true;
-  }
-
-  @Override
-  public void putNormalizedKey(
-      WindowedValue<KV<K, V>> record,
-      MemorySegment target,
-      int offset,
-      int numBytes) {
-
-    buffer1.reset();
-    try {
-      keyCoder.encode(record.getValue().getKey(), buffer1, Coder.Context.NESTED);
-    } catch (IOException e) {
-      throw new RuntimeException(
-          "Could not serializer " + record + " using coder " + coder + ": " + e);
-    }
-    final byte[] data = buffer1.getBuffer();
-    final int limit = offset + numBytes;
-
-    int numBytesPut = Math.min(numBytes, buffer1.size());
-
-    target.put(offset, data, 0, numBytesPut);
-
-    offset += numBytesPut;
-
-    while (offset < limit) {
-      target.put(offset++, (byte) 0);
-    }
-  }
-
-  @Override
-  public void writeWithKeyNormalization(
-      WindowedValue<KV<K, V>> record,
-      DataOutputView target) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public WindowedValue<KV<K, V>> readWithKeyDenormalization(
-      WindowedValue<KV<K, V>> reuse,
-      DataInputView source) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean invertNormalizedKey() {
-    return false;
-  }
-
-  @Override
-  public TypeComparator<WindowedValue<KV<K, V>>> duplicate() {
-    return new KvCoderComperator<>(coder);
-  }
-
-  @Override
-  public int extractKeys(Object record, Object[] target, int index) {
-    WindowedValue<KV<K, V>> kv = (WindowedValue<KV<K, V>>) record;
-    K k = kv.getValue().getKey();
-    target[index] = k;
-    return 1;
-  }
-
-  @Override
-  public TypeComparator[] getFlatComparators() {
-    return new TypeComparator[] {new CoderComparator<>(keyCoder)};
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
deleted file mode 100644
index ba53f64..0000000
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink.translation.types;
-
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-import java.util.List;
-
-/**
- * Flink {@link TypeInformation} for {@link KvCoder}. This creates special comparator
- * for {@link KV} that always compares on the key only.
- */
-public class KvCoderTypeInformation<K, V> extends CompositeType<WindowedValue<KV<K, V>>> {
-
-  private final WindowedValue.WindowedValueCoder<KV<K, V>> coder;
-//  private KvCoder<K, V> coder;
-
-  // We don't have the Class, so we have to pass null here. What a shame...
-  private static Object dummy = new Object();
-
-  @SuppressWarnings("unchecked")
-  public KvCoderTypeInformation(WindowedValue.WindowedValueCoder<KV<K, V>> coder) {
-    super((Class) dummy.getClass());
-    this.coder = coder;
-    Preconditions.checkNotNull(coder);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public TypeComparator<WindowedValue<KV<K, V>>> createComparator(
-      int[] logicalKeyFields,
-      boolean[] orders,
-      int logicalFieldOffset,
-      ExecutionConfig config) {
-    return new KvCoderComperator(coder);
-  }
-
-  @Override
-  public boolean isBasicType() {
-    return false;
-  }
-
-  @Override
-  public boolean isTupleType() {
-    return false;
-  }
-
-  @Override
-  public int getArity() {
-    return 2;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public Class<WindowedValue<KV<K, V>>> getTypeClass() {
-    return privateGetTypeClass();
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <X> Class<X> privateGetTypeClass() {
-    return (Class<X>) Object.class;
-  }
-
-  @Override
-  public boolean isKeyType() {
-    return true;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public TypeSerializer<WindowedValue<KV<K, V>>> createSerializer(ExecutionConfig config) {
-    return new CoderTypeSerializer<>(coder);
-  }
-
-  @Override
-  public int getTotalFields() {
-    return 2;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    KvCoderTypeInformation that = (KvCoderTypeInformation) o;
-
-    return coder.equals(that.coder);
-
-  }
-
-  @Override
-  public int hashCode() {
-    return coder.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return "CoderTypeInformation{" +
-        "coder=" + coder +
-        '}';
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <X> TypeInformation<X> getTypeAt(int pos) {
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) coder.getValueCoder();
-    if (pos == 0) {
-      return (TypeInformation<X>) new CoderTypeInformation<>(kvCoder.getKeyCoder());
-    } else if (pos == 1) {
-      return (TypeInformation<X>) new CoderTypeInformation<>(kvCoder.getValueCoder());
-    } else {
-      throw new RuntimeException("Invalid field position " + pos);
-    }
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <X> TypeInformation<X> getTypeAt(String fieldExpression) {
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) coder.getValueCoder();
-    switch (fieldExpression) {
-      case "key":
-        return (TypeInformation<X>) new CoderTypeInformation<>(kvCoder.getKeyCoder());
-      case "value":
-        return (TypeInformation<X>) new CoderTypeInformation<>(kvCoder.getValueCoder());
-      default:
-        throw new UnsupportedOperationException("Only KvCoder has fields.");
-    }
-  }
-
-  @Override
-  public String[] getFieldNames() {
-    return new String[]{"key", "value"};
-  }
-
-  @Override
-  public int getFieldIndex(String fieldName) {
-    switch (fieldName) {
-      case "key":
-        return 0;
-      case "value":
-        return 1;
-      default:
-        return -1;
-    }
-  }
-
-  @Override
-  public void getFlatFields(
-      String fieldExpression,
-      int offset,
-      List<FlatFieldDescriptor> result) {
-    KvCoder<K, V> kvCoder = (KvCoder<K, V>) coder.getValueCoder();
-
-    CoderTypeInformation keyTypeInfo =
-        new CoderTypeInformation<>(kvCoder.getKeyCoder());
-      result.add(new FlatFieldDescriptor(0, keyTypeInfo));
-  }
-
-  @Override
-  protected TypeComparatorBuilder<WindowedValue<KV<K, V>>> createTypeComparatorBuilder() {
-    return new KvCoderTypeComparatorBuilder();
-  }
-
-  private class KvCoderTypeComparatorBuilder
-      implements TypeComparatorBuilder<WindowedValue<KV<K, V>>> {
-
-    @Override
-    public void initializeTypeComparatorBuilder(int size) {}
-
-    @Override
-    public void addComparatorField(int fieldId, TypeComparator<?> comparator) {}
-
-    @Override
-    public TypeComparator<WindowedValue<KV<K, V>>> createTypeComparator(ExecutionConfig config) {
-      return new KvCoderComperator<>(coder);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java
new file mode 100644
index 0000000..80d20ca
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.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.flink.translation.types;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+/**
+ * {@link KeySelector} that extracts the key from a {@link KV} and returns
+ * it in encoded form as a {@code byte} array.
+ */
+public class KvKeySelector<InputT, K>
+    implements KeySelector<WindowedValue<KV<K, InputT>>, byte[]>, ResultTypeQueryable<byte[]> {
+
+  private final Coder<K> keyCoder;
+
+  public KvKeySelector(Coder<K> keyCoder) {
+    this.keyCoder = keyCoder;
+  }
+
+  @Override
+  public byte[] getKey(WindowedValue<KV<K, InputT>> value) throws Exception {
+    return CoderUtils.encodeToByteArray(keyCoder, value.getValue().getKey());
+  }
+
+  @Override
+  public TypeInformation<byte[]> getProducedType() {
+    return new EncodedValueTypeInformation();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java
new file mode 100644
index 0000000..68ede89
--- /dev/null
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.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.flink;
+
+import org.apache.beam.runners.flink.translation.types.EncodedValueComparator;
+import org.apache.beam.runners.flink.translation.types.EncodedValueTypeInformation;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.junit.Assert;
+
+/**
+ * Test for {@link EncodedValueComparator}.
+ */
+public class EncodedValueComparatorTest extends ComparatorTestBase<byte[]> {
+
+  @Override
+  protected TypeComparator<byte[]> createComparator(boolean ascending) {
+    return new EncodedValueTypeInformation().createComparator(ascending, new ExecutionConfig());
+  }
+
+  @Override
+  protected TypeSerializer<byte[]> createSerializer() {
+    return new EncodedValueTypeInformation().createSerializer(new ExecutionConfig());
+  }
+
+  @Override
+  protected void deepEquals(String message, byte[] should, byte[] is) {
+    Assert.assertArrayEquals(message, should, is);
+  }
+
+  @Override
+  protected byte[][] getSortedTestData() {
+    StringUtf8Coder coder = StringUtf8Coder.of();
+
+    try {
+      return new byte[][]{
+          CoderUtils.encodeToByteArray(coder, ""),
+          CoderUtils.encodeToByteArray(coder, "Lorem Ipsum Dolor Omit Longer"),
+          CoderUtils.encodeToByteArray(coder, "aaaa"),
+          CoderUtils.encodeToByteArray(coder, "abcd"),
+          CoderUtils.encodeToByteArray(coder, "abce"),
+          CoderUtils.encodeToByteArray(coder, "abdd"),
+          CoderUtils.encodeToByteArray(coder, "accd"),
+          CoderUtils.encodeToByteArray(coder, "bbcd")
+      };
+    } catch (CoderException e) {
+      throw new RuntimeException("Could not encode values.", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90d0bcfa/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
index 299def7..d6e4589 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
@@ -19,14 +19,17 @@ package org.apache.beam.sdk.transforms;
 
 import static org.apache.beam.sdk.TestUtils.KvMatcher.isKv;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
 import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
 
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.MapCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -40,6 +43,7 @@ import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.Sessions;
 import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.Reshuffle;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
@@ -47,6 +51,11 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Assert;
@@ -57,9 +66,16 @@ import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
 
 /**
  * Tests for GroupByKey.
@@ -384,4 +400,192 @@ public class GroupByKeyTest {
     assertThat(gbkDisplayData.items(), empty());
     assertThat(fewKeysDisplayData, hasDisplayItem("fewKeys", true));
   }
+
+
+  /**
+   * Verify that runners correctly hash/group on the encoded value
+   * and not the value itself.
+   */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testGroupByKeyWithBadEqualsHashCode() throws Exception {
+    final int numValues = 10;
+    final int numKeys = 5;
+
+    Pipeline p = TestPipeline.create();
+
+    p.getCoderRegistry().registerCoder(BadEqualityKey.class, DeterministicKeyCoder.class);
+
+    // construct input data
+    List<KV<BadEqualityKey, Long>> input = new ArrayList<>();
+    for (int i = 0; i < numValues; i++) {
+      for (int key = 0; key < numKeys; key++) {
+        input.add(KV.of(new BadEqualityKey(key), 1L));
+      }
+    }
+
+    // We first ensure that the values are randomly partitioned in the beginning.
+    // Some runners might otherwise keep all values on the machine where
+    // they are initially created.
+    PCollection<KV<BadEqualityKey, Long>> dataset1 = p
+        .apply(Create.of(input))
+        .apply(ParDo.of(new AssignRandomKey()))
+        .apply(Reshuffle.<Long, KV<BadEqualityKey, Long>>of())
+        .apply(Values.<KV<BadEqualityKey, Long>>create());
+
+    // Make the GroupByKey and Count implicit, in real-world code
+    // this would be a Count.perKey()
+    PCollection<KV<BadEqualityKey, Long>> result = dataset1
+        .apply(GroupByKey.<BadEqualityKey, Long>create())
+        .apply(Combine.<BadEqualityKey, Long>groupedValues(new CountFn()));
+
+    PAssert.that(result).satisfies(new AssertThatCountPerKeyCorrect(numValues));
+
+    PAssert.that(result.apply(Keys.<BadEqualityKey>create()))
+        .satisfies(new AssertThatAllKeysExist(numKeys));
+
+    p.run();
+  }
+
+  /**
+   * This is a bogus key class that returns random hash values from {@link #hashCode()} and always
+   * returns {@code false} for {@link #equals(Object)}. The results of the test are correct if
+   * the runner correctly hashes and sorts on the encoded bytes.
+   */
+  static class BadEqualityKey {
+    long key;
+
+    public BadEqualityKey() {}
+
+    public BadEqualityKey(long key) {
+      this.key = key;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return ThreadLocalRandom.current().nextInt();
+    }
+  }
+
+  /**
+   * Deterministic {@link Coder} for {@link BadEqualityKey}.
+   */
+  static class DeterministicKeyCoder extends AtomicCoder<BadEqualityKey> {
+
+    @JsonCreator
+    public static DeterministicKeyCoder of() {
+      return INSTANCE;
+    }
+
+    /////////////////////////////////////////////////////////////////////////////
+
+    private static final DeterministicKeyCoder INSTANCE =
+        new DeterministicKeyCoder();
+
+    private DeterministicKeyCoder() {}
+
+    @Override
+    public void encode(BadEqualityKey value, OutputStream outStream, Context context)
+        throws IOException {
+      new DataOutputStream(outStream).writeLong(value.key);
+    }
+
+    @Override
+    public BadEqualityKey decode(InputStream inStream, Context context)
+        throws IOException {
+      return new BadEqualityKey(new DataInputStream(inStream).readLong());
+    }
+  }
+
+  /**
+   * Creates a KV that wraps the original KV together with a random key.
+   */
+  static class AssignRandomKey
+      extends DoFn<KV<BadEqualityKey, Long>, KV<Long, KV<BadEqualityKey, Long>>> {
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(KV.of(ThreadLocalRandom.current().nextLong(), c.element()));
+    }
+  }
+
+  static class CountFn implements SerializableFunction<Iterable<Long>, Long> {
+    @Override
+    public Long apply(Iterable<Long> input) {
+      long result = 0L;
+      for (Long in: input) {
+        result += in;
+      }
+      return result;
+    }
+  }
+
+  static class AssertThatCountPerKeyCorrect
+      implements SerializableFunction<Iterable<KV<BadEqualityKey, Long>>, Void> {
+    private final int numValues;
+
+    AssertThatCountPerKeyCorrect(int numValues) {
+      this.numValues = numValues;
+    }
+
+    @Override
+    public Void apply(Iterable<KV<BadEqualityKey, Long>> input) {
+      for (KV<BadEqualityKey, Long> val: input) {
+        Assert.assertEquals(numValues, (long) val.getValue());
+      }
+      return null;
+    }
+  }
+
+  static class AssertThatAllKeysExist
+      implements SerializableFunction<Iterable<BadEqualityKey>, Void> {
+    private final int numKeys;
+
+    AssertThatAllKeysExist(int numKeys) {
+      this.numKeys = numKeys;
+    }
+
+    private static <T> Iterable<Object> asStructural(
+        final Iterable<T> iterable,
+        final Coder<T> coder) {
+
+      return Iterables.transform(
+          iterable,
+          new Function<T, Object>() {
+            @Override
+            public Object apply(T input) {
+              try {
+                return coder.structuralValue(input);
+              } catch (Exception e) {
+                Assert.fail("Could not structural values.");
+                throw new RuntimeException(); // to satisfy the compiler...
+              }
+            }
+          });
+
+    }
+    @Override
+    public Void apply(Iterable<BadEqualityKey> input) {
+      final DeterministicKeyCoder keyCoder = DeterministicKeyCoder.of();
+
+      List<BadEqualityKey> expectedList = new ArrayList<>();
+      for (int key = 0; key < numKeys; key++) {
+        expectedList.add(new BadEqualityKey(key));
+      }
+
+      Iterable<Object> structuralInput = asStructural(input, keyCoder);
+      Iterable<Object> structuralExpected = asStructural(expectedList, keyCoder);
+
+      for (Object expected: structuralExpected) {
+        assertThat(structuralInput, hasItem(expected));
+      }
+
+      return null;
+    }
+  }
 }


[41/50] [abbrv] incubator-beam git commit: Rename DataflowPipelineRunner to DataflowRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6d028ac6/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
new file mode 100644
index 0000000..91e34ac
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -0,0 +1,3229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataflow;
+
+import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName;
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.runners.dataflow.internal.AssignWindows;
+import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
+import org.apache.beam.runners.dataflow.internal.IsmFormat;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
+import org.apache.beam.runners.dataflow.internal.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.runners.dataflow.internal.ReadTranslator;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions;
+import org.apache.beam.runners.dataflow.util.DataflowTransport;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.PubsubUnboundedSink;
+import org.apache.beam.sdk.io.PubsubUnboundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.ShardNameTemplate;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.Write;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+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.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.PathValidator;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.beam.sdk.util.Reshuffle;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.ValueWithRecordId;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+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.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.services.clouddebugger.v2.Clouddebugger;
+import com.google.api.services.clouddebugger.v2.model.Debuggee;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.api.services.dataflow.model.WorkerPool;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.base.Utf8;
+import com.google.common.collect.ForwardingMap;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.format.DateTimeFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to the Dataflow representation
+ * using the {@link DataflowPipelineTranslator} and then submitting
+ * them to a Dataflow service for execution.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code DataflowRunner}, the Google cloudservices account and the Google compute engine
+ * service account of the GCP project running the Dataflow Job will need access to the corresponding
+ * source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
+  private static final Logger LOG = LoggerFactory.getLogger(DataflowRunner.class);
+
+  /** Provided configuration options. */
+  private final DataflowPipelineOptions options;
+
+  /** Client for the Dataflow service. This is used to actually submit jobs. */
+  private final Dataflow dataflowClient;
+
+  /** Translator for this DataflowRunner, based on options. */
+  private final DataflowPipelineTranslator translator;
+
+  /** Custom transforms implementations. */
+  private final Map<Class<?>, Class<?>> overrides;
+
+  /** A set of user defined functions to invoke at different points in execution. */
+  private DataflowRunnerHooks hooks;
+
+  // Environment version information.
+  private static final String ENVIRONMENT_MAJOR_VERSION = "5";
+
+  // Default Docker container images that execute Dataflow worker harness, residing in Google
+  // Container Registry, separately for Batch and Streaming.
+  public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
+      = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160613";
+  public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
+      = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160613";
+
+  // The limit of CreateJob request size.
+  private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
+
+  private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
+
+  /**
+   * Project IDs must contain lowercase letters, digits, or dashes.
+   * IDs must start with a letter and may not end with a dash.
+   * This regex isn't exact - this allows for patterns that would be rejected by
+   * the service, but this is sufficient for basic validation of project IDs.
+   */
+  public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
+
+  /**
+   * Construct a runner from the provided options.
+   *
+   * @param options Properties that configure the runner.
+   * @return The newly created runner.
+   */
+  public static DataflowRunner fromOptions(PipelineOptions options) {
+    // (Re-)register standard IO factories. Clobbers any prior credentials.
+    IOChannelUtils.registerStandardIOFactories(options);
+
+    DataflowPipelineOptions dataflowOptions =
+        PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
+    ArrayList<String> missing = new ArrayList<>();
+
+    if (dataflowOptions.getAppName() == null) {
+      missing.add("appName");
+    }
+    if (missing.size() > 0) {
+      throw new IllegalArgumentException(
+          "Missing required values: " + Joiner.on(',').join(missing));
+    }
+
+    PathValidator validator = dataflowOptions.getPathValidator();
+    Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
+        && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
+        "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+    if (dataflowOptions.getStagingLocation() != null) {
+      validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
+    }
+    if (dataflowOptions.getTempLocation() != null) {
+      validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
+    }
+    if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
+      dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
+    } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
+      try {
+        dataflowOptions.setStagingLocation(
+            IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
+            + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
+      }
+    }
+
+    if (dataflowOptions.getFilesToStage() == null) {
+      dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
+          DataflowRunner.class.getClassLoader()));
+      LOG.info("PipelineOptions.filesToStage was not specified. "
+          + "Defaulting to files from the classpath: will stage {} files. "
+          + "Enable logging at DEBUG level to see which files will be staged.",
+          dataflowOptions.getFilesToStage().size());
+      LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
+    }
+
+    // Verify jobName according to service requirements, truncating converting to lowercase if
+    // necessary.
+    String jobName =
+        dataflowOptions
+            .getJobName()
+            .toLowerCase();
+    checkArgument(
+        jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
+        "JobName invalid; the name must consist of only the characters "
+            + "[-a-z0-9], starting with a letter and ending with a letter "
+            + "or number");
+    if (!jobName.equals(dataflowOptions.getJobName())) {
+      LOG.info(
+          "PipelineOptions.jobName did not match the service requirements. "
+              + "Using {} instead of {}.",
+          jobName,
+          dataflowOptions.getJobName());
+    }
+    dataflowOptions.setJobName(jobName);
+
+    // Verify project
+    String project = dataflowOptions.getProject();
+    if (project.matches("[0-9]*")) {
+      throw new IllegalArgumentException("Project ID '" + project
+          + "' invalid. Please make sure you specified the Project ID, not project number.");
+    } else if (!project.matches(PROJECT_ID_REGEXP)) {
+      throw new IllegalArgumentException("Project ID '" + project
+          + "' invalid. Please make sure you specified the Project ID, not project description.");
+    }
+
+    DataflowPipelineDebugOptions debugOptions =
+        dataflowOptions.as(DataflowPipelineDebugOptions.class);
+    // Verify the number of worker threads is a valid value
+    if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
+      throw new IllegalArgumentException("Number of worker harness threads '"
+          + debugOptions.getNumberOfWorkerHarnessThreads()
+          + "' invalid. Please make sure the value is non-negative.");
+    }
+
+    return new DataflowRunner(dataflowOptions);
+  }
+
+  @VisibleForTesting protected DataflowRunner(DataflowPipelineOptions options) {
+    this.options = options;
+    this.dataflowClient = options.getDataflowClient();
+    this.translator = DataflowPipelineTranslator.fromOptions(options);
+    this.pcollectionsRequiringIndexedFormat = new HashSet<>();
+    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
+
+    ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
+    if (options.isStreaming()) {
+      builder.put(Combine.GloballyAsSingletonView.class,
+                  StreamingCombineGloballyAsSingletonView.class);
+      builder.put(Create.Values.class, StreamingCreate.class);
+      builder.put(View.AsMap.class, StreamingViewAsMap.class);
+      builder.put(View.AsMultimap.class, StreamingViewAsMultimap.class);
+      builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class);
+      builder.put(View.AsList.class, StreamingViewAsList.class);
+      builder.put(View.AsIterable.class, StreamingViewAsIterable.class);
+      builder.put(Write.Bound.class, StreamingWrite.class);
+      builder.put(Read.Unbounded.class, StreamingUnboundedRead.class);
+      builder.put(Read.Bounded.class, UnsupportedIO.class);
+      builder.put(AvroIO.Read.Bound.class, UnsupportedIO.class);
+      builder.put(AvroIO.Write.Bound.class, UnsupportedIO.class);
+      builder.put(BigQueryIO.Read.Bound.class, UnsupportedIO.class);
+      builder.put(TextIO.Read.Bound.class, UnsupportedIO.class);
+      builder.put(TextIO.Write.Bound.class, UnsupportedIO.class);
+      builder.put(Window.Bound.class, AssignWindows.class);
+      // In streaming mode must use either the custom Pubsub unbounded source/sink or
+      // defer to Windmill's built-in implementation.
+      builder.put(PubsubIO.Read.Bound.PubsubBoundedReader.class, UnsupportedIO.class);
+      builder.put(PubsubIO.Write.Bound.PubsubBoundedWriter.class, UnsupportedIO.class);
+      if (options.getExperiments() == null
+          || !options.getExperiments().contains("enable_custom_pubsub_source")) {
+        builder.put(PubsubUnboundedSource.class, StreamingPubsubIORead.class);
+      }
+      if (options.getExperiments() == null
+          || !options.getExperiments().contains("enable_custom_pubsub_sink")) {
+        builder.put(PubsubUnboundedSink.class, StreamingPubsubIOWrite.class);
+      }
+    } else {
+      builder.put(Read.Unbounded.class, UnsupportedIO.class);
+      builder.put(Window.Bound.class, AssignWindows.class);
+      builder.put(Write.Bound.class, BatchWrite.class);
+      builder.put(AvroIO.Write.Bound.class, BatchAvroIOWrite.class);
+      builder.put(TextIO.Write.Bound.class, BatchTextIOWrite.class);
+      // In batch mode must use the custom Pubsub bounded source/sink.
+      builder.put(PubsubUnboundedSource.class, UnsupportedIO.class);
+      builder.put(PubsubUnboundedSink.class, UnsupportedIO.class);
+      if (options.getExperiments() == null
+          || !options.getExperiments().contains("disable_ism_side_input")) {
+        builder.put(View.AsMap.class, BatchViewAsMap.class);
+        builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
+        builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
+        builder.put(View.AsList.class, BatchViewAsList.class);
+        builder.put(View.AsIterable.class, BatchViewAsIterable.class);
+      }
+    }
+    overrides = builder.build();
+  }
+
+  /**
+   * Applies the given transform to the input. For transforms with customized definitions
+   * for the Dataflow pipeline runner, the application is intercepted and modified here.
+   */
+  @Override
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+      PTransform<InputT, OutputT> transform, InputT input) {
+
+    if (Combine.GroupedValues.class.equals(transform.getClass())
+        || GroupByKey.class.equals(transform.getClass())) {
+
+      // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
+      // primitives. Returning a primitive output instead of the expanded definition
+      // signals to the translator that translation is necessary.
+      @SuppressWarnings("unchecked")
+      PCollection<?> pc = (PCollection<?>) input;
+      @SuppressWarnings("unchecked")
+      OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
+          pc.getPipeline(),
+          transform instanceof GroupByKey
+              ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
+              : pc.getWindowingStrategy(),
+          pc.isBounded());
+      return outputT;
+    } else if (Window.Bound.class.equals(transform.getClass())) {
+      /*
+       * TODO: make this the generic way overrides are applied (using super.apply() rather than
+       * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
+       * additional nodes into the graph.
+       */
+      // casting to wildcard
+      @SuppressWarnings("unchecked")
+      OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
+      return windowed;
+    } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
+        && ((PCollectionList<?>) input).size() == 0) {
+      return (OutputT) Pipeline.applyTransform(input, Create.of());
+    } else if (overrides.containsKey(transform.getClass())) {
+      // It is the responsibility of whoever constructs overrides to ensure this is type safe.
+      @SuppressWarnings("unchecked")
+      Class<PTransform<InputT, OutputT>> transformClass =
+          (Class<PTransform<InputT, OutputT>>) transform.getClass();
+
+      @SuppressWarnings("unchecked")
+      Class<PTransform<InputT, OutputT>> customTransformClass =
+          (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
+
+      PTransform<InputT, OutputT> customTransform =
+          InstanceBuilder.ofType(customTransformClass)
+          .withArg(DataflowRunner.class, this)
+          .withArg(transformClass, transform)
+          .build();
+
+      return Pipeline.applyTransform(input, customTransform);
+    } else {
+      return super.apply(transform, input);
+    }
+  }
+
+  private <T> PCollection<T> applyWindow(
+      Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
+    // types are matched at compile time
+    @SuppressWarnings("unchecked")
+    Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
+    @SuppressWarnings("unchecked")
+    PCollection<T> input = (PCollection<T>) initialInput;
+    return super.apply(new AssignWindows<>(transform), input);
+  }
+
+  private String debuggerMessage(String projectId, String uniquifier) {
+    return String.format("To debug your job, visit Google Cloud Debugger at: "
+        + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
+        projectId, uniquifier);
+  }
+
+  private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
+    if (!options.getEnableCloudDebugger()) {
+      return;
+    }
+
+    if (options.getDebuggee() != null) {
+      throw new RuntimeException("Should not specify the debuggee");
+    }
+
+    Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
+    Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
+    options.setDebuggee(debuggee);
+
+    System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
+  }
+
+  private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
+    RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
+    registerReq.setDebuggee(new Debuggee()
+        .setProject(options.getProject())
+        .setUniquifier(uniquifier)
+        .setDescription(uniquifier)
+        .setAgentVersion("google.com/cloud-dataflow-java/v1"));
+
+    try {
+      RegisterDebuggeeResponse registerResponse =
+          debuggerClient.controller().debuggees().register(registerReq).execute();
+      Debuggee debuggee = registerResponse.getDebuggee();
+      if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
+        throw new RuntimeException("Unable to register with the debugger: "
+            + debuggee.getStatus().getDescription().getFormat());
+      }
+
+      return debuggee;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to register with the debugger: ", e);
+    }
+  }
+
+  @Override
+  public DataflowPipelineJob run(Pipeline pipeline) {
+    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
+
+    LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
+        + "related to Google Compute Engine usage and other Google Cloud Services.");
+
+    List<DataflowPackage> packages = options.getStager().stageFiles();
+
+
+    // Set a unique client_request_id in the CreateJob request.
+    // This is used to ensure idempotence of job creation across retried
+    // attempts to create a job. Specifically, if the service returns a job with
+    // a different client_request_id, it means the returned one is a different
+    // job previously created with the same job name, and that the job creation
+    // has been effectively rejected. The SDK should return
+    // Error::Already_Exists to user in that case.
+    int randomNum = new Random().nextInt(9000) + 1000;
+    String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
+        .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
+
+    // Try to create a debuggee ID. This must happen before the job is translated since it may
+    // update the options.
+    DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+    maybeRegisterDebuggee(dataflowOptions, requestId);
+
+    JobSpecification jobSpecification =
+        translator.translate(pipeline, this, packages);
+    Job newJob = jobSpecification.getJob();
+    newJob.setClientRequestId(requestId);
+
+    String version = ReleaseInfo.getReleaseInfo().getVersion();
+    System.out.println("Dataflow SDK version: " + version);
+
+    newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
+    // The Dataflow Service may write to the temporary directory directly, so
+    // must be verified.
+    if (!Strings.isNullOrEmpty(options.getTempLocation())) {
+      newJob.getEnvironment().setTempStoragePrefix(
+          dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
+    }
+    newJob.getEnvironment().setDataset(options.getTempDatasetId());
+    newJob.getEnvironment().setExperiments(options.getExperiments());
+
+    // Set the Docker container image that executes Dataflow worker harness, residing in Google
+    // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
+    String workerHarnessContainerImage =
+        options.as(DataflowPipelineWorkerPoolOptions.class)
+        .getWorkerHarnessContainerImage();
+    for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
+      workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
+    }
+
+    // Requirements about the service.
+    Map<String, Object> environmentVersion = new HashMap<>();
+    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
+    newJob.getEnvironment().setVersion(environmentVersion);
+    // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
+    // autoscale if specified.
+    String jobType = "JAVA_BATCH_AUTOSCALING";
+
+    if (options.isStreaming()) {
+      jobType = "STREAMING";
+    }
+    environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
+
+    if (hooks != null) {
+      hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
+    }
+
+    if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
+      try (PrintWriter printWriter = new PrintWriter(
+          new File(options.getDataflowJobFile()))) {
+        String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
+        printWriter.print(workSpecJson);
+        LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
+      } catch (IllegalStateException ex) {
+        LOG.warn("Cannot translate workflow spec to json for debug.");
+      } catch (FileNotFoundException ex) {
+        LOG.warn("Cannot create workflow spec output file.");
+      }
+    }
+
+    String jobIdToUpdate = null;
+    if (options.isUpdate()) {
+      jobIdToUpdate = getJobIdFromName(options.getJobName());
+      newJob.setTransformNameMapping(options.getTransformNameMapping());
+      newJob.setReplaceJobId(jobIdToUpdate);
+    }
+    Job jobResult;
+    try {
+      jobResult = dataflowClient
+              .projects()
+              .jobs()
+              .create(options.getProject(), newJob)
+              .execute();
+    } catch (GoogleJsonResponseException e) {
+      String errorMessages = "Unexpected errors";
+      if (e.getDetails() != null) {
+        if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
+          errorMessages = "The size of the serialized JSON representation of the pipeline "
+              + "exceeds the allowable limit. "
+              + "For more information, please check the FAQ link below:\n"
+              + "https://cloud.google.com/dataflow/faq";
+        } else {
+          errorMessages = e.getDetails().getMessage();
+        }
+      }
+      throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to create a workflow job", e);
+    }
+
+    // Obtain all of the extractors from the PTransforms used in the pipeline so the
+    // DataflowPipelineJob has access to them.
+    AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
+    Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
+        aggregatorExtractor.getAggregatorSteps();
+
+    DataflowAggregatorTransforms aggregatorTransforms =
+        new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
+
+    // Use a raw client for post-launch monitoring, as status calls may fail
+    // regularly and need not be retried automatically.
+    DataflowPipelineJob dataflowPipelineJob =
+        new DataflowPipelineJob(options.getProject(), jobResult.getId(),
+            DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
+
+    // If the service returned client request id, the SDK needs to compare it
+    // with the original id generated in the request, if they are not the same
+    // (i.e., the returned job is not created by this request), throw
+    // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
+    // depending on whether this is a reload or not.
+    if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
+        && !jobResult.getClientRequestId().equals(requestId)) {
+      // If updating a job.
+      if (options.isUpdate()) {
+        throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
+            String.format("The job named %s with id: %s has already been updated into job id: %s "
+                + "and cannot be updated again.",
+                newJob.getName(), jobIdToUpdate, jobResult.getId()));
+      } else {
+        throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
+            String.format("There is already an active job named %s with id: %s. If you want "
+                + "to submit a second job, try again by setting a different name using --jobName.",
+                newJob.getName(), jobResult.getId()));
+      }
+    }
+
+    LOG.info("To access the Dataflow monitoring console, please navigate to {}",
+        MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
+    System.out.println("Submitted job: " + jobResult.getId());
+
+    LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
+        MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
+
+    return dataflowPipelineJob;
+  }
+
+  /**
+   * Returns the DataflowPipelineTranslator associated with this object.
+   */
+  public DataflowPipelineTranslator getTranslator() {
+    return translator;
+  }
+
+  /**
+   * Sets callbacks to invoke during execution see {@code DataflowRunnerHooks}.
+   */
+  @Experimental
+  public void setHooks(DataflowRunnerHooks hooks) {
+    this.hooks = hooks;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+
+  /** Outputs a warning about PCollection views without deterministic key coders. */
+  private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
+    // We need to wait till this point to determine the names of the transforms since only
+    // at this time do we know the hierarchy of the transforms otherwise we could
+    // have just recorded the full names during apply time.
+    if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
+      final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
+      pipeline.traverseTopologically(new PipelineVisitor() {
+        @Override
+        public void visitValue(PValue value, TransformTreeNode producer) {
+        }
+
+        @Override
+        public void visitPrimitiveTransform(TransformTreeNode node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+        }
+
+        @Override
+        public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+          if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+            ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+          }
+          return CompositeBehavior.ENTER_TRANSFORM;
+        }
+
+        @Override
+        public void leaveCompositeTransform(TransformTreeNode node) {
+        }
+      });
+
+      LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
+          + "because the key coder is not deterministic. Falling back to singleton implementation "
+          + "which may cause memory and/or performance problems. Future major versions of "
+          + "Dataflow will require deterministic key coders.",
+          ptransformViewNamesWithNonDeterministicKeyCoders);
+    }
+  }
+
+  /**
+   * Returns true if the passed in {@link PCollection} needs to be materialiazed using
+   * an indexed format.
+   */
+  boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
+    return pcollectionsRequiringIndexedFormat.contains(pcol);
+  }
+
+  /**
+   * Marks the passed in {@link PCollection} as requiring to be materialized using
+   * an indexed format.
+   */
+  private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
+    pcollectionsRequiringIndexedFormat.add(pcol);
+  }
+
+  /** A set of {@link View}s with non-deterministic key coders. */
+  Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
+
+  /**
+   * Records that the {@link PTransform} requires a deterministic key coder.
+   */
+  private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
+    ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
+  }
+
+  /**
+   * A {@link GroupByKey} transform for the {@link DataflowRunner} which sorts
+   * values using the secondary key {@code K2}.
+   *
+   * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
+   * the empty window. Care must be taken *afterwards* to either re-window
+   * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
+   * values being within a window.
+   */
+  static class GroupByKeyAndSortValuesOnly<K1, K2, V>
+      extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
+    private GroupByKeyAndSortValuesOnly() {
+    }
+
+    @Override
+    public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
+      PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
+          PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
+          input.getPipeline(),
+          WindowingStrategy.globalDefault(),
+          IsBounded.BOUNDED);
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
+      rval.setCoder(
+          KvCoder.of(inputCoder.getKeyCoder(),
+          IterableCoder.of(inputCoder.getValueCoder())));
+      return rval;
+    }
+  }
+
+  /**
+   * A {@link PTransform} that groups the values by a hash of the window's byte representation
+   * and sorts the values using the windows byte representation.
+   */
+  private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
+      PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
+
+    /**
+     * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
+     * grouping by the hash of the window's byte representation and sorting the grouped values
+     * using the window's byte representation.
+     */
+    @SystemDoFnInternal
+    private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
+        extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
+
+      private final IsmRecordCoder<?> ismCoderForHash;
+      private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
+        this.ismCoderForHash = ismCoderForHash;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        @SuppressWarnings("unchecked")
+        W window = (W) c.window();
+        c.output(
+            KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
+                KV.of(window,
+                    WindowedValue.of(
+                        c.element(),
+                        c.timestamp(),
+                        c.window(),
+                        c.pane()))));
+      }
+    }
+
+    private final IsmRecordCoder<?> ismCoderForHash;
+    private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
+      this.ismCoderForHash = ismCoderForHash;
+    }
+
+    @Override
+    public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+      PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
+          input.apply(ParDo.of(
+              new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
+      rval.setCoder(
+          KvCoder.of(
+              VarIntCoder.of(),
+              KvCoder.of(windowCoder,
+                  FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
+      return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
+   * byte representation and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsSingleton<T>
+      extends PTransform<PCollection<T>, PCollectionView<T>> {
+
+    /**
+     * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
+     * <ul>
+     *   <li>Key 1: Window
+     *   <li>Value: Windowed value
+     * </ul>
+     */
+    static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                     IsmRecord<WindowedValue<T>>> {
+
+      private final Coder<W> windowCoder;
+      IsmRecordForSingularValuePerWindowDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        T previousValue = null;
+
+        Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
+        while (iterator.hasNext()) {
+          KV<W, WindowedValue<T>> next = iterator.next();
+          Object currentWindowStructuralValue = windowCoder.structuralValue(next.getKey());
+
+          // Verify that the user isn't trying to have more than one element per window as
+          // a singleton.
+          checkState(!previousWindowStructuralValue.isPresent()
+              || !previousWindowStructuralValue.get().equals(currentWindowStructuralValue),
+              "Multiple values [%s, %s] found for singleton within window [%s].",
+              previousValue,
+              next.getValue().getValue(),
+              next.getKey());
+
+          c.output(
+              IsmRecord.of(
+                  ImmutableList.of(next.getKey()), next.getValue()));
+
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          previousValue = next.getValue().getValue();
+        }
+      }
+    }
+
+    private final DataflowRunner runner;
+    private final View.AsSingleton<T> transform;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public BatchViewAsSingleton(DataflowRunner runner, View.AsSingleton<T> transform) {
+      this.runner = runner;
+      this.transform = transform;
+    }
+
+    @Override
+    public PCollectionView<T> apply(PCollection<T> input) {
+      @SuppressWarnings("unchecked")
+      Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
+          runner,
+          input,
+          new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(windowCoder),
+          transform.hasDefaultValue(),
+          transform.defaultValue(),
+          input.getCoder());
+    }
+
+    static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
+        applyForSingleton(
+            DataflowRunner runner,
+            PCollection<T> input,
+            DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                 IsmRecord<WindowedValue<FinalT>>> doFn,
+            boolean hasDefault,
+            FinalT defaultValue,
+            Coder<FinalT> defaultValueCoder) {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      PCollectionView<ViewT> view =
+          (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
+              input.getPipeline(),
+              (WindowingStrategy) input.getWindowingStrategy(),
+              hasDefault,
+              defaultValue,
+              defaultValueCoder);
+
+      IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
+          coderForSingleton(windowCoder, defaultValueCoder);
+
+      PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
+              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+              .apply(ParDo.of(doFn));
+      reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+      return reifiedPerWindowAndSorted.apply(
+          CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsSingleton";
+    }
+
+    static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
+        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+      return IsmRecordCoder.of(
+          1, // We hash using only the window
+          0, // There are no metadata records
+          ImmutableList.<Coder<?>>of(windowCoder),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
+   * and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Key 2: Index offset within window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsIterable<T>
+      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
+
+    private final DataflowRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public BatchViewAsIterable(DataflowRunner runner, View.AsIterable<T> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
+      PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
+          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+      return BatchViewAsList.applyForIterableLike(runner, input, view);
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
+   * and with records having:
+   * <ul>
+   *   <li>Key 1: Window</li>
+   *   <li>Key 2: Index offset within window</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   */
+  static class BatchViewAsList<T>
+      extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
+     * global window. Each {@link IsmRecord} has
+     * <ul>
+     *   <li>Key 1: Global window</li>
+     *   <li>Key 2: Index offset within window</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     */
+    @SystemDoFnInternal
+    static class ToIsmRecordForGlobalWindowDoFn<T>
+        extends DoFn<T, IsmRecord<WindowedValue<T>>> {
+
+      long indexInBundle;
+      @Override
+      public void startBundle(Context c) throws Exception {
+        indexInBundle = 0;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        c.output(IsmRecord.of(
+            ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
+            WindowedValue.of(
+                c.element(),
+                c.timestamp(),
+                GlobalWindow.INSTANCE,
+                c.pane())));
+        indexInBundle += 1;
+      }
+    }
+
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+     * to locate the window boundaries. The {@link IsmRecord} has:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Key 2: Index offset within window</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     */
+    @SystemDoFnInternal
+    static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+                     IsmRecord<WindowedValue<T>>> {
+
+      private final Coder<W> windowCoder;
+      ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        long elementsInWindow = 0;
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
+          // Compare to see if this is a new window so we can reset the index counter i
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Reset i since we have a new window.
+            elementsInWindow = 0;
+          }
+          c.output(IsmRecord.of(
+              ImmutableList.of(value.getKey(), elementsInWindow),
+              value.getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          elementsInWindow += 1;
+        }
+      }
+    }
+
+    private final DataflowRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public BatchViewAsList(DataflowRunner runner, View.AsList<T> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<List<T>> apply(PCollection<T> input) {
+      PCollectionView<List<T>> view = PCollectionViews.listView(
+          input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+      return applyForIterableLike(runner, input, view);
+    }
+
+    static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
+        DataflowRunner runner,
+        PCollection<T> input,
+        PCollectionView<ViewT> view) {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
+
+      // If we are working in the global window, we do not need to do a GBK using the window
+      // as the key since all the elements of the input PCollection are already such.
+      // We just reify the windowed value while converting them to IsmRecords and generating
+      // an index based upon where we are within the bundle. Each bundle
+      // maps to one file exactly.
+      if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
+        PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
+            input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
+        reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+        runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+        return reifiedPerWindowAndSorted.apply(
+            CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+      }
+
+      PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
+              .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+              .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
+      reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+      runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+      return reifiedPerWindowAndSorted.apply(
+          CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsList";
+    }
+
+    static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
+        Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+      // TODO: swap to use a variable length long coder which has values which compare
+      // the same as their byte representation compare lexicographically within the key coder
+      return IsmRecordCoder.of(
+          1, // We hash using only the window
+          0, // There are no metadata records
+          ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+   * representation. Each record is structured as follows:
+   * <ul>
+   *   <li>Key 1: User key K</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: 0L (constant)</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   *
+   * <p>Alongside the data records, there are the following metadata records:
+   * <ul>
+   *   <li>Key 1: Metadata Key</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index [0, size of map]</li>
+   *   <li>Value: variable length long byte representation of size of map if index is 0,
+   *              otherwise the byte representation of a key</li>
+   * </ul>
+   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
+   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+   * {@code [1, size of map]}.
+   *
+   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * a warning to users to specify a deterministic key coder.
+   */
+  static class BatchViewAsMap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
+
+    /**
+     * A {@link DoFn} which groups elements by window boundaries. For each group,
+     * the group of elements is transformed into a {@link TransformedMap}.
+     * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
+     * and contains a function {@code WindowedValue<V> -> V}.
+     *
+     * <p>Outputs {@link IsmRecord}s having:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Value: Transformed map containing a transform that removes the encapsulation
+     *              of the window around each value,
+     *              {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
+     * </ul>
+     */
+    static class ToMapDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+                     IsmRecord<WindowedValue<TransformedMap<K,
+                                             WindowedValue<V>,
+                                             V>>>> {
+
+      private final Coder<W> windowCoder;
+      ToMapDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c)
+          throws Exception {
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        Optional<W> previousWindow = Optional.absent();
+        Map<K, WindowedValue<V>> map = new HashMap<>();
+        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Construct the transformed map containing all the elements since we
+            // are at a window boundary.
+            c.output(IsmRecord.of(
+                ImmutableList.of(previousWindow.get()),
+                valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+            map = new HashMap<>();
+          }
+
+          // Verify that the user isn't trying to insert the same key multiple times.
+          checkState(!map.containsKey(kv.getValue().getValue().getKey()),
+              "Multiple values [%s, %s] found for single key [%s] within window [%s].",
+              map.get(kv.getValue().getValue().getKey()),
+              kv.getValue().getValue().getValue(),
+              kv.getKey());
+          map.put(kv.getValue().getValue().getKey(),
+                  kv.getValue().withValue(kv.getValue().getValue().getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          previousWindow = Optional.of(kv.getKey());
+        }
+
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a transformed map containing all the elements since the last
+        // window boundary.
+        c.output(IsmRecord.of(
+            ImmutableList.of(previousWindow.get()),
+            valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+      }
+    }
+
+    private final DataflowRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public BatchViewAsMap(DataflowRunner runner, View.AsMap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
+      return this.<BoundedWindow>applyInternal(input);
+    }
+
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+        applyInternal(PCollection<KV<K, V>> input) {
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
+            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+        return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
+      } catch (NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+        // Since the key coder is not deterministic, we convert the map into a singleton
+        // and return a singleton view equivalent.
+        return applyForSingletonFallback(input);
+      }
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsMap";
+    }
+
+    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+        applyForSingletonFallback(PCollection<KV<K, V>> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<Function<WindowedValue<V>, V>> transformCoder =
+          (Coder) SerializableCoder.of(WindowedValueToValue.class);
+
+      Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
+          TransformedMapCoder.of(
+          transformCoder,
+          MapCoder.of(
+              inputCoder.getKeyCoder(),
+              FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
+
+      TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
+          WindowedValueToValue.<V>of(),
+          ImmutableMap.<K, WindowedValue<V>>of());
+
+      return BatchViewAsSingleton.<KV<K, V>,
+                                   TransformedMap<K, WindowedValue<V>, V>,
+                                   Map<K, V>,
+                                   W> applyForSingleton(
+          runner,
+          input,
+          new ToMapDoFn<K, V, W>(windowCoder),
+          true,
+          defaultValue,
+          finalValueCoder);
+    }
+  }
+
+  /**
+   * Specialized implementation for
+   * {@link org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
+   * Dataflow runner in batch mode.
+   *
+   * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+   * representation. Each record is structured as follows:
+   * <ul>
+   *   <li>Key 1: User key K</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index offset for a given key and window.</li>
+   *   <li>Value: Windowed value</li>
+   * </ul>
+   *
+   * <p>Alongside the data records, there are the following metadata records:
+   * <ul>
+   *   <li>Key 1: Metadata Key</li>
+   *   <li>Key 2: Window</li>
+   *   <li>Key 3: Index [0, size of map]</li>
+   *   <li>Value: variable length long byte representation of size of map if index is 0,
+   *              otherwise the byte representation of a key</li>
+   * </ul>
+   * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+   * {@code [META, Window, i]}  for {@code i} in {@code [1, size of map]} stores a the users key.
+   * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+   * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+   * {@code [1, size of map]}.
+   *
+   * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+   * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+   * a warning to users to specify a deterministic key coder.
+   */
+  static class BatchViewAsMultimap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
+    /**
+     * A {@link PTransform} that groups elements by the hash of window's byte representation
+     * if the input {@link PCollection} is not within the global window. Otherwise by the hash
+     * of the window and key's byte representation. This {@link PTransform} also sorts
+     * the values by the combination of the window and key's byte representations.
+     */
+    private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
+        extends PTransform<PCollection<KV<K, V>>,
+                           PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
+
+      @SystemDoFnInternal
+      private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
+          extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
+          implements DoFn.RequiresWindowAccess {
+
+        private final IsmRecordCoder<?> coder;
+        private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
+          this.coder = coder;
+        }
+
+        @Override
+        public void processElement(ProcessContext c) throws Exception {
+          @SuppressWarnings("unchecked")
+          W window = (W) c.window();
+
+          c.output(
+              KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
+                  KV.of(KV.of(c.element().getKey(), window),
+                      WindowedValue.of(
+                          c.element().getValue(),
+                          c.timestamp(),
+                          (BoundedWindow) window,
+                          c.pane()))));
+        }
+      }
+
+      private final IsmRecordCoder<?> coder;
+      public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
+        this.coder = coder;
+      }
+
+      @Override
+      public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
+          apply(PCollection<KV<K, V>> input) {
+
+        @SuppressWarnings("unchecked")
+        Coder<W> windowCoder = (Coder<W>)
+            input.getWindowingStrategy().getWindowFn().windowCoder();
+        @SuppressWarnings("unchecked")
+        KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+
+        PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
+        keyedByHash = input.apply(
+            ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
+        keyedByHash.setCoder(
+            KvCoder.of(
+                VarIntCoder.of(),
+                KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
+                    FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+        return keyedByHash.apply(
+            new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
+      }
+    }
+
+    /**
+     * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+     * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Key 2: User key K</li>
+     *   <li>Key 3: Index offset for a given key and window.</li>
+     *   <li>Value: Windowed value</li>
+     * </ul>
+     *
+     * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
+     * and the unique key count per window to {@code outputForSize}.
+     *
+     * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
+     * throw an {@link IllegalStateException} if more than one key per window is found.
+     */
+    static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
+                     IsmRecord<WindowedValue<V>>> {
+
+      private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
+      private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
+      private final Coder<W> windowCoder;
+      private final Coder<K> keyCoder;
+      private final IsmRecordCoder<WindowedValue<V>> ismCoder;
+      private final boolean uniqueKeysExpected;
+      ToIsmRecordForMapLikeDoFn(
+          TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
+          TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
+          Coder<W> windowCoder,
+          Coder<K> keyCoder,
+          IsmRecordCoder<WindowedValue<V>> ismCoder,
+          boolean uniqueKeysExpected) {
+        this.outputForSize = outputForSize;
+        this.outputForEntrySet = outputForEntrySet;
+        this.windowCoder = windowCoder;
+        this.keyCoder = keyCoder;
+        this.ismCoder = ismCoder;
+        this.uniqueKeysExpected = uniqueKeysExpected;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        long currentKeyIndex = 0;
+        // We use one based indexing while counting
+        long currentUniqueKeyCounter = 1;
+        Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
+
+        KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
+        Object currentKeyStructuralValue =
+            keyCoder.structuralValue(currentValue.getKey().getKey());
+        Object currentWindowStructuralValue =
+            windowCoder.structuralValue(currentValue.getKey().getValue());
+
+        while (iterator.hasNext()) {
+          KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
+          Object nextKeyStructuralValue =
+              keyCoder.structuralValue(nextValue.getKey().getKey());
+          Object nextWindowStructuralValue =
+              windowCoder.structuralValue(nextValue.getKey().getValue());
+
+          outputDataRecord(c, currentValue, currentKeyIndex);
+
+          final long nextKeyIndex;
+          final long nextUniqueKeyCounter;
+
+          // Check to see if its a new window
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            // The next value is a new window, so we output for size the number of unique keys
+            // seen and the last key of the window. We also reset the next key index the unique
+            // key counter.
+            outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+            outputMetadataRecordForEntrySet(c, currentValue);
+
+            nextKeyIndex = 0;
+            nextUniqueKeyCounter = 1;
+          } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
+            // It is a new key within the same window so output the key for the entry set,
+            // reset the key index and increase the count of unique keys seen within this window.
+            outputMetadataRecordForEntrySet(c, currentValue);
+
+            nextKeyIndex = 0;
+            nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
+          } else if (!uniqueKeysExpected) {
+            // It is not a new key so we don't have to output the number of elements in this
+            // window or increase the unique key counter. All we do is increase the key index.
+
+            nextKeyIndex = currentKeyIndex + 1;
+            nextUniqueKeyCounter = currentUniqueKeyCounter;
+          } else {
+            throw new IllegalStateException(String.format(
+                "Unique keys are expected but found key %s with values %s and %s in window %s.",
+                currentValue.getKey().getKey(),
+                currentValue.getValue().getValue(),
+                nextValue.getValue().getValue(),
+                currentValue.getKey().getValue()));
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+          currentKeyStructuralValue = nextKeyStructuralValue;
+          currentKeyIndex = nextKeyIndex;
+          currentUniqueKeyCounter = nextUniqueKeyCounter;
+        }
+
+        outputDataRecord(c, currentValue, currentKeyIndex);
+        outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a record with the number of unique keys seen.
+        outputMetadataRecordForEntrySet(c, currentValue);
+      }
+
+      /** This outputs the data record. */
+      private void outputDataRecord(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
+        IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
+            ImmutableList.of(
+                value.getKey().getKey(),
+                value.getKey().getValue(),
+                keyIndex),
+            value.getValue());
+        c.output(ismRecord);
+      }
+
+      /**
+       * This outputs records which will be used to compute the number of keys for a given window.
+       */
+      private void outputMetadataRecordForSize(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
+        c.sideOutput(outputForSize,
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+                                                 value.getKey().getValue())),
+                KV.of(value.getKey().getValue(), uniqueKeyCount)));
+      }
+
+      /** This outputs records which will be used to construct the entry set. */
+      private void outputMetadataRecordForEntrySet(
+          ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
+        c.sideOutput(outputForEntrySet,
+            KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+                                                 value.getKey().getValue())),
+                KV.of(value.getKey().getValue(), value.getKey().getKey())));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
+       * <ul>
+       *   <li>Key 1: META key</li>
+       *   <li>Key 2: window</li>
+       *   <li>Key 3: 0L (constant)</li>
+       *   <li>Value: sum of values for window</li>
+       * </ul>
+       *
+       * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
+       * per window for map and multimap side inputs.
+       */
+    static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
+      private final Coder<W> windowCoder;
+      ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
+        KV<W, Long> currentValue = iterator.next();
+        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+        long size = 0;
+        while (iterator.hasNext()) {
+          KV<W, Long> nextValue = iterator.next();
+          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+          size += currentValue.getValue();
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            c.output(IsmRecord.<WindowedValue<V>>meta(
+                ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+                CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+            size = 0;
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+        }
+
+        size += currentValue.getValue();
+        // Output the final value since it is guaranteed to be on a window boundary.
+        c.output(IsmRecord.<WindowedValue<V>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+            CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
+       * <ul>
+       *   <li>Key 1: META key</li>
+       *   <li>Key 2: window</li>
+       *   <li>Key 3: index offset (1-based index)</li>
+       *   <li>Value: key</li>
+       * </ul>
+       *
+       * <p>This {@link DoFn} is meant to be used to output index to key records
+       * per window for map and multimap side inputs.
+       */
+    static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
+
+      private final Coder<K> keyCoder;
+      private final Coder<W> windowCoder;
+      ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
+        this.keyCoder = keyCoder;
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
+        KV<W, K> currentValue = iterator.next();
+        Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+        long elementsInWindow = 1;
+        while (iterator.hasNext()) {
+          KV<W, K> nextValue = iterator.next();
+          Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+          c.output(IsmRecord.<WindowedValue<V>>meta(
+              ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+              CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+          elementsInWindow += 1;
+
+          if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+            elementsInWindow = 1;
+          }
+
+          currentValue = nextValue;
+          currentWindowStructuralValue = nextWindowStructuralValue;
+        }
+
+        // Output the final value since it is guaranteed to be on a window boundary.
+        c.output(IsmRecord.<WindowedValue<V>>meta(
+            ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+            CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+      }
+    }
+
+    /**
+     * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
+     * partition, the set of elements is transformed into a {@link TransformedMap}.
+     * The transformed {@code Map<K, Iterable<V>>} is backed by a
+     * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
+     * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
+     *
+     * <p>Outputs {@link IsmRecord}s having:
+     * <ul>
+     *   <li>Key 1: Window</li>
+     *   <li>Value: Transformed map containing a transform that removes the encapsulation
+     *              of the window around each value,
+     *              {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
+     * </ul>
+     */
+    static class ToMultimapDoFn<K, V, W extends BoundedWindow>
+        extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+                     IsmRecord<WindowedValue<TransformedMap<K,
+                                                            Iterable<WindowedValue<V>>,
+                                                            Iterable<V>>>>> {
+
+      private final Coder<W> windowCoder;
+      ToMultimapDoFn(Coder<W> windowCoder) {
+        this.windowCoder = windowCoder;
+      }
+
+      @Override
+      public void processElement(ProcessContext c)
+          throws Exception {
+        Optional<Object> previousWindowStructuralValue = Optional.absent();
+        Optional<W> previousWindow = Optional.absent();
+        Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
+        for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+          Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+          if (previousWindowStructuralValue.isPresent()
+              && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+            // Construct the transformed map containing all the elements since we
+            // are at a window boundary.
+            @SuppressWarnings({"unchecked", "rawtypes"})
+            Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+            c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+                                                             Iterable<WindowedValue<V>>,
+                                                             Iterable<V>>>>of(
+                ImmutableList.of(previousWindow.get()),
+                valueInEmptyWindows(
+                    new TransformedMap<>(
+                        IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+            multimap = HashMultimap.create();
+          }
+
+          multimap.put(kv.getValue().getValue().getKey(),
+                       kv.getValue().withValue(kv.getValue().getValue().getValue()));
+          previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+          previousWindow = Optional.of(kv.getKey());
+        }
+
+        // The last value for this hash is guaranteed to be at a window boundary
+        // so we output a transformed map containing all the elements since the last
+        // window boundary.
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+        c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+                                                         Iterable<WindowedValue<V>>,
+                                                         Iterable<V>>>>of(
+            ImmutableList.of(previousWindow.get()),
+            valueInEmptyWindows(
+                new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+      }
+    }
+
+    private final DataflowRunner runner;
+    /**
+     * Builds an instance of this class from the overridden transform.
+     */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public BatchViewAsMultimap(DataflowRunner runner, View.AsMultimap<K, V> transform) {
+      this.runner = runner;
+    }
+
+    @Override
+    public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+      return this.<BoundedWindow>applyInternal(input);
+    }
+
+    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+        applyInternal(PCollection<KV<K, V>> input) {
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      try {
+        PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
+            input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+
+        return applyForMapLike(runner, input, view, false /* unique keys not expected */);
+      } catch (NonDeterministicException e) {
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+        // Since the key coder is not deterministic, we convert the map into a singleton
+        // and return a singleton view equivalent.
+        return applyForSingletonFallback(input);
+      }
+    }
+
+    /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+        applyForSingletonFallback(PCollection<KV<K, V>> input) {
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
+          (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
+
+      Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
+          TransformedMapCoder.of(
+          transformCoder,
+          MapCoder.of(
+              inputCoder.getKeyCoder(),
+              IterableCoder.of(
+                  FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+      TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
+          new TransformedMap<>(
+              IterableWithWindowedValuesToIterable.<V>of(),
+              ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
+
+      return BatchViewAsSingleton.<KV<K, V>,
+                                   TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
+                                   Map<K, Iterable<V>>,
+                                   W> applyForSingleton(
+          runner,
+          input,
+          new ToMultimapDoFn<K, V, W>(windowCoder),
+          true,
+          defaultValue,
+          finalValueCoder);
+    }
+
+    private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
+        DataflowRunner runner,
+        PCollection<KV<K, V>> input,
+        PCollectionView<ViewT> view,
+        boolean uniqueKeysExpected) throws NonDeterministicException {
+
+      @SuppressWarnings("unchecked")
+      Coder<W> windowCoder = (Coder<W>)
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+      // If our key coder is deterministic, we can use the key portion of each KV
+      // part of a composite key containing the window , key and index.
+      inputCoder.getKeyCoder().verifyDeterministic();
+
+      IsmRecordCoder<WindowedValue<V>> ismCoder =
+          coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+
+      // Create the various output tags representing the main output containing the data stream
+      // and the side outputs containing the metadata about the size and entry set.
+      TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
+      TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
+      TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
+
+      // Process all the elements grouped by key hash, and sorted by key and then window
+      // outputting to all the outputs defined above.
+      PCollectionTuple outputTuple = input
+           .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
+           .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
+                   outputForSizeTag, outputForEntrySetTag,
+                   windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
+                       .withOutputTags(mainOutputTag,
+                                       TupleTagList.of(
+                                           ImmutableList.<TupleTag<?>>of(outputForSizeTag,
+                                                                         outputForEntrySetTag))));
+
+      // Set the coder on the main data output.
+      PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
+          outputTuple.get(mainOutputTag);
+      perHashWithReifiedWindows.setCoder(ismCoder);
+
+      // Set the coder on the metadata output for size and process the entries
+      // producing a [META, Window, 0L] record per window storing the number of unique keys
+      // for each window.
+      PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
+      outputForSize.setCoder(
+          KvCoder.of(VarIntCoder.of(),
+                     KvCoder.of(windowCoder, VarLongCoder.of())));
+      PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
+          .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
+          .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
+      windowMapSizeMetadata.setCoder(ismCoder);
+
+      // Set the coder on the metadata output destined to build the entry set and process the
+      // entries producing a [META, Window, Index] record per window key pair storing the key.
+      PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
+          outputTuple.get(outputForEntrySetTag);
+      outputForEntrySet.setCoder(
+          KvCoder.of(VarIntCoder.of(),
+                     KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
+      PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
+          .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
+          .apply(ParDo.of(
+              new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
+      windowMapKeysMetadata.setCoder(ismCoder);
+
+      // Set that all these outputs should be materialized using an indexed format.
+      runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
+      runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
+      runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
+
+      PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
+          PCollectionList.of(ImmutableList.of(
+              perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
+
+      return Pipeline.applyTransform(outputs,
+                                     Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
+          .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
+                                        ViewT>of(view));
+    }
+
+    @Override
+    protected String getKindString() {
+      return "BatchViewAsMultimap";
+    }
+
+    static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
+        Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
+      // TODO: swap to use a variable length long coder which has values which compare
+      // the same as their byte representation compare lexicographically within the key coder
+      return IsmRecordCoder.of(
+          1, // We use only the key for hashing when producing value records
+          2, // Since the key is not present, we add the window to the hash when
+             // producing metadata records
+          ImmutableList.of(
+              MetadataKeyCoder.of(keyCoder),
+              windowCoder,
+              BigEndianLongCoder.of()),
+          FullWindowedValueCoder.of(valueCoder, windowCoder));
+    }
+  }
+
+  /**
+   * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
+   * {@code V1 -> V2}.
+   */
+  static class TransformedMap<K, V1, V2>
+      extends ForwardingMap<K, V2> {
+    private final Function<V1, V2> transform;
+    private final Map<K, V1> originalMap;
+    private final Map<K, V2> transformedMap;
+
+    private TransformedMap(Function<V1, V2> transform, Map<K, V1> originalMap) {
+      this.transform = transform;
+      this.originalMap = Collections.unmodifiable

<TRUNCATED>


[10/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
new file mode 100644
index 0000000..7173129
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -0,0 +1,1428 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.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;
+
+import javax.annotation.Nullable;
+
+/**
+ * Tests for {@link WatermarkManager}.
+ */
+@RunWith(JUnit4.class)
+public class WatermarkManagerTest 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 WatermarkManager 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 = WatermarkManager.create(clock, rootTransforms, consumers);
+    bundleFactory = ImmutableListBundleFactory.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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(output)),
+        new Instant(8000L));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(intsToFlatten.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(secondPcollectionBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    // 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,
+        TimerUpdate.empty(),
+        result(flattened.getProducingTransformInternal(),
+            secondPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks transformAfterProcessing =
+        manager.getWatermarks(flattened.getProducingTransformInternal());
+    manager.updateWatermarks(secondPcollectionBundle,
+        TimerUpdate.empty(),
+        result(flattened.getProducingTransformInternal(),
+            secondPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(flattenedBundleSecondCreate)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(firstPcollectionBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(flattened.getProducingTransformInternal(),
+            firstPcollectionBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(completedFlattenBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createdBundle)),
+        new Instant(Long.MAX_VALUE));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(keyBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            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,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(keyBundle)),
+        new Instant(500L));
+    manager.refreshAll();
+    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,
+        StructuralKey.of("Odd", StringUtf8Coder.of()),
+        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,
+        StructuralKey.of("Even", StringUtf8Coder.of()),
+        createdInts)
+        .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L)))
+        .commit(clock.now());
+
+    manager.updateWatermarks(null,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            ImmutableList.of(firstKeyBundle, secondKeyBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    manager.updateWatermarks(firstKeyBundle,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            firstKeyBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(-1000L));
+    manager.updateWatermarks(secondKeyBundle,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            secondKeyBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(1234L));
+    manager.refreshAll();
+
+    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,
+        StructuralKey.of("Odd", StringUtf8Coder.of()),
+        createdInts).commit(clock.now());
+    manager.updateWatermarks(fauxFirstKeyTimerBundle,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            fauxFirstKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L)));
+
+    CommittedBundle<Integer> fauxSecondKeyTimerBundle = bundleFactory.createKeyedBundle(null,
+        StructuralKey.of("Even", StringUtf8Coder.of()),
+        createdInts).commit(clock.now());
+    manager.updateWatermarks(fauxSecondKeyTimerBundle,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            fauxSecondKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(5678L));
+    manager.refreshAll();
+    assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L)));
+
+    manager.updateWatermarks(fauxSecondKeyTimerBundle,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            fauxSecondKeyTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,  TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(firstInput)),
+        new Instant(0L));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(secondInput)),
+        new Instant(-250L));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            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,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(keyBundle)),
+        new Instant(500L));
+    manager.refreshAll();
+    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));
+  }
+
+  @Test
+  public void updateWatermarkWithUnprocessedElements() {
+    WindowedValue<Integer> first = WindowedValue.valueInGlobalWindow(1);
+    WindowedValue<Integer> second =
+        WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-1000L));
+    WindowedValue<Integer> third =
+        WindowedValue.timestampedValueInGlobalWindow(3, new Instant(1234L));
+    CommittedBundle<Integer> createdBundle = bundleFactory.createRootBundle(createdInts)
+        .add(first)
+        .add(second)
+        .add(third)
+        .commit(clock.now());
+    manager.updateWatermarks(null,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createdBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    CommittedBundle<KV<String, Integer>> keyBundle = timestampedBundle(keyed,
+        TimestampedValue.of(KV.of("MyKey", 1), BoundedWindow.TIMESTAMP_MIN_VALUE));
+    manager.updateWatermarks(createdBundle,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(ImmutableList.of(second, third)),
+            Collections.<CommittedBundle<?>>singleton(keyBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    TransformWatermarks keyedWatermarks =
+        manager.getWatermarks(keyed.getProducingTransformInternal());
+    // the unprocessed second and third are readded to pending
+    assertThat(
+        keyedWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L))));
+  }
+
+  /**
+   * 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            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,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(keyBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(lateDataBundle)),
+        new Instant(2_000_000L));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            lateDataBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(lateKeyedBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+  }
+
+  public void updateWatermarkWithDifferentWindowedValueInstances() {
+    manager.updateWatermarks(
+        null,
+        TimerUpdate.empty(), result(createdInts.getProducingTransformInternal(), null,
+        Collections.<CommittedBundle<?>>singleton(
+            bundleFactory
+                .createRootBundle(createdInts)
+                .add(WindowedValue.valueInGlobalWindow(1))
+                .commit(Instant.now()))),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    CommittedBundle<Integer> createdBundle = bundleFactory.createRootBundle(createdInts)
+        .add(WindowedValue.valueInGlobalWindow(1))
+        .commit(Instant.now());
+    manager.updateWatermarks(createdBundle,
+        TimerUpdate.empty(),
+        result(keyed.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        null);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(emptyCreateOutput)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(firstCreateOutput)),
+        new Instant(12_000L));
+
+    CommittedBundle<Integer> firstFilterOutput = multiWindowedBundle(filtered);
+    manager.updateWatermarks(firstCreateOutput,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            firstCreateOutput.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(firstFilterOutput)),
+        new Instant(10_000L));
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(emptyCreateOutput)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createOutput)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            createOutput.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filterOutputBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createdBundle)),
+        new Instant(1248L));
+    manager.refreshAll();
+
+    TransformWatermarks filteredWms =
+        manager.getWatermarks(filtered.getProducingTransformInternal());
+    TransformWatermarks filteredDoubledWms =
+        manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal());
+    Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime();
+    Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime();
+
+    StructuralKey<String> key = StructuralKey.of("key", StringUtf8Coder.of());
+    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,
+        timers,
+        result(filtered.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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<StructuralKey<?>, 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,
+        TimerUpdate.builder(key)
+            .withCompletedTimers(Collections.<TimerData>singleton(pastTimer)).build(),
+        result(filtered.getProducingTransformInternal(),
+            filteredTimerBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filteredTimerResult)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    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,
+        TimerUpdate.empty(),
+        result(filteredTimesTwo.getProducingTransformInternal(),
+            filteredTimerResult.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createOutput)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+    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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(createSecondOutput)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now()));
+  }
+
+  @Test
+  public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() {
+    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+    manager.updateWatermarks(null,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(created)),
+        new Instant(40_900L));
+    manager.refreshAll();
+
+    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,
+        TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of()))
+            .setTimer(upstreamProcessingTimer)
+            .build(),
+        result(filtered.getProducingTransformInternal(),
+            created.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    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,
+        TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of()))
+            .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)).build(),
+        result(filtered.getProducingTransformInternal(),
+            otherCreated.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>emptyList()),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now())));
+  }
+
+  @Test
+  public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() {
+    CommittedBundle<Integer> created = multiWindowedBundle(createdInts, 1, 2, 3);
+    manager.updateWatermarks(
+        null,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>singleton(created)),
+        new Instant(29_919_235L));
+
+    Instant upstreamHold = new Instant(2048L);
+    CommittedBundle<Integer> filteredBundle = bundleFactory.createKeyedBundle(created,
+        StructuralKey.of("key", StringUtf8Coder.of()),
+        filtered).commit(upstreamHold);
+    manager.updateWatermarks(
+        created,
+        TimerUpdate.empty(),
+        result(filtered.getProducingTransformInternal(),
+            created.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(filteredBundle)),
+        BoundedWindow.TIMESTAMP_MAX_VALUE);
+    manager.refreshAll();
+
+    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<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.singleton(createdBundle)),
+        new Instant(1500L));
+    manager.refreshAll();
+
+    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);
+    StructuralKey<byte[]> key = StructuralKey.of(new byte[] {1, 4, 9}, ByteArrayCoder.of());
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .setTimer(lastTimer)
+            .build();
+
+    manager.updateWatermarks(createdBundle,
+        update,
+        result(filtered.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
+        new Instant(1000L));
+    manager.refreshAll();
+
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(50_000L));
+    manager.refreshAll();
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            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);
+    StructuralKey<?> key = StructuralKey.of(-12L, VarLongCoder.of());
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(lastTimer)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .build();
+
+    manager.updateWatermarks(
+        createdBundle,
+        update,
+        result(filtered.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
+        new Instant(1000L));
+    manager.refreshAll();
+
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(50_000L));
+    manager.refreshAll();
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            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);
+    StructuralKey<byte[]> key = StructuralKey.of(new byte[] {2, -2, 22}, ByteArrayCoder.of());
+    TimerUpdate update =
+        TimerUpdate.builder(key)
+            .setTimer(lastTimer)
+            .setTimer(earliestTimer)
+            .setTimer(middleTimer)
+            .build();
+
+    manager.updateWatermarks(
+        createdBundle,
+        update,
+        result(filtered.getProducingTransformInternal(),
+            createdBundle.withElements(Collections.<WindowedValue<Integer>>emptyList()),
+            Collections.<CommittedBundle<?>>singleton(multiWindowedBundle(intsToFlatten))),
+        new Instant(1000L));
+    manager.refreshAll();
+
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> firstTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        firstTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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,
+        TimerUpdate.empty(),
+        result(createdInts.getProducingTransformInternal(),
+            null,
+            Collections.<CommittedBundle<?>>emptyList()),
+        new Instant(50_000L));
+    manager.refreshAll();
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, FiredTimers>> secondTransformFiredTimers =
+        manager.extractFiredTimers();
+    assertThat(
+        secondTransformFiredTimers.get(filtered.getProducingTransformInternal()), not(nullValue()));
+    Map<StructuralKey<?>, 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(StructuralKey.of("foo", StringUtf8Coder.of()))
+            .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);
+  }
+
+  private final CommittedResult result(
+      AppliedPTransform<?, ?, ?> transform,
+      @Nullable CommittedBundle<?> unprocessedBundle,
+      Iterable<? extends CommittedBundle<?>> bundles) {
+    return CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
+        unprocessedBundle,
+        bundles);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 8a3591b..71abcca 100644
--- 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
@@ -69,7 +69,7 @@ public class WindowEvaluatorFactoryTest {
   private PCollection<Long> input;
   private WindowEvaluatorFactory factory;
 
-  @Mock private InProcessEvaluationContext evaluationContext;
+  @Mock private EvaluationContext evaluationContext;
 
   private BundleFactory bundleFactory;
 
@@ -96,7 +96,7 @@ public class WindowEvaluatorFactoryTest {
     TestPipeline p = TestPipeline.create();
     input = p.apply(Create.of(1L, 2L, 3L));
 
-    bundleFactory = InProcessBundleFactory.create();
+    bundleFactory = ImmutableListBundleFactory.create();
     factory = new WindowEvaluatorFactory();
   }
 
@@ -112,7 +112,7 @@ public class WindowEvaluatorFactoryTest {
 
     UncommittedBundle<Long> outputBundle = createOutputBundle(triggering, inputBundle);
 
-    InProcessTransformResult result = runEvaluator(triggering, inputBundle, transform);
+    TransformResult result = runEvaluator(triggering, inputBundle, transform);
 
     assertThat(
         Iterables.getOnlyElement(result.getOutputBundles()),
@@ -134,7 +134,7 @@ public class WindowEvaluatorFactoryTest {
     BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration));
     BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH);
 
-    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+    TransformResult result = runEvaluator(windowed, inputBundle, transform);
 
     assertThat(
         Iterables.getOnlyElement(result.getOutputBundles()),
@@ -163,7 +163,7 @@ public class WindowEvaluatorFactoryTest {
     CommittedBundle<Long> inputBundle = createInputBundle();
     UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
 
-    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+    TransformResult result = runEvaluator(windowed, inputBundle, transform);
 
     assertThat(
         Iterables.getOnlyElement(result.getOutputBundles()),
@@ -205,7 +205,7 @@ public class WindowEvaluatorFactoryTest {
     CommittedBundle<Long> inputBundle = createInputBundle();
     UncommittedBundle<Long> outputBundle = createOutputBundle(windowed, inputBundle);
 
-    InProcessTransformResult result = runEvaluator(windowed, inputBundle, transform);
+    TransformResult result = runEvaluator(windowed, inputBundle, transform);
 
     assertThat(
         Iterables.getOnlyElement(result.getOutputBundles()),
@@ -251,7 +251,7 @@ public class WindowEvaluatorFactoryTest {
     return outputBundle;
   }
 
-  private InProcessTransformResult runEvaluator(
+  private TransformResult runEvaluator(
       PCollection<Long> windowed,
       CommittedBundle<Long> inputBundle,
       Window.Bound<Long> windowTransform /* Required while Window.Bound is a composite */)
@@ -265,7 +265,7 @@ public class WindowEvaluatorFactoryTest {
     evaluator.processElement(first);
     evaluator.processElement(second);
     evaluator.processElement(third);
-    InProcessTransformResult result = evaluator.finishBundle();
+    TransformResult result = evaluator.finishBundle();
     return result;
   }
 


[33/50] [abbrv] incubator-beam git commit: Fix overflow in ReduceFnRunner garbage collection times

Posted by da...@apache.org.
Fix overflow in ReduceFnRunner garbage collection times


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

Branch: refs/heads/python-sdk
Commit: 4f7a2ab47c5fdd9b3de5f091a40128e68ddd11a3
Parents: 5bf732c
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 14 16:10:09 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/ReduceFnRunner.java | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4f7a2ab4/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
index 34208da..864e8e7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
@@ -936,16 +936,21 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
   }
 
   /**
-   * Return when {@code window} should be garbage collected. If the window is the GlobalWindow,
-   * that will be the end of the window. Otherwise, add the allowed lateness to the end of
-   * the window.
+   * Return when {@code window} should be garbage collected. If the window's expiration time is on
+   * or after the end of the global window, it will be truncated to the end of the global window.
    */
   private Instant garbageCollectionTime(W window) {
-    Instant maxTimestamp = window.maxTimestamp();
-    if (maxTimestamp.isBefore(GlobalWindow.INSTANCE.maxTimestamp())) {
-      return maxTimestamp.plus(windowingStrategy.getAllowedLateness());
+
+    // If the end of the window + allowed lateness is beyond the "end of time" aka the end of the
+    // global window, then we truncate it. The conditional is phrased like it is because the
+    // addition of EOW + allowed lateness might even overflow the maximum allowed Instant
+    if (GlobalWindow.INSTANCE
+        .maxTimestamp()
+        .minus(windowingStrategy.getAllowedLateness())
+        .isBefore(window.maxTimestamp())) {
+      return GlobalWindow.INSTANCE.maxTimestamp();
     } else {
-      return maxTimestamp;
+      return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
     }
   }
 


[08/50] [abbrv] incubator-beam git commit: Explicitly set the Runner in TestFlinkPipelineRunner

Posted by da...@apache.org.
Explicitly set the Runner in TestFlinkPipelineRunner

This ensures that the created PipelineOptions are valid if the
DirectRunner is not on the classpath.


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

Branch: refs/heads/python-sdk
Commit: 90bb20ee6738c57bc25f47e2d80690fb721b562e
Parents: 0065851
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 14 15:49:34 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/90bb20ee/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
index 139aebf..a55acb7 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java
@@ -45,6 +45,7 @@ public class TestFlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
 
   public static TestFlinkPipelineRunner create(boolean streaming) {
     FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+    flinkOptions.setRunner(TestFlinkPipelineRunner.class);
     flinkOptions.setStreaming(streaming);
     return TestFlinkPipelineRunner.fromOptions(flinkOptions);
   }


[19/50] [abbrv] incubator-beam git commit: Rename InProcessPipelineRunner to DirectRunner

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index ba9815b..1fec9d8 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.util.WindowedValue;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index f2d577e..b12a34c 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index 81d2520..dfc1753 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -19,7 +19,7 @@ package org.apache.beam.runners.direct;
 
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupAlsoByWindow;
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index fb637b4..4dd1475 100644
--- 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
@@ -19,7 +19,7 @@ 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.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.util.WindowedValue;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index fceb20c..5030730 100644
--- 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
@@ -17,8 +17,8 @@
  */
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.io.Read.Unbounded;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index ffaf3fa..f4260f5 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
@@ -36,7 +36,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link CreatePCollectionView} primitive {@link PTransform}.
  *
  * <p>The {@link ViewEvaluatorFactory} produces {@link TransformEvaluator TransformEvaluators} for
@@ -49,7 +49,7 @@ class ViewEvaluatorFactory implements TransformEvaluatorFactory {
   @Override
   public <T> TransformEvaluator<T> forApplication(
       AppliedPTransform<?, ?, ?> application,
-      InProcessPipelineRunner.CommittedBundle<?> inputBundle,
+      DirectRunner.CommittedBundle<?> inputBundle,
       InProcessEvaluationContext evaluationContext) {
     @SuppressWarnings({"cast", "unchecked", "rawtypes"})
     TransformEvaluator<T> evaluator = createEvaluator(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 628f94d..89866cc 100644
--- 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
@@ -17,8 +17,8 @@
  */
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -36,7 +36,7 @@ import java.util.Collection;
 import javax.annotation.Nullable;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link Bound Window.Bound} primitive {@link PTransform}.
  */
 class WindowEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index c0c1361..d94113a 100644
--- 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
@@ -114,7 +114,7 @@ public class AvroIOShardedWriteFactoryTest {
 
   private Pipeline getPipeline() {
     PipelineOptions options = TestPipeline.testingPipelineOptions();
-    options.setRunner(InProcessPipelineRunner.class);
+    options.setRunner(DirectRunner.class);
     return TestPipeline.fromOptions(options);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index bcdc089..e26f860 100644
--- 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
@@ -25,8 +25,8 @@ 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.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.BigEndianLongCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.io.BoundedSource;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 0d1b464..4969a30 100644
--- 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
@@ -60,24 +60,24 @@ public class CommittedResultTest implements Serializable {
     CommittedResult result =
         CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
             bundleFactory.createRootBundle(created).commit(Instant.now()),
-            Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
+            Collections.<DirectRunner.CommittedBundle<?>>emptyList());
 
     assertThat(result.getTransform(), Matchers.<AppliedPTransform<?, ?, ?>>equalTo(transform));
   }
 
   @Test
   public void getUncommittedElementsEqualInput() {
-    InProcessPipelineRunner.CommittedBundle<Integer> bundle =
+    DirectRunner.CommittedBundle<Integer> bundle =
         bundleFactory.createRootBundle(created)
             .add(WindowedValue.valueInGlobalWindow(2))
             .commit(Instant.now());
     CommittedResult result =
         CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
             bundle,
-            Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
+            Collections.<DirectRunner.CommittedBundle<?>>emptyList());
 
     assertThat(result.getUnprocessedInputs(),
-        Matchers.<InProcessPipelineRunner.CommittedBundle<?>>equalTo(bundle));
+        Matchers.<DirectRunner.CommittedBundle<?>>equalTo(bundle));
   }
 
   @Test
@@ -85,14 +85,14 @@ public class CommittedResultTest implements Serializable {
     CommittedResult result =
         CommittedResult.create(StepTransformResult.withoutHold(transform).build(),
             null,
-            Collections.<InProcessPipelineRunner.CommittedBundle<?>>emptyList());
+            Collections.<DirectRunner.CommittedBundle<?>>emptyList());
 
     assertThat(result.getUnprocessedInputs(), nullValue());
   }
 
   @Test
   public void getOutputsEqualInput() {
-    List<? extends InProcessPipelineRunner.CommittedBundle<?>> outputs =
+    List<? extends DirectRunner.CommittedBundle<?>> outputs =
         ImmutableList.of(bundleFactory.createRootBundle(PCollection.createPrimitiveOutputInternal(p,
             WindowingStrategy.globalDefault(),
             PCollection.IsBounded.BOUNDED)).commit(Instant.now()),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java
new file mode 100644
index 0000000..cd44b7e
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.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.DirectRegistrar.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 DirectRegistrarTest {
+  @Test
+  public void testCorrectOptionsAreReturned() {
+    assertEquals(
+        ImmutableList.of(DirectOptions.class),
+        new DirectRegistrar.InProcessOptions().getPipelineOptions());
+  }
+
+  @Test
+  public void testCorrectRunnersAreReturned() {
+    assertEquals(
+        ImmutableList.of(DirectRunner.class),
+        new DirectRegistrar.InProcessRunner().getPipelineRunners());
+  }
+
+  @Test
+  public void testServiceLoaderForOptions() {
+    for (PipelineOptionsRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
+      if (registrar instanceof DirectRegistrar.InProcessOptions) {
+        return;
+      }
+    }
+    fail("Expected to find " + DirectRegistrar.InProcessOptions.class);
+  }
+
+  @Test
+  public void testServiceLoaderForRunner() {
+    for (PipelineRunnerRegistrar registrar :
+        Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
+      if (registrar instanceof DirectRegistrar.InProcessRunner) {
+        return;
+      }
+    }
+    fail("Expected to find " + DirectRegistrar.InProcessRunner.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
new file mode 100644
index 0000000..1de38df
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.PipelineRunner;
+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.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.IllegalMutationException;
+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.TypeDescriptor;
+
+import com.google.common.collect.ImmutableMap;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for basic {@link DirectRunner} functionality.
+ */
+@RunWith(JUnit4.class)
+public class DirectRunnerTest implements Serializable {
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+  private Pipeline getPipeline() {
+    PipelineOptions opts = PipelineOptionsFactory.create();
+    opts.setRunner(DirectRunner.class);
+
+    Pipeline p = Pipeline.create(opts);
+    return p;
+  }
+
+  @Test
+  public void defaultRunnerLoaded() {
+    assertThat(DirectRunner.class,
+        Matchers.<Class<? extends PipelineRunner>>equalTo(PipelineOptionsFactory.create()
+            .getRunner()));
+  }
+
+  @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");
+
+    DirectPipelineResult result = ((DirectPipelineResult) p.run());
+    result.awaitCompletion();
+  }
+
+  @Test(timeout = 5000L)
+  public void byteArrayCountShouldSucceed() {
+    Pipeline p = getPipeline();
+
+    SerializableFunction<Integer, byte[]> getBytes = new SerializableFunction<Integer, byte[]>() {
+      @Override
+      public byte[] apply(Integer input) {
+        try {
+          return CoderUtils.encodeToByteArray(VarIntCoder.of(), input);
+        } catch (CoderException e) {
+          fail("Unexpected Coder Exception " + e);
+          throw new AssertionError("Unreachable");
+        }
+      }
+    };
+    TypeDescriptor<byte[]> td = new TypeDescriptor<byte[]>() {
+    };
+    PCollection<byte[]> foos =
+        p.apply(Create.of(1, 1, 1, 2, 2, 3)).apply(MapElements.via(getBytes).withOutputType(td));
+    PCollection<byte[]> msync =
+        p.apply(Create.of(1, -2, -8, -16)).apply(MapElements.via(getBytes).withOutputType(td));
+    PCollection<byte[]> bytes =
+        PCollectionList.of(foos).and(msync).apply(Flatten.<byte[]>pCollections());
+    PCollection<KV<byte[], Long>> counts = bytes.apply(Count.<byte[]>perElement());
+    PCollection<KV<Integer, Long>> countsBackToString =
+        counts.apply(MapElements.via(new SimpleFunction<KV<byte[], Long>, KV<Integer, Long>>() {
+          @Override
+          public KV<Integer, Long> apply(KV<byte[], Long> input) {
+            try {
+              return KV.of(CoderUtils.decodeFromByteArray(VarIntCoder.of(), input.getKey()),
+                  input.getValue());
+            } catch (CoderException e) {
+              fail("Unexpected Coder Exception " + e);
+              throw new AssertionError("Unreachable");
+        }
+      }
+    }));
+
+    Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder().put(1, 4L)
+        .put(2, 2L)
+        .put(3, 1L)
+        .put(-2, 1L)
+        .put(-8, 1L)
+        .put(-16, 1L)
+        .build();
+    PAssert.thatMap(countsBackToString).isEqualTo(expected);
+  }
+
+  @Test
+  public void transformDisplayDataExceptionShouldFail() {
+    DoFn<Integer, Integer> brokenDoFn = new DoFn<Integer, Integer>() {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {}
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        throw new RuntimeException("oh noes!");
+      }
+    };
+
+    Pipeline p = getPipeline();
+    p
+        .apply(Create.of(1, 2, 3))
+        .apply(ParDo.of(brokenDoFn));
+
+    thrown.expectMessage(brokenDoFn.getClass().getName());
+    thrown.expectCause(ThrowableMessageMatcher.hasMessage(is("oh noes!")));
+    p.run();
+  }
+
+  @Test
+  public void pipelineOptionsDisplayDataExceptionShouldFail() {
+    Object brokenValueType = new Object() {
+      @JsonValue
+      public int getValue () {
+        return 42;
+      }
+
+      @Override
+      public String toString() {
+        throw new RuntimeException("oh noes!!");
+      }
+    };
+
+    Pipeline p = getPipeline();
+    p.getOptions().as(ObjectPipelineOptions.class).setValue(brokenValueType);
+
+    p.apply(Create.of(1, 2, 3));
+
+    thrown.expectMessage(PipelineOptions.class.getName());
+    thrown.expectCause(ThrowableMessageMatcher.hasMessage(is("oh noes!!")));
+    p.run();
+  }
+
+  /** {@link PipelineOptions} to inject bad object implementations. */
+  public interface ObjectPipelineOptions extends PipelineOptions {
+    Object getValue();
+    void setValue(Object value);
+  }
+
+
+  /**
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
+   * {@link DirectRunner}.
+   */
+  @Test
+  public void testMutatingOutputThenOutputDoFnError() throws Exception {
+    Pipeline pipeline = getPipeline();
+
+    pipeline
+        .apply(Create.of(42))
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @Override public void processElement(ProcessContext c) {
+            List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
+            c.output(outputList);
+            outputList.set(0, 37);
+            c.output(outputList);
+          }
+        }));
+
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage("output");
+    thrown.expectMessage("must not be mutated");
+    pipeline.run();
+  }
+
+  /**
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
+   * {@link DirectRunner}.
+   */
+  @Test
+  public void testMutatingOutputThenTerminateDoFnError() throws Exception {
+    Pipeline pipeline = getPipeline();
+
+    pipeline
+        .apply(Create.of(42))
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @Override public void processElement(ProcessContext c) {
+            List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
+            c.output(outputList);
+            outputList.set(0, 37);
+          }
+        }));
+
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage("output");
+    thrown.expectMessage("must not be mutated");
+    pipeline.run();
+  }
+
+  /**
+   * Tests that a {@link DoFn} that mutates an output with a bad equals() still fails
+   * in the {@link DirectRunner}.
+   */
+  @Test
+  public void testMutatingOutputCoderDoFnError() throws Exception {
+    Pipeline pipeline = getPipeline();
+
+    pipeline
+        .apply(Create.of(42))
+        .apply(ParDo.of(new DoFn<Integer, byte[]>() {
+          @Override public void processElement(ProcessContext c) {
+            byte[] outputArray = new byte[]{0x1, 0x2, 0x3};
+            c.output(outputArray);
+            outputArray[0] = 0xa;
+            c.output(outputArray);
+          }
+        }));
+
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage("output");
+    thrown.expectMessage("must not be mutated");
+    pipeline.run();
+  }
+
+  /**
+   * Tests that a {@link DoFn} that mutates its input with a good equals() fails in the
+   * {@link DirectRunner}.
+   */
+  @Test
+  public void testMutatingInputDoFnError() throws Exception {
+    Pipeline pipeline = getPipeline();
+
+    pipeline
+        .apply(Create.of(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))
+            .withCoder(ListCoder.of(VarIntCoder.of())))
+        .apply(ParDo.of(new DoFn<List<Integer>, Integer>() {
+          @Override public void processElement(ProcessContext c) {
+            List<Integer> inputList = c.element();
+            inputList.set(0, 37);
+            c.output(12);
+          }
+        }));
+
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage("Input");
+    thrown.expectMessage("must not be mutated");
+    pipeline.run();
+  }
+
+  /**
+   * Tests that a {@link DoFn} that mutates an input with a bad equals() still fails
+   * in the {@link DirectRunner}.
+   */
+  @Test
+  public void testMutatingInputCoderDoFnError() throws Exception {
+    Pipeline pipeline = getPipeline();
+
+    pipeline
+        .apply(Create.of(new byte[]{0x1, 0x2, 0x3}, new byte[]{0x4, 0x5, 0x6}))
+        .apply(ParDo.of(new DoFn<byte[], Integer>() {
+          @Override public void processElement(ProcessContext c) {
+            byte[] inputArray = c.element();
+            inputArray[0] = 0xa;
+            c.output(13);
+          }
+        }));
+
+    thrown.expect(IllegalMutationException.class);
+    thrown.expectMessage("Input");
+    thrown.expectMessage("must not be mutated");
+    pipeline.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
index 9a358dd..e129489 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -19,7 +19,7 @@ package org.apache.beam.runners.direct;
 
 import static org.hamcrest.Matchers.isA;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 66a5106..5efb090 100644
--- 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
@@ -23,8 +23,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index a4f900c..b589db0 100644
--- 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
@@ -22,8 +22,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index 20670ca..c4da86c 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -20,8 +20,8 @@ package org.apache.beam.runners.direct;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
index 6cef60d..ead9c9e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Count;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index af08d02..21c941a 100644
--- 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
@@ -29,8 +29,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index abe2a19..3a3ac8c 100644
--- 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
@@ -21,8 +21,8 @@ import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 18db400..b1cbeb1 100644
--- 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
@@ -28,9 +28,9 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -99,8 +99,8 @@ public class InProcessEvaluationContextTest {
 
   @Before
   public void setup() {
-    InProcessPipelineRunner runner =
-        InProcessPipelineRunner.fromOptions(PipelineOptionsFactory.create());
+    DirectRunner runner =
+        DirectRunner.fromOptions(PipelineOptionsFactory.create());
 
     p = TestPipeline.create();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
index 28a3cf6..e8d4711 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactoryTest.java
@@ -22,8 +22,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
deleted file mode 100644
index 54094c4..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/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.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/9400fc9a/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
deleted file mode 100644
index ab26c15..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessPipelineRunnerTest.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct;
-
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.runners.direct.InProcessPipelineRunner.InProcessPipelineResult;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
-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.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.IllegalMutationException;
-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.TypeDescriptor;
-
-import com.google.common.collect.ImmutableMap;
-
-import com.fasterxml.jackson.annotation.JsonValue;
-
-import org.hamcrest.Matchers;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests for basic {@link InProcessPipelineRunner} functionality.
- */
-@RunWith(JUnit4.class)
-public class InProcessPipelineRunnerTest implements Serializable {
-  @Rule public transient ExpectedException thrown = ExpectedException.none();
-
-  private Pipeline getPipeline() {
-    PipelineOptions opts = PipelineOptionsFactory.create();
-    opts.setRunner(InProcessPipelineRunner.class);
-
-    Pipeline p = Pipeline.create(opts);
-    return p;
-  }
-
-  @Test
-  public void defaultRunnerLoaded() {
-    assertThat(InProcessPipelineRunner.class,
-        Matchers.<Class<? extends PipelineRunner>>equalTo(PipelineOptionsFactory.create()
-            .getRunner()));
-  }
-
-  @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();
-  }
-
-  @Test(timeout = 5000L)
-  public void byteArrayCountShouldSucceed() {
-    Pipeline p = getPipeline();
-
-    SerializableFunction<Integer, byte[]> getBytes = new SerializableFunction<Integer, byte[]>() {
-      @Override
-      public byte[] apply(Integer input) {
-        try {
-          return CoderUtils.encodeToByteArray(VarIntCoder.of(), input);
-        } catch (CoderException e) {
-          fail("Unexpected Coder Exception " + e);
-          throw new AssertionError("Unreachable");
-        }
-      }
-    };
-    TypeDescriptor<byte[]> td = new TypeDescriptor<byte[]>() {
-    };
-    PCollection<byte[]> foos =
-        p.apply(Create.of(1, 1, 1, 2, 2, 3)).apply(MapElements.via(getBytes).withOutputType(td));
-    PCollection<byte[]> msync =
-        p.apply(Create.of(1, -2, -8, -16)).apply(MapElements.via(getBytes).withOutputType(td));
-    PCollection<byte[]> bytes =
-        PCollectionList.of(foos).and(msync).apply(Flatten.<byte[]>pCollections());
-    PCollection<KV<byte[], Long>> counts = bytes.apply(Count.<byte[]>perElement());
-    PCollection<KV<Integer, Long>> countsBackToString =
-        counts.apply(MapElements.via(new SimpleFunction<KV<byte[], Long>, KV<Integer, Long>>() {
-          @Override
-          public KV<Integer, Long> apply(KV<byte[], Long> input) {
-            try {
-              return KV.of(CoderUtils.decodeFromByteArray(VarIntCoder.of(), input.getKey()),
-                  input.getValue());
-            } catch (CoderException e) {
-              fail("Unexpected Coder Exception " + e);
-              throw new AssertionError("Unreachable");
-        }
-      }
-    }));
-
-    Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder().put(1, 4L)
-        .put(2, 2L)
-        .put(3, 1L)
-        .put(-2, 1L)
-        .put(-8, 1L)
-        .put(-16, 1L)
-        .build();
-    PAssert.thatMap(countsBackToString).isEqualTo(expected);
-  }
-
-  @Test
-  public void transformDisplayDataExceptionShouldFail() {
-    DoFn<Integer, Integer> brokenDoFn = new DoFn<Integer, Integer>() {
-      @Override
-      public void processElement(ProcessContext c) throws Exception {}
-
-      @Override
-      public void populateDisplayData(DisplayData.Builder builder) {
-        throw new RuntimeException("oh noes!");
-      }
-    };
-
-    Pipeline p = getPipeline();
-    p
-        .apply(Create.of(1, 2, 3))
-        .apply(ParDo.of(brokenDoFn));
-
-    thrown.expectMessage(brokenDoFn.getClass().getName());
-    thrown.expectCause(ThrowableMessageMatcher.hasMessage(is("oh noes!")));
-    p.run();
-  }
-
-  @Test
-  public void pipelineOptionsDisplayDataExceptionShouldFail() {
-    Object brokenValueType = new Object() {
-      @JsonValue
-      public int getValue () {
-        return 42;
-      }
-
-      @Override
-      public String toString() {
-        throw new RuntimeException("oh noes!!");
-      }
-    };
-
-    Pipeline p = getPipeline();
-    p.getOptions().as(ObjectPipelineOptions.class).setValue(brokenValueType);
-
-    p.apply(Create.of(1, 2, 3));
-
-    thrown.expectMessage(PipelineOptions.class.getName());
-    thrown.expectCause(ThrowableMessageMatcher.hasMessage(is("oh noes!!")));
-    p.run();
-  }
-
-  /** {@link PipelineOptions} to inject bad object implementations. */
-  public interface ObjectPipelineOptions extends PipelineOptions {
-    Object getValue();
-    void setValue(Object value);
-  }
-
-
-  /**
-   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
-   * {@link InProcessPipelineRunner}.
-   */
-  @Test
-  public void testMutatingOutputThenOutputDoFnError() throws Exception {
-    Pipeline pipeline = getPipeline();
-
-    pipeline
-        .apply(Create.of(42))
-        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
-            List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
-            c.output(outputList);
-            outputList.set(0, 37);
-            c.output(outputList);
-          }
-        }));
-
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage("output");
-    thrown.expectMessage("must not be mutated");
-    pipeline.run();
-  }
-
-  /**
-   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
-   * {@link InProcessPipelineRunner}.
-   */
-  @Test
-  public void testMutatingOutputThenTerminateDoFnError() throws Exception {
-    Pipeline pipeline = getPipeline();
-
-    pipeline
-        .apply(Create.of(42))
-        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
-            List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
-            c.output(outputList);
-            outputList.set(0, 37);
-          }
-        }));
-
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage("output");
-    thrown.expectMessage("must not be mutated");
-    pipeline.run();
-  }
-
-  /**
-   * Tests that a {@link DoFn} that mutates an output with a bad equals() still fails
-   * in the {@link InProcessPipelineRunner}.
-   */
-  @Test
-  public void testMutatingOutputCoderDoFnError() throws Exception {
-    Pipeline pipeline = getPipeline();
-
-    pipeline
-        .apply(Create.of(42))
-        .apply(ParDo.of(new DoFn<Integer, byte[]>() {
-          @Override public void processElement(ProcessContext c) {
-            byte[] outputArray = new byte[]{0x1, 0x2, 0x3};
-            c.output(outputArray);
-            outputArray[0] = 0xa;
-            c.output(outputArray);
-          }
-        }));
-
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage("output");
-    thrown.expectMessage("must not be mutated");
-    pipeline.run();
-  }
-
-  /**
-   * Tests that a {@link DoFn} that mutates its input with a good equals() fails in the
-   * {@link InProcessPipelineRunner}.
-   */
-  @Test
-  public void testMutatingInputDoFnError() throws Exception {
-    Pipeline pipeline = getPipeline();
-
-    pipeline
-        .apply(Create.of(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))
-            .withCoder(ListCoder.of(VarIntCoder.of())))
-        .apply(ParDo.of(new DoFn<List<Integer>, Integer>() {
-          @Override public void processElement(ProcessContext c) {
-            List<Integer> inputList = c.element();
-            inputList.set(0, 37);
-            c.output(12);
-          }
-        }));
-
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage("Input");
-    thrown.expectMessage("must not be mutated");
-    pipeline.run();
-  }
-
-  /**
-   * Tests that a {@link DoFn} that mutates an input with a bad equals() still fails
-   * in the {@link InProcessPipelineRunner}.
-   */
-  @Test
-  public void testMutatingInputCoderDoFnError() throws Exception {
-    Pipeline pipeline = getPipeline();
-
-    pipeline
-        .apply(Create.of(new byte[]{0x1, 0x2, 0x3}, new byte[]{0x4, 0x5, 0x6}))
-        .apply(ParDo.of(new DoFn<byte[], Integer>() {
-          @Override public void processElement(ProcessContext c) {
-            byte[] inputArray = c.element();
-            inputArray[0] = 0xa;
-            c.output(13);
-          }
-        }));
-
-    thrown.expect(IllegalMutationException.class);
-    thrown.expectMessage("Input");
-    thrown.expectMessage("must not be mutated");
-    pipeline.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
index 0f7afa1..b78eb40 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoInProcessEvaluatorTest.java
@@ -25,8 +25,8 @@ import static org.mockito.Mockito.when;
 
 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.UncommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
@@ -139,7 +139,7 @@ public class ParDoInProcessEvaluatorTest {
   private ParDoInProcessEvaluator<Integer> createEvaluator(
       PCollectionView<Integer> singletonView,
       RecorderFn fn,
-      InProcessPipelineRunner.CommittedBundle<Integer> inputBundle,
+      DirectRunner.CommittedBundle<Integer> inputBundle,
       PCollection<Integer> output) {
     when(
             evaluationContext.createSideInputReader(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index a6f31c0..e61881e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -26,8 +26,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index a1480e5..8b8d44f 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -26,8 +26,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index fe9866c..5ede931 100644
--- 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
@@ -114,7 +114,7 @@ public class TextIOShardedWriteFactoryTest {
 
   private Pipeline getPipeline() {
     PipelineOptions options = TestPipeline.testingPipelineOptions();
-    options.setRunner(InProcessPipelineRunner.class);
+    options.setRunner(DirectRunner.class);
     return TestPipeline.fromOptions(options);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 0345662..a5e6cee 100644
--- 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
@@ -27,7 +27,7 @@ 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.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 05656eb..be5c489 100644
--- 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
@@ -25,8 +25,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.BigEndianLongCoder;
 import org.apache.beam.sdk.coders.Coder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 859418b..714e9c9 100644
--- 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
@@ -23,8 +23,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index a2f971a..8a3591b 100644
--- 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
@@ -21,8 +21,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
index 01f3070..5fdfb49 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java
@@ -21,7 +21,7 @@ package org.apache.beam.runners.spark.translation;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner;
+import org.apache.beam.runners.direct.DirectRunner;
 import org.apache.beam.runners.spark.SparkPipelineRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
@@ -58,7 +58,7 @@ public class TransformTranslatorTest {
    */
   @Test
   public void testTextIOReadAndWriteTransforms() throws IOException {
-    String directOut = runPipeline(InProcessPipelineRunner.class);
+    String directOut = runPipeline(DirectRunner.class);
     String sparkOut = runPipeline(SparkPipelineRunner.class);
 
     List<String> directOutput =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index b1b5280..456b6ae 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -277,7 +277,7 @@ public interface PipelineOptions extends HasDisplayData {
       try {
         @SuppressWarnings({"unchecked", "rawtypes"})
         Class<? extends PipelineRunner> direct = (Class<? extends PipelineRunner>) Class.forName(
-            "org.apache.beam.runners.direct.InProcessPipelineRunner");
+            "org.apache.beam.runners.direct.DirectRunner");
         return direct;
       } catch (ClassNotFoundException e) {
         throw new IllegalArgumentException(String.format(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/testing/travis/test_wordcount.sh
----------------------------------------------------------------------
diff --git a/testing/travis/test_wordcount.sh b/testing/travis/test_wordcount.sh
index b00b0d6..e059a35 100755
--- a/testing/travis/test_wordcount.sh
+++ b/testing/travis/test_wordcount.sh
@@ -70,7 +70,7 @@ function run_via_mvn {
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='mvn exec:java -f pom.xml -pl examples/java \
     -Dexec.mainClass=org.apache.beam.examples.WordCount \
-    -Dexec.args="--runner=InProcessPipelineRunner --inputFile='"$input"' --output='"$outfile_prefix"'"'
+    -Dexec.args="--runner=DirectRunner --inputFile='"$input"' --output='"$outfile_prefix"'"'
   echo "$name: Running $cmd" >&2
   sh -c "$cmd"
   check_result_hash "$name" "$outfile_prefix" "$expected_hash"
@@ -84,7 +84,7 @@ function run_bundled {
   local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2
   local cmd='java -cp '"$JAR_FILE"' \
     org.apache.beam.examples.WordCount \
-    --runner=InProcessPipelineRunner \
+    --runner=DirectRunner \
     --inputFile='"'$input'"' \
     --output='"$outfile_prefix"
   echo "$name: Running $cmd" >&2


[18/50] [abbrv] incubator-beam git commit: Remove InProcess Prefixes

Posted by da...@apache.org.
Remove InProcess Prefixes

These prefixes are out of date with the rename of the runner. Most of
the prefixes are be droped in their entirety, as the classes are scoped
to the direct runner 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/babddbbc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/babddbbc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/babddbbc

Branch: refs/heads/python-sdk
Commit: babddbbc8247bc7322c3fd519a5bf0fa23c57064
Parents: 9400fc9
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 15 11:21:41 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../direct/AbstractModelEnforcement.java        |    2 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   12 +-
 .../beam/runners/direct/BundleFactory.java      |    4 +-
 .../beam/runners/direct/CommittedResult.java    |    4 +-
 .../beam/runners/direct/CompletionCallback.java |    2 +-
 .../runners/direct/DirectExecutionContext.java  |  106 ++
 .../beam/runners/direct/DirectGroupByKey.java   |  132 ++
 .../direct/DirectGroupByKeyOverrideFactory.java |   41 +
 .../beam/runners/direct/DirectRegistrar.java    |   16 +-
 .../beam/runners/direct/DirectRunner.java       |   29 +-
 .../runners/direct/DirectTimerInternals.java    |   84 ++
 .../runners/direct/EmptyTransformEvaluator.java |    4 +-
 .../beam/runners/direct/EvaluationContext.java  |  429 ++++++
 .../beam/runners/direct/EvaluatorKey.java       |    4 +-
 .../direct/ExecutorServiceParallelExecutor.java |   28 +-
 .../runners/direct/FlattenEvaluatorFactory.java |   12 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  127 ++
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |  186 +++
 .../direct/ImmutabilityEnforcementFactory.java  |    2 +-
 .../direct/ImmutableListBundleFactory.java      |  163 ++
 .../direct/InMemoryWatermarkManager.java        | 1420 -----------------
 .../runners/direct/InProcessBundleFactory.java  |  161 --
 .../direct/InProcessBundleOutputManager.java    |   51 -
 .../direct/InProcessEvaluationContext.java      |  429 ------
 .../direct/InProcessExecutionContext.java       |  105 --
 .../beam/runners/direct/InProcessExecutor.java  |   48 -
 ...rocessGroupAlsoByWindowEvaluatorFactory.java |  127 --
 .../runners/direct/InProcessGroupByKey.java     |  132 --
 ...InProcessGroupByKeyOnlyEvaluatorFactory.java |  185 ---
 .../InProcessGroupByKeyOverrideFactory.java     |   41 -
 .../direct/InProcessSideInputContainer.java     |  277 ----
 .../runners/direct/InProcessTimerInternals.java |   84 --
 .../direct/InProcessTransformResult.java        |   84 --
 .../beam/runners/direct/ModelEnforcement.java   |    6 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  186 +++
 .../runners/direct/ParDoInProcessEvaluator.java |  186 ---
 .../direct/ParDoMultiEvaluatorFactory.java      |    6 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |    6 +-
 .../direct/PassthroughTransformEvaluator.java   |    2 +-
 .../beam/runners/direct/PipelineExecutor.java   |   48 +
 .../beam/runners/direct/SideInputContainer.java |  277 ++++
 .../runners/direct/StepTransformResult.java     |    6 +-
 ...readLocalInvalidatingTransformEvaluator.java |    2 +-
 .../beam/runners/direct/TransformEvaluator.java |    4 +-
 .../direct/TransformEvaluatorFactory.java       |    2 +-
 .../direct/TransformEvaluatorRegistry.java      |   10 +-
 .../beam/runners/direct/TransformExecutor.java  |   14 +-
 .../beam/runners/direct/TransformResult.java    |   84 ++
 .../direct/UnboundedReadEvaluatorFactory.java   |   12 +-
 .../direct/UncommittedBundleOutputManager.java  |   51 +
 .../runners/direct/ViewEvaluatorFactory.java    |   16 +-
 .../beam/runners/direct/WatermarkManager.java   | 1420 +++++++++++++++++
 .../runners/direct/WindowEvaluatorFactory.java  |   12 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |   10 +-
 .../runners/direct/CommittedResultTest.java     |    2 +-
 .../runners/direct/DirectRegistrarTest.java     |   18 +-
 .../direct/DirectTimerInternalsTest.java        |  134 ++
 .../EncodabilityEnforcementFactoryTest.java     |    2 +-
 .../runners/direct/EvaluationContextTest.java   |  545 +++++++
 .../direct/FlattenEvaluatorFactoryTest.java     |   12 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |    9 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  197 +++
 .../ImmutabilityCheckingBundleFactoryTest.java  |    2 +-
 .../ImmutabilityEnforcementFactoryTest.java     |    2 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  231 +++
 .../direct/InMemoryWatermarkManagerTest.java    | 1428 ------------------
 .../direct/InProcessBundleFactoryTest.java      |  231 ---
 .../direct/InProcessEvaluationContextTest.java  |  545 -------
 ...ocessGroupByKeyOnlyEvaluatorFactoryTest.java |  196 ---
 .../direct/InProcessSideInputContainerTest.java |  520 -------
 .../direct/InProcessTimerInternalsTest.java     |  134 --
 .../beam/runners/direct/ParDoEvaluatorTest.java |  214 +++
 .../direct/ParDoInProcessEvaluatorTest.java     |  214 ---
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |   32 +-
 .../direct/ParDoSingleEvaluatorFactoryTest.java |   32 +-
 .../runners/direct/SideInputContainerTest.java  |  520 +++++++
 ...LocalInvalidatingTransformEvaluatorTest.java |    4 +-
 .../runners/direct/TransformExecutorTest.java   |   42 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   14 +-
 .../direct/ViewEvaluatorFactoryTest.java        |    4 +-
 .../runners/direct/WatermarkManagerTest.java    | 1428 ++++++++++++++++++
 .../direct/WindowEvaluatorFactoryTest.java      |   16 +-
 82 files changed, 6814 insertions(+), 6805 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 2ae0275..81f0f5f 100644
--- 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
@@ -33,6 +33,6 @@ abstract class AbstractModelEnforcement<T> implements ModelEnforcement<T> {
   @Override
   public void afterFinish(
       CommittedBundle<T> input,
-      InProcessTransformResult result,
+      TransformResult result,
       Iterable<? extends CommittedBundle<?>> outputs) {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 63d248a..e550f54 100644
--- 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
@@ -55,14 +55,14 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   public <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application,
       @Nullable CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext)
+      EvaluationContext evaluationContext)
       throws IOException {
     return getTransformEvaluator((AppliedPTransform) application, evaluationContext);
   }
 
   private <OutputT> TransformEvaluator<?> getTransformEvaluator(
       final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
+      final EvaluationContext evaluationContext) {
     return getTransformEvaluatorQueue(transform, evaluationContext).poll();
   }
 
@@ -76,7 +76,7 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   @SuppressWarnings("unchecked")
   private <OutputT> Queue<BoundedReadEvaluator<OutputT>> getTransformEvaluatorQueue(
       final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-      final InProcessEvaluationContext evaluationContext) {
+      final EvaluationContext evaluationContext) {
     // Key by the application and the context the evaluation is occurring in (which call to
     // Pipeline#run).
     EvaluatorKey key = new EvaluatorKey(transform, evaluationContext);
@@ -110,7 +110,7 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
    */
   private static class BoundedReadEvaluator<OutputT> implements TransformEvaluator<Object> {
     private final AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform;
-    private final InProcessEvaluationContext evaluationContext;
+    private final EvaluationContext 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.
@@ -119,7 +119,7 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
 
     public BoundedReadEvaluator(
         AppliedPTransform<?, PCollection<OutputT>, Bounded<OutputT>> transform,
-        InProcessEvaluationContext evaluationContext,
+        EvaluationContext evaluationContext,
         BoundedSource<OutputT> source) {
       this.transform = transform;
       this.evaluationContext = evaluationContext;
@@ -130,7 +130,7 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
     public void processElement(WindowedValue<Object> element) {}
 
     @Override
-    public InProcessTransformResult finishBundle() throws IOException {
+    public TransformResult finishBundle() throws IOException {
       try (final BoundedReader<OutputT> reader =
               source.createReader(evaluationContext.getPipelineOptions());) {
         boolean contentsRemaining = reader.start();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index a546cfb..0241d87 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -41,7 +41,7 @@ public interface BundleFactory {
 
   /**
    * 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
+   * {@link DirectGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
    * belong to the {@code output} {@link PCollection}.
    */
   public <K, T> UncommittedBundle<T> createKeyedBundle(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index b241493..e86f07d 100644
--- 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
@@ -26,7 +26,7 @@ import com.google.auto.value.AutoValue;
 import javax.annotation.Nullable;
 
 /**
- * A {@link InProcessTransformResult} that has been committed.
+ * A {@link TransformResult} that has been committed.
  */
 @AutoValue
 abstract class CommittedResult {
@@ -50,7 +50,7 @@ abstract class CommittedResult {
   public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
 
   public static CommittedResult create(
-      InProcessTransformResult original,
+      TransformResult original,
       CommittedBundle<?> unprocessedElements,
       Iterable<? extends CommittedBundle<?>> outputs) {
     return new AutoValue_CommittedResult(original.getTransform(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 8ee4b44..0c5fe24 100644
--- 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
@@ -27,7 +27,7 @@ interface CompletionCallback {
    * Handle a successful result, returning the committed outputs of the result.
    */
   CommittedResult handleResult(
-      CommittedBundle<?> inputBundle, InProcessTransformResult result);
+      CommittedBundle<?> inputBundle, TransformResult result);
 
   /**
    * Handle a result that terminated abnormally due to the provided {@link Throwable}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
new file mode 100644
index 0000000..2d2b87d
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.WatermarkManager.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 DirectRunner}.
+ *
+ * This implementation is not thread safe. A new {@link DirectExecutionContext} must be created
+ * for each thread that requires it.
+ */
+class DirectExecutionContext
+    extends BaseExecutionContext<DirectStepContext> {
+  private final Clock clock;
+  private final StructuralKey<?> key;
+  private final CopyOnAccessInMemoryStateInternals<Object> existingState;
+  private final TransformWatermarks watermarks;
+
+  public DirectExecutionContext(Clock clock, StructuralKey<?> key,
+      CopyOnAccessInMemoryStateInternals<Object> existingState, TransformWatermarks watermarks) {
+    this.clock = clock;
+    this.key = key;
+    this.existingState = existingState;
+    this.watermarks = watermarks;
+  }
+
+  @Override
+  protected DirectStepContext createStepContext(String stepName, String transformName) {
+    return new DirectStepContext(this, stepName, transformName);
+  }
+
+  /**
+   * Step Context for the {@link DirectRunner}.
+   */
+  public class DirectStepContext
+      extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
+    private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
+    private DirectTimerInternals timerInternals;
+
+    public DirectStepContext(
+        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 DirectTimerInternals timerInternals() {
+      if (timerInternals == null) {
+        timerInternals =
+            DirectTimerInternals.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 DirectStepContext},
+     * 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/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
new file mode 100644
index 0000000..0200676
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+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.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+class DirectGroupByKey<K, V>
+    extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+  private final GroupByKey<K, V> original;
+
+  DirectGroupByKey(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) {
+    @SuppressWarnings("unchecked")
+    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();
+
+    // 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 DirectGroupByKeyOnly<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", new DirectGroupAlsoByWindow<K, V>(windowingStrategy))
+
+        // And update the windowing strategy as appropriate.
+        .setWindowingStrategyInternal(original.updateWindowingStrategy(windowingStrategy))
+        .setCoder(
+            KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())));
+  }
+
+  static final class DirectGroupByKeyOnly<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());
+    }
+
+    DirectGroupByKeyOnly() {}
+  }
+
+  static final class DirectGroupAlsoByWindow<K, V>
+      extends PTransform<PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+
+    private final WindowingStrategy<?, ?> windowingStrategy;
+
+    public DirectGroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
+      this.windowingStrategy = windowingStrategy;
+    }
+
+    public WindowingStrategy<?, ?> getWindowingStrategy() {
+      return windowingStrategy;
+    }
+
+    private KeyedWorkItemCoder<K, V> getKeyedWorkItemCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
+      // Coder<KV<...>> --> KvCoder<...>
+      checkArgument(
+          inputCoder instanceof KeyedWorkItemCoder,
+          "%s requires a %s<...> but got %s",
+          getClass().getSimpleName(),
+          KvCoder.class.getSimpleName(),
+          inputCoder);
+      @SuppressWarnings("unchecked")
+      KeyedWorkItemCoder<K, V> kvCoder = (KeyedWorkItemCoder<K, V>) inputCoder;
+      return kvCoder;
+    }
+
+    public Coder<K> getKeyCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
+      return getKeyedWorkItemCoder(inputCoder).getKeyCoder();
+    }
+
+    public Coder<V> getValueCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
+      return getKeyedWorkItemCoder(inputCoder).getElementCoder();
+    }
+
+    @Override
+    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KeyedWorkItem<K, V>> input) {
+      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
+          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java
new file mode 100644
index 0000000..c64f3f0
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+/**
+ * A {@link PTransformOverrideFactory} for {@link GroupByKey} PTransforms.
+ */
+final class DirectGroupByKeyOverrideFactory
+    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 =
+          (PTransform) new DirectGroupByKey((GroupByKey) transform);
+      return override;
+    }
+    return transform;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
index eb027fa..7c094ae 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
@@ -27,29 +27,31 @@ import com.google.common.collect.ImmutableList;
 
 /**
  * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
- * {@link DirectRunner}.
+ * {@link org.apache.beam.runners.direct.DirectRunner}.
  */
 public class DirectRegistrar {
   private DirectRegistrar() {}
   /**
-   * Registers the {@link DirectRunner}.
+   * Registers the {@link org.apache.beam.runners.direct.DirectRunner}.
    */
   @AutoService(PipelineRunnerRegistrar.class)
-  public static class InProcessRunner implements PipelineRunnerRegistrar {
+  public static class DirectRunner implements PipelineRunnerRegistrar {
     @Override
     public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(DirectRunner.class);
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
+          org.apache.beam.runners.direct.DirectRunner.class);
     }
   }
 
   /**
-   * Registers the {@link DirectOptions}.
+   * Registers the {@link org.apache.beam.runners.direct.DirectOptions}.
    */
   @AutoService(PipelineOptionsRegistrar.class)
-  public static class InProcessOptions implements PipelineOptionsRegistrar {
+  public static class DirectOptions implements PipelineOptionsRegistrar {
     @Override
     public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
-      return ImmutableList.<Class<? extends PipelineOptions>>of(DirectOptions.class);
+      return ImmutableList.<Class<? extends PipelineOptions>>of(
+          org.apache.beam.runners.direct.DirectOptions.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 2f5a0bc..2584739 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.runners.direct;
 
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.ViewEvaluatorFactory.InProcessViewOverrideFactory;
+import org.apache.beam.runners.direct.ViewEvaluatorFactory.ViewOverrideFactory;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.PipelineResult;
@@ -78,8 +78,8 @@ public class DirectRunner
   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(GroupByKey.class, new DirectGroupByKeyOverrideFactory())
+              .put(CreatePCollectionView.class, new ViewOverrideFactory())
               .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
               .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
               .build();
@@ -97,6 +97,7 @@ public class DirectRunner
      */
     PCollection<T> getPCollection();
 
+
     /**
      * Outputs an element to this bundle.
      *
@@ -217,13 +218,13 @@ public class DirectRunner
     KeyedPValueTrackingVisitor keyedPValueVisitor =
         KeyedPValueTrackingVisitor.create(
             ImmutableSet.<Class<? extends PTransform>>of(
-                GroupByKey.class, InProcessGroupByKeyOnly.class));
+                GroupByKey.class, DirectGroupByKeyOnly.class));
     pipeline.traverseTopologically(keyedPValueVisitor);
 
     DisplayDataValidator.validatePipeline(pipeline);
 
-    InProcessEvaluationContext context =
-        InProcessEvaluationContext.create(
+    EvaluationContext context =
+        EvaluationContext.create(
             getPipelineOptions(),
             createBundleFactory(getPipelineOptions()),
             consumerTrackingVisitor.getRootTransforms(),
@@ -234,7 +235,7 @@ public class DirectRunner
     // independent executor service for each run
     ExecutorService executorService =
         context.getPipelineOptions().getExecutorServiceFactory().create();
-    InProcessExecutor executor =
+    PipelineExecutor executor =
         ExecutorServiceParallelExecutor.create(
             executorService,
             consumerTrackingVisitor.getValueToConsumers(),
@@ -283,7 +284,7 @@ public class DirectRunner
   }
 
   private BundleFactory createBundleFactory(DirectOptions pipelineOptions) {
-    BundleFactory bundleFactory = InProcessBundleFactory.create();
+    BundleFactory bundleFactory = ImmutableListBundleFactory.create();
     if (pipelineOptions.isTestImmutability()) {
       bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
     }
@@ -296,14 +297,14 @@ public class DirectRunner
    * Throws {@link UnsupportedOperationException} for all methods.
    */
   public static class DirectPipelineResult implements PipelineResult {
-    private final InProcessExecutor executor;
-    private final InProcessEvaluationContext evaluationContext;
+    private final PipelineExecutor executor;
+    private final EvaluationContext evaluationContext;
     private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
     private State state;
 
     private DirectPipelineResult(
-        InProcessExecutor executor,
-        InProcessEvaluationContext evaluationContext,
+        PipelineExecutor executor,
+        EvaluationContext evaluationContext,
         Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps) {
       this.executor = executor;
       this.evaluationContext = evaluationContext;
@@ -350,7 +351,7 @@ public class DirectRunner
      * {@link DirectOptions#isShutdownUnboundedProducersWithMaxWatermark()} set to false,
      * this method will never return.
      *
-     * See also {@link InProcessExecutor#awaitCompletion()}.
+     * See also {@link PipelineExecutor#awaitCompletion()}.
      */
     public State awaitCompletion() throws Throwable {
       if (!state.isTerminal()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
new file mode 100644
index 0000000..a4705dd
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.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.WatermarkManager.TimerUpdate;
+import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
+import org.apache.beam.runners.direct.WatermarkManager.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 DirectTimerInternals implements TimerInternals {
+  private final Clock processingTimeClock;
+  private final TransformWatermarks watermarks;
+  private final TimerUpdateBuilder timerUpdateBuilder;
+
+  public static DirectTimerInternals create(
+      Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
+    return new DirectTimerInternals(clock, watermarks, timerUpdateBuilder);
+  }
+
+  private DirectTimerInternals(
+      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/babddbbc/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
index 5379038..778c5aa 100644
--- 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
@@ -24,7 +24,7 @@ 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 TransformResult} 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.
  */
@@ -43,7 +43,7 @@ final class EmptyTransformEvaluator<T> implements TransformEvaluator<T> {
   public void processElement(WindowedValue<T> element) throws Exception {}
 
   @Override
-  public InProcessTransformResult finishBundle() throws Exception {
+  public TransformResult finishBundle() throws Exception {
     return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE)
         .build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
new file mode 100644
index 0000000..bee878b
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectGroupByKey.DirectGroupByKeyOnly;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
+import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
+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.ReadyCheckingSideInputReader;
+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.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.MoreExecutors;
+
+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 DirectRunner}. Contains state shared within the execution across all
+ * transforms.
+ *
+ * <p>{@link EvaluationContext} contains shared state for an execution of the
+ * {@link DirectRunner} 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 EvaluationContext} 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 EvaluationContext {
+  /** 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 DirectOptions options;
+
+  private final BundleFactory bundleFactory;
+  /** The current processing time and event time watermarks and timers. */
+  private final WatermarkManager 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 SideInputContainer sideInputContainer;
+
+  private final CounterSet mergedCounters;
+
+  public static EvaluationContext create(
+      DirectOptions options,
+      BundleFactory bundleFactory,
+      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
+      Collection<PCollectionView<?>> views) {
+    return new EvaluationContext(
+        options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
+  }
+
+  private EvaluationContext(
+      DirectOptions 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 =
+        WatermarkManager.create(
+            NanosOffsetClock.create(), rootTransforms, valueToConsumers);
+    this.sideInputContainer = SideInputContainer.create(this, views);
+
+    this.applicationStateInternals = new ConcurrentHashMap<>();
+    this.mergedCounters = new CounterSet();
+
+    this.callbackExecutor =
+        WatermarkCallbackExecutor.create(MoreExecutors.directExecutor());
+  }
+
+  /**
+   * Handle the provided {@link TransformResult}, 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 TransformResult} 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 CommittedResult handleResult(
+      @Nullable CommittedBundle<?> completedBundle,
+      Iterable<TimerData> completedTimers,
+      TransformResult result) {
+    Iterable<? extends CommittedBundle<?>> committedBundles =
+        commitBundles(result.getOutputBundles());
+    // Update watermarks and timers
+    CommittedResult committedResult = CommittedResult.create(result,
+        completedBundle == null
+            ? null
+            : completedBundle.withElements((Iterable) result.getUnprocessedElements()),
+        committedBundles);
+    watermarkManager.updateWatermarks(
+        completedBundle,
+        result.getTimerUpdate().withCompletedTimers(completedTimers),
+        committedResult,
+        result.getWatermarkHold());
+    // 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;
+  }
+
+  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 DirectGroupByKeyOnly} {@link PTransform PTransforms}.
+   */
+  public <K, T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, StructuralKey<K> 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 DirectOptions getPipelineOptions() {
+    return options;
+  }
+
+  /**
+   * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key.
+   */
+  public DirectExecutionContext getExecutionContext(
+      AppliedPTransform<?, ?, ?> application, StructuralKey<?> key) {
+    StepAndKey stepAndKey = StepAndKey.of(application, key);
+    return new DirectExecutionContext(
+        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);
+  }
+
+
+  /**
+   * 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 EvaluationContext} 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;
+  }
+
+  @VisibleForTesting
+  void forceRefresh() {
+    watermarkManager.refreshAll();
+    fireAllAvailableCallbacks();
+  }
+
+  /**
+   * 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<StructuralKey<?>, FiredTimers>> extractFiredTimers() {
+    forceRefresh();
+    Map<AppliedPTransform<?, ?, ?>, Map<StructuralKey<?>, 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 DirectOptions#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/babddbbc/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
index 1c36751..164e05a 100644
--- 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
@@ -31,9 +31,9 @@ import java.util.Objects;
  */
 final class EvaluatorKey {
   private final AppliedPTransform<?, ?, ?> transform;
-  private final InProcessEvaluationContext context;
+  private final EvaluationContext context;
 
-  public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, InProcessEvaluationContext context) {
+  public EvaluatorKey(AppliedPTransform<?, ?, ?> transform, EvaluationContext context) {
     this.transform = transform;
     this.context = context;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index 14570a5..4bb5021 100644
--- 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
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -57,10 +57,10 @@ import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 
 /**
- * An {@link InProcessExecutor} that uses an underlying {@link ExecutorService} and
- * {@link InProcessEvaluationContext} to execute a {@link Pipeline}.
+ * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and
+ * {@link EvaluationContext} to execute a {@link Pipeline}.
  */
-final class ExecutorServiceParallelExecutor implements InProcessExecutor {
+final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
 
   private final ExecutorService executorService;
@@ -72,7 +72,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
   private final Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
       transformEnforcements;
 
-  private final InProcessEvaluationContext evaluationContext;
+  private final EvaluationContext evaluationContext;
 
   private final LoadingCache<StepAndKey, TransformExecutorService> executorServices;
 
@@ -91,7 +91,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
       TransformEvaluatorRegistry registry,
       @SuppressWarnings("rawtypes")
       Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
+      EvaluationContext context) {
     return new ExecutorServiceParallelExecutor(
         executorService, valueToConsumers, keyedPValues, registry, transformEnforcements, context);
   }
@@ -103,7 +103,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
       TransformEvaluatorRegistry registry,
       @SuppressWarnings("rawtypes")
       Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>> transformEnforcements,
-      InProcessEvaluationContext context) {
+      EvaluationContext context) {
     this.executorService = executorService;
     this.valueToConsumers = valueToConsumers;
     this.keyedPValues = keyedPValues;
@@ -215,18 +215,18 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
 
   /**
    * The base implementation of {@link CompletionCallback} that provides implementations for
-   * {@link #handleResult(CommittedBundle, InProcessTransformResult)} and
+   * {@link #handleResult(CommittedBundle, TransformResult)} and
    * {@link #handleThrowable(CommittedBundle, Throwable)}, given an implementation of
-   * {@link #getCommittedResult(CommittedBundle, InProcessTransformResult)}.
+   * {@link #getCommittedResult(CommittedBundle, TransformResult)}.
    */
   private abstract class CompletionCallbackBase implements CompletionCallback {
     protected abstract CommittedResult getCommittedResult(
         CommittedBundle<?> inputBundle,
-        InProcessTransformResult result);
+        TransformResult result);
 
     @Override
     public final CommittedResult handleResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+        CommittedBundle<?> inputBundle, TransformResult result) {
       CommittedResult committedResult = getCommittedResult(inputBundle, result);
       for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
         allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle,
@@ -254,7 +254,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
   private class DefaultCompletionCallback extends CompletionCallbackBase {
     @Override
     public CommittedResult getCommittedResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+        CommittedBundle<?> inputBundle, TransformResult result) {
       return evaluationContext.handleResult(inputBundle,
           Collections.<TimerData>emptyList(),
           result);
@@ -264,7 +264,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
   /**
    * 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}
+   * timers used to create the input to the {@link EvaluationContext evaluation context}
    * as part of the result.
    */
   private class TimerCompletionCallback extends CompletionCallbackBase {
@@ -276,7 +276,7 @@ final class ExecutorServiceParallelExecutor implements InProcessExecutor {
 
     @Override
     public CommittedResult getCommittedResult(
-        CommittedBundle<?> inputBundle, InProcessTransformResult result) {
+        CommittedBundle<?> inputBundle, TransformResult result) {
           return evaluationContext.handleResult(inputBundle, timers, result);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index bbe8787..c84f620 100644
--- 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
@@ -36,7 +36,7 @@ class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
   public <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application,
       CommittedBundle<?> inputBundle,
-      InProcessEvaluationContext evaluationContext) {
+      EvaluationContext evaluationContext) {
     @SuppressWarnings({"cast", "unchecked", "rawtypes"})
     TransformEvaluator<InputT> evaluator = (TransformEvaluator<InputT>) createInMemoryEvaluator(
             (AppliedPTransform) application, inputBundle, evaluationContext);
@@ -48,7 +48,7 @@ class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
               PCollectionList<InputT>, PCollection<InputT>, FlattenPCollectionList<InputT>>
           application,
       final CommittedBundle<InputT> inputBundle,
-      final InProcessEvaluationContext evaluationContext) {
+      final EvaluationContext 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())
@@ -57,17 +57,17 @@ class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
     }
     final UncommittedBundle<InputT> outputBundle =
         evaluationContext.createBundle(inputBundle, application.getOutput());
-    final InProcessTransformResult result =
+    final TransformResult 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;
+    private final TransformResult result;
 
     public FlattenEvaluator(
-        UncommittedBundle<InputT> outputBundle, InProcessTransformResult result) {
+        UncommittedBundle<InputT> outputBundle, TransformResult result) {
       this.outputBundle = outputBundle;
       this.result = result;
     }
@@ -78,7 +78,7 @@ class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
     }
 
     @Override
-    public InProcessTransformResult finishBundle() {
+    public TransformResult finishBundle() {
       return result;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
new file mode 100644
index 0000000..9782ab1
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+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.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+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.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Collections;
+
+/**
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
+ * {@link GroupByKeyOnly} {@link PTransform}.
+ */
+class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      EvaluationContext evaluationContext) {
+    @SuppressWarnings({"cast", "unchecked", "rawtypes"})
+    TransformEvaluator<InputT> evaluator =
+        createEvaluator(
+            (AppliedPTransform) application, (CommittedBundle) inputBundle, evaluationContext);
+    return evaluator;
+  }
+
+  private <K, V> TransformEvaluator<KeyedWorkItem<K, V>> createEvaluator(
+      AppliedPTransform<
+              PCollection<KeyedWorkItem<K, V>>,
+              PCollection<KV<K, Iterable<V>>>,
+              DirectGroupAlsoByWindow<K, V>> application,
+      CommittedBundle<KeyedWorkItem<K, V>> inputBundle,
+      EvaluationContext evaluationContext) {
+    return new GroupAlsoByWindowEvaluator<>(
+        evaluationContext, inputBundle, application);
+  }
+
+  /**
+   * A transform evaluator for the pseudo-primitive {@link GroupAlsoByWindow}. Windowing is ignored;
+   * all input should be in the global window since all output will be as well.
+   *
+   * @see GroupByKeyViaGroupByKeyOnly
+   */
+  private static class GroupAlsoByWindowEvaluator<K, V>
+      implements TransformEvaluator<KeyedWorkItem<K, V>> {
+
+    private final TransformEvaluator<KeyedWorkItem<K, V>> gabwParDoEvaluator;
+
+    public GroupAlsoByWindowEvaluator(
+        final EvaluationContext evaluationContext,
+        CommittedBundle<KeyedWorkItem<K, V>> inputBundle,
+        final AppliedPTransform<
+                PCollection<KeyedWorkItem<K, V>>,
+                PCollection<KV<K, Iterable<V>>>,
+                DirectGroupAlsoByWindow<K, V>> application) {
+
+      Coder<V> valueCoder =
+          application.getTransform().getValueCoder(inputBundle.getPCollection().getCoder());
+
+      @SuppressWarnings("unchecked")
+      WindowingStrategy<?, BoundedWindow> windowingStrategy =
+          (WindowingStrategy<?, BoundedWindow>) application.getTransform().getWindowingStrategy();
+
+      DoFn<KeyedWorkItem<K, V>, KV<K, Iterable<V>>> gabwDoFn =
+          GroupAlsoByWindowViaWindowSetDoFn.create(
+              windowingStrategy,
+              SystemReduceFn.<K, V, BoundedWindow>buffering(valueCoder));
+
+      TupleTag<KV<K, Iterable<V>>> mainOutputTag = new TupleTag<KV<K, Iterable<V>>>() {};
+
+      // Not technically legit, as the application is not a ParDo
+      this.gabwParDoEvaluator =
+          ParDoEvaluator.create(
+              evaluationContext,
+              inputBundle,
+              application,
+              gabwDoFn,
+              Collections.<PCollectionView<?>>emptyList(),
+              mainOutputTag,
+              Collections.<TupleTag<?>>emptyList(),
+              ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
+    }
+
+    @Override
+    public void processElement(WindowedValue<KeyedWorkItem<K, V>> element) throws Exception {
+      gabwParDoEvaluator.processElement(element);
+    }
+
+    @Override
+    public TransformResult finishBundle() throws Exception {
+      return gabwParDoEvaluator.finishBundle();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
new file mode 100644
index 0000000..0e419c3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.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.KvCoder;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
+ * {@link GroupByKeyOnly} {@link PTransform}.
+ */
+class GroupByKeyOnlyEvaluatorFactory implements TransformEvaluatorFactory {
+  @Override
+  public <InputT> TransformEvaluator<InputT> forApplication(
+      AppliedPTransform<?, ?, ?> application,
+      CommittedBundle<?> inputBundle,
+      EvaluationContext 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>>,
+          DirectGroupByKeyOnly<K, V>>
+          application,
+      final CommittedBundle<KV<K, WindowedValue<V>>> inputBundle,
+      final EvaluationContext evaluationContext) {
+    return new GroupByKeyOnlyEvaluator<>(evaluationContext, inputBundle, application);
+  }
+
+  /**
+   * A transform evaluator for the pseudo-primitive {@link GroupByKeyOnly}. Windowing is ignored;
+   * all input should be in the global window since all output will be as well.
+   *
+   * @see GroupByKeyViaGroupByKeyOnly
+   */
+  private static class GroupByKeyOnlyEvaluator<K, V>
+      implements TransformEvaluator<KV<K, WindowedValue<V>>> {
+    private final EvaluationContext evaluationContext;
+
+    private final CommittedBundle<KV<K, WindowedValue<V>>> inputBundle;
+    private final AppliedPTransform<
+            PCollection<KV<K, WindowedValue<V>>>,
+            PCollection<KeyedWorkItem<K, V>>,
+            DirectGroupByKeyOnly<K, V>> application;
+    private final Coder<K> keyCoder;
+    private Map<GroupingKey<K>, List<WindowedValue<V>>> groupingMap;
+
+    public GroupByKeyOnlyEvaluator(
+        EvaluationContext evaluationContext,
+        CommittedBundle<KV<K, WindowedValue<V>>> inputBundle,
+        AppliedPTransform<
+                PCollection<KV<K, WindowedValue<V>>>,
+                PCollection<KeyedWorkItem<K, V>>,
+            DirectGroupByKeyOnly<K, V>> application) {
+      this.evaluationContext = evaluationContext;
+      this.inputBundle = inputBundle;
+      this.application = application;
+      this.keyCoder = getKeyCoder(application.getInput().getCoder());
+      this.groupingMap = new HashMap<>();
+    }
+
+    private Coder<K> getKeyCoder(Coder<KV<K, WindowedValue<V>>> coder) {
+      checkState(
+          coder instanceof KvCoder,
+          "%s requires a coder of class %s."
+              + " This is an internal error; this is checked during pipeline construction"
+              + " but became corrupted.",
+          getClass().getSimpleName(),
+          KvCoder.class.getSimpleName());
+      @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 TransformResult 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,
+            StructuralKey.of(key, keyCoder),
+            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);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/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
index d121442..b0eb38f 100644
--- 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
@@ -77,7 +77,7 @@ class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
     @Override
     public void afterFinish(
         CommittedBundle<T> input,
-        InProcessTransformResult result,
+        TransformResult result,
         Iterable<? extends CommittedBundle<?>> outputs) {
       for (MutationDetector detector : mutationElements.values()) {
         verifyUnmodified(detector);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/babddbbc/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
new file mode 100644
index 0000000..25a0d05
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.VoidCoder;
+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;
+
+/**
+ * A factory that produces bundles that perform no additional validation.
+ */
+class ImmutableListBundleFactory implements BundleFactory {
+  public static ImmutableListBundleFactory create() {
+    return new ImmutableListBundleFactory();
+  }
+
+  private ImmutableListBundleFactory() {}
+
+  @Override
+  public <T> UncommittedBundle<T> createRootBundle(PCollection<T> output) {
+    return UncommittedImmutableListBundle.create(output, StructuralKey.of(null, VoidCoder.of()));
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createBundle(CommittedBundle<?> input, PCollection<T> output) {
+    return UncommittedImmutableListBundle.create(output, input.getKey());
+  }
+
+  @Override
+  public <K, T> UncommittedBundle<T> createKeyedBundle(
+      CommittedBundle<?> input, StructuralKey<K> key, PCollection<T> output) {
+    return UncommittedImmutableListBundle.create(output, key);
+  }
+
+  /**
+   * A {@link UncommittedBundle} that buffers elements in memory.
+   */
+  private static final class UncommittedImmutableListBundle<T> implements UncommittedBundle<T> {
+    private final PCollection<T> pcollection;
+    private final StructuralKey<?> key;
+    private boolean committed = false;
+    private ImmutableList.Builder<WindowedValue<T>> elements;
+
+    /**
+     * Create a new {@link UncommittedImmutableListBundle} for the specified {@link PCollection}.
+     */
+    public static <T> UncommittedImmutableListBundle<T> create(
+        PCollection<T> pcollection,
+        StructuralKey<?> key) {
+      return new UncommittedImmutableListBundle<>(pcollection, key);
+    }
+
+    private UncommittedImmutableListBundle(PCollection<T> pcollection, StructuralKey<?> key) {
+      this.pcollection = pcollection;
+      this.key = key;
+      this.elements = ImmutableList.builder();
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return pcollection;
+    }
+
+    @Override
+    public UncommittedImmutableListBundle<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 CommittedImmutableListBundle<>(
+          pcollection, key, committedElements, synchronizedCompletionTime);
+    }
+  }
+
+  private static class CommittedImmutableListBundle<T> implements CommittedBundle<T> {
+    public CommittedImmutableListBundle(
+        PCollection<T> pcollection,
+        StructuralKey<?> key,
+        Iterable<WindowedValue<T>> committedElements,
+        Instant synchronizedCompletionTime) {
+      this.pcollection = pcollection;
+      this.key = key;
+      this.committedElements = committedElements;
+      this.synchronizedCompletionTime = synchronizedCompletionTime;
+    }
+
+    private final PCollection<T> pcollection;
+    /** The structural value key of the Bundle, as specified by the coder that created it. */
+    private final StructuralKey<?> key;
+    private final Iterable<WindowedValue<T>> committedElements;
+    private final Instant synchronizedCompletionTime;
+
+    @Override
+    public StructuralKey<?> 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 CommittedImmutableListBundle<>(
+          pcollection, key, ImmutableList.copyOf(elements), synchronizedCompletionTime);
+    }
+  }
+}



[20/50] [abbrv] incubator-beam git commit: Rename InProcessPipelineRunner to DirectRunner

Posted by da...@apache.org.
Rename InProcessPipelineRunner to DirectRunner

Completes BEAM-243


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

Branch: refs/heads/python-sdk
Commit: 9400fc9a699f218a7948c21639428f5f00134ec5
Parents: d2ceaf5
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 15 10:45:15 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 runners/direct-java/pom.xml                     |   2 +-
 .../direct/AbstractModelEnforcement.java        |   2 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   4 +-
 .../beam/runners/direct/BundleFactory.java      |   4 +-
 .../beam/runners/direct/CommittedResult.java    |   2 +-
 .../beam/runners/direct/CompletionCallback.java |   2 +-
 .../beam/runners/direct/DirectOptions.java      | 101 +++++
 .../beam/runners/direct/DirectRegistrar.java    |  55 +++
 .../beam/runners/direct/DirectRunner.java       | 371 +++++++++++++++++++
 .../direct/EncodabilityEnforcementFactory.java  |   2 +-
 .../direct/ExecutorServiceParallelExecutor.java |   2 +-
 .../runners/direct/FlattenEvaluatorFactory.java |   6 +-
 .../ImmutabilityCheckingBundleFactory.java      |   4 +-
 .../direct/ImmutabilityEnforcementFactory.java  |   2 +-
 .../direct/InMemoryWatermarkManager.java        |   2 +-
 .../runners/direct/InProcessBundleFactory.java  |   4 +-
 .../direct/InProcessBundleOutputManager.java    |   6 +-
 .../direct/InProcessEvaluationContext.java      |  20 +-
 .../direct/InProcessExecutionContext.java       |   4 +-
 .../beam/runners/direct/InProcessExecutor.java  |   2 +-
 ...rocessGroupAlsoByWindowEvaluatorFactory.java |   4 +-
 ...InProcessGroupByKeyOnlyEvaluatorFactory.java |   6 +-
 .../direct/InProcessPipelineOptions.java        | 101 -----
 .../runners/direct/InProcessPipelineRunner.java | 370 ------------------
 .../beam/runners/direct/InProcessRegistrar.java |  55 ---
 .../direct/InProcessTransformResult.java        |   2 +-
 .../beam/runners/direct/ModelEnforcement.java   |   2 +-
 .../runners/direct/ModelEnforcementFactory.java |   2 +-
 .../runners/direct/ParDoInProcessEvaluator.java |   4 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |   4 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   4 +-
 .../direct/PassthroughTransformEvaluator.java   |   2 +-
 .../runners/direct/StepTransformResult.java     |   2 +-
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorFactory.java       |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |   2 +-
 .../beam/runners/direct/TransformExecutor.java  |   2 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   4 +-
 .../runners/direct/ViewEvaluatorFactory.java    |   6 +-
 .../runners/direct/WindowEvaluatorFactory.java  |   6 +-
 .../direct/AvroIOShardedWriteFactoryTest.java   |   2 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |   4 +-
 .../runners/direct/CommittedResultTest.java     |  12 +-
 .../runners/direct/DirectRegistrarTest.java     |  74 ++++
 .../beam/runners/direct/DirectRunnerTest.java   | 339 +++++++++++++++++
 .../EncodabilityEnforcementFactoryTest.java     |   2 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |   4 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   4 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   4 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   2 +-
 .../direct/InMemoryWatermarkManagerTest.java    |   4 +-
 .../direct/InProcessBundleFactoryTest.java      |   4 +-
 .../direct/InProcessEvaluationContextTest.java  |  10 +-
 ...ocessGroupByKeyOnlyEvaluatorFactoryTest.java |   4 +-
 .../direct/InProcessPipelineRegistrarTest.java  |  74 ----
 .../direct/InProcessPipelineRunnerTest.java     | 339 -----------------
 .../direct/ParDoInProcessEvaluatorTest.java     |   6 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |   4 +-
 .../direct/ParDoSingleEvaluatorFactoryTest.java |   4 +-
 .../direct/TextIOShardedWriteFactoryTest.java   |   2 +-
 .../runners/direct/TransformExecutorTest.java   |   2 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   4 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   4 +-
 .../direct/WindowEvaluatorFactoryTest.java      |   4 +-
 .../translation/TransformTranslatorTest.java    |   4 +-
 .../beam/sdk/options/PipelineOptions.java       |   2 +-
 testing/travis/test_wordcount.sh                |   4 +-
 67 files changed, 1051 insertions(+), 1050 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 8667aee..3010757 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -102,7 +102,7 @@
               <systemPropertyVariables>
                 <beamTestPipelineOptions>
                   [
-                    "--runner=org.apache.beam.runners.direct.InProcessPipelineRunner"
+                    "--runner=DirectRunner"
                   ]
                 </beamTestPipelineOptions>
               </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 948beb6..2ae0275 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.util.WindowedValue;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index f15d446..63d248a 100644
--- 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
@@ -17,8 +17,8 @@
  */
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
 import org.apache.beam.sdk.io.Read.Bounded;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index a0511df..a546cfb 100644
--- 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
@@ -18,8 +18,8 @@
 package org.apache.beam.runners.direct;
 
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 4a42e34..b241493 100644
--- 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
@@ -18,7 +18,7 @@
 
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 
 import com.google.auto.value.AutoValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 7c2c068..8ee4b44 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 
 /**
  * A callback for completing a bundle of input.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java
new file mode 100644
index 0000000..3901c04
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.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 org.apache.beam.runners.direct.DirectRunner}.
+ */
+public interface DirectOptions 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 FixedThreadPoolExecutorServiceFactory}, which produces instances of
+   * {@link Executors#newCachedThreadPool()}.
+   */
+  @JsonIgnore
+  @Required
+  @Hidden
+  @Default.InstanceFactory(FixedThreadPoolExecutorServiceFactory.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 DirectPipelineResult#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/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java
new file mode 100644
index 0000000..eb027fa
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.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 DirectRunner}.
+ */
+public class DirectRegistrar {
+  private DirectRegistrar() {}
+  /**
+   * Registers the {@link DirectRunner}.
+   */
+  @AutoService(PipelineRunnerRegistrar.class)
+  public static class InProcessRunner implements PipelineRunnerRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(DirectRunner.class);
+    }
+  }
+
+  /**
+   * Registers the {@link DirectOptions}.
+   */
+  @AutoService(PipelineOptionsRegistrar.class)
+  public static class InProcessOptions implements PipelineOptionsRegistrar {
+    @Override
+    public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+      return ImmutableList.<Class<? extends PipelineOptions>>of(DirectOptions.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
new file mode 100644
index 0000000..2f5a0bc
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
+import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
+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.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;
+
+/**
+ * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded
+ * {@link PCollection PCollections}.
+ */
+@Experimental
+public class DirectRunner
+    extends PipelineRunner<DirectPipelineResult> {
+  /**
+   * The default set of transform overrides to use in the {@link DirectRunner}.
+   *
+   * <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 key that was output in the most recent {@link GroupByKey} in the
+     * execution of this bundle.
+     */
+    StructuralKey<?> 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 DirectOptions options;
+
+  public static DirectRunner fromOptions(PipelineOptions options) {
+    return new DirectRunner(options.as(DirectOptions.class));
+  }
+
+  private DirectRunner(DirectOptions options) {
+    this.options = options;
+  }
+
+  /**
+   * Returns the {@link PipelineOptions} used to create this {@link DirectRunner}.
+   */
+  public DirectOptions 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 DirectPipelineResult 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);
+
+    DisplayDataValidator.validatePipeline(pipeline);
+
+    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();
+    DirectPipelineResult result =
+        new DirectPipelineResult(executor, context, aggregatorSteps);
+    if (options.isBlockOnRun()) {
+      try {
+        result.awaitCompletion();
+      } catch (UserCodeException userException) {
+        throw new PipelineExecutionException(userException.getCause());
+      } catch (Throwable t) {
+        if (t instanceof RuntimeException) {
+          throw (RuntimeException) t;
+        }
+        throw new RuntimeException(t);
+      }
+    }
+    return result;
+  }
+
+  private Map<Class<? extends PTransform>, Collection<ModelEnforcementFactory>>
+      defaultModelEnforcements(DirectOptions 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(
+      DirectOptions options) {
+    ImmutableList.Builder<ModelEnforcementFactory> enforcements = ImmutableList.builder();
+    if (options.isTestImmutability()) {
+      enforcements.add(ImmutabilityEnforcementFactory.create());
+    }
+    return enforcements.build();
+  }
+
+  private BundleFactory createBundleFactory(DirectOptions pipelineOptions) {
+    BundleFactory bundleFactory = InProcessBundleFactory.create();
+    if (pipelineOptions.isTestImmutability()) {
+      bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
+    }
+    return bundleFactory;
+  }
+
+  /**
+   * The result of running a {@link Pipeline} with the {@link DirectRunner}.
+   *
+   * Throws {@link UnsupportedOperationException} for all methods.
+   */
+  public static class DirectPipelineResult implements PipelineResult {
+    private final InProcessExecutor executor;
+    private final InProcessEvaluationContext evaluationContext;
+    private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps;
+    private State state;
+
+    private DirectPipelineResult(
+        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 DirectPipelineResult} 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 DirectOptions#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/9400fc9a/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
index ccf4c2b..bed61ec 100644
--- 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
@@ -19,7 +19,7 @@ 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.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.util.CoderUtils;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 980d764..14570a5 100644
--- 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
@@ -18,7 +18,7 @@
 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.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 2efaad3..bbe8787 100644
--- 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
@@ -17,8 +17,8 @@
  */
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList;
@@ -28,7 +28,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Flatten}
  * {@link PTransform}.
  */
 class FlattenEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 2a965ed..dcbe3d1 100644
--- 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
@@ -19,8 +19,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index bfecc9d..d121442 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.transforms.AppliedPTransform;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 95095fa..fb8eb7c 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 52bc575..0c7449c 100644
--- 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
@@ -19,8 +19,8 @@ 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index f374f99..bd07040 100644
--- 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
@@ -17,8 +17,8 @@
  */
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.TupleTag;
@@ -27,7 +27,7 @@ import java.util.Map;
 
 /**
  * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the
- * {@link InProcessPipelineRunner}.
+ * {@link DirectRunner}.
  */
 public class InProcessBundleOutputManager implements OutputManager {
   private final Map<TupleTag<?>, UncommittedBundle<?>> bundles;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index db8baa0..220ff83 100644
--- 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
@@ -22,9 +22,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import org.apache.beam.runners.direct.InMemoryWatermarkManager.FiredTimers;
 import org.apache.beam.runners.direct.InMemoryWatermarkManager.TransformWatermarks;
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -58,11 +58,11 @@ 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
+ * {@link DirectRunner}. 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
+ * {@link DirectRunner} 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
@@ -79,7 +79,7 @@ class InProcessEvaluationContext {
   private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
 
   /** The options that were used to create this {@link Pipeline}. */
-  private final InProcessPipelineOptions options;
+  private final DirectOptions options;
 
   private final BundleFactory bundleFactory;
   /** The current processing time and event time watermarks and timers. */
@@ -97,7 +97,7 @@ class InProcessEvaluationContext {
   private final CounterSet mergedCounters;
 
   public static InProcessEvaluationContext create(
-      InProcessPipelineOptions options,
+      DirectOptions options,
       BundleFactory bundleFactory,
       Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
       Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
@@ -108,7 +108,7 @@ class InProcessEvaluationContext {
   }
 
   private InProcessEvaluationContext(
-      InProcessPipelineOptions options,
+      DirectOptions options,
       BundleFactory bundleFactory,
       Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
       Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
@@ -295,7 +295,7 @@ class InProcessEvaluationContext {
   /**
    * Get the options used by this {@link Pipeline}.
    */
-  public InProcessPipelineOptions getPipelineOptions() {
+  public DirectOptions getPipelineOptions() {
     return options;
   }
 
@@ -389,7 +389,7 @@ class InProcessEvaluationContext {
    *
    * <p>If the provided transform produces any {@link IsBounded#UNBOUNDED}
    * {@link PCollection PCollections}, returns the value of
-   * {@link InProcessPipelineOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
+   * {@link DirectOptions#isShutdownUnboundedProducersWithMaxWatermark()}.
    */
   public boolean isDone(AppliedPTransform<?, ?, ?> transform) {
     // if the PTransform's watermark isn't at the max value, it isn't done

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 4f10b3a..d2558ce 100644
--- 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
@@ -25,7 +25,7 @@ import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals;
 
 /**
- * Execution Context for the {@link InProcessPipelineRunner}.
+ * Execution Context for the {@link DirectRunner}.
  *
  * This implementation is not thread safe. A new {@link InProcessExecutionContext} must be created
  * for each thread that requires it.
@@ -51,7 +51,7 @@ class InProcessExecutionContext
   }
 
   /**
-   * Step Context for the {@link InProcessPipelineRunner}.
+   * Step Context for the {@link DirectRunner}.
    */
   public class InProcessStepContext
       extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index d811e1b..1cfa544 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
index 5ded8b6..53b93d0 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupAlsoByWindowEvaluatorFactory.java
@@ -19,7 +19,7 @@ package org.apache.beam.runners.direct;
 
 import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupAlsoByWindow;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -41,7 +41,7 @@ import com.google.common.collect.ImmutableMap;
 import java.util.Collections;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link GroupByKeyOnly} {@link PTransform}.
  */
 class InProcessGroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
index a10d496..3604bca 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessGroupByKeyOnlyEvaluatorFactory.java
@@ -22,8 +22,8 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
 import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
-import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.runners.direct.StepTransformResult.Builder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -45,7 +45,7 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link GroupByKeyOnly} {@link PTransform}.
  */
 class InProcessGroupByKeyOnlyEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
deleted file mode 100644
index 0498521..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.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 FixedThreadPoolExecutorServiceFactory}, which produces instances of
-   * {@link Executors#newCachedThreadPool()}.
-   */
-  @JsonIgnore
-  @Required
-  @Hidden
-  @Default.InstanceFactory(FixedThreadPoolExecutorServiceFactory.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/9400fc9a/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
deleted file mode 100644
index 8847c58..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.runners.direct;
-
-import org.apache.beam.runners.direct.InProcessGroupByKey.InProcessGroupByKeyOnly;
-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.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;
-
-/**
- * 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 key that was output in the most recent {@link GroupByKey} in the
-     * execution of this bundle.
-     */
-    StructuralKey<?> 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);
-
-    DisplayDataValidator.validatePipeline(pipeline);
-
-    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) {
-        if (t instanceof RuntimeException) {
-          throw (RuntimeException) t;
-        }
-        throw new RuntimeException(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/9400fc9a/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
deleted file mode 100644
index 4a09de7..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/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.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/9400fc9a/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
index 0bc3ea1..92127b4 100644
--- 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
@@ -18,7 +18,7 @@
 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.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index cc9b6da..758ee24 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index 6162ba0..e0bbfcb 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index a3e2f18..b9f4808 100644
--- 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
@@ -18,8 +18,8 @@
 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.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.util.DoFnRunner;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
index 8945242..58d6f00 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -32,7 +32,7 @@ import com.google.common.cache.LoadingCache;
 import java.util.Map;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link BoundMulti} primitive {@link PTransform}.
  */
 class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
index 15704d7..afbb6ed 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -32,7 +32,7 @@ import com.google.common.collect.ImmutableMap;
 import java.util.Collections;
 
 /**
- * The {@link InProcessPipelineRunner} {@link TransformEvaluatorFactory} for the
+ * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the
  * {@link Bound ParDo.Bound} primitive {@link PTransform}.
  */
 class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index aef62b2..ba792d3 100644
--- 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.direct;
 
-import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.util.WindowedValue;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9400fc9a/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
index b2e3897..eacea91 100644
--- 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
@@ -20,7 +20,7 @@ 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.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.WindowedValue;



[07/50] [abbrv] incubator-beam git commit: Package javadoc for org.apache.beam.sdk.transforms.display

Posted by da...@apache.org.
Package javadoc for org.apache.beam.sdk.transforms.display


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

Branch: refs/heads/python-sdk
Commit: 340fe3ebcfef0b57b163483d7d7243ad5456ae72
Parents: 90bb20e
Author: Scott Wegner <sw...@google.com>
Authored: Wed Jun 15 10:17:01 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:28 2016 -0700

----------------------------------------------------------------------
 .../sdk/transforms/display/package-info.java    | 26 ++++++++++++++++++++
 1 file changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/340fe3eb/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/package-info.java
new file mode 100644
index 0000000..4af3327
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Defines {@link org.apache.beam.sdk.transforms.display.HasDisplayData} for annotating components
+ * which provide {@link org.apache.beam.sdk.transforms.display.DisplayData display data} used
+ * within UIs and diagnostic tools.
+ *
+ * @see org.apache.beam.sdk.transforms.display.HasDisplayData
+ */
+package org.apache.beam.sdk.transforms.display;


[09/50] [abbrv] incubator-beam git commit: Add test for ReduceFnRunner GC time overflow

Posted by da...@apache.org.
Add test for ReduceFnRunner GC time overflow


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

Branch: refs/heads/python-sdk
Commit: 5bf732cd3e598321a5c51e1239eda0fe2877a65d
Parents: 6058330
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 14 16:04:10 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:29 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/WindowMatchers.java     |  5 ++
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 68 ++++++++++++++++++++
 2 files changed, 73 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5bf732cd/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
index b47c32c..7a5e2fb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
@@ -47,6 +47,11 @@ public class WindowMatchers {
     return new WindowedValueMatcher<>(valueMatcher, timestampMatcher, Matchers.anything());
   }
 
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      Matcher<? super T> valueMatcher) {
+    return new WindowedValueMatcher<>(valueMatcher, Matchers.anything(), Matchers.anything());
+  }
+
   public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
       T value, long timestamp, long windowStart, long windowEnd) {
     return WindowMatchers.<T>isSingleWindowedValue(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5bf732cd/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
index 0df4bc6..b7ec540 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.util;
 
 import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
+import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
 
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -51,6 +52,7 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Never;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
@@ -59,6 +61,7 @@ import org.apache.beam.sdk.transforms.windowing.Sessions;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
 import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -79,6 +82,7 @@ import org.mockito.MockitoAnnotations;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
@@ -226,6 +230,70 @@ public class ReduceFnRunnerTest {
     tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
   }
 
+  /**
+   * Tests that the garbage collection time for a fixed window does not overflow the end of time.
+   */
+  @Test
+  public void testFixedWindowEndOfTimeGarbageCollection() throws Exception {
+
+    Duration allowedLateness = Duration.standardDays(365);
+    Duration windowSize = Duration.millis(10);
+    WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(windowSize);
+
+    // This timestamp falls into a window where the end of the window is before the end of the
+    // global window - the "end of time" - yet its expiration time is after.
+    final Instant elementTimestamp =
+        GlobalWindow.INSTANCE.maxTimestamp().minus(allowedLateness).plus(1);
+
+    IntervalWindow window = Iterables.getOnlyElement(
+        windowFn.assignWindows(
+            windowFn.new AssignContext() {
+              @Override
+              public Object element() {
+                throw new UnsupportedOperationException();
+              }
+              @Override
+              public Instant timestamp() {
+                return elementTimestamp;
+              }
+
+              @Override
+              public Collection<? extends BoundedWindow> windows() {
+                throw new UnsupportedOperationException();
+              }
+            }));
+
+    assertTrue(
+        window.maxTimestamp().isBefore(GlobalWindow.INSTANCE.maxTimestamp()));
+    assertTrue(
+        window.maxTimestamp().plus(allowedLateness).isAfter(GlobalWindow.INSTANCE.maxTimestamp()));
+
+    // Test basic execution of a trigger using a non-combining window set and accumulating mode.
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+        ReduceFnTester.combining(
+            windowFn,
+            AfterWatermark.pastEndOfWindow().withLateFirings(Never.ever()).buildTrigger(),
+            AccumulationMode.DISCARDING_FIRED_PANES,
+            new Sum.SumIntegerFn().<String>asKeyedFn(),
+            VarIntCoder.of(),
+            allowedLateness);
+
+    tester.injectElements(TimestampedValue.of(13, elementTimestamp));
+
+    // Should fire ON_TIME pane and there will be a checkState that the cleanup time
+    // is prior to timestamp max value
+    tester.advanceInputWatermark(window.maxTimestamp());
+
+    // Nothing in the ON_TIME pane (not governed by triggers, but by ReduceFnRunner)
+    assertThat(tester.extractOutput(), emptyIterable());
+
+    tester.injectElements(TimestampedValue.of(42, elementTimestamp));
+
+    // Now the final pane should fire, demonstrating that the GC time was truncated
+    tester.advanceInputWatermark(GlobalWindow.INSTANCE.maxTimestamp());
+    assertThat(tester.extractOutput(), contains(isWindowedValue(equalTo(55))));
+  }
+
   @Test
   public void testOnElementCombiningAccumulating() throws Exception {
     // Test basic execution of a trigger using a non-combining window set and accumulating mode.


[36/50] [abbrv] incubator-beam git commit: Turn on failOnWarning and ignore unused runners modules in example.

Posted by da...@apache.org.
Turn on failOnWarning and ignore unused runners modules in example.


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

Branch: refs/heads/python-sdk
Commit: c9550101373c399c15c722e995bdbaf429eabf30
Parents: 6d028ac
Author: Pei He <pe...@google.com>
Authored: Thu Jun 16 15:50:21 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:31 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c9550101/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 9458962..5167810 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -59,6 +59,14 @@
       <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>
@@ -216,6 +224,7 @@
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-direct-java</artifactId>
       <version>${project.version}</version>
+      <scope>runtime</scope>
     </dependency>
 
     <dependency>