You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/09/13 00:40:32 UTC

[01/50] [abbrv] incubator-beam git commit: [BEAM-589] Fixing IO.Read transformation

Repository: incubator-beam
Updated Branches:
  refs/heads/gearpump-runner bf0a2edae -> ed7c4aaf9


[BEAM-589] Fixing IO.Read transformation


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

Branch: refs/heads/gearpump-runner
Commit: 4023167b30f7a2b282f45073d4f9dd9cc532236d
Parents: 79491eb
Author: gaurav gupta <ga...@cisco.com>
Authored: Thu Aug 25 14:00:06 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/examples/complete/TfIdf.java   |  6 +++---
 .../runners/core/UnboundedReadFromBoundedSource.java    |  6 +++---
 .../org/apache/beam/runners/flink/examples/TFIDF.java   |  6 +++---
 .../apache/beam/runners/dataflow/DataflowRunner.java    |  6 +++---
 .../DataflowUnboundedReadFromBoundedSource.java         |  6 +++---
 .../beam/runners/dataflow/DataflowRunnerTest.java       |  4 ++--
 .../org/apache/beam/runners/spark/io/CreateStream.java  |  7 +++----
 .../java/org/apache/beam/runners/spark/io/KafkaIO.java  |  6 +++---
 .../apache/beam/runners/spark/io/hadoop/HadoopIO.java   |  6 +++---
 .../src/main/java/org/apache/beam/sdk/io/AvroIO.java    |  6 +++---
 .../beam/sdk/io/BoundedReadFromUnboundedSource.java     |  6 +++---
 .../src/main/java/org/apache/beam/sdk/io/PubsubIO.java  |  6 +++---
 .../core/src/main/java/org/apache/beam/sdk/io/Read.java | 10 +++++-----
 .../src/main/java/org/apache/beam/sdk/io/TextIO.java    |  6 +++---
 .../java/org/apache/beam/sdk/transforms/Create.java     | 12 ++++++------
 .../test/java/org/apache/beam/sdk/io/AvroIOTest.java    |  2 +-
 .../test/java/org/apache/beam/sdk/io/PubsubIOTest.java  |  2 +-
 .../test/java/org/apache/beam/sdk/io/TextIOTest.java    |  2 +-
 .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java |  8 ++++----
 .../apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java |  4 ++--
 .../src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java |  3 +--
 .../main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java |  3 +--
 22 files changed, 60 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 87023ed..6684553 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
@@ -51,11 +51,11 @@ import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.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.TupleTag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -152,7 +152,7 @@ public class TfIdf {
    * from the documents tagged with which document they are from.
    */
   public static class ReadDocuments
-      extends PTransform<PInput, PCollection<KV<URI, String>>> {
+      extends PTransform<PBegin, PCollection<KV<URI, String>>> {
     private Iterable<URI> uris;
 
     public ReadDocuments(Iterable<URI> uris) {
@@ -165,7 +165,7 @@ public class TfIdf {
     }
 
     @Override
-    public PCollection<KV<URI, String>> apply(PInput input) {
+    public PCollection<KV<URI, String>> apply(PBegin input) {
       Pipeline pipeline = input.getPipeline();
 
       // Create one TextIO.Read transform for each document

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
index 73688d4..91a1715 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
@@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
@@ -74,7 +74,7 @@ import org.slf4j.LoggerFactory;
  * <p>This transform is intended to be used by a runner during pipeline translation to convert
  * a Read.Bounded into a Read.Unbounded.
  */
-public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PCollection<T>> {
+public class UnboundedReadFromBoundedSource<T> extends PTransform<PBegin, PCollection<T>> {
 
   private static final Logger LOG = LoggerFactory.getLogger(UnboundedReadFromBoundedSource.class);
 
@@ -88,7 +88,7 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
   }
 
   @Override
-  public PCollection<T> apply(PInput input) {
+  public PCollection<T> apply(PBegin input) {
     return input.getPipeline().apply(
         Read.from(new BoundedToUnboundedSourceAdapter<>(source)));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 0ca94a1..a92d339 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
@@ -53,11 +53,11 @@ import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.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.TupleTag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -154,7 +154,7 @@ public class TFIDF {
    * from the documents tagged with which document they are from.
    */
   public static class ReadDocuments
-      extends PTransform<PInput, PCollection<KV<URI, String>>> {
+      extends PTransform<PBegin, PCollection<KV<URI, String>>> {
     private static final long serialVersionUID = 0;
 
     private Iterable<URI> uris;
@@ -169,7 +169,7 @@ public class TFIDF {
     }
 
     @Override
-    public PCollection<KV<URI, String>> apply(PInput input) {
+    public PCollection<KV<URI, String>> apply(PBegin input) {
       Pipeline pipeline = input.getPipeline();
 
       // Create one TextIO.Read transform for each document

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 a0e24b1..0ce4b58 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
@@ -400,7 +400,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       return windowed;
     } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
         && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform(input, Create.of());
+      return (OutputT) Pipeline.applyTransform((PBegin) 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")
@@ -2318,7 +2318,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
    * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the
    * Dataflow runner in streaming mode.
    */
-  private static class StreamingBoundedRead<T> extends PTransform<PInput, PCollection<T>> {
+  private static class StreamingBoundedRead<T> extends PTransform<PBegin, PCollection<T>> {
     private final BoundedSource<T> source;
 
     /** Builds an instance of this class from the overridden transform. */
@@ -2333,7 +2333,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     }
 
     @Override
-    public final PCollection<T> apply(PInput input) {
+    public final PCollection<T> apply(PBegin input) {
       source.validate();
 
       return Pipeline.applyTransform(input, new DataflowUnboundedReadFromBoundedSource<>(source))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
index 85f5e73..866da13 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
@@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
@@ -78,7 +78,7 @@ import org.slf4j.LoggerFactory;
  * time dependency. It should be replaced in the dataflow worker as an execution time dependency.
  */
 @Deprecated
-public class DataflowUnboundedReadFromBoundedSource<T> extends PTransform<PInput, PCollection<T>> {
+public class DataflowUnboundedReadFromBoundedSource<T> extends PTransform<PBegin, PCollection<T>> {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(DataflowUnboundedReadFromBoundedSource.class);
@@ -93,7 +93,7 @@ public class DataflowUnboundedReadFromBoundedSource<T> extends PTransform<PInput
   }
 
   @Override
-  public PCollection<T> apply(PInput input) {
+  public PCollection<T> apply(PBegin input) {
     return input.getPipeline().apply(
         Read.from(new BoundedToUnboundedSourceAdapter<>(source)));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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
index 208e84c..58a01aa 100644
--- 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
@@ -101,8 +101,8 @@ 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.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.hamcrest.Description;
@@ -970,7 +970,7 @@ public class DataflowRunnerTest {
     return options;
   }
 
-  private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
+  private void testUnsupportedSource(PTransform<PBegin, ?> source, String name, boolean streaming)
       throws Exception {
     String mode = streaming ? "streaming" : "batch";
     thrown.expect(UnsupportedOperationException.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
index b3beae6..a08c54e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
@@ -21,9 +21,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
-
 
 /**
  * Create an input stream from Queue.
@@ -49,7 +48,7 @@ public final class CreateStream<T> {
   /**
    * {@link PTransform} for queueing values.
    */
-  public static final class QueuedValues<T> extends PTransform<PInput, PCollection<T>> {
+  public static final class QueuedValues<T> extends PTransform<PBegin, PCollection<T>> {
 
     private final Iterable<Iterable<T>> queuedValues;
 
@@ -64,7 +63,7 @@ public final class CreateStream<T> {
     }
 
     @Override
-    public PCollection<T> apply(PInput input) {
+    public PCollection<T> apply(PBegin input) {
       // Spark streaming micro batches are bounded by default
       return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
           WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
index f57c114..8cf2083 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
@@ -25,8 +25,8 @@ import kafka.serializer.Decoder;
 import org.apache.beam.sdk.transforms.PTransform;
 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.PInput;
 
 /**
  * Read stream from Kafka.
@@ -68,7 +68,7 @@ public final class KafkaIO {
     /**
      * A {@link PTransform} reading from Kafka topics and providing {@link PCollection}.
      */
-    public static class Unbound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
+    public static class Unbound<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
 
       private final Class<? extends Decoder<K>> keyDecoderClass;
       private final Class<? extends Decoder<V>> valueDecoderClass;
@@ -120,7 +120,7 @@ public final class KafkaIO {
       }
 
       @Override
-      public PCollection<KV<K, V>> apply(PInput input) {
+      public PCollection<KV<K, V>> apply(PBegin input) {
         // Spark streaming micro batches are bounded by default
         return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
             WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
index 70bec78..042c316 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
@@ -26,9 +26,9 @@ import org.apache.beam.sdk.io.ShardNameTemplate;
 import org.apache.beam.sdk.transforms.PTransform;
 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.PDone;
-import org.apache.beam.sdk.values.PInput;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
@@ -58,7 +58,7 @@ public final class HadoopIO {
      * @param <K> the type of the keys
      * @param <V> the type of the values
      */
-    public static class Bound<K, V> extends PTransform<PInput, PCollection<KV<K, V>>> {
+    public static class Bound<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
 
       private final String filepattern;
       private final Class<? extends FileInputFormat<K, V>> formatClass;
@@ -94,7 +94,7 @@ public final class HadoopIO {
       }
 
       @Override
-      public PCollection<KV<K, V>> apply(PInput input) {
+      public PCollection<KV<K, V>> apply(PBegin input) {
         return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
             WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index e7c302b..267265d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -40,9 +40,9 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
 
 /**
  * {@link PTransform}s for reading and writing Avro files.
@@ -184,7 +184,7 @@ public class AvroIO {
      * @param <T> the type of each of the elements of the resulting
      * PCollection
      */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
+    public static class Bound<T> extends PTransform<PBegin, PCollection<T>> {
       /** The filepattern to read from. */
       @Nullable
       final String filepattern;
@@ -270,7 +270,7 @@ public class AvroIO {
       }
 
       @Override
-      public PCollection<T> apply(PInput input) {
+      public PCollection<T> apply(PBegin input) {
         if (filepattern == null) {
           throw new IllegalStateException(
               "need to set the filepattern of an AvroIO.Read transform");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
index ede65a9..28d7746 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
@@ -36,8 +36,8 @@ import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff;
 import org.apache.beam.sdk.util.ValueWithRecordId;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -48,7 +48,7 @@ import org.joda.time.Instant;
  *
  * <p>Created by {@link Read}.
  */
-class BoundedReadFromUnboundedSource<T> extends PTransform<PInput, PCollection<T>> {
+class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PCollection<T>> {
   private final UnboundedSource<T, ?> source;
   private final long maxNumRecords;
   private final Duration maxReadTime;
@@ -82,7 +82,7 @@ class BoundedReadFromUnboundedSource<T> extends PTransform<PInput, PCollection<T
   }
 
   @Override
-  public PCollection<T> apply(PInput input) {
+  public PCollection<T> apply(PBegin input) {
     PCollection<ValueWithRecordId<T>> read = Pipeline.applyTransform(input,
         Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime)));
     if (source.requiresDeduping()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 b137f15..d113457 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
@@ -46,9 +46,9 @@ import org.apache.beam.sdk.util.PubsubClient.ProjectPath;
 import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath;
 import org.apache.beam.sdk.util.PubsubClient.TopicPath;
 import org.apache.beam.sdk.util.PubsubJsonClient;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
@@ -481,7 +481,7 @@ public class PubsubIO {
      * A {@link PTransform} that reads from a Cloud Pub/Sub source and returns
      * a unbounded {@link PCollection} containing the items from the stream.
      */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
+    public static class Bound<T> extends PTransform<PBegin, PCollection<T>> {
       /** The Cloud Pub/Sub topic to read from. */
       @Nullable private final PubsubTopic topic;
 
@@ -610,7 +610,7 @@ public class PubsubIO {
       }
 
       @Override
-      public PCollection<T> apply(PInput input) {
+      public PCollection<T> apply(PBegin input) {
         if (topic == null && subscription == null) {
           throw new IllegalStateException("Need to set either the topic or the subscription for "
               + "a PubsubIO.Read transform");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 f99877d..29c4e47 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
@@ -25,9 +25,9 @@ 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.WindowingStrategy;
+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.PInput;
 import org.joda.time.Duration;
 
 /**
@@ -87,7 +87,7 @@ public class Read {
   /**
    * {@link PTransform} that reads from a {@link BoundedSource}.
    */
-  public static class Bounded<T> extends PTransform<PInput, PCollection<T>> {
+  public static class Bounded<T> extends PTransform<PBegin, PCollection<T>> {
     private final BoundedSource<T> source;
 
     private Bounded(@Nullable String name, BoundedSource<T> source) {
@@ -101,7 +101,7 @@ public class Read {
     }
 
     @Override
-    public final PCollection<T> apply(PInput input) {
+    public final PCollection<T> apply(PBegin input) {
       source.validate();
 
       return PCollection.<T>createPrimitiveOutputInternal(input.getPipeline(),
@@ -134,7 +134,7 @@ public class Read {
   /**
    * {@link PTransform} that reads from a {@link UnboundedSource}.
    */
-  public static class Unbounded<T> extends PTransform<PInput, PCollection<T>> {
+  public static class Unbounded<T> extends PTransform<PBegin, PCollection<T>> {
     private final UnboundedSource<T, ?> source;
 
     private Unbounded(@Nullable String name, UnboundedSource<T, ?> source) {
@@ -169,7 +169,7 @@ public class Read {
     }
 
     @Override
-    public final PCollection<T> apply(PInput input) {
+    public final PCollection<T> apply(PBegin input) {
       source.validate();
 
       return PCollection.<T>createPrimitiveOutputInternal(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/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 ed9a627..242470b 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
@@ -43,9 +43,9 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
 
 /**
  * {@link PTransform}s for reading and writing text files.
@@ -189,7 +189,7 @@ public class TextIO {
      * may use {@link #withCoder(Coder)} to supply a {@code Coder<T>} to produce a
      * {@code PCollection<T>} instead.
      */
-    public static class Bound<T> extends PTransform<PInput, PCollection<T>> {
+    public static class Bound<T> extends PTransform<PBegin, PCollection<T>> {
       /** The filepattern to read from. */
       @Nullable private final String filepattern;
 
@@ -269,7 +269,7 @@ public class TextIO {
       }
 
       @Override
-      public PCollection<T> apply(PInput input) {
+      public PCollection<T> apply(PBegin input) {
         if (filepattern == null) {
           throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform");
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
index e261db2..7cd4711 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
@@ -46,8 +46,8 @@ import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptions;
 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.PInput;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder;
 import org.apache.beam.sdk.values.TypeDescriptor;
@@ -218,7 +218,7 @@ public class Create<T> {
   /**
    * A {@code PTransform} that creates a {@code PCollection} from a set of in-memory objects.
    */
-  public static class Values<T> extends PTransform<PInput, PCollection<T>> {
+  public static class Values<T> extends PTransform<PBegin, PCollection<T>> {
     /**
      * Returns a {@link Create.Values} PTransform like this one that uses the given
      * {@code Coder<T>} to decode each of the objects into a
@@ -240,7 +240,7 @@ public class Create<T> {
     }
 
     @Override
-    public PCollection<T> apply(PInput input) {
+    public PCollection<T> apply(PBegin input) {
       try {
         Coder<T> coder = getDefaultOutputCoder(input);
         try {
@@ -257,7 +257,7 @@ public class Create<T> {
     }
 
     @Override
-    public Coder<T> getDefaultOutputCoder(PInput input) throws CannotProvideCoderException {
+    public Coder<T> getDefaultOutputCoder(PBegin input) throws CannotProvideCoderException {
       if (coder.isPresent()) {
         return coder.get();
       } else {
@@ -421,7 +421,7 @@ public class Create<T> {
    * A {@code PTransform} that creates a {@code PCollection} whose elements have
    * associated timestamps.
    */
-  public static class TimestampedValues<T> extends PTransform<PInput, PCollection<T>>{
+  public static class TimestampedValues<T> extends PTransform<PBegin, PCollection<T>>{
     /**
      * Returns a {@link Create.TimestampedValues} PTransform like this one that uses the given
      * {@code Coder<T>} to decode each of the objects into a
@@ -440,7 +440,7 @@ public class Create<T> {
     }
 
     @Override
-    public PCollection<T> apply(PInput input) {
+    public PCollection<T> apply(PBegin input) {
       try {
         Iterable<T> rawElements =
             Iterables.transform(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
index a8a7746..81f05d7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
@@ -291,7 +291,7 @@ public class AvroIOTest {
         .withSchema(Schema.create(Schema.Type.STRING))
         .withoutValidation();
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("AvroIO.Read should include the file pattern in its primitive transform",
         displayData, hasItem(hasDisplayItem("filePattern")));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
index 4067055..086b726 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
@@ -111,7 +111,7 @@ public class PubsubIOTest {
         PubsubIO.Read.subscription("projects/project/subscriptions/subscription")
             .maxNumRecords(1);
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("PubsubIO.Read should include the subscription in its primitive display data",
         displayData, hasItem(hasDisplayItem("subscription")));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 358a30f..8f94766 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -210,7 +210,7 @@ public class TextIOTest {
         .from("foobar")
         .withoutValidation();
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("TextIO.Read should include the file prefix in its primitive display data",
         displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar")))));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 01a8a1c..304dc82 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -122,11 +122,11 @@ import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.Transport;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.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.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Instant;
@@ -377,7 +377,7 @@ public class BigQueryIO {
      * A {@link PTransform} that reads from a BigQuery table and returns a bounded
      * {@link PCollection} of {@link TableRow TableRows}.
      */
-    public static class Bound extends PTransform<PInput, PCollection<TableRow>> {
+    public static class Bound extends PTransform<PBegin, PCollection<TableRow>> {
       @Nullable final String jsonTableRef;
       @Nullable final String query;
 
@@ -480,7 +480,7 @@ public class BigQueryIO {
       }
 
       @Override
-      public void validate(PInput input) {
+      public void validate(PBegin input) {
         // Even if existence validation is disabled, we need to make sure that the BigQueryIO
         // read is properly specified.
         BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
@@ -524,7 +524,7 @@ public class BigQueryIO {
       }
 
       @Override
-      public PCollection<TableRow> apply(PInput input) {
+      public PCollection<TableRow> apply(PBegin input) {
         String uuid = randomUUIDString();
         final String jobIdToken = "beam_job_" + uuid;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
index 7a7575b..57eb4ff 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -671,7 +671,7 @@ public class BigQueryIOTest implements Serializable {
             .withJobService(mockJobService))
         .withoutValidation();
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("BigQueryIO.Read should include the table spec in its primitive display data",
         displayData, hasItem(hasDisplayItem("table")));
   }
@@ -688,7 +688,7 @@ public class BigQueryIOTest implements Serializable {
             .withJobService(mockJobService))
         .withoutValidation();
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("BigQueryIO.Read should include the query in its primitive display data",
         displayData, hasItem(hasDisplayItem("query")));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
index f92dbd4..29d0c5f 100644
--- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
+++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
@@ -51,7 +51,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
@@ -140,7 +139,7 @@ public class JmsIO {
       // handles unbounded source to bounded conversion if maxNumRecords is set.
       Unbounded<JmsRecord> unbounded = org.apache.beam.sdk.io.Read.from(createSource());
 
-      PTransform<PInput, PCollection<JmsRecord>> transform = unbounded;
+      PTransform<PBegin, PCollection<JmsRecord>> transform = unbounded;
 
       if (maxNumRecords != Long.MAX_VALUE) {
         transform = unbounded.withMaxNumRecords(maxNumRecords);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4023167b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index 885d5d1..f639422 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -73,7 +73,6 @@ 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.PDone;
-import org.apache.beam.sdk.values.PInput;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
@@ -450,7 +449,7 @@ public class KafkaIO {
       Unbounded<KafkaRecord<K, V>> unbounded =
           org.apache.beam.sdk.io.Read.from(makeSource());
 
-      PTransform<PInput, PCollection<KafkaRecord<K, V>>> transform = unbounded;
+      PTransform<PBegin, PCollection<KafkaRecord<K, V>>> transform = unbounded;
 
       if (maxNumRecords < Long.MAX_VALUE) {
         transform = unbounded.withMaxNumRecords(maxNumRecords);



[42/50] [abbrv] incubator-beam git commit: Remove timeout in JAXBCoderTest

Posted by dh...@apache.org.
Remove timeout in JAXBCoderTest

The infrastructure should kill the test if the await does not return.


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

Branch: refs/heads/gearpump-runner
Commit: 8f68085bf4f66425d7b5b4835c844d523deb3e5e
Parents: c8052b6
Author: Thomas Groh <tg...@google.com>
Authored: Tue Sep 6 11:13:07 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/coders/JAXBCoderTest.java | 13 +++++--------
 1 file changed, 5 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f68085b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
index 129be61..36190f9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.coders;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
 import com.google.common.collect.ImmutableList;
 import java.io.IOException;
@@ -28,7 +27,6 @@ import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.beam.sdk.testing.CoderProperties;
@@ -119,7 +117,7 @@ public class JAXBCoderTest {
   @Test
   public void testEncodeDecodeMultithreaded() throws Throwable {
     final JAXBCoder<TestType> coder = JAXBCoder.of(TestType.class);
-    int numThreads = 1000;
+    int numThreads = 100;
 
     final CountDownLatch ready = new CountDownLatch(numThreads);
     final CountDownLatch start = new CountDownLatch(1);
@@ -155,11 +153,10 @@ public class JAXBCoderTest {
     ready.await();
     start.countDown();
 
-    if (!done.await(10L, TimeUnit.SECONDS)) {
-      fail("Should be able to clone " + numThreads + " elements in 10 seconds");
-    }
-    if (thrown.get() != null) {
-      throw thrown.get();
+    done.await();
+    Throwable actuallyThrown = thrown.get();
+    if (actuallyThrown != null) {
+      throw actuallyThrown;
     }
   }
 


[49/50] [abbrv] incubator-beam git commit: Added even more javadoc to TextIO#withHeader and TextIO#withFooter (2).

Posted by dh...@apache.org.
Added even more javadoc to TextIO#withHeader and TextIO#withFooter (2).


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

Branch: refs/heads/gearpump-runner
Commit: 34c731fd23b5984c1389783d2b3bfd561650c216
Parents: e5db1c7
Author: Stas Levin <st...@gmail.com>
Authored: Thu Sep 8 00:02:52 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/io/TextIO.java    | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34c731fd/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 f09ab6c..e4fcc32 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
@@ -632,11 +632,27 @@ public class TextIO {
             shardTemplate, false);
       }
 
+      /**
+       * Returns a transform for writing to text files that adds a header string to the files
+       * it writes.
+       *
+       *<p> A <code>null</code> value will clear any previously configured header.</p>
+       *
+       * @param header the string to be added as file header
+       */
       public Bound<T> withHeader(String header) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
 
+      /**
+       * Returns a transform for writing to text files that adds a footer string to the files
+       * it writes.
+       *
+       *<p> A <code>null</code> value will clear any previously configured footer.</p>
+       *
+       * @param footer the string to be added as file footer
+       */
       public Bound<T> withFooter(String footer) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);


[04/50] [abbrv] incubator-beam git commit: DataflowRunner: get PBegin from PInput

Posted by dh...@apache.org.
DataflowRunner: get PBegin from PInput

Fixes an invalid cast that breaks some PCollectionList-related 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/8454d5ca
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8454d5ca
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8454d5ca

Branch: refs/heads/gearpump-runner
Commit: 8454d5ca18c265ab1c5f54f2222c13d9ff17e4ef
Parents: 1c1115e
Author: Daniel Halperin <dh...@users.noreply.github.com>
Authored: Sat Aug 27 09:42:08 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8454d5ca/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 0ce4b58..e5b6614 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
@@ -400,7 +400,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       return windowed;
     } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
         && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform((PBegin) input, Create.of());
+      return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of());
     } else if (overrides.containsKey(transform.getClass())) {
       // It is the responsibility of whoever constructs overrides to ensure this is type safe.
       @SuppressWarnings("unchecked")


[26/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
deleted file mode 100644
index 24e33dd..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
+++ /dev/null
@@ -1,784 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-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.Sets;
-import java.io.IOException;
-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.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.PriorityQueue;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-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.StringUtf8Coder;
-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.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-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.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.InMemoryStateInternals;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub
- * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to
- * run {@code Trigger}s and {@code ReduceFn}s.
- *
- * @param <InputT> The element types.
- * @param <OutputT> The final type for elements in the window (for instance,
- *     {@code Iterable<InputT>})
- * @param <W> The type of windows being used.
- */
-public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
-  private static final String KEY = "TEST_KEY";
-
-  private final TestInMemoryStateInternals<String> stateInternals =
-      new TestInMemoryStateInternals<>(KEY);
-  private final TestTimerInternals timerInternals = new TestTimerInternals();
-
-  private final WindowFn<Object, W> windowFn;
-  private final TestWindowingInternals windowingInternals;
-  private final Coder<OutputT> outputCoder;
-  private final WindowingStrategy<Object, W> objectStrategy;
-  private final ReduceFn<String, InputT, OutputT, W> reduceFn;
-  private final PipelineOptions options;
-
-  /**
-   * If true, the output watermark is automatically advanced to the latest possible
-   * point when the input watermark is advanced. This is the default for most tests.
-   * If false, the output watermark must be explicitly advanced by the test, which can
-   * be used to exercise some of the more subtle behavior of WatermarkHold.
-   */
-  private boolean autoAdvanceOutputWatermark;
-
-  private ExecutableTrigger executableTrigger;
-
-  private final InMemoryLongSumAggregator droppedDueToClosedWindow =
-      new InMemoryLongSumAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER);
-
-  public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
-      nonCombining(WindowingStrategy<?, W> windowingStrategy) throws Exception {
-    return new ReduceFnTester<Integer, Iterable<Integer>, W>(
-        windowingStrategy,
-        SystemReduceFn.<String, Integer, W>buffering(VarIntCoder.of()),
-        IterableCoder.of(VarIntCoder.of()),
-        PipelineOptionsFactory.create(),
-        NullSideInputReader.empty());
-  }
-
-  public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
-      nonCombining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
-          Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception {
-    WindowingStrategy<?, W> strategy =
-        WindowingStrategy.of(windowFn)
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withTrigger(trigger)
-            .withMode(mode)
-            .withAllowedLateness(allowedDataLateness)
-            .withClosingBehavior(closingBehavior);
-    return nonCombining(strategy);
-  }
-
-  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
-      combining(WindowingStrategy<?, W> strategy,
-          KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn,
-          Coder<OutputT> outputCoder) throws Exception {
-
-    CoderRegistry registry = new CoderRegistry();
-    registry.registerStandardCoders();
-    AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
-        AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
-            combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
-
-    return new ReduceFnTester<Integer, OutputT, W>(
-        strategy,
-        SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
-        outputCoder,
-        PipelineOptionsFactory.create(),
-        NullSideInputReader.empty());
-  }
-
-  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
-  combining(WindowingStrategy<?, W> strategy,
-      KeyedCombineFnWithContext<String, Integer, AccumT, OutputT> combineFn,
-      Coder<OutputT> outputCoder,
-      PipelineOptions options,
-      SideInputReader sideInputReader) throws Exception {
-    CoderRegistry registry = new CoderRegistry();
-    registry.registerStandardCoders();
-    AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
-        AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
-            combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
-
-    return new ReduceFnTester<Integer, OutputT, W>(
-        strategy,
-        SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
-        outputCoder,
-        options,
-        sideInputReader);
-  }
-  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
-      combining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
-          KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder,
-          Duration allowedDataLateness) throws Exception {
-
-    WindowingStrategy<?, W> strategy =
-        WindowingStrategy.of(windowFn)
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withTrigger(trigger)
-            .withMode(mode)
-            .withAllowedLateness(allowedDataLateness);
-
-    return combining(strategy, combineFn, outputCoder);
-  }
-
-  private ReduceFnTester(WindowingStrategy<?, W> wildcardStrategy,
-      ReduceFn<String, InputT, OutputT, W> reduceFn, Coder<OutputT> outputCoder,
-      PipelineOptions options, SideInputReader sideInputReader) throws Exception {
-    @SuppressWarnings("unchecked")
-    WindowingStrategy<Object, W> objectStrategy = (WindowingStrategy<Object, W>) wildcardStrategy;
-
-    this.objectStrategy = objectStrategy;
-    this.reduceFn = reduceFn;
-    this.windowFn = objectStrategy.getWindowFn();
-    this.windowingInternals = new TestWindowingInternals(sideInputReader);
-    this.outputCoder = outputCoder;
-    this.autoAdvanceOutputWatermark = true;
-    this.executableTrigger = wildcardStrategy.getTrigger();
-    this.options = options;
-  }
-
-  public void setAutoAdvanceOutputWatermark(boolean autoAdvanceOutputWatermark) {
-    this.autoAdvanceOutputWatermark = autoAdvanceOutputWatermark;
-  }
-
-  @Nullable
-  public Instant getNextTimer(TimeDomain domain) {
-    return timerInternals.getNextTimer(domain);
-  }
-
-  ReduceFnRunner<String, InputT, OutputT, W> createRunner() {
-    return new ReduceFnRunner<>(
-        KEY,
-        objectStrategy,
-        stateInternals,
-        timerInternals,
-        windowingInternals,
-        droppedDueToClosedWindow,
-        reduceFn,
-        options);
-  }
-
-  public ExecutableTrigger getTrigger() {
-    return executableTrigger;
-  }
-
-  public boolean isMarkedFinished(W window) {
-    return createRunner().isFinished(window);
-  }
-
-  public boolean hasNoActiveWindows() {
-    return createRunner().hasNoActiveWindows();
-  }
-
-  @SafeVarargs
-  public final void assertHasOnlyGlobalAndFinishedSetsFor(W... expectedWindows) {
-    assertHasOnlyGlobalAndAllowedTags(
-        ImmutableSet.copyOf(expectedWindows),
-        ImmutableSet.<StateTag<? super String, ?>>of(TriggerRunner.FINISHED_BITS_TAG));
-  }
-
-  @SafeVarargs
-  public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expectedWindows) {
-    assertHasOnlyGlobalAndAllowedTags(
-        ImmutableSet.copyOf(expectedWindows),
-        ImmutableSet.<StateTag<? super String, ?>>of(
-            TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG,
-            WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
-            WatermarkHold.EXTRA_HOLD_TAG));
-  }
-
-  public final void assertHasOnlyGlobalState() {
-    assertHasOnlyGlobalAndAllowedTags(
-        Collections.<W>emptySet(), Collections.<StateTag<? super String, ?>>emptySet());
-  }
-
-  @SafeVarargs
-  public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) {
-    assertHasOnlyGlobalAndAllowedTags(
-        ImmutableSet.copyOf(expectedWindows),
-        ImmutableSet.<StateTag<? super String, ?>>of(
-            PaneInfoTracker.PANE_INFO_TAG,
-            WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
-            WatermarkHold.EXTRA_HOLD_TAG));
-  }
-
-  /**
-   * Verifies that the the set of windows that have any state stored is exactly
-   * {@code expectedWindows} and that each of these windows has only tags from {@code allowedTags}.
-   */
-  private void assertHasOnlyGlobalAndAllowedTags(
-      Set<W> expectedWindows, Set<StateTag<? super String, ?>> allowedTags) {
-    Set<StateNamespace> expectedWindowsSet = new HashSet<>();
-    for (W expectedWindow : expectedWindows) {
-      expectedWindowsSet.add(windowNamespace(expectedWindow));
-    }
-    Map<StateNamespace, Set<StateTag<? super String, ?>>> actualWindows = new HashMap<>();
-
-    for (StateNamespace namespace : stateInternals.getNamespacesInUse()) {
-      if (namespace instanceof StateNamespaces.GlobalNamespace) {
-        continue;
-      } else if (namespace instanceof StateNamespaces.WindowNamespace) {
-        Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
-        if (tagsInUse.isEmpty()) {
-          continue;
-        }
-        actualWindows.put(namespace, tagsInUse);
-        Set<StateTag<? super String, ?>> unexpected = Sets.difference(tagsInUse, allowedTags);
-        if (unexpected.isEmpty()) {
-          continue;
-        } else {
-          fail(namespace + " has unexpected states: " + tagsInUse);
-        }
-      } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) {
-        Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
-        assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty());
-      } else {
-        fail("Unrecognized namespace " + namespace);
-      }
-    }
-
-    assertEquals("Still in use: " + actualWindows.toString(), expectedWindowsSet,
-        actualWindows.keySet());
-  }
-
-  private StateNamespace windowNamespace(W window) {
-    return StateNamespaces.window(windowFn.windowCoder(), window);
-  }
-
-  public Instant getWatermarkHold() {
-    return stateInternals.earliestWatermarkHold();
-  }
-
-  public Instant getOutputWatermark() {
-    return timerInternals.currentOutputWatermarkTime();
-  }
-
-  public long getElementsDroppedDueToClosedWindow() {
-    return droppedDueToClosedWindow.getSum();
-  }
-
-  /**
-   * How many panes do we have in the output?
-   */
-  public int getOutputSize() {
-    return windowingInternals.outputs.size();
-  }
-
-  /**
-   * Retrieve the values that have been output to this time, and clear out the output accumulator.
-   */
-  public List<WindowedValue<OutputT>> extractOutput() {
-    ImmutableList<WindowedValue<OutputT>> result =
-        FluentIterable.from(windowingInternals.outputs)
-            .transform(new Function<WindowedValue<KV<String, OutputT>>, WindowedValue<OutputT>>() {
-              @Override
-              public WindowedValue<OutputT> apply(WindowedValue<KV<String, OutputT>> input) {
-                return input.withValue(input.getValue().getValue());
-              }
-            })
-            .toList();
-    windowingInternals.outputs.clear();
-    return result;
-  }
-
-  /**
-   * Advance the input watermark to the specified time, firing any timers that should
-   * fire. Then advance the output watermark as far as possible.
-   */
-  public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
-    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
-    timerInternals.advanceInputWatermark(runner, newInputWatermark);
-    runner.persist();
-  }
-
-  /**
-   * If {@link #autoAdvanceOutputWatermark} is {@literal false}, advance the output watermark
-   * to the given value. Otherwise throw.
-   */
-  public void advanceOutputWatermark(Instant newOutputWatermark) throws Exception {
-    timerInternals.advanceOutputWatermark(newOutputWatermark);
-  }
-
-  /** Advance the processing time to the specified time, firing any timers that should fire. */
-  public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
-    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
-    timerInternals.advanceProcessingTime(runner, newProcessingTime);
-    runner.persist();
-  }
-
-  /**
-   * Advance the synchronized processing time to the specified time,
-   * firing any timers that should fire.
-   */
-  public void advanceSynchronizedProcessingTime(Instant newProcessingTime) throws Exception {
-    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
-    timerInternals.advanceSynchronizedProcessingTime(runner, newProcessingTime);
-    runner.persist();
-  }
-
-  /**
-   * Inject all the timestamped values (after passing through the window function) as if they
-   * arrived in a single chunk of a bundle (or work-unit).
-   */
-  @SafeVarargs
-  public final void injectElements(TimestampedValue<InputT>... values) throws Exception {
-    for (TimestampedValue<InputT> value : values) {
-      WindowTracing.trace("TriggerTester.injectElements: {}", value);
-    }
-    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
-    runner.processElements(
-        Iterables.transform(
-            Arrays.asList(values),
-            new Function<TimestampedValue<InputT>, WindowedValue<InputT>>() {
-              @Override
-              public WindowedValue<InputT> apply(TimestampedValue<InputT> input) {
-                try {
-                  InputT value = input.getValue();
-                  Instant timestamp = input.getTimestamp();
-                  Collection<W> windows =
-                      windowFn.assignWindows(
-                          new TestAssignContext<W>(
-                              windowFn, value, timestamp, GlobalWindow.INSTANCE));
-                  return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING);
-                } catch (Exception e) {
-                  throw new RuntimeException(e);
-                }
-              }
-            }));
-
-    // Persist after each bundle.
-    runner.persist();
-  }
-
-  public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception {
-    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
-    runner.onTimer(
-        TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain));
-    runner.persist();
-  }
-
-  /**
-   * Simulate state.
-   */
-  private static class TestInMemoryStateInternals<K> extends InMemoryStateInternals<K> {
-
-    public TestInMemoryStateInternals(K key) {
-      super(key);
-    }
-
-    public Set<StateTag<? super K, ?>> getTagsInUse(StateNamespace namespace) {
-      Set<StateTag<? super K, ?>> inUse = new HashSet<>();
-      for (Entry<StateTag<? super K, ?>, State> entry :
-        inMemoryState.getTagsInUse(namespace).entrySet()) {
-        if (!isEmptyForTesting(entry.getValue())) {
-          inUse.add(entry.getKey());
-        }
-      }
-      return inUse;
-    }
-
-    public Set<StateNamespace> getNamespacesInUse() {
-      return inMemoryState.getNamespacesInUse();
-    }
-
-    /** Return the earliest output watermark hold in state, or null if none. */
-    public Instant earliestWatermarkHold() {
-      Instant minimum = null;
-      for (State storage : inMemoryState.values()) {
-        if (storage instanceof WatermarkHoldState) {
-          Instant hold = ((WatermarkHoldState<?>) storage).read();
-          if (minimum == null || (hold != null && hold.isBefore(minimum))) {
-            minimum = hold;
-          }
-        }
-      }
-      return minimum;
-    }
-  }
-
-  /**
-   * Convey the simulated state and implement {@link #outputWindowedValue} to capture all output
-   * elements.
-   */
-  private class TestWindowingInternals implements WindowingInternals<InputT, KV<String, OutputT>> {
-    private List<WindowedValue<KV<String, OutputT>>> outputs = new ArrayList<>();
-    private SideInputReader sideInputReader;
-
-    private TestWindowingInternals(SideInputReader sideInputReader) {
-      this.sideInputReader = sideInputReader;
-    }
-
-    @Override
-    public void outputWindowedValue(KV<String, OutputT> output, Instant timestamp,
-        Collection<? extends BoundedWindow> windows, PaneInfo pane) {
-      // Copy the output value (using coders) before capturing it.
-      KV<String, OutputT> copy = SerializableUtils.<KV<String, OutputT>>ensureSerializableByCoder(
-          KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow");
-      WindowedValue<KV<String, OutputT>> value = WindowedValue.of(copy, timestamp, windows, pane);
-      outputs.add(value);
-    }
-
-    @Override
-    public TimerInternals timerInternals() {
-      throw new UnsupportedOperationException(
-          "Testing triggers should not use timers from WindowingInternals.");
-    }
-
-    @Override
-    public Collection<? extends BoundedWindow> windows() {
-      throw new UnsupportedOperationException(
-          "Testing triggers should not use windows from WindowingInternals.");
-    }
-
-    @Override
-    public PaneInfo pane() {
-      throw new UnsupportedOperationException(
-          "Testing triggers should not use pane from WindowingInternals.");
-    }
-
-    @Override
-    public <T> void writePCollectionViewData(
-        TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
-      throw new UnsupportedOperationException(
-          "Testing triggers should not use writePCollectionViewData from WindowingInternals.");
-    }
-
-    @Override
-    public StateInternals<Object> stateInternals() {
-      // Safe for testing only
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      TestInMemoryStateInternals<Object> untypedStateInternals =
-          (TestInMemoryStateInternals) stateInternals;
-      return untypedStateInternals;
-    }
-
-    @Override
-    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-      if (!sideInputReader.contains(view)) {
-        throw new IllegalArgumentException("calling sideInput() with unknown view");
-      }
-      BoundedWindow sideInputWindow =
-          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
-      return sideInputReader.get(view, sideInputWindow);
-    }
-  }
-
-  private static class TestAssignContext<W extends BoundedWindow>
-      extends WindowFn<Object, W>.AssignContext {
-    private Object element;
-    private Instant timestamp;
-    private BoundedWindow window;
-
-    public TestAssignContext(
-        WindowFn<Object, W> windowFn, Object element, Instant timestamp, BoundedWindow window) {
-      windowFn.super();
-      this.element = element;
-      this.timestamp = timestamp;
-      this.window = window;
-    }
-
-    @Override
-    public Object element() {
-      return element;
-    }
-
-    @Override
-    public Instant timestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public BoundedWindow window() {
-      return window;
-    }
-  }
-
-  private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
-    private final String name;
-    private long sum = 0;
-
-    public InMemoryLongSumAggregator(String name) {
-      this.name = name;
-    }
-
-    @Override
-    public void addValue(Long value) {
-      sum += value;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public CombineFn<Long, ?, Long> getCombineFn() {
-      return new Sum.SumLongFn();
-    }
-
-    public long getSum() {
-      return sum;
-    }
-  }
-
-  /**
-   * Simulate the firing of timers and progression of input and output watermarks for a
-   * single computation and key in a Windmill-like streaming environment. Similar to
-   * {@link BatchTimerInternals}, but also tracks the output watermark.
-   */
-  private class TestTimerInternals implements TimerInternals {
-    /** At most one timer per timestamp is kept. */
-    private Set<TimerData> existingTimers = new HashSet<>();
-
-    /** Pending input watermark timers, in timestamp order. */
-    private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
-
-    /** Pending processing time timers, in timestamp order. */
-    private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
-
-    /** Current input watermark. */
-    @Nullable
-    private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
-    /** Current output watermark. */
-    @Nullable
-    private Instant outputWatermarkTime = null;
-
-    /** Current processing time. */
-    private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
-    /** Current synchronized processing time. */
-    @Nullable
-    private Instant synchronizedProcessingTime = null;
-
-    @Nullable
-    public Instant getNextTimer(TimeDomain domain) {
-      TimerData data = null;
-      switch (domain) {
-        case EVENT_TIME:
-           data = watermarkTimers.peek();
-           break;
-        case PROCESSING_TIME:
-        case SYNCHRONIZED_PROCESSING_TIME:
-          data = processingTimers.peek();
-          break;
-      }
-      checkNotNull(data); // cases exhaustive
-      return data == null ? null : data.getTimestamp();
-    }
-
-    private PriorityQueue<TimerData> queue(TimeDomain domain) {
-      switch (domain) {
-        case EVENT_TIME:
-          return watermarkTimers;
-        case PROCESSING_TIME:
-        case SYNCHRONIZED_PROCESSING_TIME:
-          return processingTimers;
-      }
-      throw new RuntimeException(); // cases exhaustive
-    }
-
-    @Override
-    public void setTimer(TimerData timer) {
-      WindowTracing.trace("TestTimerInternals.setTimer: {}", timer);
-      if (existingTimers.add(timer)) {
-        queue(timer.getDomain()).add(timer);
-      }
-    }
-
-    @Override
-    public void deleteTimer(TimerData timer) {
-      WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer);
-      existingTimers.remove(timer);
-      queue(timer.getDomain()).remove(timer);
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return processingTime;
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return synchronizedProcessingTime;
-    }
-
-    @Override
-    public Instant currentInputWatermarkTime() {
-      return checkNotNull(inputWatermarkTime);
-    }
-
-    @Override
-    @Nullable
-    public Instant currentOutputWatermarkTime() {
-      return outputWatermarkTime;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(getClass())
-          .add("watermarkTimers", watermarkTimers)
-          .add("processingTimers", processingTimers)
-          .add("inputWatermarkTime", inputWatermarkTime)
-          .add("outputWatermarkTime", outputWatermarkTime)
-          .add("processingTime", processingTime)
-          .toString();
-    }
-
-    public void advanceInputWatermark(
-        ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark) throws Exception {
-      checkNotNull(newInputWatermark);
-      checkState(
-          !newInputWatermark.isBefore(inputWatermarkTime),
-          "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
-          newInputWatermark);
-      WindowTracing.trace("TestTimerInternals.advanceInputWatermark: from {} to {}",
-          inputWatermarkTime, newInputWatermark);
-      inputWatermarkTime = newInputWatermark;
-      advanceAndFire(runner, newInputWatermark, TimeDomain.EVENT_TIME);
-
-      Instant hold = stateInternals.earliestWatermarkHold();
-      if (hold == null) {
-        WindowTracing.trace("TestTimerInternals.advanceInputWatermark: no holds, "
-            + "so output watermark = input watermark");
-        hold = inputWatermarkTime;
-      }
-      if (autoAdvanceOutputWatermark) {
-        advanceOutputWatermark(hold);
-      }
-    }
-
-    public void advanceOutputWatermark(Instant newOutputWatermark) {
-      checkNotNull(newOutputWatermark);
-      if (newOutputWatermark.isAfter(inputWatermarkTime)) {
-        WindowTracing.trace(
-            "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}",
-            newOutputWatermark, inputWatermarkTime);
-        newOutputWatermark = inputWatermarkTime;
-      }
-      checkState(
-          outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime),
-          "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime,
-          newOutputWatermark);
-      WindowTracing.trace("TestTimerInternals.advanceOutputWatermark: from {} to {}",
-          outputWatermarkTime, newOutputWatermark);
-      outputWatermarkTime = newOutputWatermark;
-    }
-
-    public void advanceProcessingTime(
-        ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime) throws Exception {
-      checkState(!newProcessingTime.isBefore(processingTime),
-          "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
-      WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime,
-          newProcessingTime);
-      processingTime = newProcessingTime;
-      advanceAndFire(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
-    }
-
-    public void advanceSynchronizedProcessingTime(
-        ReduceFnRunner<?, ?, ?, ?> runner, Instant newSynchronizedProcessingTime) throws Exception {
-      checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
-          "Cannot move processing time backwards from %s to %s", processingTime,
-          newSynchronizedProcessingTime);
-      WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}",
-          synchronizedProcessingTime, newSynchronizedProcessingTime);
-      synchronizedProcessingTime = newSynchronizedProcessingTime;
-      advanceAndFire(
-          runner, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    }
-
-    private void advanceAndFire(
-        ReduceFnRunner<?, ?, ?, ?> runner, Instant currentTime, TimeDomain domain)
-            throws Exception {
-      PriorityQueue<TimerData> queue = queue(domain);
-      boolean shouldFire = false;
-
-      do {
-        TimerData timer = queue.peek();
-        // Timers fire when the current time progresses past the timer time.
-        shouldFire = timer != null && currentTime.isAfter(timer.getTimestamp());
-        if (shouldFire) {
-          WindowTracing.trace(
-              "TestTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime);
-          // Remove before firing, so that if the trigger adds another identical
-          // timer we don't remove it.
-          queue.remove();
-
-          runner.onTimer(timer);
-        }
-      } while (shouldFire);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
deleted file mode 100644
index 156b4a9..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.is;
-import static org.mockito.Mockito.mock;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for base {@link DoFnRunnerBase} functionality.
- */
-@RunWith(JUnit4.class)
-public class SimpleDoFnRunnerTest {
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testExceptionsWrappedAsUserCodeException() {
-    ThrowingDoFn fn = new ThrowingDoFn();
-    DoFnRunner<String, String> runner = createRunner(fn);
-
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(is(fn.exceptionToThrow));
-
-    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
-  }
-
-  @Test
-  public void testSystemDoFnInternalExceptionsNotWrapped() {
-    ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
-    DoFnRunner<String, String> runner = createRunner(fn);
-
-    thrown.expect(is(fn.exceptionToThrow));
-
-    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
-  }
-
-  private DoFnRunner<String, String> createRunner(OldDoFn<String, String> fn) {
-    // Pass in only necessary parameters for the test
-    List<TupleTag<?>> sideOutputTags = Arrays.asList();
-    StepContext context = mock(StepContext.class);
-    return new SimpleDoFnRunner<>(
-          null, fn, null, null, null, sideOutputTags, context, null, null);
-  }
-
-  static class ThrowingDoFn extends OldDoFn<String, String> {
-    final Exception exceptionToThrow =
-        new UnsupportedOperationException("Expected exception");
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      throw exceptionToThrow;
-    }
-  }
-
-  @SystemDoFnInternal
-  static class ThrowingSystemDoFn extends ThrowingDoFn {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/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
index 2da70bb..c08c229 100644
--- 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
@@ -20,6 +20,10 @@ package org.apache.beam.runners.direct;
 import com.google.common.collect.ImmutableMap;
 import java.util.Collections;
 import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
@@ -28,11 +32,7 @@ import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
-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.util.state.StateInternals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/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
index f085a39..17dc0be 100644
--- 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
@@ -25,6 +25,8 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
@@ -34,8 +36,6 @@ 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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
index 85a1c6a..99ab22a 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -23,15 +23,15 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 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.OldDoFn;
-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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/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
index 41f7e8d..d40dc11 100644
--- 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
@@ -18,9 +18,9 @@
 package org.apache.beam.runners.direct;
 
 import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
 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;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
index 07e2191..3719fa8 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
@@ -63,7 +63,7 @@ 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.Reshuffle;
-import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index 2c7ebc6..3b0fccc 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -40,11 +40,11 @@ import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.NullSideInputReader;
-import org.apache.beam.sdk.util.PushbackSideInputDoFnRunner;
+import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index 01cfa5b..b893116 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -43,7 +43,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.KeyedWorkItem;
 import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
index 9f1a839..03db811 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
@@ -18,6 +18,7 @@
 
 package org.apache.beam.runners.spark;
 
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
 import org.apache.beam.runners.spark.translation.EvaluationContext;
 import org.apache.beam.runners.spark.translation.SparkContextFactory;
 import org.apache.beam.runners.spark.translation.SparkPipelineEvaluator;
@@ -36,7 +37,6 @@ import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/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 eaceb85..8341c6d 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
@@ -37,6 +37,11 @@ import java.util.Map;
 import org.apache.avro.mapred.AvroKey;
 import org.apache.avro.mapreduce.AvroJob;
 import org.apache.avro.mapreduce.AvroKeyInputFormat;
+import org.apache.beam.runners.core.AssignWindowsDoFn;
+import org.apache.beam.runners.core.GroupAlsoByWindowsViaOutputBufferDoFn;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.spark.coders.CoderHelpers;
 import org.apache.beam.runners.spark.io.hadoop.HadoopIO;
 import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper;
@@ -61,11 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AssignWindowsDoFn;
-import org.apache.beam.sdk.util.GroupAlsoByWindowsViaOutputBufferDoFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.sdk.util.SystemReduceFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
 import org.apache.beam.sdk.util.WindowingStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
index 43dcef6..c55be3d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import kafka.serializer.Decoder;
+import org.apache.beam.runners.core.AssignWindowsDoFn;
 import org.apache.beam.runners.spark.io.ConsoleIO;
 import org.apache.beam.runners.spark.io.CreateStream;
 import org.apache.beam.runners.spark.io.KafkaIO;
@@ -49,7 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AssignWindowsDoFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionList;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
index cd50408..3c01690 100644
--- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
@@ -110,7 +110,7 @@
     <!--[BEAM-419] Non-transient non-serializable instance field in serializable class-->
   </Match>
   <Match>
-    <Class name="org.apache.beam.sdk.util.WatermarkHold"/>
+    <Class name="org.apache.beam.runners.core.WatermarkHold"/>
     <Field name="timerInternals"/>
     <Bug pattern="SE_BAD_FIELD"/>
     <!--[BEAM-420] Non-transient non-serializable instance field in serializable class-->

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
index fde90af..72524bd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java
@@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.CoderException;
 /**
  * Coder for the BitSet used to track child-trigger finished states.
  */
-class BitSetCoder extends AtomicCoder<BitSet> {
+public class BitSetCoder extends AtomicCoder<BitSet> {
 
   private static final BitSetCoder INSTANCE = new BitSetCoder();
   private static final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of();


[36/50] [abbrv] incubator-beam git commit: [BEAM-242] Enable checkstyle and fix checkstyle errors in Flink runner

Posted by dh...@apache.org.
[BEAM-242] Enable checkstyle and fix checkstyle errors in Flink runner


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

Branch: refs/heads/gearpump-runner
Commit: 5eb44aa01157ca62f1a618d1738eb064ca3a10e4
Parents: 9ae5cc7
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Thu Aug 25 16:19:54 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 runners/flink/runner/pom.xml                    |   2 -
 .../FlinkPipelineExecutionEnvironment.java      |  17 +-
 .../runners/flink/FlinkPipelineOptions.java     |  27 ++--
 .../runners/flink/FlinkRunnerRegistrar.java     |   6 +
 .../beam/runners/flink/FlinkRunnerResult.java   |  17 +-
 .../beam/runners/flink/TestFlinkRunner.java     |   8 +-
 .../apache/beam/runners/flink/package-info.java |  22 +++
 .../FlinkBatchPipelineTranslator.java           |  15 +-
 .../FlinkBatchTranslationContext.java           |  10 +-
 .../translation/FlinkPipelineTranslator.java    |   2 +-
 .../FlinkStreamingTransformTranslators.java     |   5 +-
 .../flink/translation/TranslationMode.java      |   8 +-
 .../translation/functions/package-info.java     |  22 +++
 .../runners/flink/translation/package-info.java |  22 +++
 .../translation/types/CoderTypeSerializer.java  |   2 +-
 .../types/EncodedValueSerializer.java           | 162 ++++++++++---------
 .../flink/translation/types/package-info.java   |  22 +++
 .../utils/SerializedPipelineOptions.java        |   2 +-
 .../flink/translation/utils/package-info.java   |  22 +++
 .../wrappers/DataOutputViewWrapper.java         |   2 +-
 .../translation/wrappers/package-info.java      |  22 +++
 .../wrappers/streaming/DoFnOperator.java        |  12 +-
 .../streaming/SingletonKeyedWorkItem.java       |   5 +
 .../streaming/SingletonKeyedWorkItemCoder.java  |  14 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   2 +-
 .../wrappers/streaming/WorkItemKeySelector.java |   3 +-
 .../streaming/io/UnboundedFlinkSink.java        |  13 +-
 .../streaming/io/UnboundedFlinkSource.java      |  29 ++--
 .../streaming/io/UnboundedSocketSource.java     |  46 ++++--
 .../wrappers/streaming/io/package-info.java     |  22 +++
 .../wrappers/streaming/package-info.java        |  22 +++
 .../beam/runners/flink/PipelineOptionsTest.java |   3 +
 .../beam/runners/flink/WriteSinkITCase.java     |   3 +-
 .../apache/beam/runners/flink/package-info.java |  22 +++
 .../streaming/FlinkStateInternalsTest.java      |   3 +-
 .../flink/streaming/GroupByNullKeyTest.java     |   6 +
 .../streaming/TopWikipediaSessionsITCase.java   |   2 +-
 .../streaming/UnboundedSourceWrapperTest.java   |  33 ++--
 .../runners/flink/streaming/package-info.java   |  22 +++
 39 files changed, 490 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 08adc60..7c32280 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -234,12 +234,10 @@
         </executions>
       </plugin>
 
-      <!-- Checkstyle errors for now
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
-      -->
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
index d1977a4..a5d33b4 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -37,14 +37,15 @@ import org.slf4j.LoggerFactory;
 /**
  * The class that instantiates and manages the execution of a given job.
  * Depending on if the job is a Streaming or Batch processing one, it creates
- * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}),
- * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or
- * {@link FlinkStreamingPipelineTranslator}) to transform the Beam job into a Flink one, and
- * executes the (translated) job.
+ * the adequate execution environment ({@link ExecutionEnvironment}
+ * or {@link StreamExecutionEnvironment}), the necessary {@link FlinkPipelineTranslator}
+ * ({@link FlinkBatchPipelineTranslator} or {@link FlinkStreamingPipelineTranslator}) to
+ * transform the Beam job into a Flink one, and executes the (translated) job.
  */
 public class FlinkPipelineExecutionEnvironment {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
 
   private final FlinkPipelineOptions options;
 
@@ -79,8 +80,8 @@ public class FlinkPipelineExecutionEnvironment {
    * Depending on if the job is a Streaming or a Batch one, this method creates
    * the necessary execution environment and pipeline translator, and translates
    * the {@link org.apache.beam.sdk.values.PCollection} program into
-   * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
-   * one.
+   * a {@link org.apache.flink.api.java.DataSet}
+   * or {@link org.apache.flink.streaming.api.datastream.DataStream} one.
    * */
   public void translate(Pipeline pipeline) {
     this.flinkBatchEnv = null;
@@ -213,7 +214,7 @@ public class FlinkPipelineExecutionEnvironment {
     // If the value is not -1, then the validity checks are applied.
     // By default, checkpointing is disabled.
     long checkpointInterval = options.getCheckpointingInterval();
-    if(checkpointInterval != -1) {
+    if (checkpointInterval != -1) {
       if (checkpointInterval < 1) {
         throw new IllegalArgumentException("The checkpoint interval must be positive");
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index 6561fa5..1fb23ec 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -29,7 +29,8 @@ import org.apache.beam.sdk.options.StreamingOptions;
 /**
  * Options which can be used to configure a Flink PipelineRunner.
  */
-public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
+public interface FlinkPipelineOptions
+    extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
 
   /**
    * List of local files to make available to workers.
@@ -38,8 +39,8 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp
    * <p>
    * The default value is the list of jars from the main program's classpath.
    */
-  @Description("Jar-Files to send to all workers and put on the classpath. " +
-      "The default value is all files from the classpath.")
+  @Description("Jar-Files to send to all workers and put on the classpath. "
+      + "The default value is all files from the classpath.")
   @JsonIgnore
   List<String> getFilesToStage();
   void setFilesToStage(List<String> value);
@@ -51,9 +52,9 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp
    * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
    * "[auto]" will let the system decide where to execute the pipeline based on the environment.
    */
-  @Description("Address of the Flink Master where the Pipeline should be executed. Can" +
-      " either be of the form \"host:port\" or one of the special values [local], " +
-      "[collection] or [auto].")
+  @Description("Address of the Flink Master where the Pipeline should be executed. Can"
+      + " either be of the form \"host:port\" or one of the special values [local], "
+      + "[collection] or [auto].")
   String getFlinkMaster();
   void setFlinkMaster(String value);
 
@@ -62,21 +63,23 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp
   Integer getParallelism();
   void setParallelism(Integer value);
 
-  @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
-      "fault tolerance).")
+  @Description("The interval between consecutive checkpoints (i.e. snapshots of the current"
+      + "pipeline state used for fault tolerance).")
   @Default.Long(-1L)
   Long getCheckpointingInterval();
   void setCheckpointingInterval(Long interval);
 
-  @Description("Sets the number of times that failed tasks are re-executed. " +
-      "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
-      "that the system default value (as defined in the configuration) should be used.")
+  @Description("Sets the number of times that failed tasks are re-executed. "
+      + "A value of zero effectively disables fault tolerance. A value of -1 indicates "
+      + "that the system default value (as defined in the configuration) should be used.")
   @Default.Integer(-1)
   Integer getNumberOfExecutionRetries();
   void setNumberOfExecutionRetries(Integer retries);
 
-  @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
+  @Description("Sets the delay between executions. A value of {@code -1} "
+      + "indicates that the default value should be used.")
   @Default.Long(-1L)
   Long getExecutionRetryDelay();
   void setExecutionRetryDelay(Long delay);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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 f328279..0e4b513 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
@@ -36,6 +36,9 @@ import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 public class FlinkRunnerRegistrar {
   private FlinkRunnerRegistrar() { }
 
+  /**
+   * Pipeline runner registrar.
+   */
   @AutoService(PipelineRunnerRegistrar.class)
   public static class Runner implements PipelineRunnerRegistrar {
     @Override
@@ -46,6 +49,9 @@ public class FlinkRunnerRegistrar {
     }
   }
 
+  /**
+   * Pipeline options registrar.
+   */
   @AutoService(PipelineOptionsRegistrar.class)
   public static class Options implements PipelineOptionsRegistrar {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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 dd0733a..90bb64d 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
@@ -35,9 +35,9 @@ public class FlinkRunnerResult implements PipelineResult {
   private final Map<String, Object> aggregators;
   private final long runtime;
   public FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
-    this.aggregators = (aggregators == null || aggregators.isEmpty()) ?
-        Collections.<String, Object>emptyMap() :
-        Collections.unmodifiableMap(aggregators);
+    this.aggregators = (aggregators == null || aggregators.isEmpty())
+        ? Collections.<String, Object>emptyMap()
+        : Collections.unmodifiableMap(aggregators);
     this.runtime = runtime;
   }
 
@@ -47,7 +47,8 @@ public class FlinkRunnerResult implements PipelineResult {
   }
 
   @Override
-  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator) throws AggregatorRetrievalException {
+  public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
+      throws AggregatorRetrievalException {
     // TODO provide a list of all accumulator step values
     Object value = aggregators.get(aggregator.getName());
     if (value != null) {
@@ -65,10 +66,10 @@ public class FlinkRunnerResult implements PipelineResult {
 
   @Override
   public String toString() {
-    return "FlinkRunnerResult{" +
-        "aggregators=" + aggregators +
-        ", runtime=" + runtime +
-        '}';
+    return "FlinkRunnerResult{"
+        + "aggregators=" + aggregators
+        + ", runtime=" + runtime
+        + '}';
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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
index dd231d6..67a7d38 100644
--- 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
@@ -26,6 +26,9 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 
+/**
+ * Test Flink runner.
+ */
 public class TestFlinkRunner extends PipelineRunner<FlinkRunnerResult> {
 
   private FlinkRunner delegate;
@@ -37,7 +40,8 @@ public class TestFlinkRunner extends PipelineRunner<FlinkRunnerResult> {
   }
 
   public static TestFlinkRunner fromOptions(PipelineOptions options) {
-    FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+    FlinkPipelineOptions flinkOptions =
+        PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
     return new TestFlinkRunner(flinkOptions);
   }
 
@@ -50,7 +54,7 @@ public class TestFlinkRunner extends PipelineRunner<FlinkRunnerResult> {
 
   @Override
   public <OutputT extends POutput, InputT extends PInput>
-      OutputT apply(PTransform<InputT,OutputT> transform, InputT input) {
+      OutputT apply(PTransform<InputT, OutputT> transform, InputT input) {
     return delegate.apply(transform, input);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
new file mode 100644
index 0000000..57f1e59
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
index 66c48b0..aa38bfb 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -91,15 +91,20 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
     // get the transformation corresponding to the node we are
     // currently visiting and translate it into its Flink alternative.
     PTransform<?, ?> transform = node.getTransform();
-    BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+    BatchTransformTranslator<?> translator =
+        FlinkBatchTransformTranslators.getTranslator(transform);
     if (translator == null) {
       LOG.info(node.getTransform().getClass().toString());
-      throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+      throw new UnsupportedOperationException("The transform " + transform
+          + " is currently not supported.");
     }
     applyBatchTransform(transform, node, translator);
   }
 
-  private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
+  private <T extends PTransform<?, ?>> void applyBatchTransform(
+      PTransform<?, ?> transform,
+      TransformTreeNode node,
+      BatchTransformTranslator<?> translator) {
 
     @SuppressWarnings("unchecked")
     T typedTransform = (T) transform;
@@ -116,8 +121,8 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
   /**
    * A translator of a {@link PTransform}.
    */
-  public interface BatchTransformTranslator<Type extends PTransform> {
-    void translateNode(Type transform, FlinkBatchTranslationContext context);
+  public interface BatchTransformTranslator<TransformT extends PTransform> {
+    void translateNode(TransformT transform, FlinkBatchTranslationContext context);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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 835648e..611f5e6 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
@@ -40,7 +40,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  * {@link FlinkBatchTransformTranslators}.
  */
 public class FlinkBatchTranslationContext {
-  
+
   private final Map<PValue, DataSet<?>> dataSets;
   private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
 
@@ -55,9 +55,9 @@ public class FlinkBatchTranslationContext {
   private final PipelineOptions options;
 
   private AppliedPTransform<?, ?, ?> currentTransform;
-  
+
   // ------------------------------------------------------------------------
-  
+
   public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
     this.env = env;
     this.options = options;
@@ -66,7 +66,7 @@ public class FlinkBatchTranslationContext {
 
     this.danglingDataSets = new HashMap<>();
   }
-  
+
   // ------------------------------------------------------------------------
 
   public Map<PValue, DataSet<?>> getDanglingDataSets() {
@@ -80,7 +80,7 @@ public class FlinkBatchTranslationContext {
   public PipelineOptions getPipelineOptions() {
     return options;
   }
-  
+
   @SuppressWarnings("unchecked")
   public <T> DataSet<WindowedValue<T>> getInputDataSet(PValue value) {
     // assume that the DataSet is used as an input if retrieved here

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
index 4db929b..cba28e4 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
@@ -39,7 +39,7 @@ public abstract class FlinkPipelineTranslator extends Pipeline.PipelineVisitor.D
   }
 
   /**
-   * Utility formatting method
+   * Utility formatting method.
    * @param n number of spaces to generate
    * @return String with "|" followed by n spaces
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
index 3719fa8..4b819b7 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
@@ -18,9 +18,10 @@
 
 package org.apache.beam.runners.flink.translation;
 
-import com.google.api.client.util.Maps;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -29,6 +30,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.flink.FlinkRunner;
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.types.FlinkCoder;
@@ -63,7 +65,6 @@ 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.Reshuffle;
-import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java
index 71eb655..57b69aa 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java
@@ -18,14 +18,14 @@
 package org.apache.beam.runners.flink.translation;
 
 /**
- * The translation mode of the Beam Pipeline
+ * The translation mode of the Beam Pipeline.
  */
 public enum TranslationMode {
 
-  /** Uses the batch mode of Flink */
+  /** Uses the batch mode of Flink. */
   BATCH,
 
-  /** Uses the streaming mode of Flink */
+  /** Uses the streaming mode of Flink. */
   STREAMING
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
new file mode 100644
index 0000000..9f11212
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.functions;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
new file mode 100644
index 0000000..af4b354
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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 4eda357..e210ed9 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
@@ -33,7 +33,7 @@ import org.apache.flink.core.memory.DataOutputView;
  * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}.
  */
 public class CoderTypeSerializer<T> extends TypeSerializer<T> {
-  
+
   private Coder<T> coder;
 
   public CoderTypeSerializer(Coder<T> coder) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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
index f3e667d..41db61e 100644
--- 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
@@ -18,7 +18,9 @@
 package org.apache.beam.runners.flink.translation.types;
 
 import java.io.IOException;
+
 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;
@@ -28,84 +30,84 @@ import org.apache.flink.core.memory.DataOutputView;
  */
 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;
-	}
+  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/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
new file mode 100644
index 0000000..6fb3182
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.types;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
index 0c6cea8..fe2602b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
@@ -33,7 +33,7 @@ public class SerializedPipelineOptions implements Serializable {
 
   private final byte[] serializedOptions;
 
-  /** Lazily initialized copy of deserialized options */
+  /** Lazily initialized copy of deserialized options. */
   private transient PipelineOptions pipelineOptions;
 
   public SerializedPipelineOptions(PipelineOptions options) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
new file mode 100644
index 0000000..5dedd53
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.utils;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
index 2cb9b18..f2d9db2 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
@@ -28,7 +28,7 @@ import org.apache.flink.core.memory.DataOutputView;
  * {@link java.io.OutputStream}.
  */
 public class DataOutputViewWrapper extends OutputStream {
-  
+
   private DataOutputView outputView;
 
   public DataOutputViewWrapper(DataOutputView outputView) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
new file mode 100644
index 0000000..72f7deb
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index 3b0fccc..3b917e2 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -27,6 +27,9 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
@@ -40,11 +43,8 @@ import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.runners.core.DoFnRunner;
-import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.NullSideInputReader;
-import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -79,8 +79,8 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
  *
  * @param <InputT> the input type of the {@link OldDoFn}
  * @param <FnOutputT> the output type of the {@link OldDoFn}
- * @param <OutputT> the output type of the operator, this can be different from the fn output type when we have
- *                 side outputs
+ * @param <OutputT> the output type of the operator, this can be different from the fn output
+ *                 type when we have side outputs
  */
 public class DoFnOperator<InputT, FnOutputT, OutputT>
     extends AbstractStreamOperator<OutputT>
@@ -166,7 +166,7 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
     currentInputWatermark = Long.MIN_VALUE;
     currentOutputWatermark = currentInputWatermark;
 
-   	Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() {
+    Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() {
       @Override
       public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
           Class<?> fnClass,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
index 5751aac..6d2582b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java
@@ -22,6 +22,11 @@ import org.apache.beam.sdk.util.KeyedWorkItem;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 
+/**
+ * Singleton keyed word item.
+ * @param <K>
+ * @param <ElemT>
+ */
 public class SingletonKeyedWorkItem<K, ElemT> implements KeyedWorkItem<K, ElemT> {
 
   final K key;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
index 5e583e9..37454a3 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java
@@ -35,7 +35,13 @@ import org.apache.beam.sdk.util.KeyedWorkItemCoder;
 import org.apache.beam.sdk.util.PropertyNames;
 import org.apache.beam.sdk.util.WindowedValue;
 
-public class SingletonKeyedWorkItemCoder<K, ElemT> extends StandardCoder<SingletonKeyedWorkItem<K, ElemT>> {
+/**
+ * Singleton keyed word iteam coder.
+ * @param <K>
+ * @param <ElemT>
+ */
+public class SingletonKeyedWorkItemCoder<K, ElemT>
+    extends StandardCoder<SingletonKeyedWorkItem<K, ElemT>> {
   /**
    * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window
    * coder.
@@ -68,7 +74,7 @@ public class SingletonKeyedWorkItemCoder<K, ElemT> extends StandardCoder<Singlet
     this.keyCoder = keyCoder;
     this.elemCoder = elemCoder;
     this.windowCoder = windowCoder;
-    valueCoder= WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder);
+    valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder);
   }
 
   public Coder<K> getKeyCoder() {
@@ -80,7 +86,9 @@ public class SingletonKeyedWorkItemCoder<K, ElemT> extends StandardCoder<Singlet
   }
 
   @Override
-  public void encode(SingletonKeyedWorkItem<K, ElemT> value, OutputStream outStream, Context context)
+  public void encode(SingletonKeyedWorkItem<K, ElemT> value,
+                     OutputStream outStream,
+                     Context context)
       throws CoderException, IOException {
     Context nestedContext = context.nested();
     keyCoder.encode(value.key(), outStream, nestedContext);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index b893116..29ae6ae 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -34,6 +34,7 @@ import java.util.Queue;
 import java.util.Set;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.KeyedWorkItem;
 import org.apache.beam.sdk.util.KeyedWorkItems;
-import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
index 51d9e0c..7829163 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java
@@ -33,7 +33,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
  * that all key comparisons/hashing happen on the encoded form.
  */
 public class WorkItemKeySelector<K, V>
-    implements KeySelector<WindowedValue<SingletonKeyedWorkItem<K, V>>, ByteBuffer>, ResultTypeQueryable<ByteBuffer> {
+    implements KeySelector<WindowedValue<SingletonKeyedWorkItem<K, V>>, ByteBuffer>,
+    ResultTypeQueryable<ByteBuffer> {
 
   private final Coder<K> keyCoder;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
index 2117e9d..5b01796 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
@@ -62,7 +62,8 @@ public class UnboundedFlinkSink<T> extends Sink<T> {
       }
 
       @Override
-      public void finalize(Iterable<Object> writerResults, PipelineOptions options) throws Exception {
+      public void finalize(Iterable<Object> writerResults, PipelineOptions options)
+          throws Exception {
 
       }
 
@@ -70,12 +71,14 @@ public class UnboundedFlinkSink<T> extends Sink<T> {
       public Coder<Object> getWriterResultCoder() {
         return new Coder<Object>() {
           @Override
-          public void encode(Object value, OutputStream outStream, Context context) throws CoderException, IOException {
+          public void encode(Object value, OutputStream outStream, Context context)
+              throws CoderException, IOException {
 
           }
 
           @Override
-          public Object decode(InputStream inStream, Context context) throws CoderException, IOException {
+          public Object decode(InputStream inStream, Context context)
+              throws CoderException, IOException {
             return null;
           }
 
@@ -110,7 +113,9 @@ public class UnboundedFlinkSink<T> extends Sink<T> {
           }
 
           @Override
-          public void registerByteSizeObserver(Object value, ElementByteSizeObserver observer, Context context) throws Exception {
+          public void registerByteSizeObserver(Object value,
+                                               ElementByteSizeObserver observer,
+                                               Context context) throws Exception {
 
           }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/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 c6e0825..ac20c34 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
@@ -36,17 +36,19 @@ public class UnboundedFlinkSource<T> extends UnboundedSource<T, UnboundedSource.
 
   private final SourceFunction<T> flinkSource;
 
-  /** Coder set during translation */
+  /** Coder set during translation. */
   private Coder<T> coder;
 
-  /** Timestamp / watermark assigner for source; defaults to ingestion time */
-  private AssignerWithPeriodicWatermarks<T> flinkTimestampAssigner = new IngestionTimeExtractor<T>();
+  /** Timestamp / watermark assigner for source; defaults to ingestion time. */
+  private AssignerWithPeriodicWatermarks<T> flinkTimestampAssigner =
+      new IngestionTimeExtractor<T>();
 
   public UnboundedFlinkSource(SourceFunction<T> source) {
     flinkSource = checkNotNull(source);
   }
 
-  public UnboundedFlinkSource(SourceFunction<T> source, AssignerWithPeriodicWatermarks<T> timestampAssigner) {
+  public UnboundedFlinkSource(SourceFunction<T> source,
+                              AssignerWithPeriodicWatermarks<T> timestampAssigner) {
     flinkSource = checkNotNull(source);
     flinkTimestampAssigner = checkNotNull(timestampAssigner);
   }
@@ -60,19 +62,25 @@ 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 FlinkRunner.");
+  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 FlinkRunner.");
   }
 
   @Override
-  public UnboundedReader<T> createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) {
-    throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner.");
+  public UnboundedReader<T> createReader(PipelineOptions options,
+                                         @Nullable CheckpointMark checkpointMark) {
+    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 FlinkRunner.");
+    throw new RuntimeException("Flink Sources are supported only when "
+        + "running with the FlinkRunner.");
   }
 
 
@@ -100,7 +108,8 @@ public class UnboundedFlinkSource<T> extends UnboundedSource<T, UnboundedSource.
    * @param <T> The type that the source function produces.
    * @return The wrapped source function.
    */
-  public static <T> UnboundedSource<T, UnboundedSource.CheckpointMark> of(SourceFunction<T> flinkSource) {
+  public static <T> UnboundedSource<T, UnboundedSource.CheckpointMark> of(
+      SourceFunction<T> flinkSource) {
     return new UnboundedFlinkSource<>(flinkSource);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
index 8d37fe7..96b5138 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -38,9 +38,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging.
+ * An example unbounded Beam source that reads input from a socket.
+ * This is used mainly for testing and debugging.
  * */
-public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
+public class UnboundedSocketSource<CheckpointMarkT extends UnboundedSource.CheckpointMark>
+    extends UnboundedSource<String, CheckpointMarkT> {
 
   private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
 
@@ -60,7 +62,11 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
     this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
   }
 
-  public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
+  public UnboundedSocketSource(String hostname,
+                               int port,
+                               char delimiter,
+                               long maxNumRetries,
+                               long delayBetweenRetries) {
     this.hostname = hostname;
     this.port = port;
     this.delimiter = delimiter;
@@ -89,12 +95,15 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
   }
 
   @Override
-  public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
-    return Collections.<UnboundedSource<String, C>>singletonList(this);
+  public List<? extends UnboundedSource<String, CheckpointMarkT>> generateInitialSplits(
+      int desiredNumSplits,
+      PipelineOptions options) throws Exception {
+    return Collections.<UnboundedSource<String, CheckpointMarkT>>singletonList(this);
   }
 
   @Override
-  public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+  public UnboundedReader<String> createReader(PipelineOptions options,
+                                              @Nullable CheckpointMarkT checkpointMark) {
     return new UnboundedSocketReader(this);
   }
 
@@ -109,7 +118,8 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
   @Override
   public void validate() {
     checkArgument(port > 0 && port < 65536, "port is out of range");
-    checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
+    checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), "
+        + "or -1 (infinite retries)");
     checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
   }
 
@@ -118,7 +128,11 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
     return DEFAULT_SOCKET_CODER;
   }
 
-  public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
+  /**
+   * Unbounded socket reader.
+   */
+  public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String>
+      implements Serializable {
 
     private static final long serialVersionUID = 7526472295622776147L;
     private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
@@ -138,7 +152,8 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
 
     private void openConnection() throws IOException {
       this.socket = new Socket();
-      this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
+      this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()),
+          CONNECTION_TIMEOUT_TIME);
       this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
       this.isRunning = true;
     }
@@ -149,11 +164,14 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
       while (!isRunning) {
         try {
           openConnection();
-          LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
+          LOG.info("Connected to server socket " + this.source.getHostname() + ':'
+              + this.source.getPort());
 
           return advance();
         } catch (IOException e) {
-          LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
+          LOG.info("Lost connection to server socket " + this.source.getHostname() + ':'
+              + this.source.getPort() + ". Retrying in "
+              + this.source.getDelayBetweenRetries() + " msecs...");
 
           if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
             try {
@@ -167,7 +185,8 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
           }
         }
       }
-      LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
+      LOG.error("Unable to connect to host " + this.source.getHostname()
+          + " : " + this.source.getPort());
       return false;
     }
 
@@ -211,7 +230,8 @@ public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> ext
       this.reader.close();
       this.socket.close();
       this.isRunning = false;
-      LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
+      LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":"
+          + this.source.getPort() + ".");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
new file mode 100644
index 0000000..b431ce7
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
new file mode 100644
index 0000000..0674871
--- /dev/null
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.translation.wrappers.streaming;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
index 32339dc..3c30fed 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
@@ -52,6 +52,9 @@ import org.junit.Test;
  */
 public class PipelineOptionsTest {
 
+  /**
+   * Pipeline options.
+   */
   public interface MyOptions extends FlinkPipelineOptions {
     @Description("Bla bla bla")
     @Default.String("Hello")

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
index 0988146..37eedb2 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
@@ -118,7 +118,8 @@ public class WriteSinkITCase extends JavaProgramTestBase {
       }
 
       @Override
-      public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
+      public void finalize(Iterable<String> writerResults, PipelineOptions options)
+          throws Exception {
 
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
new file mode 100644
index 0000000..57f1e59
--- /dev/null
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
index 711ae00..628212a 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
@@ -56,7 +56,8 @@ import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /**
- * Tests for {@link FlinkStateInternals}. This is based on the tests for {@code InMemoryStateInternals}.
+ * Tests for {@link FlinkStateInternals}. This is based on the tests for
+ * {@code InMemoryStateInternals}.
  */
 @RunWith(JUnit4.class)
 public class FlinkStateInternalsTest {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
index ab98c27..c6381ee 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -36,6 +36,9 @@ import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
+/**
+ * Test for GroupByNullKey.
+ */
 public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
 
 
@@ -58,6 +61,9 @@ public class GroupByNullKeyTest extends StreamingProgramTestBase implements Seri
     compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
   }
 
+  /**
+   * DoFn extracting user and timestamp.
+   */
   public static class ExtractUserAndTimestamp extends OldDoFn<KV<Integer, String>, String> {
     private static final long serialVersionUID = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
index 64f978f..9410481 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
@@ -38,7 +38,7 @@ import org.joda.time.Instant;
 
 
 /**
- * Session window test
+ * Session window test.
  */
 public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
   protected String resultPath;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
index a70ad49..73124a9 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java
@@ -56,14 +56,14 @@ public class UnboundedSourceWrapperTest {
    */
   @Test
   public void testWithOneReader() throws Exception {
-    final int NUM_ELEMENTS = 20;
+    final int numElements = 20;
     final Object checkpointLock = new Object();
     PipelineOptions options = PipelineOptionsFactory.create();
 
     // this source will emit exactly NUM_ELEMENTS across all parallel readers,
     // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
     // elements later.
-    TestCountingSource source = new TestCountingSource(NUM_ELEMENTS);
+    TestCountingSource source = new TestCountingSource(numElements);
     UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
         new UnboundedSourceWrapper<>(options, source, 1);
 
@@ -92,7 +92,7 @@ public class UnboundedSourceWrapperTest {
                 StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
 
               count++;
-              if (count >= NUM_ELEMENTS) {
+              if (count >= numElements) {
                 throw new SuccessException();
               }
             }
@@ -116,14 +116,14 @@ public class UnboundedSourceWrapperTest {
    */
   @Test
   public void testWithMultipleReaders() throws Exception {
-    final int NUM_ELEMENTS = 20;
+    final int numElements = 20;
     final Object checkpointLock = new Object();
     PipelineOptions options = PipelineOptionsFactory.create();
 
     // this source will emit exactly NUM_ELEMENTS across all parallel readers,
     // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
     // elements later.
-    TestCountingSource source = new TestCountingSource(NUM_ELEMENTS);
+    TestCountingSource source = new TestCountingSource(numElements);
     UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
         new UnboundedSourceWrapper<>(options, source, 4);
 
@@ -149,10 +149,10 @@ public class UnboundedSourceWrapperTest {
 
             @Override
             public void collect(
-                StreamRecord<WindowedValue<KV<Integer,Integer>>> windowedValueStreamRecord) {
+                StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
 
               count++;
-              if (count >= NUM_ELEMENTS) {
+              if (count >= numElements) {
                 throw new SuccessException();
               }
             }
@@ -177,14 +177,14 @@ public class UnboundedSourceWrapperTest {
    */
   @Test
   public void testRestore() throws Exception {
-    final int NUM_ELEMENTS = 20;
+    final int numElements = 20;
     final Object checkpointLock = new Object();
     PipelineOptions options = PipelineOptionsFactory.create();
 
     // this source will emit exactly NUM_ELEMENTS across all parallel readers,
     // afterwards it will stall. We check whether we also receive NUM_ELEMENTS
     // elements later.
-    TestCountingSource source = new TestCountingSource(NUM_ELEMENTS);
+    TestCountingSource source = new TestCountingSource(numElements);
     UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> flinkWrapper =
         new UnboundedSourceWrapper<>(options, source, 1);
 
@@ -213,11 +213,11 @@ public class UnboundedSourceWrapperTest {
 
             @Override
             public void collect(
-                StreamRecord<WindowedValue<KV<Integer,Integer>>> windowedValueStreamRecord) {
+                StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
 
               emittedElements.add(windowedValueStreamRecord.getValue().getValue());
               count++;
-              if (count >= NUM_ELEMENTS / 2) {
+              if (count >= numElements / 2) {
                 throw new SuccessException();
               }
             }
@@ -238,7 +238,7 @@ public class UnboundedSourceWrapperTest {
     byte[] snapshot = flinkWrapper.snapshotState(0, 0);
 
     // create a completely new source but restore from the snapshot
-    TestCountingSource restoredSource = new TestCountingSource(NUM_ELEMENTS);
+    TestCountingSource restoredSource = new TestCountingSource(numElements);
     UnboundedSourceWrapper<
         KV<Integer, Integer>, TestCountingSource.CounterMark> restoredFlinkWrapper =
         new UnboundedSourceWrapper<>(options, restoredSource, 1);
@@ -271,10 +271,10 @@ public class UnboundedSourceWrapperTest {
 
             @Override
             public void collect(
-                StreamRecord<WindowedValue<KV<Integer,Integer>>> windowedValueStreamRecord) {
+                StreamRecord<WindowedValue<KV<Integer, Integer>>> windowedValueStreamRecord) {
               emittedElements.add(windowedValueStreamRecord.getValue().getValue());
               count++;
-              if (count >= NUM_ELEMENTS / 2) {
+              if (count >= numElements / 2) {
                 throw new SuccessException();
               }
             }
@@ -292,7 +292,7 @@ public class UnboundedSourceWrapperTest {
     assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements);
 
     // verify that we saw all NUM_ELEMENTS elements
-    assertTrue(emittedElements.size() == NUM_ELEMENTS);
+    assertTrue(emittedElements.size() == numElements);
   }
 
   @SuppressWarnings("unchecked")
@@ -310,7 +310,8 @@ public class UnboundedSourceWrapperTest {
     when(mockTask.getConfiguration()).thenReturn(cfg);
     when(mockTask.getEnvironment()).thenReturn(env);
     when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
-    when(mockTask.getAccumulatorMap()).thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap());
+    when(mockTask.getAccumulatorMap())
+        .thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap());
 
     operator.setup(mockTask, cfg, (Output< StreamRecord<T>>) mock(Output.class));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eb44aa0/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
new file mode 100644
index 0000000..08a1e03
--- /dev/null
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Internal implementation of the Beam runner for Apache Flink.
+ */
+package org.apache.beam.runners.flink.streaming;


[27/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
deleted file mode 100644
index f8ad291..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.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.sdk.util;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.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.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-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;
-
-/**
- * Tests for {@link PushbackSideInputDoFnRunner}.
- */
-@RunWith(JUnit4.class)
-public class PushbackSideInputDoFnRunnerTest {
-  @Mock private ReadyCheckingSideInputReader reader;
-  private TestDoFnRunner<Integer, Integer> underlying;
-  private PCollectionView<Integer> singletonView;
-
-  @Before
-  public void setup() {
-    MockitoAnnotations.initMocks(this);
-    TestPipeline p = TestPipeline.create();
-    PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
-    singletonView =
-        created
-            .apply(Window.into(new IdentitySideInputWindowFn()))
-            .apply(Sum.integersGlobally().asSingletonView());
-
-    underlying = new TestDoFnRunner<>();
-  }
-
-  private PushbackSideInputDoFnRunner<Integer, Integer> createRunner(
-      ImmutableList<PCollectionView<?>> views) {
-    PushbackSideInputDoFnRunner<Integer, Integer> runner =
-        PushbackSideInputDoFnRunner.create(underlying, views, reader);
-    runner.startBundle();
-    return runner;
-  }
-
-  @Test
-  public void startFinishBundleDelegates() {
-    PushbackSideInputDoFnRunner runner =
-        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
-    assertThat(underlying.started, is(true));
-    assertThat(underlying.finished, is(false));
-    runner.finishBundle();
-    assertThat(underlying.finished, is(true));
-  }
-
-  @Test
-  public void processElementSideInputNotReady() {
-    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
-        .thenReturn(false);
-
-    PushbackSideInputDoFnRunner<Integer, Integer> runner =
-        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
-    WindowedValue<Integer> oneWindow =
-        WindowedValue.of(
-            2,
-            new Instant(-2),
-            new IntervalWindow(new Instant(-500L), new Instant(0L)),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    Iterable<WindowedValue<Integer>> oneWindowPushback =
-        runner.processElementInReadyWindows(oneWindow);
-    assertThat(oneWindowPushback, containsInAnyOrder(oneWindow));
-    assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
-  }
-
-  @Test
-  public void processElementSideInputNotReadyMultipleWindows() {
-    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
-        .thenReturn(false);
-
-    PushbackSideInputDoFnRunner<Integer, Integer> runner =
-        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
-    WindowedValue<Integer> multiWindow =
-        WindowedValue.of(
-            2,
-            new Instant(-2),
-            ImmutableList.of(
-                new IntervalWindow(new Instant(-500L), new Instant(0L)),
-                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
-                GlobalWindow.INSTANCE),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    Iterable<WindowedValue<Integer>> multiWindowPushback =
-        runner.processElementInReadyWindows(multiWindow);
-    assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows()));
-    assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
-  }
-
-  @Test
-  public void processElementSideInputNotReadySomeWindows() {
-    when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE)))
-        .thenReturn(false);
-    when(
-            reader.isReady(
-                Mockito.eq(singletonView),
-                org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE))))
-        .thenReturn(true);
-
-    PushbackSideInputDoFnRunner<Integer, Integer> runner =
-        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
-    IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L));
-    IntervalWindow bigWindow =
-        new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L));
-    WindowedValue<Integer> multiWindow =
-        WindowedValue.of(
-            2,
-            new Instant(-2),
-            ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE),
-            PaneInfo.NO_FIRING);
-    Iterable<WindowedValue<Integer>> multiWindowPushback =
-        runner.processElementInReadyWindows(multiWindow);
-    assertThat(
-        multiWindowPushback,
-        containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L))));
-    assertThat(underlying.inputElems,
-        containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING),
-            WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING)));
-  }
-
-  @Test
-  public void processElementSideInputReadyAllWindows() {
-    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
-        .thenReturn(true);
-
-    ImmutableList<PCollectionView<?>> views = ImmutableList.<PCollectionView<?>>of(singletonView);
-    PushbackSideInputDoFnRunner<Integer, Integer> runner = createRunner(views);
-
-    WindowedValue<Integer> multiWindow =
-        WindowedValue.of(
-            2,
-            new Instant(-2),
-            ImmutableList.of(
-                new IntervalWindow(new Instant(-500L), new Instant(0L)),
-                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
-                GlobalWindow.INSTANCE),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    Iterable<WindowedValue<Integer>> multiWindowPushback =
-        runner.processElementInReadyWindows(multiWindow);
-    assertThat(multiWindowPushback, emptyIterable());
-    assertThat(underlying.inputElems,
-        containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray()));
-  }
-
-  @Test
-  public void processElementNoSideInputs() {
-    PushbackSideInputDoFnRunner<Integer, Integer> runner =
-        createRunner(ImmutableList.<PCollectionView<?>>of());
-
-    WindowedValue<Integer> multiWindow =
-        WindowedValue.of(
-            2,
-            new Instant(-2),
-            ImmutableList.of(
-                new IntervalWindow(new Instant(-500L), new Instant(0L)),
-                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
-                GlobalWindow.INSTANCE),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING);
-    Iterable<WindowedValue<Integer>> multiWindowPushback =
-        runner.processElementInReadyWindows(multiWindow);
-    assertThat(multiWindowPushback, emptyIterable());
-    assertThat(underlying.inputElems, containsInAnyOrder(multiWindow));
-  }
-
-  private static class TestDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-    List<WindowedValue<InputT>> inputElems;
-    private boolean started = false;
-    private boolean finished = false;
-
-    @Override
-    public void startBundle() {
-      started = true;
-      inputElems = new ArrayList<>();
-    }
-
-    @Override
-    public void processElement(WindowedValue<InputT> elem) {
-      inputElems.add(elem);
-    }
-
-    @Override
-    public void finishBundle() {
-      finished = true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
deleted file mode 100644
index 647495c..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ /dev/null
@@ -1,1442 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static 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;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.withSettings;
-
-import com.google.common.collect.Iterables;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.beam.sdk.WindowMatchers;
-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.transforms.CombineWithContext.CombineFnWithContext;
-import org.apache.beam.sdk.transforms.CombineWithContext.Context;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.AfterEach;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.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;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-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;
-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.Matchers;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of
- * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger}
- * (sometimes mocked). They proceed by injecting elements and advancing watermark and
- * processing time, then verifying produced panes and counters.
- */
-@RunWith(JUnit4.class)
-public class ReduceFnRunnerTest {
-  @Mock private SideInputReader mockSideInputReader;
-  private Trigger mockTrigger;
-  private PCollectionView<Integer> mockView;
-
-  private IntervalWindow firstWindow;
-
-  private static Trigger.TriggerContext anyTriggerContext() {
-    return Mockito.<Trigger.TriggerContext>any();
-  }
-  private static Trigger.OnElementContext anyElementContext() {
-    return Mockito.<Trigger.OnElementContext>any();
-  }
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-
-    mockTrigger = mock(Trigger.class, withSettings().serializable());
-
-    @SuppressWarnings("unchecked")
-    PCollectionView<Integer> mockViewUnchecked =
-        mock(PCollectionView.class, withSettings().serializable());
-    mockView = mockViewUnchecked;
-    firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
-  }
-
-  private void injectElement(ReduceFnTester<Integer, ?, IntervalWindow> tester, int element)
-      throws Exception {
-    doNothing().when(mockTrigger).onElement(anyElementContext());
-    tester.injectElements(TimestampedValue.of(element, new Instant(element)));
-  }
-
-  private void triggerShouldFinish(Trigger mockTrigger) throws Exception {
-    doAnswer(new Answer<Void>() {
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Exception {
-        @SuppressWarnings("unchecked")
-        Trigger.TriggerContext context =
-            (Trigger.TriggerContext) invocation.getArguments()[0];
-        context.trigger().setFinished(true);
-        return null;
-      }
-    })
-    .when(mockTrigger).onFire(anyTriggerContext());
- }
-
-  @Test
-  public void testOnElementBufferingDiscarding() throws Exception {
-    // Test basic execution of a trigger using a non-combining window set and discarding mode.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Pane of {1, 2}
-    injectElement(tester, 1);
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 2);
-    assertThat(tester.extractOutput(),
-        contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
-    // Pane of just 3, and finish
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 3);
-    assertThat(tester.extractOutput(),
-            contains(isSingleWindowedValue(containsInAnyOrder(3), 3, 0, 10)));
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
-    // This element shouldn't be seen, because the trigger has finished
-    injectElement(tester, 4);
-
-    assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
-  }
-
-  @Test
-  public void testOnElementBufferingAccumulating() throws Exception {
-    // Test basic execution of a trigger using a non-combining window set and accumulating mode.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    injectElement(tester, 1);
-
-    // Fires {1, 2}
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 2);
-
-    // Fires {1, 2, 3} because we are in accumulating mode
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 3);
-
-    // This element shouldn't be seen, because the trigger has finished
-    injectElement(tester, 4);
-
-    assertThat(
-        tester.extractOutput(),
-        contains(
-            isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
-            isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 3, 0, 10)));
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-  }
-
-  @Test
-  public void testOnElementCombiningDiscarding() throws Exception {
-    // Test basic execution of a trigger using a non-combining window set and discarding mode.
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
-        FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.DISCARDING_FIRED_PANES,
-        new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of(), Duration.millis(100));
-
-    injectElement(tester, 2);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 3);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 4);
-
-    // This element shouldn't be seen, because the trigger has finished
-    injectElement(tester, 6);
-
-    assertThat(
-        tester.extractOutput(),
-        contains(
-            isSingleWindowedValue(equalTo(5), 2, 0, 10),
-            isSingleWindowedValue(equalTo(4), 4, 0, 10)));
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    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 BoundedWindow window() {
-                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()),
-            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.
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
-        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
-            VarIntCoder.of(), Duration.millis(100));
-
-    injectElement(tester, 1);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 2);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 3);
-
-    // This element shouldn't be seen, because the trigger has finished
-    injectElement(tester, 4);
-
-    assertThat(
-        tester.extractOutput(),
-        contains(
-            isSingleWindowedValue(equalTo(3), 1, 0, 10),
-            isSingleWindowedValue(equalTo(6), 3, 0, 10)));
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-  }
-
-  @Test
-  public void testOnElementCombiningWithContext() throws Exception {
-    Integer expectedValue = 5;
-    WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy
-        .of(FixedWindows.of(Duration.millis(10)))
-        .withTrigger(mockTrigger)
-        .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
-        .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-        .withAllowedLateness(Duration.millis(100));
-
-    TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
-    options.setValue(5);
-
-    when(mockSideInputReader.contains(Matchers.<PCollectionView<Integer>>any())).thenReturn(true);
-    when(mockSideInputReader.get(
-        Matchers.<PCollectionView<Integer>>any(), any(BoundedWindow.class))).thenReturn(5);
-
-    @SuppressWarnings({"rawtypes", "unchecked", "unused"})
-    Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal())
-        .thenReturn((WindowingStrategy) windowingStrategy);
-
-    SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue);
-    // Test basic execution of a trigger using a non-combining window set and discarding mode.
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
-        windowingStrategy, combineFn.<String>asKeyedFn(),
-        VarIntCoder.of(), options, mockSideInputReader);
-
-    injectElement(tester, 2);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 3);
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 4);
-
-    // This element shouldn't be seen, because the trigger has finished
-    injectElement(tester, 6);
-
-    assertThat(
-        tester.extractOutput(),
-        contains(
-            isSingleWindowedValue(equalTo(5), 2, 0, 10),
-            isSingleWindowedValue(equalTo(4), 4, 0, 10)));
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-  }
-
-  @Test
-  public void testWatermarkHoldAndLateData() throws Exception {
-    // Test handling of late data. Specifically, ensure the watermark hold is correct.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Input watermark -> null
-    assertEquals(null, tester.getWatermarkHold());
-    assertEquals(null, tester.getOutputWatermark());
-
-    // All on time data, verify watermark hold.
-    injectElement(tester, 1);
-    injectElement(tester, 3);
-    assertEquals(new Instant(1), tester.getWatermarkHold());
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 2);
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output, contains(
-        isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
-            1, // timestamp
-            0, // window start
-            10))); // window end
-    assertThat(output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
-
-    // Holding for the end-of-window transition.
-    assertEquals(new Instant(9), tester.getWatermarkHold());
-    // Nothing dropped.
-    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
-    // Input watermark -> 4, output watermark should advance that far as well
-    tester.advanceInputWatermark(new Instant(4));
-    assertEquals(new Instant(4), tester.getOutputWatermark());
-
-    // Some late, some on time. Verify that we only hold to the minimum of on-time.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(4));
-    injectElement(tester, 2);
-    injectElement(tester, 3);
-    assertEquals(new Instant(9), tester.getWatermarkHold());
-    injectElement(tester, 5);
-    assertEquals(new Instant(5), tester.getWatermarkHold());
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 4);
-    output = tester.extractOutput();
-    assertThat(output,
-        contains(
-            isSingleWindowedValue(containsInAnyOrder(
-                1, 2, 3, // earlier firing
-                2, 3, 4, 5), // new elements
-            4, // timestamp
-            0, // window start
-            10))); // window end
-    assertThat(output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)));
-
-    // All late -- output at end of window timestamp.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(8));
-    injectElement(tester, 6);
-    injectElement(tester, 5);
-    assertEquals(new Instant(9), tester.getWatermarkHold());
-    injectElement(tester, 4);
-
-    // Fire the ON_TIME pane
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.advanceInputWatermark(new Instant(10));
-
-    // Output time is end of the window, because all the new data was late, but the pane
-    // is the ON_TIME pane.
-    output = tester.extractOutput();
-    assertThat(output,
-        contains(isSingleWindowedValue(
-            containsInAnyOrder(1, 2, 3, // earlier firing
-                2, 3, 4, 5, // earlier firing
-                4, 5, 6), // new elements
-            9, // timestamp
-            0, // window start
-            10))); // window end
-    assertThat(output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0)));
-
-    // This is "pending" at the time the watermark makes it way-late.
-    // Because we're about to expire the window, we output it.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    injectElement(tester, 8);
-    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
-    // Exceed the GC limit, triggering the last pane to be fired
-    tester.advanceInputWatermark(new Instant(50));
-    output = tester.extractOutput();
-    // Output time is still end of the window, because the new data (8) was behind
-    // the output watermark.
-    assertThat(output,
-        contains(isSingleWindowedValue(
-            containsInAnyOrder(1, 2, 3, // earlier firing
-                2, 3, 4, 5, // earlier firing
-                4, 5, 6, // earlier firing
-                8), // new element prior to window becoming expired
-            9, // timestamp
-            0, // window start
-            10))); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1)));
-    assertEquals(new Instant(50), tester.getOutputWatermark());
-    assertEquals(null, tester.getWatermarkHold());
-
-    // Late timers are ignored
-    tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), new Instant(12),
-        TimeDomain.EVENT_TIME);
-
-    // And because we're past the end of window + allowed lateness, everything should be cleaned up.
-    assertFalse(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor();
-  }
-
-  @Test
-  public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception {
-    // Make sure holds are only set if they are accompanied by an end-of-window timer.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
-            ClosingBehavior.FIRE_ALWAYS);
-    tester.setAutoAdvanceOutputWatermark(false);
-
-    // Case: Unobservably late
-    tester.advanceInputWatermark(new Instant(15));
-    tester.advanceOutputWatermark(new Instant(11));
-    injectElement(tester, 14);
-    // Hold was applied, waiting for end-of-window timer.
-    assertEquals(new Instant(14), tester.getWatermarkHold());
-    assertEquals(new Instant(19), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
-    // Trigger the end-of-window timer.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.advanceInputWatermark(new Instant(20));
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    // Hold has been replaced with garbage collection hold. Waiting for garbage collection.
-    assertEquals(new Instant(29), tester.getWatermarkHold());
-    assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
-    // Case: Maybe late 1
-    injectElement(tester, 13);
-    // No change to hold or timers.
-    assertEquals(new Instant(29), tester.getWatermarkHold());
-    assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
-    // Trigger the garbage collection timer.
-    tester.advanceInputWatermark(new Instant(30));
-
-    // Everything should be cleaned up.
-    assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20))));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor();
-  }
-
-  @Test
-  public void testPaneInfoAllStates() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    tester.advanceInputWatermark(new Instant(0));
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 1);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY))));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 2);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.advanceInputWatermark(new Instant(15));
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 3);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    injectElement(tester, 4);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(false, false, Timing.LATE, 3, 1))));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 5);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 4, 2))));
-  }
-
-  @Test
-  public void testPaneInfoAllStatesAfterWatermark() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-            .withTrigger(Repeatedly.forever(AfterFirst.of(
-                AfterPane.elementCountAtLeast(2),
-                AfterWatermark.pastEndOfWindow())))
-            .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
-            .withAllowedLateness(Duration.millis(100))
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
-    tester.advanceInputWatermark(new Instant(50));
-
-    // We should get the ON_TIME pane even though it is empty,
-    // because we have an AfterWatermark.pastEndOfWindow() trigger.
-    output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
-
-    // We should get the final pane even though it is empty.
-    tester.advanceInputWatermark(new Instant(150));
-    output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
-  }
-
-  @Test
-  public void noEmptyPanesFinalIfNonEmpty() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-            .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
-                AfterPane.elementCountAtLeast(2),
-                AfterWatermark.pastEndOfWindow())))
-            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
-            .withAllowedLateness(Duration.millis(100))
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)),
-        TimestampedValue.of(2, new Instant(2)));
-    tester.advanceInputWatermark(new Instant(20));
-    tester.advanceInputWatermark(new Instant(250));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output, contains(
-        // Trigger with 2 elements
-        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
-        // Trigger for the empty on time pane
-        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
-  }
-
-  @Test
-  public void noEmptyPanesFinalAlways() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-            .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
-                AfterPane.elementCountAtLeast(2),
-                AfterWatermark.pastEndOfWindow())))
-            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
-            .withAllowedLateness(Duration.millis(100))
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)),
-        TimestampedValue.of(2, new Instant(2)));
-    tester.advanceInputWatermark(new Instant(20));
-    tester.advanceInputWatermark(new Instant(250));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output, contains(
-        // Trigger with 2 elements
-        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
-        // Trigger for the empty on time pane
-        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10),
-        // Trigger for the final pane
-        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
-  }
-
-  @Test
-  public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-            .withTrigger(Repeatedly.forever(AfterFirst.of(
-                AfterPane.elementCountAtLeast(2),
-                AfterWatermark.pastEndOfWindow())))
-            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
-            .withAllowedLateness(Duration.millis(100))
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
-    tester.advanceInputWatermark(new Instant(50));
-
-    // We should get the ON_TIME pane even though it is empty,
-    // because we have an AfterWatermark.pastEndOfWindow() trigger.
-    output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
-
-    // We should get the final pane even though it is empty.
-    tester.advanceInputWatermark(new Instant(150));
-    output = tester.extractOutput();
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
-    assertThat(
-        output,
-        contains(
-            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
-  }
-
-  @Test
-  public void testPaneInfoFinalAndOnTime() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-            .withTrigger(
-                Repeatedly.forever(AfterPane.elementCountAtLeast(2))
-                    .orFinally(AfterWatermark.pastEndOfWindow()))
-            .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
-            .withAllowedLateness(Duration.millis(100))
-            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
-    tester.advanceInputWatermark(new Instant(0));
-
-    // Should trigger due to element count
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
-    assertThat(
-        tester.extractOutput(),
-        contains(WindowMatchers.valueWithPaneInfo(
-            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
-
-    tester.advanceInputWatermark(new Instant(150));
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0))));
-  }
-
-  @Test
-  public void testPaneInfoSkipToFinish() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    tester.advanceInputWatermark(new Instant(0));
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 1);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY))));
-  }
-
-  @Test
-  public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    tester.advanceInputWatermark(new Instant(15));
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 1);
-    assertThat(tester.extractOutput(), contains(
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE))));
-  }
-
-  @Test
-  public void testMergeBeforeFinalizing() throws Exception {
-    // Verify that we merge windows before producing output so users don't see undesired
-    // unmerged windows.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(0),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // All on time data, verify watermark hold.
-    // These two windows should pre-merge immediately to [1, 20)
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)), // in [1, 11)
-        TimestampedValue.of(10, new Instant(10))); // in [10, 20)
-
-    // And this should fire the end-of-window timer
-    tester.advanceInputWatermark(new Instant(100));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(1));
-    assertThat(output.get(0),
-        isSingleWindowedValue(containsInAnyOrder(1, 10),
-            1, // timestamp
-            1, // window start
-            20)); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
-  }
-
-  /**
-   * It is possible for a session window's trigger to be closed at the point at which
-   * the (merged) session window is garbage collected. Make sure we don't accidentally
-   * assume the window is still active.
-   */
-  @Test
-  public void testMergingWithCloseBeforeGC() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Two elements in two overlapping session windows.
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)), // in [1, 11)
-        TimestampedValue.of(10, new Instant(10))); // in [10, 20)
-
-    // Close the trigger, but the gargbage collection timer is still pending.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    tester.advanceInputWatermark(new Instant(30));
-
-    // Now the garbage collection timer will fire, finding the trigger already closed.
-    tester.advanceInputWatermark(new Instant(100));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(1));
-    assertThat(output.get(0),
-        isSingleWindowedValue(containsInAnyOrder(1, 10),
-            1, // timestamp
-            1, // window start
-            20)); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
-  }
-
-  /**
-   * Ensure a closed trigger has its state recorded in the merge result window.
-   */
-  @Test
-  public void testMergingWithCloseTrigger() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
-                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Create a new merged session window.
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
-                          TimestampedValue.of(2, new Instant(2)));
-
-    // Force the trigger to be closed for the merged window.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    tester.advanceInputWatermark(new Instant(13));
-
-    // Trigger is now closed.
-    assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
-
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-
-    // Revisit the same session window.
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
-                          TimestampedValue.of(2, new Instant(2)));
-
-    // Trigger is still closed.
-    assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
-  }
-
-  /**
-   * If a later event tries to reuse an earlier session window which has been closed, we
-   * should reject that element and not fail due to the window no longer being active.
-   */
-  @Test
-  public void testMergingWithReusedWindow() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
-                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // One elements in one session window.
-    tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
-
-    // Close the trigger, but the gargbage collection timer is still pending.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    tester.advanceInputWatermark(new Instant(15));
-
-    // Another element in the same session window.
-    // Should be discarded with 'window closed'.
-    tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
-
-    // And nothing should be left in the active window state.
-    assertTrue(tester.hasNoActiveWindows());
-
-    // Now the garbage collection timer will fire, finding the trigger already closed.
-    tester.advanceInputWatermark(new Instant(100));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(1));
-    assertThat(output.get(0),
-               isSingleWindowedValue(containsInAnyOrder(1),
-                                     1, // timestamp
-                                     1, // window start
-                                     11)); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
-  }
-
-  /**
-   * When a merged window's trigger is closed we record that state using the merged window rather
-   * than the original windows.
-   */
-  @Test
-  public void testMergingWithClosedRepresentative() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
-                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // 2 elements into merged session window.
-    // Close the trigger, but the garbage collection timer is still pending.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),       // in [1, 11), gc at 21.
-                          TimestampedValue.of(8, new Instant(8)));      // in [8, 18), gc at 28.
-
-    // More elements into the same merged session window.
-    // It has not yet been gced.
-    // Should be discarded with 'window closed'.
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),      // in [1, 11), gc at 21.
-                          TimestampedValue.of(2, new Instant(2)),      // in [2, 12), gc at 22.
-                          TimestampedValue.of(8, new Instant(8)));     // in [8, 18), gc at 28.
-
-    // Now the garbage collection timer will fire, finding the trigger already closed.
-    tester.advanceInputWatermark(new Instant(100));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-
-    assertThat(output.size(), equalTo(1));
-    assertThat(output.get(0),
-               isSingleWindowedValue(containsInAnyOrder(1, 8),
-                                     1, // timestamp
-                                     1, // window start
-                                     18)); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
-  }
-
-  /**
-   * If an element for a closed session window ends up being merged into other still-open
-   * session windows, the resulting session window is not 'poisoned'.
-   */
-  @Test
-  public void testMergingWithClosedDoesNotPoison() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // 1 element, force its trigger to close.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    tester.injectElements(TimestampedValue.of(2, new Instant(2)));
-
-    // 3 elements, one already closed.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
-        TimestampedValue.of(2, new Instant(2)),
-        TimestampedValue.of(3, new Instant(3)));
-
-    tester.advanceInputWatermark(new Instant(100));
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(2));
-    assertThat(output.get(0),
-        isSingleWindowedValue(containsInAnyOrder(2),
-            2, // timestamp
-            2, // window start
-            12)); // window end
-    assertThat(
-        output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
-    assertThat(output.get(1),
-        isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
-            1, // timestamp
-            1, // window start
-            13)); // window end
-    assertThat(
-        output.get(1).getPane(),
-        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
-  }
-
-  /**
-   * Tests that when data is assigned to multiple windows but some of those windows have
-   * had their triggers finish, then the data is dropped and counted accurately.
-   */
-  @Test
-  public void testDropDataMultipleWindowsFinishedTrigger() throws Exception {
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
-        WindowingStrategy.of(
-            SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
-        .withTrigger(AfterWatermark.pastEndOfWindow())
-        .withAllowedLateness(Duration.millis(1000)),
-        new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of());
-
-    tester.injectElements(
-        // assigned to [-60, 40), [-30, 70), [0, 100)
-        TimestampedValue.of(10, new Instant(23)),
-        // assigned to [-30, 70), [0, 100), [30, 130)
-        TimestampedValue.of(12, new Instant(40)));
-
-    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
-    tester.advanceInputWatermark(new Instant(70));
-    tester.injectElements(
-        // assigned to [-30, 70), [0, 100), [30, 130)
-        // but [-30, 70) is closed by the trigger
-        TimestampedValue.of(14, new Instant(60)));
-
-    assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
-
-    tester.advanceInputWatermark(new Instant(130));
-    // assigned to [-30, 70), [0, 100), [30, 130)
-    // but they are all closed
-    tester.injectElements(TimestampedValue.of(16, new Instant(40)));
-
-    assertEquals(4, tester.getElementsDroppedDueToClosedWindow());
-  }
-
-  @Test
-  public void testIdempotentEmptyPanesDiscarding() throws Exception {
-    // Test uninteresting (empty) panes don't increment the index or otherwise
-    // modify PaneInfo.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Inject a couple of on-time elements and fire at the window end.
-    injectElement(tester, 1);
-    injectElement(tester, 2);
-    tester.advanceInputWatermark(new Instant(12));
-
-    // Fire the on-time pane
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-
-    // Fire another timer (with no data, so it's an uninteresting pane that should not be output).
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-
-    // Finish it off with another datum.
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 3);
-
-    // The intermediate trigger firing shouldn't result in any output.
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(2));
-
-    // The on-time pane is as expected.
-    assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
-
-    // The late pane has the correct indices.
-    assertThat(output.get(1).getValue(), contains(3));
-    assertThat(
-        output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
-
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
-    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-  }
-
-  @Test
-  public void testIdempotentEmptyPanesAccumulating() throws Exception {
-    // Test uninteresting (empty) panes don't increment the index or otherwise
-    // modify PaneInfo.
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
-            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    // Inject a couple of on-time elements and fire at the window end.
-    injectElement(tester, 1);
-    injectElement(tester, 2);
-    tester.advanceInputWatermark(new Instant(12));
-
-    // Trigger the on-time pane
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertThat(output.size(), equalTo(1));
-    assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
-    assertThat(output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0)));
-
-    // Fire another timer with no data; the empty pane should not be output even though the
-    // trigger is ready to fire
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-    assertThat(tester.extractOutput().size(), equalTo(0));
-
-    // Finish it off with another datum, which is late
-    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
-    triggerShouldFinish(mockTrigger);
-    injectElement(tester, 3);
-    output = tester.extractOutput();
-    assertThat(output.size(), equalTo(1));
-
-    // The late pane has the correct indices.
-    assertThat(output.get(0).getValue(), containsInAnyOrder(1, 2, 3));
-    assertThat(output.get(0).getPane(),
-        equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
-
-    assertTrue(tester.isMarkedFinished(firstWindow));
-    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
-    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-  }
-
-  /**
-   * Test that we receive an empty on-time pane when an or-finally waiting for the watermark fires.
-   * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
-   * when the on-time pane is empty.
-   */
-  @Test
-  public void testEmptyOnTimeFromOrFinally() throws Exception {
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
-        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
-            AfterEach.<IntervalWindow>inOrder(
-                Repeatedly
-                    .forever(
-                        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
-                            new Duration(5)))
-                    .orFinally(AfterWatermark.pastEndOfWindow()),
-                Repeatedly.forever(
-                    AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
-                        new Duration(25)))),
-            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
-            VarIntCoder.of(), Duration.millis(100));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.advanceProcessingTime(new Instant(0));
-
-    // Processing time timer for 5
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(1)),
-        TimestampedValue.of(1, new Instant(3)),
-        TimestampedValue.of(1, new Instant(7)),
-        TimestampedValue.of(1, new Instant(5)));
-
-    // Should fire early pane
-    tester.advanceProcessingTime(new Instant(6));
-
-    // Should fire empty on time pane
-    tester.advanceInputWatermark(new Instant(11));
-    List<WindowedValue<Integer>> output = tester.extractOutput();
-    assertEquals(2, output.size());
-
-    assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
-    assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(4, 9, 0, 10));
-
-    assertThat(
-        output.get(0),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
-    assertThat(
-        output.get(1),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
-  }
-
-  /**
-   * Tests for processing time firings after the watermark passes the end of the window.
-   * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
-   * when the on-time pane is non-empty.
-   */
-  @Test
-  public void testProcessingTime() throws Exception {
-    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
-        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
-            AfterEach.<IntervalWindow>inOrder(
-                Repeatedly
-                    .forever(
-                        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
-                            new Duration(5)))
-                    .orFinally(AfterWatermark.pastEndOfWindow()),
-                Repeatedly.forever(
-                    AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
-                        new Duration(25)))),
-            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
-            VarIntCoder.of(), Duration.millis(100));
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.advanceProcessingTime(new Instant(0));
-
-    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
-        TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)),
-        TimestampedValue.of(1, new Instant(5)));
-    // 4 elements all at processing time 0
-
-    tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(8)),
-        TimestampedValue.of(1, new Instant(4)));
-    // 6 elements
-
-    tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(8)),
-        TimestampedValue.of(1, new Instant(4)),
-        TimestampedValue.of(1, new Instant(5)));
-    // 9 elements
-
-    tester.advanceInputWatermark(new Instant(12));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(3)));
-    // 10 elements
-
-    tester.advanceProcessingTime(new Instant(15));
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(5)));
-    // 11 elements
-    tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25
-
-    tester.injectElements(
-        TimestampedValue.of(1, new Instant(3)));
-    // 12 elements
-    // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25
-    tester.advanceInputWatermark(new Instant(125));
-
-    List<WindowedValue<Integer>> output = tester.extractOutput();
-    assertEquals(4, output.size());
-
-    assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
-    assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(6, 4, 0, 10));
-    assertThat(output.get(2), WindowMatchers.isSingleWindowedValue(11, 9, 0, 10));
-    assertThat(output.get(3), WindowMatchers.isSingleWindowedValue(12, 9, 0, 10));
-
-    assertThat(
-        output.get(0),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
-    assertThat(
-        output.get(1),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
-    assertThat(
-        output.get(2),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 1)));
-    assertThat(
-        output.get(3),
-        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 3, 2)));
-  }
-
-  /**
-   * We should fire a non-empty ON_TIME pane in the GlobalWindow when the watermark moves to
-   * end-of-time.
-   */
-  @Test
-  public void fireNonEmptyOnDrainInGlobalWindow() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
-        ReduceFnTester.nonCombining(
-            WindowingStrategy.of(new GlobalWindows())
-                             .withTrigger(Repeatedly.<GlobalWindow>forever(
-                                 AfterPane.elementCountAtLeast(3)))
-                             .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
-
-    tester.advanceInputWatermark(new Instant(0));
-
-    final int n = 20;
-    for (int i = 0; i < n; i++) {
-      tester.injectElements(TimestampedValue.of(i, new Instant(i)));
-    }
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertEquals(n / 3, output.size());
-    for (int i = 0; i < output.size(); i++) {
-      assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
-      assertEquals(i, output.get(i).getPane().getIndex());
-      assertEquals(3, Iterables.size(output.get(i).getValue()));
-    }
-
-    tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    output = tester.extractOutput();
-    assertEquals(1, output.size());
-    assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
-    assertEquals(n / 3, output.get(0).getPane().getIndex());
-    assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue()));
-  }
-
-  /**
-   * We should fire an empty ON_TIME pane in the GlobalWindow when the watermark moves to
-   * end-of-time.
-   */
-  @Test
-  public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
-        ReduceFnTester.nonCombining(
-            WindowingStrategy.of(new GlobalWindows())
-                             .withTrigger(Repeatedly.<GlobalWindow>forever(
-                                 AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
-                                     new Duration(3))))
-                             .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
-
-    final int n = 20;
-    for (int i = 0; i < n; i++) {
-      tester.advanceProcessingTime(new Instant(i));
-      tester.injectElements(TimestampedValue.of(i, new Instant(i)));
-    }
-    tester.advanceProcessingTime(new Instant(n + 4));
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertEquals((n + 3) / 4, output.size());
-    for (int i = 0; i < output.size(); i++) {
-      assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
-      assertEquals(i, output.get(i).getPane().getIndex());
-      assertEquals(4, Iterables.size(output.get(i).getValue()));
-    }
-
-    tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    output = tester.extractOutput();
-    assertEquals(1, output.size());
-    assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
-    assertEquals((n + 3) / 4, output.get(0).getPane().getIndex());
-    assertEquals(0, Iterables.size(output.get(0).getValue()));
-  }
-
-  /**
-   * Late elements should still have a garbage collection hold set so that they
-   * can make a late pane rather than be dropped due to lateness.
-   */
-  @Test
-  public void setGarbageCollectionHoldOnLateElements() throws Exception {
-    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
-        ReduceFnTester.nonCombining(
-            FixedWindows.of(Duration.millis(10)),
-            AfterWatermark.pastEndOfWindow().withLateFirings(AfterPane.elementCountAtLeast(2)),
-            AccumulationMode.DISCARDING_FIRED_PANES,
-            Duration.millis(100),
-            ClosingBehavior.FIRE_IF_NON_EMPTY);
-
-    tester.advanceInputWatermark(new Instant(0));
-    tester.advanceOutputWatermark(new Instant(0));
-    tester.injectElements(TimestampedValue.of(1,  new Instant(1)));
-
-    // Fire ON_TIME pane @ 9 with 1
-
-    tester.advanceInputWatermark(new Instant(109));
-    tester.advanceOutputWatermark(new Instant(109));
-    tester.injectElements(TimestampedValue.of(2,  new Instant(2)));
-    // We should have set a garbage collection hold for the final pane.
-    Instant hold = tester.getWatermarkHold();
-    assertEquals(new Instant(109), hold);
-
-    tester.advanceInputWatermark(new Instant(110));
-    tester.advanceOutputWatermark(new Instant(110));
-
-    // Fire final LATE pane @ 9 with 2
-
-    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-    assertEquals(2, output.size());
-  }
-
-  private static class SumAndVerifyContextFn extends CombineFnWithContext<Integer, int[], Integer> {
-
-    private final PCollectionView<Integer> view;
-    private final int expectedValue;
-
-    private SumAndVerifyContextFn(PCollectionView<Integer> view, int expectedValue) {
-      this.view = view;
-      this.expectedValue = expectedValue;
-    }
-    @Override
-    public int[] createAccumulator(Context c) {
-      checkArgument(
-          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      checkArgument(c.sideInput(view) == expectedValue);
-      return wrap(0);
-    }
-
-    @Override
-    public int[] addInput(int[] accumulator, Integer input, Context c) {
-      checkArgument(
-          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      checkArgument(c.sideInput(view) == expectedValue);
-      accumulator[0] += input.intValue();
-      return accumulator;
-    }
-
-    @Override
-    public int[] mergeAccumulators(Iterable<int[]> accumulators, Context c) {
-      checkArgument(
-          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      checkArgument(c.sideInput(view) == expectedValue);
-      Iterator<int[]> iter = accumulators.iterator();
-      if (!iter.hasNext()) {
-        return createAccumulator(c);
-      } else {
-        int[] running = iter.next();
-        while (iter.hasNext()) {
-          running[0] += iter.next()[0];
-        }
-        return running;
-      }
-    }
-
-    @Override
-    public Integer extractOutput(int[] accumulator, Context c) {
-      checkArgument(
-          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      checkArgument(c.sideInput(view) == expectedValue);
-      return accumulator[0];
-    }
-
-    private int[] wrap(int value) {
-      return new int[] { value };
-    }
-  }
-
-  /**
-   * A {@link PipelineOptions} to test combining with context.
-   */
-  public interface TestOptions extends PipelineOptions {
-    Integer getValue();
-    void setValue(Integer value);
-  }
-}


[09/50] [abbrv] incubator-beam git commit: Add LeaderBoardTest

Posted by dh...@apache.org.
Add LeaderBoardTest

This test exercises the PTransforms that make up the LeaderBoard
example. This includes speculative and late trigger firings to produce
team and individual scores on a global and fixed window basis.

Refactor LeaderBoard to expose the team and user score calculations as
composite PTransforms to enable this testing.


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

Branch: refs/heads/gearpump-runner
Commit: 00b4e95148eb98d7fea5877274f2fcf2252ac432
Parents: 74d0195
Author: Thomas Groh <tg...@google.com>
Authored: Fri Aug 5 14:20:56 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../examples/complete/game/LeaderBoard.java     | 113 ++++--
 .../examples/complete/game/LeaderBoardTest.java | 362 +++++++++++++++++++
 2 files changed, 440 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/00b4e951/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 8dd4e39..13bbf44 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
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.examples.complete.game;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.TimeZone;
@@ -32,6 +33,7 @@ import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
 import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
@@ -68,7 +70,7 @@ import org.joda.time.format.DateTimeFormatter;
  * here we're using an unbounded data source, which lets us provide speculative results, and allows
  * handling of late data, at much lower latency. We can use the early/speculative results to keep a
  * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct
- * results, e.g. for 'team prizes'. We're now outputing window results as they're
+ * results, e.g. for 'team prizes'. We're now outputting window results as they're
  * calculated, giving us much lower latency than with the previous batch examples.
  *
  * <p> Run {@link injector.Injector} to generate pubsub data for this pipeline.  The Injector
@@ -186,50 +188,91 @@ public class LeaderBoard extends HourlyTeamScore {
         .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic()))
         .apply("ParseGameEvent", ParDo.of(new ParseEventFn()));
 
-    // [START DocInclude_WindowAndTrigger]
-    // Extract team/score pairs from the event stream, using hour-long windows by default.
-    gameEvents
-        .apply("LeaderboardTeamFixedWindows", Window.<GameActionInfo>into(
-            FixedWindows.of(Duration.standardMinutes(options.getTeamWindowDuration())))
-          // We will get early (speculative) results as well as cumulative
-          // processing of late data.
-          .triggering(
-            AfterWatermark.pastEndOfWindow()
-            .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(FIVE_MINUTES))
-            .withLateFirings(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(TEN_MINUTES)))
-          .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))
-          .accumulatingFiredPanes())
-        // Extract and sum teamname/score pairs from the event data.
-        .apply("ExtractTeamScore", new ExtractAndSumScore("team"))
+    gameEvents.apply("CalculateTeamScores",
+        new CalculateTeamScores(
+            Duration.standardMinutes(options.getTeamWindowDuration()),
+            Duration.standardMinutes(options.getAllowedLateness())))
         // Write the results to BigQuery.
         .apply("WriteTeamScoreSums",
                new WriteWindowedToBigQuery<KV<String, Integer>>(
                   options.getTableName() + "_team", configureWindowedTableWrite()));
-    // [END DocInclude_WindowAndTrigger]
-
-    // [START DocInclude_ProcTimeTrigger]
-    // Extract user/score pairs from the event stream using processing time, via global windowing.
-    // Get periodic updates on all users' running scores.
     gameEvents
-        .apply("LeaderboardUserGlobalWindow", Window.<GameActionInfo>into(new GlobalWindows())
-          // Get periodic results every ten minutes.
-              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
-                  .plusDelayOf(TEN_MINUTES)))
-              .accumulatingFiredPanes()
-              .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())))
-        // Extract and sum username/score pairs from the event data.
-        .apply("ExtractUserScore", new ExtractAndSumScore("user"))
+        .apply(
+            "CalculateUserScores",
+            new CalculateUserScores(Duration.standardMinutes(options.getAllowedLateness())))
         // Write the results to BigQuery.
-        .apply("WriteUserScoreSums",
-               new WriteToBigQuery<KV<String, Integer>>(
-                  options.getTableName() + "_user", configureGlobalWindowBigQueryWrite()));
-    // [END DocInclude_ProcTimeTrigger]
+        .apply(
+            "WriteUserScoreSums",
+            new WriteToBigQuery<KV<String, Integer>>(
+                options.getTableName() + "_user", configureGlobalWindowBigQueryWrite()));
 
     // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the
     // command line.
     PipelineResult result = pipeline.run();
     exampleUtils.waitToFinish(result);
   }
+
+  /**
+   * Calculates scores for each team within the configured window duration.
+   */
+  // [START DocInclude_WindowAndTrigger]
+  // Extract team/score pairs from the event stream, using hour-long windows by default.
+  @VisibleForTesting
+  static class CalculateTeamScores
+      extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> {
+    private final Duration teamWindowDuration;
+    private final Duration allowedLateness;
+
+    CalculateTeamScores(Duration teamWindowDuration, Duration allowedLateness) {
+      this.teamWindowDuration = teamWindowDuration;
+      this.allowedLateness = allowedLateness;
+    }
+
+    @Override
+    public PCollection<KV<String, Integer>> apply(PCollection<GameActionInfo> infos) {
+      return infos.apply("LeaderboardTeamFixedWindows",
+          Window.<GameActionInfo>into(FixedWindows.of(teamWindowDuration))
+              // We will get early (speculative) results as well as cumulative
+              // processing of late data.
+              .triggering(AfterWatermark.pastEndOfWindow()
+                  .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane()
+                      .plusDelayOf(FIVE_MINUTES))
+                  .withLateFirings(AfterProcessingTime.pastFirstElementInPane()
+                      .plusDelayOf(TEN_MINUTES)))
+              .withAllowedLateness(allowedLateness)
+              .accumulatingFiredPanes())
+          // Extract and sum teamname/score pairs from the event data.
+          .apply("ExtractTeamScore", new ExtractAndSumScore("team"));
+    }
+  }
+  // [END DocInclude_WindowAndTrigger]
+
+  // [START DocInclude_ProcTimeTrigger]
+  /**
+   * Extract user/score pairs from the event stream using processing time, via global windowing.
+   * Get periodic updates on all users' running scores.
+   */
+  @VisibleForTesting
+  static class CalculateUserScores
+      extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> {
+    private final Duration allowedLateness;
+
+    CalculateUserScores(Duration allowedLateness) {
+      this.allowedLateness = allowedLateness;
+    }
+
+    @Override
+    public PCollection<KV<String, Integer>> apply(PCollection<GameActionInfo> input) {
+      return input.apply("LeaderboardUserGlobalWindow",
+          Window.<GameActionInfo>into(new GlobalWindows())
+              // Get periodic results every ten minutes.
+              .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
+                  .plusDelayOf(TEN_MINUTES)))
+              .accumulatingFiredPanes()
+              .withAllowedLateness(allowedLateness))
+          // Extract and sum username/score pairs from the event data.
+          .apply("ExtractUserScore", new ExtractAndSumScore("user"));
+    }
+  }
+  // [END DocInclude_ProcTimeTrigger]
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/00b4e951/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
new file mode 100644
index 0000000..40cac36
--- /dev/null
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
@@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.examples.complete.game;
+
+import static org.apache.beam.sdk.testing.PAssert.that;
+import static org.hamcrest.Matchers.hasItem;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.Serializable;
+import org.apache.beam.examples.complete.game.LeaderBoard.CalculateTeamScores;
+import org.apache.beam.examples.complete.game.LeaderBoard.CalculateUserScores;
+import org.apache.beam.examples.complete.game.UserScore.GameActionInfo;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link LeaderBoard}.
+ */
+@RunWith(JUnit4.class)
+public class LeaderBoardTest implements Serializable {
+  private static final Duration ALLOWED_LATENESS = Duration.standardHours(1);
+  private static final Duration TEAM_WINDOW_DURATION = Duration.standardMinutes(20);
+  private Instant baseTime = new Instant(0);
+
+  /**
+   * Some example users, on two separate teams.
+   */
+  private enum TestUser {
+    RED_ONE("scarlet", "red"), RED_TWO("burgundy", "red"),
+    BLUE_ONE("navy", "blue"), BLUE_TWO("sky", "blue");
+
+    private final String userName;
+    private final String teamName;
+
+    TestUser(String userName, String teamName) {
+      this.userName = userName;
+      this.teamName = teamName;
+    }
+
+    public String getUser() {
+      return userName;
+    }
+
+    public String getTeam() {
+      return teamName;
+    }
+  }
+
+  /**
+   * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive
+   * on time (ahead of the watermark).
+   */
+  @Test
+  public void testTeamScoresOnTime() {
+    TestPipeline p = TestPipeline.create();
+
+    TestStream<GameActionInfo> createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class))
+        // Start at the epoch
+        .advanceWatermarkTo(baseTime)
+        // add some elements ahead of the watermark
+        .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)),
+            event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1)),
+            event(TestUser.RED_TWO, 3, Duration.standardSeconds(22)),
+            event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(3)))
+        // The watermark advances slightly, but not past the end of the window
+        .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3)))
+        // Add some more on time elements
+        .addElements(event(TestUser.RED_ONE, 1, Duration.standardMinutes(4)),
+            event(TestUser.BLUE_ONE, 2, Duration.standardSeconds(270)))
+        // The window should close and emit an ON_TIME pane
+        .advanceWatermarkToInfinity();
+
+    PCollection<KV<String, Integer>> teamScores = p.apply(createEvents)
+        .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS));
+
+    String blueTeam = TestUser.BLUE_ONE.getTeam();
+    String redTeam = TestUser.RED_ONE.getTeam();
+    that(teamScores)
+        .inOnTimePane(new IntervalWindow(baseTime, TEAM_WINDOW_DURATION))
+        .containsInAnyOrder(KV.of(blueTeam, 12), KV.of(redTeam, 4));
+
+    p.run();
+  }
+
+  /**
+   * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive
+   * on time, and the processing time advances far enough for speculative panes.
+   */
+  @Test
+  public void testTeamScoresSpeculative() {
+    TestPipeline p = TestPipeline.create();
+
+    TestStream<GameActionInfo> createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class))
+        // Start at the epoch
+        .advanceWatermarkTo(baseTime)
+        .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)),
+            event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1)))
+        // Some time passes within the runner, which causes a speculative pane containing the blue
+        // team's score to be emitted
+        .advanceProcessingTime(Duration.standardMinutes(10))
+        .addElements(event(TestUser.RED_TWO, 5, Duration.standardMinutes(3)))
+        // Some additional time passes and we get a speculative pane for the red team
+        .advanceProcessingTime(Duration.standardMinutes(12))
+        .addElements(event(TestUser.BLUE_TWO, 3, Duration.standardSeconds(22)))
+        // More time passes and a speculative pane containing a refined value for the blue pane is
+        // emitted
+        .advanceProcessingTime(Duration.standardMinutes(10))
+        // Some more events occur
+        .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(4)),
+            event(TestUser.BLUE_TWO, 2, Duration.standardMinutes(2)))
+        // The window closes and we get an ON_TIME pane that contains all of the updates
+        .advanceWatermarkToInfinity();
+
+    PCollection<KV<String, Integer>> teamScores = p.apply(createEvents)
+        .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS));
+
+    String blueTeam = TestUser.BLUE_ONE.getTeam();
+    String redTeam = TestUser.RED_ONE.getTeam();
+    IntervalWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION);
+    // The window contains speculative panes alongside the on-time pane
+    PAssert.that(teamScores)
+        .inWindow(window)
+        .containsInAnyOrder(KV.of(blueTeam, 10) /* The on-time blue pane */,
+            KV.of(redTeam, 9) /* The on-time red pane */,
+            KV.of(blueTeam, 5) /* The first blue speculative pane */,
+            KV.of(blueTeam, 8) /* The second blue speculative pane */,
+            KV.of(redTeam, 5) /* The red speculative pane */);
+     PAssert.that(teamScores)
+        .inOnTimePane(window)
+        .containsInAnyOrder(KV.of(blueTeam, 10), KV.of(redTeam, 9));
+
+    p.run();
+  }
+
+  /**
+   * A test where elements arrive behind the watermark (late data), but before the end of the
+   * window. These elements are emitted on time.
+   */
+  @Test
+  public void testTeamScoresUnobservablyLate() {
+    TestPipeline p = TestPipeline.create();
+
+    BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION);
+    TestStream<GameActionInfo> createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class))
+        .advanceWatermarkTo(baseTime)
+        .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)),
+            event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8)),
+            event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)),
+            event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5)))
+        .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1)))
+        // These events are late, but the window hasn't closed yet, so the elements are in the
+        // on-time pane
+        .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO),
+            event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)),
+            event(TestUser.BLUE_TWO, 2, Duration.standardSeconds(90)),
+            event(TestUser.RED_TWO, 3, Duration.standardMinutes(3)))
+        .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1)))
+        .advanceWatermarkToInfinity();
+    PCollection<KV<String, Integer>> teamScores = p.apply(createEvents)
+        .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS));
+
+    String blueTeam = TestUser.BLUE_ONE.getTeam();
+    String redTeam = TestUser.RED_ONE.getTeam();
+    // The On Time pane contains the late elements that arrived before the end of the window
+    PAssert.that(teamScores)
+        .inOnTimePane(window)
+        .containsInAnyOrder(KV.of(redTeam, 14), KV.of(blueTeam, 13));
+
+    p.run();
+  }
+
+  /**
+   * A test where elements arrive behind the watermark (late data) after the watermark passes the
+   * end of the window, but before the maximum allowed lateness. These elements are emitted in a
+   * late pane.
+   */
+  @Test
+  public void testTeamScoresObservablyLate() {
+    TestPipeline p = TestPipeline.create();
+
+    Instant firstWindowCloses = baseTime.plus(ALLOWED_LATENESS).plus(TEAM_WINDOW_DURATION);
+    TestStream<GameActionInfo> createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class))
+        .advanceWatermarkTo(baseTime)
+        .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)),
+            event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8)))
+        .advanceProcessingTime(Duration.standardMinutes(10))
+        .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3)))
+        .addElements(event(TestUser.RED_ONE, 3, Duration.standardMinutes(1)),
+            event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)),
+            event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5)))
+        .advanceWatermarkTo(firstWindowCloses.minus(Duration.standardMinutes(1)))
+        // These events are late but should still appear in a late pane
+        .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO),
+            event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)),
+            event(TestUser.RED_TWO, 3, Duration.standardMinutes(3)))
+        // A late refinement is emitted due to the advance in processing time, but the window has
+        // not yet closed because the watermark has not advanced
+        .advanceProcessingTime(Duration.standardMinutes(12))
+        // These elements should appear in the final pane
+        .addElements(event(TestUser.RED_TWO, 9, Duration.standardMinutes(1)),
+            event(TestUser.RED_TWO, 1, Duration.standardMinutes(3)))
+        .advanceWatermarkToInfinity();
+
+    PCollection<KV<String, Integer>> teamScores = p.apply(createEvents)
+        .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS));
+
+    BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION);
+    String blueTeam = TestUser.BLUE_ONE.getTeam();
+    String redTeam = TestUser.RED_ONE.getTeam();
+    PAssert.that(teamScores)
+        .inWindow(window)
+        .satisfies((SerializableFunction<Iterable<KV<String, Integer>>, Void>) input -> {
+          // The final sums need not exist in the same pane, but must appear in the output
+          // PCollection
+          assertThat(input, hasItem(KV.of(blueTeam, 11)));
+          assertThat(input, hasItem(KV.of(redTeam, 27)));
+          return null;
+        });
+    PAssert.thatMap(teamScores)
+        // The closing behavior of CalculateTeamScores precludes an inFinalPane matcher
+        .inOnTimePane(window)
+        .isEqualTo(ImmutableMap.<String, Integer>builder().put(redTeam, 7)
+            .put(blueTeam, 11)
+            .build());
+
+    // No final pane is emitted for the blue team, as all of their updates have been taken into
+    // account in earlier panes
+    PAssert.that(teamScores).inFinalPane(window).containsInAnyOrder(KV.of(redTeam, 27));
+
+    p.run();
+  }
+
+  /**
+   * A test where elements arrive beyond the maximum allowed lateness. These elements are dropped
+   * within {@link CalculateTeamScores} and do not impact the final result.
+   */
+  @Test
+  public void testTeamScoresDroppablyLate() {
+    TestPipeline p = TestPipeline.create();
+
+    BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION);
+    TestStream<GameActionInfo> infos = TestStream.create(AvroCoder.of(GameActionInfo.class))
+        .addElements(event(TestUser.BLUE_ONE, 12, Duration.ZERO),
+            event(TestUser.RED_ONE, 3, Duration.ZERO))
+        .advanceWatermarkTo(window.maxTimestamp())
+        .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)),
+            event(TestUser.BLUE_TWO, 3, Duration.ZERO),
+            event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3)))
+        // Move the watermark past the end of the allowed lateness plus the end of the window
+        .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS)
+            .plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1)))
+        // These elements within the expired window are droppably late, and will not appear in the
+        // output
+        .addElements(
+            event(TestUser.BLUE_TWO, 3, TEAM_WINDOW_DURATION.minus(Duration.standardSeconds(5))),
+            event(TestUser.RED_ONE, 7, Duration.standardMinutes(4)))
+        .advanceWatermarkToInfinity();
+    PCollection<KV<String, Integer>> teamScores = p.apply(infos)
+        .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS));
+
+    String blueTeam = TestUser.BLUE_ONE.getTeam();
+    String redTeam = TestUser.RED_ONE.getTeam();
+    // Only one on-time pane and no late panes should be emitted
+    PAssert.that(teamScores)
+        .inWindow(window)
+        .containsInAnyOrder(KV.of(redTeam, 7), KV.of(blueTeam, 18));
+    // No elements are added before the watermark passes the end of the window plus the allowed
+    // lateness, so no refinement should be emitted
+    PAssert.that(teamScores).inFinalPane(window).empty();
+  }
+
+  /**
+   * A test where elements arrive both on-time and late in {@link CalculateUserScores}, which emits
+   * output into the {@link GlobalWindow}. All elements that arrive should be taken into account,
+   * even if they arrive later than the maximum allowed lateness.
+   */
+  @Test
+  public void testUserScore() {
+    TestPipeline p = TestPipeline.create();
+
+    TestStream<GameActionInfo> infos =
+        TestStream.create(AvroCoder.of(GameActionInfo.class))
+            .addElements(
+                event(TestUser.BLUE_ONE, 12, Duration.ZERO),
+                event(TestUser.RED_ONE, 3, Duration.ZERO))
+            .advanceProcessingTime(Duration.standardMinutes(7))
+            .addElements(
+                event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)),
+                event(TestUser.BLUE_TWO, 3, Duration.ZERO),
+                event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3)))
+            .advanceProcessingTime(Duration.standardMinutes(5))
+            .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS).plus(Duration.standardHours(12)))
+            // Late elements are always observable within the global window - they arrive before
+            // the window closes, so they will appear in a pane, even if they arrive after the
+            // allowed lateness, and are taken into account alongside on-time elements
+            .addElements(
+                event(TestUser.RED_ONE, 3, Duration.standardMinutes(7)),
+                event(TestUser.RED_ONE, 2, (ALLOWED_LATENESS).plus(Duration.standardHours(13))))
+            .advanceProcessingTime(Duration.standardMinutes(6))
+            .addElements(event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(12)))
+            .advanceProcessingTime(Duration.standardMinutes(20))
+            .advanceWatermarkToInfinity();
+
+    PCollection<KV<String, Integer>> userScores =
+        p.apply(infos).apply(new CalculateUserScores(ALLOWED_LATENESS));
+
+    // User scores are emitted in speculative panes in the Global Window - this matcher choice
+    // ensures that panes emitted by the watermark advancing to positive infinity are not included,
+    // as that will not occur outside of tests
+    that(userScores)
+        .inEarlyGlobalWindowPanes()
+        .containsInAnyOrder(KV.of(TestUser.BLUE_ONE.getUser(), 15),
+            KV.of(TestUser.RED_ONE.getUser(), 7),
+            KV.of(TestUser.RED_ONE.getUser(), 12),
+            KV.of(TestUser.BLUE_TWO.getUser(), 3),
+            KV.of(TestUser.BLUE_TWO.getUser(), 8));
+
+    p.run();
+  }
+
+  private TimestampedValue<GameActionInfo> event(
+      TestUser user,
+      int score,
+      Duration baseTimeOffset) {
+    return TimestampedValue.of(new GameActionInfo(user.getUser(),
+        user.getTeam(),
+        score,
+        baseTime.plus(baseTimeOffset).getMillis()), baseTime.plus(baseTimeOffset));
+  }
+}


[25/50] [abbrv] incubator-beam git commit: Delegate populateDipslayData to wrapped combineFn's

Posted by dh...@apache.org.
Delegate populateDipslayData to wrapped combineFn's


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

Branch: refs/heads/gearpump-runner
Commit: 60d8cd90ab9f0947baaf7b50168f8a7b570215f8
Parents: 9943fd7d
Author: Scott Wegner <sw...@google.com>
Authored: Fri Aug 19 13:32:45 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/Combine.java | 39 ++++++--------------
 1 file changed, 11 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60d8cd90/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index d432e15..2b89372 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -1740,7 +1740,7 @@ public class Combine {
   public static class PerKey<K, InputT, OutputT>
     extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>> {
 
-    private final transient PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
+    private final PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
     private final DisplayData.Item<? extends Class<?>> fnDisplayData;
     private final boolean fewKeys;
     private final List<PCollectionView<?>> sideInputs;
@@ -1820,8 +1820,8 @@ public class Combine {
             @Override
             public void populateDisplayData(Builder builder) {
               super.populateDisplayData(builder);
-              builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout)
-                  .withLabel("Key Fanout Size"), 0);
+              builder.add(DisplayData.item("fanout", hotKeyFanout)
+                  .withLabel("Key Fanout Size"));
             }
 
             @Override
@@ -1866,7 +1866,7 @@ public class Combine {
   public static class PerKeyWithHotKeyFanout<K, InputT, OutputT>
       extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>> {
 
-    private final transient PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
+    private final PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
     private final DisplayData.Item<? extends Class<?>> fnDisplayData;
     private final SerializableFunction<? super K, Integer> hotKeyFanout;
 
@@ -1955,12 +1955,7 @@ public class Combine {
 
               @Override
               public void populateDisplayData(DisplayData.Builder builder) {
-                super.populateDisplayData(builder);
-                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
-                    .withLabel("Fanout Function"));
-                if (hotKeyFanout instanceof HasDisplayData) {
-                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
-                }
+                builder.include(PerKeyWithHotKeyFanout.this);
               }
             };
         postCombine =
@@ -2008,12 +2003,7 @@ public class Combine {
               }
               @Override
               public void populateDisplayData(DisplayData.Builder builder) {
-                super.populateDisplayData(builder);
-                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
-                    .withLabel("Fanout Function"));
-                if (hotKeyFanout instanceof HasDisplayData) {
-                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
-                }
+                builder.include(PerKeyWithHotKeyFanout.this);
               }
             };
       } else {
@@ -2057,12 +2047,7 @@ public class Combine {
               }
               @Override
               public void populateDisplayData(DisplayData.Builder builder) {
-                super.populateDisplayData(builder);
-                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
-                    .withLabel("Fanout Function"));
-                if (hotKeyFanout instanceof HasDisplayData) {
-                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
-                }
+                builder.include(PerKeyWithHotKeyFanout.this);
               }
             };
         postCombine =
@@ -2111,12 +2096,7 @@ public class Combine {
               }
               @Override
               public void populateDisplayData(DisplayData.Builder builder) {
-                super.populateDisplayData(builder);
-                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
-                    .withLabel("Fanout Function"));
-                if (hotKeyFanout instanceof HasDisplayData) {
-                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
-                }
+                builder.include(PerKeyWithHotKeyFanout.this);
               }
             };
       }
@@ -2200,6 +2180,9 @@ public class Combine {
       super.populateDisplayData(builder);
 
       Combine.populateDisplayData(builder, fn, fnDisplayData);
+      if (hotKeyFanout instanceof HasDisplayData) {
+        builder.include((HasDisplayData) hotKeyFanout);
+      }
       builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
         .withLabel("Fanout Function"));
     }


[45/50] [abbrv] incubator-beam git commit: Added even more javadoc to TextIO#withHeader and TextIO#withFooter.

Posted by dh...@apache.org.
Added even more javadoc to TextIO#withHeader and TextIO#withFooter.


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

Branch: refs/heads/gearpump-runner
Commit: e5db1c78bbc910db31f7a324ea2f3f48b4a0de3e
Parents: 5084580
Author: Stas Levin <st...@gmail.com>
Authored: Wed Sep 7 18:38:28 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5db1c78/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 e75aaa9..f09ab6c 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
@@ -452,6 +452,8 @@ public class TextIO {
      * Returns a transform for writing to text files that adds a header string to the files
      * it writes.
      *
+     *<p> A <code>null</code> value will clear any previously configured header.</p>
+     *
      * @param header the string to be added as file header
      */
     public static Bound<String> withHeader(String header) {
@@ -462,6 +464,8 @@ public class TextIO {
      * Returns a transform for writing to text files that adds a footer string to the files
      * it writes.
      *
+     *<p> A <code>null</code> value will clear any previously configured footer.</p>
+     *
      * @param footer the string to be added as file footer
      */
     public static Bound<String> withFooter(String footer) {


[43/50] [abbrv] incubator-beam git commit: [BEAM-242] Enable and fix checkstyle in Flink runner examples

Posted by dh...@apache.org.
[BEAM-242] Enable and fix checkstyle in Flink runner examples


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

Branch: refs/heads/gearpump-runner
Commit: 50c1c88160c013dcc36ba6488b43405ad2463842
Parents: 8f68085
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Tue Sep 6 07:26:45 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 runners/flink/examples/pom.xml                  |  2 --
 .../beam/runners/flink/examples/WordCount.java  |  9 ++++++
 .../runners/flink/examples/package-info.java    | 22 +++++++++++++
 .../flink/examples/streaming/AutoComplete.java  |  5 +--
 .../flink/examples/streaming/JoinExamples.java  |  3 +-
 .../examples/streaming/KafkaIOExamples.java     | 34 ++++++++++----------
 .../KafkaWindowedWordCountExample.java          | 27 +++++++++++++---
 .../examples/streaming/WindowedWordCount.java   | 19 +++++++----
 .../flink/examples/streaming/package-info.java  | 22 +++++++++++++
 9 files changed, 110 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index 9f705db..b8489fc 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -109,12 +109,10 @@
         </executions>
       </plugin>
 
-      <!-- Checkstyle errors for now
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
-      -->
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 ab9297f..9cce757 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
@@ -36,8 +36,14 @@ import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
+/**
+ * Wordcount pipeline.
+ */
 public class WordCount {
 
+  /**
+   * Function to extract words.
+   */
   public static class ExtractWordsFn extends DoFn<String, String> {
     private final Aggregator<Long, Long> emptyLines =
         createAggregator("emptyLines", new Sum.SumLongFn());
@@ -60,6 +66,9 @@ public class WordCount {
     }
   }
 
+  /**
+   * PTransform counting words.
+   */
   public static class CountWords extends PTransform<PCollection<String>,
                     PCollection<KV<String, Long>>> {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
new file mode 100644
index 0000000..b0ecb56
--- /dev/null
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Flink Beam runner exemple.
+ */
+package org.apache.beam.runners.flink.examples;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 9b5e31d..4636e3f 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
@@ -96,7 +96,8 @@ public class AutoComplete {
 
               @ProcessElement
               public void processElement(ProcessContext c) {
-                CompletionCandidate cand = new CompletionCandidate(c.element().getKey(), c.element().getValue());
+                CompletionCandidate cand = new CompletionCandidate(c.element().getKey(),
+                    c.element().getValue());
                 c.output(cand);
               }
             }));
@@ -349,7 +350,7 @@ public class AutoComplete {
       StringBuilder str = new StringBuilder();
       KV<String, List<CompletionCandidate>> elem = c.element();
 
-      str.append(elem.getKey() +" @ "+ window +" -> ");
+      str.append(elem.getKey() + " @ " + window + " -> ");
       for (CompletionCandidate cand: elem.getValue()) {
         str.append(cand.toString() + " ");
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 bf5dfc4..96638aa 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
@@ -132,7 +132,8 @@ public class JoinExamples {
     options.setExecutionRetryDelay(3000L);
     options.setRunner(FlinkRunner.class);
 
-    WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+    WindowFn<Object, ?> windowFn = FixedWindows.of(
+        Duration.standardSeconds(options.getWindowSize()));
 
     Pipeline p = Pipeline.create(options);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 27faefe..f0bf188 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
@@ -56,15 +56,15 @@ public class KafkaIOExamples {
   private static final String KAFKA_AVRO_TOPIC = "output";  // Default kafka topic to read from
   private static final String KAFKA_BROKER = "localhost:9092";  // Default kafka broker to contact
   private static final String GROUP_ID = "myGroup";  // Default groupId
-  private static final String ZOOKEEPER = "localhost:2181";  // Default zookeeper to connect to for Kafka
+  private static final String ZOOKEEPER = "localhost:2181";  // Default zookeeper to connect (Kafka)
 
   /**
-   * Read/Write String data to Kafka
+   * Read/Write String data to Kafka.
    */
   public static class KafkaString {
 
     /**
-     * Read String data from Kafka
+     * Read String data from Kafka.
      */
     public static class ReadStringFromKafka {
 
@@ -88,7 +88,7 @@ public class KafkaIOExamples {
     }
 
     /**
-     * Write String data to Kafka
+     * Write String data to Kafka.
      */
     public static class WriteStringToKafka {
 
@@ -113,12 +113,12 @@ public class KafkaIOExamples {
   }
 
   /**
-   * Read/Write Avro data to Kafka
+   * Read/Write Avro data to Kafka.
    */
   public static class KafkaAvro {
 
     /**
-     * Read Avro data from Kafka
+     * Read Avro data from Kafka.
      */
     public static class ReadAvroFromKafka {
 
@@ -142,7 +142,7 @@ public class KafkaIOExamples {
     }
 
     /**
-     * Write Avro data to Kafka
+     * Write Avro data to Kafka.
      */
     public static class WriteAvroToKafka {
 
@@ -169,7 +169,7 @@ public class KafkaIOExamples {
     }
 
     /**
-     * Serialiation/Deserialiation schema for Avro types
+     * Serialiation/Deserialiation schema for Avro types.
      * @param <T>
      */
     static class AvroSerializationDeserializationSchema<T>
@@ -217,7 +217,7 @@ public class KafkaIOExamples {
     }
 
     /**
-     * Custom type for Avro serialization
+     * Custom type for Avro serialization.
      */
     static class MyType implements Serializable {
 
@@ -233,10 +233,10 @@ public class KafkaIOExamples {
 
       @Override
       public String toString() {
-        return "MyType{" +
-            "word='" + word + '\'' +
-            ", count=" + count +
-            '}';
+        return "MyType{"
+            + "word='" + word + '\''
+            + ", count=" + count
+            + '}';
       }
     }
   }
@@ -244,7 +244,7 @@ public class KafkaIOExamples {
   // -------------- Utilities --------------
 
   /**
-   * Custom options for the Pipeline
+   * Custom options for the Pipeline.
    */
   public interface KafkaOptions extends FlinkPipelineOptions {
     @Description("The Kafka topic to read from")
@@ -279,7 +279,7 @@ public class KafkaIOExamples {
   }
 
   /**
-   * Initializes some options for the Flink runner
+   * Initializes some options for the Flink runner.
    * @param args The command line args
    * @return the pipeline
    */
@@ -298,7 +298,7 @@ public class KafkaIOExamples {
   }
 
   /**
-   * Gets KafkaOptions from the Pipeline
+   * Gets KafkaOptions from the Pipeline.
    * @param p the pipeline
    * @return KafkaOptions
    */
@@ -322,7 +322,7 @@ public class KafkaIOExamples {
   }
 
   /**
-   * Print contents to stdout
+   * Print contents to stdout.
    * @param <T> type of the input
    */
   private static class PrintFn<T> extends DoFn<T, T> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 365fb7b..42c42f3 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
@@ -40,6 +40,9 @@ import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.joda.time.Duration;
 
+/**
+ * Wordcount example using Kafka topic.
+ */
 public class KafkaWindowedWordCountExample {
 
   static final String KAFKA_TOPIC = "test";  // Default kafka topic to read from
@@ -47,6 +50,9 @@ public class KafkaWindowedWordCountExample {
   static final String GROUP_ID = "myGroup";  // Default groupId
   static final String ZOOKEEPER = "localhost:2181";  // Default zookeeper to connect to for Kafka
 
+  /**
+   * Function to extract words.
+   */
   public static class ExtractWordsFn extends DoFn<String, String> {
     private final Aggregator<Long, Long> emptyLines =
         createAggregator("emptyLines", new Sum.SumLongFn());
@@ -69,16 +75,24 @@ public class KafkaWindowedWordCountExample {
     }
   }
 
+  /**
+   * Function to format KV as String.
+   */
   public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
     @ProcessElement
     public void processElement(ProcessContext c) {
-      String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+      String row = c.element().getKey() + " - " + c.element().getValue() + " @ "
+          + c.timestamp().toString();
       System.out.println(row);
       c.output(row);
     }
   }
 
-  public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
+  /**
+   * Pipeline options.
+   */
+  public interface KafkaStreamingWordCountOptions
+      extends WindowedWordCount.StreamingWordCountOptions {
     @Description("The Kafka topic to read from")
     @Default.String(KAFKA_TOPIC)
     String getKafkaTopic();
@@ -107,7 +121,8 @@ public class KafkaWindowedWordCountExample {
 
   public static void main(String[] args) {
     PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
-    KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
+    KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args)
+        .as(KafkaStreamingWordCountOptions.class);
     options.setJobName("KafkaExample - WindowSize: " + options.getWindowSize() + " seconds");
     options.setStreaming(true);
     options.setCheckpointingInterval(1000L);
@@ -115,7 +130,8 @@ public class KafkaWindowedWordCountExample {
     options.setExecutionRetryDelay(3000L);
     options.setRunner(FlinkRunner.class);
 
-    System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
+    System.out.println(options.getKafkaTopic() + " " + options.getZookeeper() + " "
+        + options.getBroker() + " " + options.getGroup());
     Pipeline pipeline = Pipeline.create(options);
 
     Properties p = new Properties();
@@ -132,7 +148,8 @@ public class KafkaWindowedWordCountExample {
     PCollection<String> words = pipeline
         .apply("StreamingWordCount", Read.from(UnboundedFlinkSource.of(kafkaConsumer)))
         .apply(ParDo.of(new ExtractWordsFn()))
-        .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
+        .apply(Window.<String>into(FixedWindows.of(
+            Duration.standardSeconds(options.getWindowSize())))
             .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
             .discardingFiredPanes());
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/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 f3361c5..0e250b8 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
@@ -60,7 +60,8 @@ public class WindowedWordCount {
   static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
     @ProcessElement
     public void processElement(ProcessContext c) {
-      String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+      String row = c.element().getKey() + " - " + c.element().getValue() + " @ "
+          + c.timestamp().toString();
       c.output(row);
     }
   }
@@ -87,7 +88,11 @@ public class WindowedWordCount {
     }
   }
 
-  public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options {
+  /**
+   * Pipeline options.
+   */
+  public interface StreamingWordCountOptions
+      extends org.apache.beam.runners.flink.examples.WordCount.Options {
     @Description("Sliding window duration, in seconds")
     @Default.Long(WINDOW_SIZE)
     Long getWindowSize();
@@ -102,7 +107,8 @@ public class WindowedWordCount {
   }
 
   public static void main(String[] args) throws IOException {
-    StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
+    StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
+        .as(StreamingWordCountOptions.class);
     options.setStreaming(true);
     options.setWindowSize(10L);
     options.setSlide(5L);
@@ -111,8 +117,8 @@ public class WindowedWordCount {
     options.setExecutionRetryDelay(3000L);
     options.setRunner(FlinkRunner.class);
 
-    LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
-        " sec. and a slide of " + options.getSlide());
+    LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize()
+        + " sec. and a slide of " + options.getSlide());
 
     Pipeline pipeline = Pipeline.create(options);
 
@@ -120,7 +126,8 @@ public class WindowedWordCount {
         .apply("StreamingWordCount",
             Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
         .apply(ParDo.of(new ExtractWordsFn()))
-        .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
+        .apply(Window.<String>into(SlidingWindows.of(
+            Duration.standardSeconds(options.getWindowSize()))
             .every(Duration.standardSeconds(options.getSlide())))
             .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
             .discardingFiredPanes());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/50c1c881/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
new file mode 100644
index 0000000..58f41b6
--- /dev/null
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Flink Beam runner exemple.
+ */
+package org.apache.beam.runners.flink.examples.streaming;


[22/50] [abbrv] incubator-beam git commit: FluentBackoff: a replacement for a variety of custom backoff implementations

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 6aff3b0..8b5e8c2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -53,14 +53,13 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.options.BigQueryOptions;
 import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.Transport;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,14 +71,14 @@ class BigQueryServicesImpl implements BigQueryServices {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class);
 
-  // The maximum number of attempts to execute a BigQuery RPC.
-  private static final int MAX_RPC_ATTEMPTS = 10;
+  // The maximum number of retries to execute a BigQuery RPC.
+  private static final int MAX_RPC_RETRIES = 9;
 
   // The initial backoff for executing a BigQuery RPC.
-  private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+  private static final Duration INITIAL_RPC_BACKOFF = Duration.standardSeconds(1);
 
   // The initial backoff for polling the status of a BigQuery job.
-  private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+  private static final Duration INITIAL_JOB_STATUS_POLL_BACKOFF = Duration.standardSeconds(1);
 
   @Override
   public JobService getJobService(BigQueryOptions options) {
@@ -121,9 +120,9 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void startLoadJob(
@@ -139,9 +138,9 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig)
@@ -157,9 +156,9 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig)
@@ -175,9 +174,9 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig)
@@ -194,7 +193,8 @@ class BigQueryServicesImpl implements BigQueryServices {
       ApiErrorExtractor errorExtractor,
       Bigquery client) throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff);
     }
 
@@ -227,15 +227,17 @@ class BigQueryServicesImpl implements BigQueryServices {
       throw new IOException(
           String.format(
               "Unable to insert job: %s, aborting after %d .",
-              jobRef.getJobId(), MAX_RPC_ATTEMPTS),
+              jobRef.getJobId(), MAX_RPC_RETRIES),
           lastException);
     }
 
     @Override
     public Job pollJob(JobReference jobRef, int maxAttempts)
         throws InterruptedException {
-      BackOff backoff = new AttemptBoundedExponentialBackOff(
-          maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS);
+      BackOff backoff =
+          FluentBackoff.DEFAULT
+              .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF)
+              .backoff();
       return pollJob(jobRef, Sleeper.DEFAULT, backoff);
     }
 
@@ -270,12 +272,13 @@ class BigQueryServicesImpl implements BigQueryServices {
                   .setQuery(query))
               .setDryRun(true));
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       return executeWithRetries(
           client.jobs().insert(projectId, job),
           String.format(
               "Unable to dry run query: %s, aborting after %d retries.",
-              query, MAX_RPC_ATTEMPTS),
+              query, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff).getStatistics();
     }
@@ -289,15 +292,14 @@ class BigQueryServicesImpl implements BigQueryServices {
     // The maximum number of rows to upload per InsertAll request.
     private static final long MAX_ROWS_PER_BATCH = 500;
 
-    // The maximum number of times to retry inserting rows into BigQuery.
-    private static final int MAX_INSERT_ATTEMPTS = 5;
-
-    // The initial backoff after a failure inserting rows into BigQuery.
-    private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L;
+    private static final FluentBackoff INSERT_BACKOFF_FACTORY =
+        FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5);
 
-    // Backoff time bounds for rate limit exceeded errors.
-    private static final long INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.SECONDS.toMillis(1);
-    private static final long MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.MINUTES.toMillis(2);
+    // A backoff for rate limit exceeded errors. Retries forever.
+    private static final FluentBackoff DEFAULT_BACKOFF_FACTORY =
+        FluentBackoff.DEFAULT
+            .withInitialBackoff(Duration.standardSeconds(1))
+            .withMaxBackoff(Duration.standardMinutes(2));
 
     private final ApiErrorExtractor errorExtractor;
     private final Bigquery client;
@@ -335,20 +337,21 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public Table getTable(String projectId, String datasetId, String tableId)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       return executeWithRetries(
           client.tables().get(projectId, datasetId, tableId),
           String.format(
               "Unable to get table: %s, aborting after %d retries.",
-              tableId, MAX_RPC_ATTEMPTS),
+              tableId, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff);
     }
@@ -356,20 +359,21 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void deleteTable(String projectId, String datasetId, String tableId)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       executeWithRetries(
           client.tables().delete(projectId, datasetId, tableId),
           String.format(
               "Unable to delete table: %s, aborting after %d retries.",
-              tableId, MAX_RPC_ATTEMPTS),
+              tableId, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff);
     }
@@ -378,12 +382,13 @@ class BigQueryServicesImpl implements BigQueryServices {
     public boolean isTableEmpty(String projectId, String datasetId, String tableId)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       TableDataList dataList = executeWithRetries(
           client.tabledata().list(projectId, datasetId, tableId),
           String.format(
               "Unable to list table data: %s, aborting after %d retries.",
-              tableId, MAX_RPC_ATTEMPTS),
+              tableId, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff);
       return dataList.getRows() == null || dataList.getRows().isEmpty();
@@ -392,20 +397,21 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public Dataset getDataset(String projectId, String datasetId)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       return executeWithRetries(
           client.datasets().get(projectId, datasetId),
           String.format(
               "Unable to get dataset: %s, aborting after %d retries.",
-              datasetId, MAX_RPC_ATTEMPTS),
+              datasetId, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff);
     }
@@ -413,21 +419,21 @@ class BigQueryServicesImpl implements BigQueryServices {
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void createDataset(
         String projectId, String datasetId, String location, String description)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       createDataset(projectId, datasetId, location, description, Sleeper.DEFAULT, backoff);
     }
 
-    @VisibleForTesting
-    void createDataset(
+    private void createDataset(
         String projectId,
         String datasetId,
         String location,
@@ -464,27 +470,28 @@ class BigQueryServicesImpl implements BigQueryServices {
       throw new IOException(
           String.format(
               "Unable to create dataset: %s, aborting after %d .",
-              datasetId, MAX_RPC_ATTEMPTS),
+              datasetId, MAX_RPC_RETRIES),
           lastException);
     }
 
     /**
      * {@inheritDoc}
      *
-     * <p>Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     * <p>Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds.
      *
-     * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts.
+     * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts.
      */
     @Override
     public void deleteDataset(String projectId, String datasetId)
         throws IOException, InterruptedException {
       BackOff backoff =
-          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff();
       executeWithRetries(
           client.datasets().delete(projectId, datasetId),
           String.format(
               "Unable to delete table: %s, aborting after %d retries.",
-              datasetId, MAX_RPC_ATTEMPTS),
+              datasetId, MAX_RPC_RETRIES),
           Sleeper.DEFAULT,
           backoff);
     }
@@ -502,9 +509,7 @@ class BigQueryServicesImpl implements BigQueryServices {
             + "as many elements as rowList");
       }
 
-      AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
-          MAX_INSERT_ATTEMPTS,
-          INITIAL_INSERT_BACKOFF_INTERVAL_MS);
+      BackOff backoff = INSERT_BACKOFF_FACTORY.backoff();
 
       long retTotalDataSize = 0;
       List<TableDataInsertAllResponse.InsertErrors> allErrors = new ArrayList<>();
@@ -547,8 +552,7 @@ class BigQueryServicesImpl implements BigQueryServices {
                 executor.submit(new Callable<List<TableDataInsertAllResponse.InsertErrors>>() {
                   @Override
                   public List<TableDataInsertAllResponse.InsertErrors> call() throws IOException {
-                    BackOff backoff = new IntervalBoundedExponentialBackOff(
-                        MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS, INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS);
+                    BackOff backoff = DEFAULT_BACKOFF_FACTORY.backoff();
                     while (true) {
                       try {
                         return insert.execute().getInsertErrors();
@@ -603,21 +607,24 @@ class BigQueryServicesImpl implements BigQueryServices {
           throw new RuntimeException(e.getCause());
         }
 
-        if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) {
-          try {
-            Thread.sleep(backoff.nextBackOffMillis());
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException(
-                "Interrupted while waiting before retrying insert of " + retryRows);
-          }
-          LOG.info("Retrying failed inserts to BigQuery");
-          rowsToPublish = retryRows;
-          idsToPublish = retryIds;
-          allErrors.clear();
-        } else {
+        if (allErrors.isEmpty()) {
+          break;
+        }
+        long nextBackoffMillis = backoff.nextBackOffMillis();
+        if (nextBackoffMillis == BackOff.STOP) {
           break;
         }
+        try {
+          Thread.sleep(backoff.nextBackOffMillis());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException(
+              "Interrupted while waiting before retrying insert of " + retryRows);
+        }
+        LOG.info("Retrying failed inserts to BigQuery");
+        rowsToPublish = retryRows;
+        idsToPublish = retryIds;
+        allErrors.clear();
       }
       if (!allErrors.isEmpty()) {
         throw new IOException("Insert failed: " + allErrors);
@@ -628,7 +635,7 @@ class BigQueryServicesImpl implements BigQueryServices {
   }
 
   private static class BigQueryJsonReaderImpl implements BigQueryJsonReader {
-    BigQueryTableRowIterator iterator;
+    private BigQueryTableRowIterator iterator;
 
     private BigQueryJsonReaderImpl(BigQueryTableRowIterator iterator) {
       this.iterator = iterator;
@@ -706,7 +713,6 @@ class BigQueryServicesImpl implements BigQueryServices {
 
   /**
    * Identical to {@link BackOffUtils#next} but without checked IOException.
-   * @throws InterruptedException
    */
   private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
index 729da97..677c661 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java
@@ -58,7 +58,7 @@ import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -426,7 +426,8 @@ class BigQueryTableRowIterator implements AutoCloseable {
       Object... errorArgs) throws IOException, InterruptedException {
     Sleeper sleeper = Sleeper.DEFAULT;
     BackOff backOff =
-        new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_TIME.getMillis());
+        FluentBackoff.DEFAULT
+            .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_TIME).backoff();
 
     T result = null;
     while (true) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index 6bd03b5..45871f1 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -76,12 +76,13 @@ import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.transforms.Values;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
 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.PDone;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -862,16 +863,11 @@ public class DatastoreV1 {
     private final V1DatastoreFactory datastoreFactory;
     // Current batch of mutations to be written.
     private final List<Mutation> mutations = new ArrayList<>();
-    /**
-     * Since a bundle is written in batches, we should retry the commit of a batch in order to
-     * prevent transient errors from causing the bundle to fail.
-     */
-    private static final int MAX_RETRIES = 5;
 
-    /**
-     * Initial backoff time for exponential backoff for retry attempts.
-     */
-    private static final int INITIAL_BACKOFF_MILLIS = 5000;
+    private static final int MAX_RETRIES = 5;
+    private static final FluentBackoff BUNDLE_WRITE_BACKOFF =
+        FluentBackoff.DEFAULT
+            .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5));
 
     DatastoreWriterFn(String projectId) {
       this(projectId, new V1DatastoreFactory());
@@ -906,10 +902,10 @@ public class DatastoreV1 {
     /**
      * Writes a batch of mutations to Cloud Datastore.
      *
-     * <p>If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES}
-     * times). All mutations in the batch will be committed again, even if the commit was partially
-     * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will
-     * be thrown.
+     * <p>If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. All
+     * mutations in the batch will be committed again, even if the commit was partially
+     * successful. If the retry limit is exceeded, the last exception from Cloud Datastore will be
+     * thrown.
      *
      * @throws DatastoreException if the commit fails or IOException or InterruptedException if
      * backing off between retries fails.
@@ -917,7 +913,7 @@ public class DatastoreV1 {
     private void flushBatch() throws DatastoreException, IOException, InterruptedException {
       LOG.debug("Writing batch of {} mutations", mutations.size());
       Sleeper sleeper = Sleeper.DEFAULT;
-      BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS);
+      BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff();
 
       while (true) {
         // Batch upsert entities.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
index eb5fbe6..16cb004 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
@@ -56,7 +56,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.ExpectedLogs;
 import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
 import org.apache.beam.sdk.util.Transport;
 import org.junit.Before;
@@ -117,9 +117,8 @@ public class BigQueryServicesImplTest {
     when(response.getContent()).thenReturn(toStream(testJob));
 
     Sleeper sleeper = new FastNanoClockAndSleeper();
-    BackOff backoff = new AttemptBoundedExponentialBackOff(
-        5 /* attempts */, 1000 /* initialIntervalMillis */);
-    JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+    JobServiceImpl.startJob(
+        testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff());
 
     verify(response, times(1)).getStatusCode();
     verify(response, times(1)).getContent();
@@ -141,9 +140,8 @@ public class BigQueryServicesImplTest {
     when(response.getStatusCode()).thenReturn(409); // 409 means already exists
 
     Sleeper sleeper = new FastNanoClockAndSleeper();
-    BackOff backoff = new AttemptBoundedExponentialBackOff(
-        5 /* attempts */, 1000 /* initialIntervalMillis */);
-    JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+    JobServiceImpl.startJob(
+        testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff());
 
     verify(response, times(1)).getStatusCode();
     verify(response, times(1)).getContent();
@@ -169,9 +167,8 @@ public class BigQueryServicesImplTest {
         .thenReturn(toStream(testJob));
 
     Sleeper sleeper = new FastNanoClockAndSleeper();
-    BackOff backoff = new AttemptBoundedExponentialBackOff(
-        5 /* attempts */, 1000 /* initialIntervalMillis */);
-    JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff);
+    JobServiceImpl.startJob(
+        testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff());
 
     verify(response, times(2)).getStatusCode();
     verify(response, times(2)).getContent();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
index a596bb3..b680a0e 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
@@ -54,8 +54,9 @@ import javax.annotation.Nullable;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -232,7 +233,7 @@ class V1TestUtil {
     // Number of times to retry on update failure
     private static final int MAX_RETRIES = 5;
     //Initial backoff time for exponential backoff for retry attempts.
-    private static final int INITIAL_BACKOFF_MILLIS = 5000;
+    private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(5);
 
     // Returns true if a Datastore key is complete. A key is complete if its last element
     // has either an id or a name.
@@ -279,7 +280,9 @@ class V1TestUtil {
     private void flushBatch() throws DatastoreException, IOException, InterruptedException {
       LOG.info("Writing batch of {} entities", entities.size());
       Sleeper sleeper = Sleeper.DEFAULT;
-      BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS);
+      BackOff backoff =
+          FluentBackoff.DEFAULT
+              .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF).backoff();
 
       while (true) {
         // Batch mutate entities.


[44/50] [abbrv] incubator-beam git commit: fix import order

Posted by dh...@apache.org.
fix import order


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

Branch: refs/heads/gearpump-runner
Commit: 59ae94c59931732d5cf78c5431147d580f9ff747
Parents: 6cd48c4
Author: manuzhang <ow...@gmail.com>
Authored: Mon Sep 12 11:45:15 2016 +0800
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 runners/gearpump/pom.xml                        |  7 ++---
 .../gearpump/GearpumpPipelineOptions.java       |  8 +++---
 .../gearpump/GearpumpPipelineResult.java        |  4 ++-
 .../gearpump/GearpumpPipelineRunner.java        | 28 ++++++++++----------
 .../GearpumpPipelineRunnerRegistrar.java        |  9 +++----
 .../gearpump/GearpumpPipelineTranslator.java    |  7 +++--
 .../gearpump/examples/StreamingWordCount.java   | 15 +++--------
 .../gearpump/examples/UnboundedTextSource.java  | 15 ++++++-----
 .../translators/GroupByKeyTranslator.java       | 14 +++++-----
 .../translators/ParDoBoundMultiTranslator.java  | 16 +++++------
 .../translators/TransformTranslator.java        |  3 +--
 .../translators/TranslationContext.java         |  6 ++---
 .../translators/functions/DoFnFunction.java     | 14 +++++-----
 .../translators/io/BoundedSourceWrapper.java    |  4 +--
 .../gearpump/translators/io/GearpumpSource.java | 12 +++++----
 .../translators/io/UnboundedSourceWrapper.java  |  4 +--
 .../gearpump/translators/io/ValuesSource.java   | 12 ++++-----
 .../translators/utils/GearpumpDoFnRunner.java   | 28 ++++++++++----------
 .../translators/utils/NoOpSideInputReader.java  |  8 +++---
 .../translators/utils/NoOpStepContext.java      |  6 ++---
 .../main/java/org/apache/beam/sdk/Pipeline.java |  2 +-
 .../apache/beam/sdk/runners/PipelineRunner.java |  1 +
 .../beam/sdk/transforms/DoFnAdapters.java       |  5 ++++
 23 files changed, 114 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/pom.xml
----------------------------------------------------------------------
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index cc99a7a..296de6b 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -122,7 +122,6 @@
       <groupId>org.apache.gearpump</groupId>
       <artifactId>gearpump-daemon_2.11</artifactId>
       <version>${gearpump.version}</version>
-      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <groupId>org.apache.gearpump</groupId>
@@ -186,10 +185,6 @@
       <artifactId>jackson-annotations</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.http-client</groupId>
-      <artifactId>google-http-client</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
@@ -225,6 +220,7 @@
       <artifactId>auto-service</artifactId>
       <version>1.0-rc2</version>
     </dependency>
+
   </dependencies>
 
   <build>
@@ -287,6 +283,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
+
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java
index 5b6ee96..e02cbbc 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java
@@ -18,17 +18,17 @@
 
 package org.apache.beam.runners.gearpump;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.Map;
+
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
 
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
 import org.apache.gearpump.cluster.client.ClientContext;
 import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
 
-import java.util.Map;
-
 /**
  * Options that configure the Gearpump pipeline.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
index 6184bc3..2011a4b 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
@@ -17,14 +17,16 @@
  */
 package org.apache.beam.runners.gearpump;
 
+import java.io.IOException;
+
 import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.AggregatorValues;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.transforms.Aggregator;
+
 import org.joda.time.Duration;
 
-import java.io.IOException;
 
 /**
  * Result of executing a {@link Pipeline} with Gearpump.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
index 4182ee4..ad7bb3e 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java
@@ -17,6 +17,13 @@
  */
 package org.apache.beam.runners.gearpump;
 
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.beam.runners.core.AssignWindows;
 import org.apache.beam.runners.gearpump.translators.TranslationContext;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -30,25 +37,18 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AssignWindows;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigValueFactory;
-
 import org.apache.gearpump.cluster.ClusterConfig;
 import org.apache.gearpump.cluster.UserConfig;
 import org.apache.gearpump.cluster.client.ClientContext;
 import org.apache.gearpump.cluster.embedded.EmbeddedCluster;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * A {@link PipelineRunner} that executes the operations in the
  * pipeline by first translating them to Gearpump Stream DSL
@@ -79,16 +79,16 @@ public class GearpumpPipelineRunner extends PipelineRunner<GearpumpPipelineResul
       PTransform<InputT, OutputT> transform, InputT input) {
     if (Window.Bound.class.equals(transform.getClass())) {
       return (OutputT) super.apply(
-          new AssignWindowsAndSetStrategy((Window.Bound) transform), input);
+              new AssignWindowsAndSetStrategy((Window.Bound) transform), input);
     } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
-        && ((PCollectionList<?>) input).size() == 0) {
-      return (OutputT) Pipeline.applyTransform(input, Create.of());
+            && ((PCollectionList<?>) input).size() == 0) {
+      return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of());
     } else if (Create.Values.class.equals(transform.getClass())) {
       return (OutputT) PCollection
-          .<OutputT>createPrimitiveOutputInternal(
-              input.getPipeline(),
-              WindowingStrategy.globalDefault(),
-              PCollection.IsBounded.BOUNDED);
+              .<OutputT>createPrimitiveOutputInternal(
+                      input.getPipeline(),
+                      WindowingStrategy.globalDefault(),
+                      PCollection.IsBounded.BOUNDED);
     } else {
       return super.apply(transform, input);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
index 2b9e89e..ca173d1 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java
@@ -18,14 +18,14 @@
 
 package org.apache.beam.runners.gearpump;
 
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
 
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
 /**
  * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
  * {@link GearpumpPipelineRunner}.
@@ -44,8 +44,7 @@ public class GearpumpPipelineRunnerRegistrar {
 
     @Override
     public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
-      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
-          TestGearpumpRunner.class);
+      return ImmutableList.<Class<? extends PipelineRunner<?>>>of(TestGearpumpRunner.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
index 59f0df7..5045ae4 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
@@ -18,6 +18,8 @@
 
 package org.apache.beam.runners.gearpump;
 
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator;
 import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator;
 import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator;
@@ -41,9 +43,6 @@ import org.apache.gearpump.util.Graph;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects
  * into Gearpump {@link Graph}.
@@ -109,7 +108,7 @@ public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor {
 
   @Override
   public void visitValue(PValue value, TransformTreeNode producer) {
-    LOG.debug("visiting value {}", value);
+    LOG.info("visiting value {}", value);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
index 5f35c6b..ba50de7 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java
@@ -23,11 +23,9 @@ import org.apache.beam.runners.gearpump.GearpumpPipelineRunner;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.KV;
@@ -45,15 +43,9 @@ import org.slf4j.LoggerFactory;
 public class StreamingWordCount {
 
   static class ExtractWordsFn extends OldDoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
 
     @Override
     public void processElement(ProcessContext c) {
-      if (c.element().trim().isEmpty()) {
-        emptyLines.addValue(1L);
-      }
-
       // Split the line into words.
       String[] words = c.element().split("[^a-zA-Z']+");
 
@@ -81,11 +73,12 @@ public class StreamingWordCount {
 
 
   public static void main(String[] args) {
-    GearpumpPipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-        .as(GearpumpPipelineOptions.class);
-    options.setApplicationName("StreamingWordCount");
+    GearpumpPipelineOptions options = PipelineOptionsFactory
+            .fromArgs(args).as(GearpumpPipelineOptions.class);
     options.setRunner(GearpumpPipelineRunner.class);
+    options.setApplicationName("StreamingWordCount");
     options.setParallelism(1);
+
     Pipeline p = Pipeline.create(options);
 
     PCollection<KV<String, Long>> wordCounts =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java
index caf066c..b014432 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java
@@ -18,13 +18,6 @@
 
 package org.apache.beam.runners.gearpump.examples;
 
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collections;
@@ -33,6 +26,14 @@ import java.util.NoSuchElementException;
 
 import javax.annotation.Nullable;
 
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import org.joda.time.Instant;
+
+
 /**
  * unbounded source that reads from text.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
index f36b908..43e3336 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
@@ -18,23 +18,25 @@
 
 package org.apache.beam.runners.gearpump.translators;
 
+import com.google.common.collect.Iterables;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 
-import com.google.common.collect.Iterables;
-
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
+
 
 /**
  * {@link GroupByKey} is translated to Gearpump groupBy function.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java
index d5ed0d2..2b49684 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java
@@ -18,6 +18,14 @@
 
 package org.apache.beam.runners.gearpump.translators;
 
+import com.google.common.collect.Lists;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
 import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner;
 import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader;
@@ -25,8 +33,6 @@ import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
@@ -35,17 +41,11 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
-import com.google.common.collect.Lists;
-
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.FilterFunction;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction;
 import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction;
 
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 /**
  * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function
  * with {@link DoFn} wrapped in {@link DoFnMultiFunction}. The outputs are

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
index 1ed6d5d..c8587d3 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java
@@ -18,11 +18,10 @@
 
 package org.apache.beam.runners.gearpump.translators;
 
+import java.io.Serializable;
 
 import org.apache.beam.sdk.transforms.PTransform;
 
-import java.io.Serializable;
-
 /**
  * translates {@link PTransform} to Gearpump functions.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
index b9b2c7a..d3bc75d 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
@@ -20,6 +20,9 @@ package org.apache.beam.runners.gearpump.translators;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
 import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
@@ -33,9 +36,6 @@ import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp;
 import org.apache.gearpump.streaming.source.DataSource;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * Maintains context data for {@link TransformTranslator}s.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java
index b1ebd2a..8d16356 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java
@@ -18,26 +18,26 @@
 
 package org.apache.beam.runners.gearpump.translators.functions;
 
+import com.google.common.collect.Lists;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
 import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner;
 import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
 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.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
-import com.google.api.client.util.Lists;
-
 import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction;
 
-import java.util.Iterator;
-import java.util.List;
-
 /**
  * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java
index f25d113..f889101 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java
@@ -18,12 +18,12 @@
 
 package org.apache.beam.runners.gearpump.translators.io;
 
+import java.io.IOException;
+
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.Source;
 import org.apache.beam.sdk.options.PipelineOptions;
 
-import java.io.IOException;
-
 /**
  * wrapper over BoundedSource for Gearpump DataSource API.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java
index 892ccc3..8f2beb2 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java
@@ -18,23 +18,23 @@
 
 package org.apache.beam.runners.gearpump.translators.io;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+
 import org.apache.beam.sdk.io.Source;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.WindowedValue;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 import org.apache.gearpump.Message;
 import org.apache.gearpump.streaming.source.DataSource;
 import org.apache.gearpump.streaming.task.TaskContext;
 
 import org.joda.time.Instant;
 
-import java.io.IOException;
-
 /**
  * common methods for {@link BoundedSourceWrapper} and {@link UnboundedSourceWrapper}.
  */
@@ -61,6 +61,7 @@ public abstract class GearpumpSource<T> implements DataSource {
       PipelineOptions options = new ObjectMapper()
           .readValue(serializedOptions, PipelineOptions.class);
       this.reader = createReader(options);
+      this.available = reader.start();
     } catch (Exception e) {
       throw new RuntimeException(e);
     } finally {
@@ -97,4 +98,5 @@ public abstract class GearpumpSource<T> implements DataSource {
       throw new RuntimeException(e);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
index b39f29f..dfdecb2 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java
@@ -18,12 +18,12 @@
 
 package org.apache.beam.runners.gearpump.translators.io;
 
+import java.io.IOException;
+
 import org.apache.beam.sdk.io.Source;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
 
-import java.io.IOException;
-
 /**
  * wrapper over UnboundedSource for Gearpump DataSource API.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
index 24055f7..9359e35 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
@@ -18,12 +18,6 @@
 
 package org.apache.beam.runners.gearpump.translators.io;
 
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import org.joda.time.Instant;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -36,6 +30,12 @@ import java.util.NoSuchElementException;
 
 import javax.annotation.Nullable;
 
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import org.joda.time.Instant;
+
 /**
  * unbounded source that reads from a Java {@link Iterable}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
index be0d025..e205575 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java
@@ -18,6 +18,20 @@
 
 package org.apache.beam.runners.gearpump.translators.utils;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.SimpleDoFnRunner;
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
@@ -31,11 +45,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.DoFnRunner;
-import org.apache.beam.sdk.util.DoFnRunners;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.util.SimpleDoFnRunner;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.UserCodeException;
@@ -46,19 +57,8 @@ import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-
 import org.joda.time.Instant;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
 /**
  * a serializable {@link SimpleDoFnRunner}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java
index 600ebfb..d1a9198 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java
@@ -18,14 +18,14 @@
 
 package org.apache.beam.runners.gearpump.translators.utils;
 
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.SideInputReader;
-import org.apache.beam.sdk.values.PCollectionView;
-
 import java.io.Serializable;
 
 import javax.annotation.Nullable;
 
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.values.PCollectionView;
+
 /**
  * no-op side input reader.
  */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java
index ce0935a..45f146b 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java
@@ -18,6 +18,9 @@
 
 package org.apache.beam.runners.gearpump.translators.utils;
 
+import java.io.IOException;
+import java.io.Serializable;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.ExecutionContext;
@@ -26,9 +29,6 @@ import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.TupleTag;
 
-import java.io.IOException;
-import java.io.Serializable;
-
 /**
  * serializable {@link ExecutionContext.StepContext} that basically does nothing.
  */

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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59ae94c5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
index 4803d77..642971f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.transforms;
 
 import java.io.IOException;
+
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
@@ -31,6 +32,8 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}.
@@ -72,6 +75,8 @@ public class DoFnAdapters {
   private static class SimpleDoFnAdapter<InputT, OutputT> extends OldDoFn<InputT, OutputT> {
     private final DoFn<InputT, OutputT> fn;
     private transient DoFnInvoker<InputT, OutputT> invoker;
+    private static final Logger LOG =
+            LoggerFactory.getLogger(SimpleDoFnAdapter.class);
 
     SimpleDoFnAdapter(DoFn<InputT, OutputT> fn) {
       super(fn.aggregators);



[05/50] [abbrv] incubator-beam git commit: [BEAM-592] Fix SparkRunner Dependency Problem in WordCount

Posted by dh...@apache.org.
[BEAM-592] Fix SparkRunner Dependency Problem 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/1c1115ee
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/1c1115ee
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/1c1115ee

Branch: refs/heads/gearpump-runner
Commit: 1c1115eeeb59707a401513e903dfbae15a85a029
Parents: 973081e
Author: Mark Liu <ma...@markliu-macbookpro.roam.corp.google.com>
Authored: Fri Aug 26 14:23:54 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml  | 6 ++++++
 examples/java8/pom.xml | 6 ++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1c1115ee/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 096bc4e..47f0366 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -97,6 +97,12 @@
           <version>${spark.version}</version>
           <scope>runtime</scope>
           <optional>true</optional>
+          <exclusions>
+            <exclusion>
+              <groupId>org.slf4j</groupId>
+              <artifactId>jul-to-slf4j</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1c1115ee/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index f3f2c3c..44cf918 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -98,6 +98,12 @@
           <version>${spark.version}</version>
           <scope>runtime</scope>
           <optional>true</optional>
+          <exclusions>
+            <exclusion>
+              <groupId>org.slf4j</groupId>
+              <artifactId>jul-to-slf4j</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
 
         <dependency>


[46/50] [abbrv] incubator-beam git commit: !fixup Minor javadoc clean-up

Posted by dh...@apache.org.
!fixup Minor javadoc clean-up


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

Branch: refs/heads/gearpump-runner
Commit: 6cd48c4256539aa45e535729a7a1042f7ccb66fd
Parents: 34c731f
Author: Luke Cwik <lc...@google.com>
Authored: Wed Sep 7 15:38:16 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/io/TextIO.java    | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6cd48c42/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 e4fcc32..c0761b1 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
@@ -452,7 +452,7 @@ public class TextIO {
      * Returns a transform for writing to text files that adds a header string to the files
      * it writes.
      *
-     *<p> A <code>null</code> value will clear any previously configured header.</p>
+     * <p>A {@code null} value will clear any previously configured header.
      *
      * @param header the string to be added as file header
      */
@@ -464,7 +464,7 @@ public class TextIO {
      * Returns a transform for writing to text files that adds a footer string to the files
      * it writes.
      *
-     *<p> A <code>null</code> value will clear any previously configured footer.</p>
+     * <p>A {@code null} value will clear any previously configured footer.
      *
      * @param footer the string to be added as file footer
      */
@@ -636,7 +636,9 @@ public class TextIO {
        * Returns a transform for writing to text files that adds a header string to the files
        * it writes.
        *
-       *<p> A <code>null</code> value will clear any previously configured header.</p>
+       * <p>A {@code null} value will clear any previously configured header.
+       *
+       * <p>Does not modify this object.
        *
        * @param header the string to be added as file header
        */
@@ -649,7 +651,9 @@ public class TextIO {
        * Returns a transform for writing to text files that adds a footer string to the files
        * it writes.
        *
-       *<p> A <code>null</code> value will clear any previously configured footer.</p>
+       * <p>A {@code null} value will clear any previously configured footer.
+       *
+       * <p>Does not modify this object.
        *
        * @param footer the string to be added as file footer
        */


[32/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
deleted file mode 100644
index cc418da..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import com.google.common.collect.Iterables;
-import java.util.List;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
-import org.apache.beam.sdk.values.KV;
-import org.joda.time.Instant;
-
-/**
- * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path"
- * implementation is applicable.
- */
-@SystemDoFnInternal
-public class GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, OutputT, W extends BoundedWindow>
-   extends GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> {
-
-  private final WindowingStrategy<?, W> strategy;
-  private final StateInternalsFactory<K> stateInternalsFactory;
-  private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
-
-  public GroupAlsoByWindowsViaOutputBufferDoFn(
-      WindowingStrategy<?, W> windowingStrategy,
-      StateInternalsFactory<K> stateInternalsFactory,
-      SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
-    this.strategy = windowingStrategy;
-    this.reduceFn = reduceFn;
-    this.stateInternalsFactory = stateInternalsFactory;
-  }
-
-  @Override
-  public void processElement(
-      OldDoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>.ProcessContext c)
-          throws Exception {
-    K key = c.element().getKey();
-    // Used with Batch, we know that all the data is available for this key. We can't use the
-    // timer manager from the context because it doesn't exist. So we create one and emulate the
-    // watermark, knowing that we have all data and it is in timestamp order.
-    BatchTimerInternals timerInternals = new BatchTimerInternals(Instant.now());
-    StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);
-
-    ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
-        new ReduceFnRunner<K, InputT, OutputT, W>(
-            key,
-            strategy,
-            stateInternals,
-            timerInternals,
-            c.windowingInternals(),
-            droppedDueToClosedWindow,
-            reduceFn,
-            c.getPipelineOptions());
-
-    Iterable<List<WindowedValue<InputT>>> chunks =
-        Iterables.partition(c.element().getValue(), 1000);
-    for (Iterable<WindowedValue<InputT>> chunk : chunks) {
-      // Process the chunk of elements.
-      reduceFnRunner.processElements(chunk);
-
-      // Then, since elements are sorted by their timestamp, advance the input watermark
-      // to the first element, and fire any timers that may have been scheduled.
-      timerInternals.advanceInputWatermark(reduceFnRunner, chunk.iterator().next().getTimestamp());
-
-      // Fire any processing timers that need to fire
-      timerInternals.advanceProcessingTime(reduceFnRunner, Instant.now());
-
-      // Leave the output watermark undefined. Since there's no late data in batch mode
-      // there's really no need to track it as we do for streaming.
-    }
-
-    // Finish any pending windows by advancing the input watermark to infinity.
-    timerInternals.advanceInputWatermark(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    // Finally, advance the processing time to infinity to fire any timers.
-    timerInternals.advanceProcessingTime(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
-
-    reduceFnRunner.persist();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
deleted file mode 100644
index fdad17a..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-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.OldDoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly}
- * primitive.
- *
- * <p>This implementation of {@link GroupByKey} proceeds via the following steps:
- * <ol>
- *   <li>{@code ReifyTimestampsAndWindowsDoFn ParDo(ReifyTimestampsAndWindows)}: This embeds
- *       the previously-implicit timestamp and window into the elements themselves, so a
- *       window-and-timestamp-unaware transform can operate on them.</li>
- *   <li>{@code GroupByKeyOnly}: This lower-level primitive groups by keys, ignoring windows
- *       and timestamps. Many window-unaware runners have such a primitive already.</li>
- *   <li>{@code SortValuesByTimestamp ParDo(SortValuesByTimestamp)}: The values in the iterables
- *       output by {@link GroupByKeyOnly} are sorted by timestamp.</li>
- *   <li>{@code GroupAlsoByWindow}: This primitive processes the sorted values. Today it is
- *       implemented as a {@link ParDo} that calls reserved internal methods.</li>
- * </ol>
- *
- * <p>This implementation of {@link GroupByKey} has severe limitations unless its component
- * transforms are replaced. As-is, it is only applicable for in-memory runners using a batch-style
- * execution strategy. Specifically:
- *
- * <ul>
- *   <li>Every iterable output by {@link GroupByKeyOnly} must contain all elements for that key.
- *       A streaming-style partition, with multiple elements for the same key, will not yield
- *       correct results.</li>
- *   <li>Sorting of values by timestamp is performed on an in-memory list. It will not succeed
- *       for large iterables.</li>
- *   <li>The implementation of {@code GroupAlsoByWindow} does not support timers. This is only
- *       appropriate for runners which also do not support timers.</li>
- * </ul>
- */
-public class GroupByKeyViaGroupByKeyOnly<K, V>
-    extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
-  private final GroupByKey<K, V> gbkTransform;
-
-  public GroupByKeyViaGroupByKeyOnly(GroupByKey<K, V> originalTransform) {
-    this.gbkTransform = originalTransform;
-  }
-
-  @Override
-  public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-    WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
-    return input
-        // Make each input element's timestamp and assigned windows
-        // explicit, in the value part.
-        .apply(new ReifyTimestampsAndWindows<K, V>())
-
-        // Group by just the key.
-        // Combiner lifting will not happen regardless of the disallowCombinerLifting value.
-        // There will be no combiners right after the GroupByKeyOnly because of the two ParDos
-        // introduced in here.
-        .apply(new GroupByKeyOnly<K, WindowedValue<V>>())
-
-        // Sort each key's values by timestamp. GroupAlsoByWindow requires
-        // its input to be sorted by timestamp.
-        .apply(new SortValuesByTimestamp<K, V>())
-
-        // Group each key's values by window, merging windows as needed.
-        .apply(new GroupAlsoByWindow<K, V>(windowingStrategy))
-
-        // And update the windowing strategy as appropriate.
-        .setWindowingStrategyInternal(
-            gbkTransform.updateWindowingStrategy(windowingStrategy));
-  }
-
-  /**
-   * Runner-specific primitive that groups by key only, ignoring any window assignments. A
-   * runner that uses {@link GroupByKeyViaGroupByKeyOnly} should have a primitive way to translate
-   * or evaluate this class.
-   */
-  public static class GroupByKeyOnly<K, V>
-      extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    @Override
-    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
-      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
-          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
-    }
-
-    @Override
-    public Coder<KV<K, Iterable<V>>> getDefaultOutputCoder(PCollection<KV<K, V>> input) {
-      return GroupByKey.getOutputKvCoder(input.getCoder());
-    }
-  }
-
-  /**
-   * Helper transform that sorts the values associated with each key by timestamp.
-   */
-  private static class SortValuesByTimestamp<K, V>
-      extends PTransform<
-          PCollection<KV<K, Iterable<WindowedValue<V>>>>,
-          PCollection<KV<K, Iterable<WindowedValue<V>>>>> {
-    @Override
-    public PCollection<KV<K, Iterable<WindowedValue<V>>>> apply(
-        PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
-      return input
-          .apply(
-              ParDo.of(
-                  new OldDoFn<
-                      KV<K, Iterable<WindowedValue<V>>>,
-                      KV<K, Iterable<WindowedValue<V>>>>() {
-                    @Override
-                    public void processElement(ProcessContext c) {
-                      KV<K, Iterable<WindowedValue<V>>> kvs = c.element();
-                      K key = kvs.getKey();
-                      Iterable<WindowedValue<V>> unsortedValues = kvs.getValue();
-                      List<WindowedValue<V>> sortedValues = new ArrayList<>();
-                      for (WindowedValue<V> value : unsortedValues) {
-                        sortedValues.add(value);
-                      }
-                      Collections.sort(
-                          sortedValues,
-                          new Comparator<WindowedValue<V>>() {
-                            @Override
-                            public int compare(WindowedValue<V> e1, WindowedValue<V> e2) {
-                              return e1.getTimestamp().compareTo(e2.getTimestamp());
-                            }
-                          });
-                      c.output(KV.<K, Iterable<WindowedValue<V>>>of(key, sortedValues));
-                    }
-                  }))
-          .setCoder(input.getCoder());
-    }
-  }
-
-  /**
-   * Runner-specific primitive that takes a collection of timestamp-ordered values associated with
-   * each key, groups the values by window, merges windows as needed, and for each window in each
-   * key, outputs a collection of key/value-list pairs implicitly assigned to the window and with
-   * the timestamp derived from that window.
-   */
-  public static class GroupAlsoByWindow<K, V>
-      extends PTransform<
-          PCollection<KV<K, Iterable<WindowedValue<V>>>>, PCollection<KV<K, Iterable<V>>>> {
-    private final WindowingStrategy<?, ?> windowingStrategy;
-
-    public GroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
-      this.windowingStrategy = windowingStrategy;
-    }
-
-    public WindowingStrategy<?, ?> getWindowingStrategy() {
-      return windowingStrategy;
-    }
-
-    private KvCoder<K, Iterable<WindowedValue<V>>> getKvCoder(
-        Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
-      // Coder<KV<...>> --> KvCoder<...>
-      checkArgument(inputCoder instanceof KvCoder,
-          "%s requires a %s<...> but got %s",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName(),
-          inputCoder);
-      @SuppressWarnings("unchecked")
-      KvCoder<K, Iterable<WindowedValue<V>>> kvCoder =
-          (KvCoder<K, Iterable<WindowedValue<V>>>) inputCoder;
-      return kvCoder;
-    }
-
-    public Coder<K> getKeyCoder(Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
-      return getKvCoder(inputCoder).getKeyCoder();
-    }
-
-    public Coder<V> getValueCoder(Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
-      // Coder<Iterable<...>> --> IterableCoder<...>
-      Coder<Iterable<WindowedValue<V>>> iterableWindowedValueCoder =
-          getKvCoder(inputCoder).getValueCoder();
-      checkArgument(iterableWindowedValueCoder instanceof IterableCoder,
-          "%s requires a %s<..., %s> but got a %s",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName(),
-          IterableCoder.class.getSimpleName(),
-          iterableWindowedValueCoder);
-      IterableCoder<WindowedValue<V>> iterableCoder =
-          (IterableCoder<WindowedValue<V>>) iterableWindowedValueCoder;
-
-      // Coder<WindowedValue<...>> --> WindowedValueCoder<...>
-      Coder<WindowedValue<V>> iterableElementCoder = iterableCoder.getElemCoder();
-      checkArgument(iterableElementCoder instanceof WindowedValueCoder,
-          "%s requires a %s<..., %s<%s>> but got a %s",
-          getClass().getSimpleName(),
-          KvCoder.class.getSimpleName(),
-          IterableCoder.class.getSimpleName(),
-          WindowedValueCoder.class.getSimpleName(),
-          iterableElementCoder);
-      WindowedValueCoder<V> windowedValueCoder =
-          (WindowedValueCoder<V>) iterableElementCoder;
-
-      return windowedValueCoder.getValueCoder();
-    }
-
-    @Override
-    public PCollection<KV<K, Iterable<V>>> apply(
-        PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
-      @SuppressWarnings("unchecked")
-      KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
-          (KvCoder<K, Iterable<WindowedValue<V>>>) input.getCoder();
-
-      Coder<K> keyCoder = inputKvCoder.getKeyCoder();
-      Coder<Iterable<WindowedValue<V>>> inputValueCoder = inputKvCoder.getValueCoder();
-
-      IterableCoder<WindowedValue<V>> inputIterableValueCoder =
-          (IterableCoder<WindowedValue<V>>) inputValueCoder;
-      Coder<WindowedValue<V>> inputIterableElementCoder = inputIterableValueCoder.getElemCoder();
-      WindowedValueCoder<V> inputIterableWindowedValueCoder =
-          (WindowedValueCoder<V>) inputIterableElementCoder;
-
-      Coder<V> inputIterableElementValueCoder = inputIterableWindowedValueCoder.getValueCoder();
-      Coder<Iterable<V>> outputValueCoder = IterableCoder.of(inputIterableElementValueCoder);
-      Coder<KV<K, Iterable<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
-
-      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
-          input.getPipeline(), windowingStrategy, input.isBounded())
-          .setCoder(outputKvCoder);
-    }
-
-    private <W extends BoundedWindow>
-        GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W> groupAlsoByWindowsFn(
-            WindowingStrategy<?, W> strategy,
-            StateInternalsFactory<K> stateInternalsFactory,
-            Coder<V> inputIterableElementValueCoder) {
-      return new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
-          strategy,
-          stateInternalsFactory,
-          SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
deleted file mode 100644
index 08c670e..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-import org.joda.time.Instant;
-
-/**
- * A customized {@link DoFnRunner} that handles late data dropping for
- * a {@link KeyedWorkItem} input {@link OldDoFn}.
- *
- * <p>It expands windows before checking data lateness.
- *
- * <p>{@link KeyedWorkItem KeyedWorkItems} are always in empty windows.
- *
- * @param <K> key type
- * @param <InputT> input value element type
- * @param <OutputT> output value element type
- * @param <W> window type
- */
-public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
-    implements DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
-  private final DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner;
-  private final LateDataFilter lateDataFilter;
-
-  public LateDataDroppingDoFnRunner(
-      DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner,
-      WindowingStrategy<?, ?> windowingStrategy,
-      TimerInternals timerInternals,
-      Aggregator<Long, Long> droppedDueToLateness) {
-    this.doFnRunner = doFnRunner;
-    lateDataFilter = new LateDataFilter(windowingStrategy, timerInternals, droppedDueToLateness);
-  }
-
-  @Override
-  public void startBundle() {
-    doFnRunner.startBundle();
-  }
-
-  @Override
-  public void processElement(WindowedValue<KeyedWorkItem<K, InputT>> elem) {
-    Iterable<WindowedValue<InputT>> nonLateElements = lateDataFilter.filter(
-        elem.getValue().key(), elem.getValue().elementsIterable());
-    KeyedWorkItem<K, InputT> keyedWorkItem = KeyedWorkItems.workItem(
-        elem.getValue().key(), elem.getValue().timersIterable(), nonLateElements);
-    doFnRunner.processElement(elem.withValue(keyedWorkItem));
-  }
-
-  @Override
-  public void finishBundle() {
-    doFnRunner.finishBundle();
-  }
-
-  /**
-   * It filters late data in a {@link KeyedWorkItem}.
-   */
-  @VisibleForTesting
-  static class LateDataFilter {
-    private final WindowingStrategy<?, ?> windowingStrategy;
-    private final TimerInternals timerInternals;
-    private final Aggregator<Long, Long> droppedDueToLateness;
-
-    public LateDataFilter(
-        WindowingStrategy<?, ?> windowingStrategy,
-        TimerInternals timerInternals,
-        Aggregator<Long, Long> droppedDueToLateness) {
-      this.windowingStrategy = windowingStrategy;
-      this.timerInternals = timerInternals;
-      this.droppedDueToLateness = droppedDueToLateness;
-    }
-
-    /**
-     * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains
-     * non-late input elements.
-     */
-    public <K, InputT> Iterable<WindowedValue<InputT>> filter(
-        final K key, Iterable<WindowedValue<InputT>> elements) {
-      Iterable<Iterable<WindowedValue<InputT>>> windowsExpandedElements = Iterables.transform(
-          elements,
-          new Function<WindowedValue<InputT>, Iterable<WindowedValue<InputT>>>() {
-            @Override
-            public Iterable<WindowedValue<InputT>> apply(final WindowedValue<InputT> input) {
-              return Iterables.transform(
-                  input.getWindows(),
-                  new Function<BoundedWindow, WindowedValue<InputT>>() {
-                    @Override
-                    public WindowedValue<InputT> apply(BoundedWindow window) {
-                      return WindowedValue.of(
-                          input.getValue(), input.getTimestamp(), window, input.getPane());
-                    }
-                  });
-            }});
-
-      Iterable<WindowedValue<InputT>> nonLateElements = Iterables.filter(
-          Iterables.concat(windowsExpandedElements),
-          new Predicate<WindowedValue<InputT>>() {
-            @Override
-            public boolean apply(WindowedValue<InputT> input) {
-              BoundedWindow window = Iterables.getOnlyElement(input.getWindows());
-              if (canDropDueToExpiredWindow(window)) {
-                // The element is too late for this window.
-                droppedDueToLateness.addValue(1L);
-                WindowTracing.debug(
-                    "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
-                    + "since too far behind inputWatermark:{}; outputWatermark:{}",
-                    input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
-                    timerInternals.currentOutputWatermarkTime());
-                return false;
-              } else {
-                return true;
-              }
-            }
-          });
-      return nonLateElements;
-    }
-
-    /** Is {@code window} expired w.r.t. the garbage collection watermark? */
-    private boolean canDropDueToExpiredWindow(BoundedWindow window) {
-      Instant inputWM = timerInternals.currentInputWatermarkTime();
-      return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
deleted file mode 100644
index e809c24..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
+++ /dev/null
@@ -1,150 +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 org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-
-/**
- * Tracks which windows have non-empty panes. Specifically, which windows have new elements since
- * their last triggering.
- *
- * @param <W> The kind of windows being tracked.
- */
-public abstract class NonEmptyPanes<K, W extends BoundedWindow> {
-
-  static <K, W extends BoundedWindow> NonEmptyPanes<K, W> create(
-      WindowingStrategy<?, W> strategy, ReduceFn<K, ?, ?, W> reduceFn) {
-    if (strategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
-      return new DiscardingModeNonEmptyPanes<>(reduceFn);
-    } else {
-      return new GeneralNonEmptyPanes<>();
-    }
-  }
-
-  /**
-   * Record that some content has been added to the window in {@code context}, and therefore the
-   * current pane is not empty.
-   */
-  public abstract void recordContent(StateAccessor<K> context);
-
-  /**
-   * Record that the given pane is empty.
-   */
-  public abstract void clearPane(StateAccessor<K> state);
-
-  /**
-   * Return true if the current pane for the window in {@code context} is empty.
-   */
-  public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
-
-  /**
-   * Prefetch in preparation for merging.
-   */
-  public abstract void prefetchOnMerge(MergingStateAccessor<K, W> state);
-
-  /**
-   * Eagerly merge backing state.
-   */
-  public abstract void onMerge(MergingStateAccessor<K, W> context);
-
-  /**
-   * An implementation of {@code NonEmptyPanes} optimized for use with discarding mode. Uses the
-   * presence of data in the accumulation buffer to record non-empty panes.
-   */
-  private static class DiscardingModeNonEmptyPanes<K, W extends BoundedWindow>
-      extends NonEmptyPanes<K, W> {
-
-    private ReduceFn<K, ?, ?, W> reduceFn;
-
-    private DiscardingModeNonEmptyPanes(ReduceFn<K, ?, ?, W> reduceFn) {
-      this.reduceFn = reduceFn;
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
-      return reduceFn.isEmpty(state);
-    }
-
-    @Override
-    public void recordContent(StateAccessor<K> state) {
-      // Nothing to do -- the reduceFn is tracking contents
-    }
-
-    @Override
-    public void clearPane(StateAccessor<K> state) {
-      // Nothing to do -- the reduceFn is tracking contents
-    }
-
-    @Override
-    public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
-      // Nothing to do -- the reduceFn is tracking contents
-    }
-
-    @Override
-    public void onMerge(MergingStateAccessor<K, W> context) {
-      // Nothing to do -- the reduceFn is tracking contents
-    }
-  }
-
-  /**
-   * An implementation of {@code NonEmptyPanes} for general use.
-   */
-  private static class GeneralNonEmptyPanes<K, W extends BoundedWindow>
-      extends NonEmptyPanes<K, W> {
-
-    private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
-        PANE_ADDITIONS_TAG =
-        StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
-            "count", VarLongCoder.of(), new Sum.SumLongFn()));
-
-    @Override
-    public void recordContent(StateAccessor<K> state) {
-      state.access(PANE_ADDITIONS_TAG).add(1L);
-    }
-
-    @Override
-    public void clearPane(StateAccessor<K> state) {
-      state.access(PANE_ADDITIONS_TAG).clear();
-    }
-
-    @Override
-    public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
-      return state.access(PANE_ADDITIONS_TAG).isEmpty();
-    }
-
-    @Override
-    public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
-      StateMerging.prefetchCombiningValues(state, PANE_ADDITIONS_TAG);
-    }
-
-    @Override
-    public void onMerge(MergingStateAccessor<K, W> context) {
-      StateMerging.mergeCombiningValues(context, PANE_ADDITIONS_TAG);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
deleted file mode 100644
index 90c10b5..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.joda.time.Instant;
-
-/**
- * Determine the timing and other properties of a new pane for a given computation, key and window.
- * Incorporates any previous pane, whether the pane has been produced because an
- * on-time {@link AfterWatermark} trigger firing, and the relation between the element's timestamp
- * and the current output watermark.
- */
-public class PaneInfoTracker {
-  private TimerInternals timerInternals;
-
-  public PaneInfoTracker(TimerInternals timerInternals) {
-    this.timerInternals = timerInternals;
-  }
-
-  @VisibleForTesting
-  static final StateTag<Object, ValueState<PaneInfo>> PANE_INFO_TAG =
-      StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE));
-
-  public void clear(StateAccessor<?> state) {
-    state.access(PANE_INFO_TAG).clear();
-  }
-
-  /**
-   * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane
-   * info includes the timing for the pane, who's calculation is quite subtle.
-   *
-   * @param isFinal should be {@code true} only if the triggering machinery can guarantee
-   * no further firings for the
-   */
-  public ReadableState<PaneInfo> getNextPaneInfo(
-      ReduceFn<?, ?, ?, ?>.Context context, final boolean isFinal) {
-    final Object key = context.key();
-    final ReadableState<PaneInfo> previousPaneFuture =
-        context.state().access(PaneInfoTracker.PANE_INFO_TAG);
-    final Instant windowMaxTimestamp = context.window().maxTimestamp();
-
-    return new ReadableState<PaneInfo>() {
-      @Override
-      @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-          justification = "prefetch side effect")
-      public ReadableState<PaneInfo> readLater() {
-        previousPaneFuture.readLater();
-        return this;
-      }
-
-      @Override
-      public PaneInfo read() {
-        PaneInfo previousPane = previousPaneFuture.read();
-        return describePane(key, windowMaxTimestamp, previousPane, isFinal);
-      }
-    };
-  }
-
-  public void storeCurrentPaneInfo(ReduceFn<?, ?, ?, ?>.Context context, PaneInfo currentPane) {
-    context.state().access(PANE_INFO_TAG).write(currentPane);
-  }
-
-  private <W> PaneInfo describePane(
-      Object key, Instant windowMaxTimestamp, PaneInfo previousPane, boolean isFinal) {
-    boolean isFirst = previousPane == null;
-    Timing previousTiming = isFirst ? null : previousPane.getTiming();
-    long index = isFirst ? 0 : previousPane.getIndex() + 1;
-    long nonSpeculativeIndex = isFirst ? 0 : previousPane.getNonSpeculativeIndex() + 1;
-    Instant outputWM = timerInternals.currentOutputWatermarkTime();
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-
-    // True if it is not possible to assign the element representing this pane a timestamp
-    // which will make an ON_TIME pane for any following computation.
-    // Ie true if the element's latest possible timestamp is before the current output watermark.
-    boolean isLateForOutput = outputWM != null && windowMaxTimestamp.isBefore(outputWM);
-
-    // True if all emitted panes (if any) were EARLY panes.
-    // Once the ON_TIME pane has fired, all following panes must be considered LATE even
-    // if the output watermark is behind the end of the window.
-    boolean onlyEarlyPanesSoFar = previousTiming == null || previousTiming == Timing.EARLY;
-
-    // True is the input watermark hasn't passed the window's max timestamp.
-    boolean isEarlyForInput = !inputWM.isAfter(windowMaxTimestamp);
-
-    Timing timing;
-    if (isLateForOutput || !onlyEarlyPanesSoFar) {
-      // The output watermark has already passed the end of this window, or we have already
-      // emitted a non-EARLY pane. Irrespective of how this pane was triggered we must
-      // consider this pane LATE.
-      timing = Timing.LATE;
-    } else if (isEarlyForInput) {
-      // This is an EARLY firing.
-      timing = Timing.EARLY;
-      nonSpeculativeIndex = -1;
-    } else {
-      // This is the unique ON_TIME firing for the window.
-      timing = Timing.ON_TIME;
-    }
-
-    WindowTracing.debug(
-        "describePane: {} pane (prev was {}) for key:{}; windowMaxTimestamp:{}; "
-        + "inputWatermark:{}; outputWatermark:{}; isLateForOutput:{}",
-        timing, previousTiming, key, windowMaxTimestamp, inputWM, outputWM, isLateForOutput);
-
-    if (previousPane != null) {
-      // Timing transitions should follow EARLY* ON_TIME? LATE*
-      switch (previousTiming) {
-        case EARLY:
-          checkState(
-              timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE,
-              "EARLY cannot transition to %s", timing);
-          break;
-        case ON_TIME:
-          checkState(
-              timing == Timing.LATE, "ON_TIME cannot transition to %s", timing);
-          break;
-        case LATE:
-          checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
-          break;
-        case UNKNOWN:
-          break;
-      }
-      checkState(!previousPane.isLast(), "Last pane was not last after all.");
-    }
-
-    return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
deleted file mode 100644
index d9f1fbf..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
- * them via the {@link #processElementInReadyWindows(WindowedValue)}.
- */
-public class PushbackSideInputDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-  private final DoFnRunner<InputT, OutputT> underlying;
-  private final Collection<PCollectionView<?>> views;
-  private final ReadyCheckingSideInputReader sideInputReader;
-
-  private Set<BoundedWindow> notReadyWindows;
-
-  public static <InputT, OutputT> PushbackSideInputDoFnRunner<InputT, OutputT> create(
-      DoFnRunner<InputT, OutputT> underlying,
-      Collection<PCollectionView<?>> views,
-      ReadyCheckingSideInputReader sideInputReader) {
-    return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader);
-  }
-
-  private PushbackSideInputDoFnRunner(
-      DoFnRunner<InputT, OutputT> underlying,
-      Collection<PCollectionView<?>> views,
-      ReadyCheckingSideInputReader sideInputReader) {
-    this.underlying = underlying;
-    this.views = views;
-    this.sideInputReader = sideInputReader;
-  }
-
-  @Override
-  public void startBundle() {
-    notReadyWindows = new HashSet<>();
-    underlying.startBundle();
-  }
-
-  /**
-   * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element
-   * for each window the element is in that is ready.
-   *
-   * @param elem the element to process in all ready windows
-   * @return each element that could not be processed because it requires a side input window
-   * that is not ready.
-   */
-  public Iterable<WindowedValue<InputT>> processElementInReadyWindows(WindowedValue<InputT> elem) {
-    if (views.isEmpty()) {
-      processElement(elem);
-      return Collections.emptyList();
-    }
-    ImmutableList.Builder<WindowedValue<InputT>> pushedBack = ImmutableList.builder();
-    for (WindowedValue<InputT> windowElem : elem.explodeWindows()) {
-      BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows());
-      boolean isReady = !notReadyWindows.contains(mainInputWindow);
-      for (PCollectionView<?> view : views) {
-        BoundedWindow sideInputWindow =
-            view.getWindowingStrategyInternal()
-                .getWindowFn()
-                .getSideInputWindow(mainInputWindow);
-        if (!sideInputReader.isReady(view, sideInputWindow)) {
-          isReady = false;
-          break;
-        }
-      }
-      if (isReady) {
-        processElement(windowElem);
-      } else {
-        notReadyWindows.add(mainInputWindow);
-        pushedBack.add(windowElem);
-      }
-    }
-    return pushedBack.build();
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> elem) {
-    underlying.processElement(elem);
-  }
-
-  /**
-   * Call the underlying {@link DoFnRunner#finishBundle()}.
-   */
-  @Override
-  public void finishBundle() {
-    notReadyWindows = null;
-    underlying.finishBundle();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
deleted file mode 100644
index 8135a5b..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import java.io.Serializable;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.joda.time.Instant;
-
-/**
- * Specification for processing to happen after elements have been grouped by key.
- *
- * @param <K> The type of key being processed.
- * @param <InputT> The type of input values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W> The type of windows this operates on.
- */
-public abstract class ReduceFn<K, InputT, OutputT, W extends BoundedWindow>
-    implements Serializable {
-
-  /** Information accessible to all the processing methods in this {@code ReduceFn}. */
-  public abstract class Context {
-    /** Return the key that is being processed. */
-    public abstract K key();
-
-    /** The window that is being processed. */
-    public abstract W window();
-
-    /** Access the current {@link WindowingStrategy}. */
-    public abstract WindowingStrategy<?, W> windowingStrategy();
-
-    /** Return the interface for accessing state. */
-    public abstract StateAccessor<K> state();
-
-    /** Return the interface for accessing timers. */
-    public abstract Timers timers();
-  }
-
-  /** Information accessible within {@link #processValue}. */
-  public abstract class ProcessValueContext extends Context {
-    /** Return the actual value being processed. */
-    public abstract InputT value();
-
-    /** Return the timestamp associated with the value. */
-    public abstract Instant timestamp();
-  }
-
-  /** Information accessible within {@link #onMerge}. */
-  public abstract class OnMergeContext extends Context {
-    /** Return the interface for accessing state. */
-    @Override
-    public abstract MergingStateAccessor<K, W> state();
-  }
-
-  /** Information accessible within {@link #onTrigger}. */
-  public abstract class OnTriggerContext extends Context {
-    /** Returns the {@link PaneInfo} for the trigger firing being processed. */
-    public abstract PaneInfo paneInfo();
-
-    /** Output the given value in the current window. */
-    public abstract void output(OutputT value);
-  }
-
-  //////////////////////////////////////////////////////////////////////////////////////////////////
-
-  /**
-   * Called for each value of type {@code InputT} associated with the current key.
-   */
-  public abstract void processValue(ProcessValueContext c) throws Exception;
-
-  /**
-   * Called when windows are merged.
-   */
-  public abstract void onMerge(OnMergeContext context) throws Exception;
-
-  /**
-   * Called when triggers fire.
-   *
-   * <p>Implementations of {@link ReduceFn} should call {@link OnTriggerContext#output} to emit
-   * any results that should be included in the pane produced by this trigger firing.
-   */
-  public abstract void onTrigger(OnTriggerContext context) throws Exception;
-
-  /**
-   * Called before {@link #onMerge} is invoked to provide an opportunity to prefetch any needed
-   * state.
-   *
-   * @param c Context to use prefetch from.
-   */
-  public void prefetchOnMerge(MergingStateAccessor<K, W> c) throws Exception {}
-
-  /**
-   * Called before {@link #onTrigger} is invoked to provide an opportunity to prefetch any needed
-   * state.
-   *
-   * @param context Context to use prefetch from.
-   */
-  public void prefetchOnTrigger(StateAccessor<K> context) {}
-
-  /**
-   * Called to clear any persisted state that the {@link ReduceFn} may be holding. This will be
-   * called when the windowing is closing and will receive no future interactions.
-   */
-  public abstract void clearState(Context context) throws Exception;
-
-  /**
-   * Returns true if the there is no buffered state.
-   */
-  public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
deleted file mode 100644
index 5d27d51..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
+++ /dev/null
@@ -1,493 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.ImmutableMap;
-import java.util.Collection;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.joda.time.Instant;
-
-/**
- * Factory for creating instances of the various {@link ReduceFn} contexts.
- */
-class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
-  public interface OnTriggerCallbacks<OutputT> {
-    void output(OutputT toOutput);
-  }
-
-  private final K key;
-  private final ReduceFn<K, InputT, OutputT, W> reduceFn;
-  private final WindowingStrategy<?, W> windowingStrategy;
-  private final StateInternals<K> stateInternals;
-  private final ActiveWindowSet<W> activeWindows;
-  private final TimerInternals timerInternals;
-  private final WindowingInternals<?, ?> windowingInternals;
-  private final PipelineOptions options;
-
-  ReduceFnContextFactory(K key, ReduceFn<K, InputT, OutputT, W> reduceFn,
-      WindowingStrategy<?, W> windowingStrategy, StateInternals<K> stateInternals,
-      ActiveWindowSet<W> activeWindows, TimerInternals timerInternals,
-      WindowingInternals<?, ?> windowingInternals, PipelineOptions options) {
-    this.key = key;
-    this.reduceFn = reduceFn;
-    this.windowingStrategy = windowingStrategy;
-    this.stateInternals = stateInternals;
-    this.activeWindows = activeWindows;
-    this.timerInternals = timerInternals;
-    this.windowingInternals = windowingInternals;
-    this.options = options;
-  }
-
-  /** Where should we look for state associated with a given window? */
-  public static enum StateStyle {
-    /** All state is associated with the window itself. */
-    DIRECT,
-    /** State is associated with the 'state address' windows tracked by the active window set. */
-    RENAMED
-  }
-
-  private StateAccessorImpl<K, W> stateAccessor(W window, StateStyle style) {
-    return new StateAccessorImpl<K, W>(
-        activeWindows, windowingStrategy.getWindowFn().windowCoder(),
-        stateInternals, StateContexts.createFromComponents(options, windowingInternals, window),
-        style);
-  }
-
-  public ReduceFn<K, InputT, OutputT, W>.Context base(W window, StateStyle style) {
-    return new ContextImpl(stateAccessor(window, style));
-  }
-
-  public ReduceFn<K, InputT, OutputT, W>.ProcessValueContext forValue(
-      W window, InputT value, Instant timestamp, StateStyle style) {
-    return new ProcessValueContextImpl(stateAccessor(window, style), value, timestamp);
-  }
-
-  public ReduceFn<K, InputT, OutputT, W>.OnTriggerContext forTrigger(W window,
-      ReadableState<PaneInfo> pane, StateStyle style, OnTriggerCallbacks<OutputT> callbacks) {
-    return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks);
-  }
-
-  public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forMerge(
-      Collection<W> activeToBeMerged, W mergeResult, StateStyle style) {
-    return new OnMergeContextImpl(
-        new MergingStateAccessorImpl<K, W>(activeWindows,
-            windowingStrategy.getWindowFn().windowCoder(),
-            stateInternals, style, activeToBeMerged, mergeResult));
-  }
-
-  public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forPremerge(W window) {
-    return new OnPremergeContextImpl(new PremergingStateAccessorImpl<K, W>(
-        activeWindows, windowingStrategy.getWindowFn().windowCoder(), stateInternals, window));
-  }
-
-  private class TimersImpl implements Timers {
-    private final StateNamespace namespace;
-
-    public TimersImpl(StateNamespace namespace) {
-      checkArgument(namespace instanceof WindowNamespace);
-      this.namespace = namespace;
-    }
-
-    @Override
-    public void setTimer(Instant timestamp, TimeDomain timeDomain) {
-      timerInternals.setTimer(TimerData.of(namespace, timestamp, timeDomain));
-    }
-
-    @Override
-    public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
-      timerInternals.deleteTimer(TimerData.of(namespace, timestamp, timeDomain));
-    }
-
-    @Override
-    public Instant currentProcessingTime() {
-      return timerInternals.currentProcessingTime();
-    }
-
-    @Override
-    @Nullable
-    public Instant currentSynchronizedProcessingTime() {
-      return timerInternals.currentSynchronizedProcessingTime();
-    }
-
-    @Override
-    public Instant currentEventTime() {
-      return timerInternals.currentInputWatermarkTime();
-    }
-  }
-
-  // ======================================================================
-  // StateAccessors
-  // ======================================================================
-  static class StateAccessorImpl<K, W extends BoundedWindow> implements StateAccessor<K> {
-
-
-    protected final ActiveWindowSet<W> activeWindows;
-    protected final StateContext<W> context;
-    protected final StateNamespace windowNamespace;
-    protected final Coder<W> windowCoder;
-    protected final StateInternals<K> stateInternals;
-    protected final StateStyle style;
-
-    public StateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
-        StateInternals<K> stateInternals, StateContext<W> context, StateStyle style) {
-
-      this.activeWindows = activeWindows;
-      this.windowCoder = windowCoder;
-      this.stateInternals = stateInternals;
-      this.context = checkNotNull(context);
-      this.windowNamespace = namespaceFor(context.window());
-      this.style = style;
-    }
-
-    protected StateNamespace namespaceFor(W window) {
-      return StateNamespaces.window(windowCoder, window);
-    }
-
-    protected StateNamespace windowNamespace() {
-      return windowNamespace;
-    }
-
-    W window() {
-      return context.window();
-    }
-
-    StateNamespace namespace() {
-      return windowNamespace();
-    }
-
-    @Override
-    public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
-      switch (style) {
-        case DIRECT:
-          return stateInternals.state(windowNamespace(), address, context);
-        case RENAMED:
-          return stateInternals.state(
-              namespaceFor(activeWindows.writeStateAddress(context.window())), address, context);
-      }
-      throw new RuntimeException(); // cases are exhaustive.
-    }
-  }
-
-  static class MergingStateAccessorImpl<K, W extends BoundedWindow>
-      extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
-    private final Collection<W> activeToBeMerged;
-
-    public MergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
-        StateInternals<K> stateInternals, StateStyle style, Collection<W> activeToBeMerged,
-        W mergeResult) {
-      super(activeWindows, windowCoder, stateInternals,
-          StateContexts.windowOnly(mergeResult), style);
-      this.activeToBeMerged = activeToBeMerged;
-    }
-
-    @Override
-    public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
-      switch (style) {
-        case DIRECT:
-          return stateInternals.state(windowNamespace(), address, context);
-        case RENAMED:
-          return stateInternals.state(
-              namespaceFor(activeWindows.mergedWriteStateAddress(
-                  activeToBeMerged, context.window())),
-              address,
-              context);
-      }
-      throw new RuntimeException(); // cases are exhaustive.
-    }
-
-    @Override
-    public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
-        StateTag<? super K, StateT> address) {
-      ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
-      for (W mergingWindow : activeToBeMerged) {
-        StateNamespace namespace = null;
-        switch (style) {
-          case DIRECT:
-            namespace = namespaceFor(mergingWindow);
-            break;
-          case RENAMED:
-            namespace = namespaceFor(activeWindows.writeStateAddress(mergingWindow));
-            break;
-        }
-        checkNotNull(namespace); // cases are exhaustive.
-        builder.put(mergingWindow, stateInternals.state(namespace, address, context));
-      }
-      return builder.build();
-    }
-  }
-
-  static class PremergingStateAccessorImpl<K, W extends BoundedWindow>
-      extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
-    public PremergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
-        StateInternals<K> stateInternals, W window) {
-      super(activeWindows, windowCoder, stateInternals,
-          StateContexts.windowOnly(window), StateStyle.RENAMED);
-    }
-
-    Collection<W> mergingWindows() {
-      return activeWindows.readStateAddresses(context.window());
-    }
-
-    @Override
-    public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
-        StateTag<? super K, StateT> address) {
-      ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
-      for (W stateAddressWindow : activeWindows.readStateAddresses(context.window())) {
-        StateT stateForWindow =
-            stateInternals.state(namespaceFor(stateAddressWindow), address, context);
-        builder.put(stateAddressWindow, stateForWindow);
-      }
-      return builder.build();
-    }
-  }
-
-  // ======================================================================
-  // Contexts
-  // ======================================================================
-
-  private class ContextImpl extends ReduceFn<K, InputT, OutputT, W>.Context {
-    private final StateAccessorImpl<K, W> state;
-    private final TimersImpl timers;
-
-    private ContextImpl(StateAccessorImpl<K, W> state) {
-      reduceFn.super();
-      this.state = state;
-      this.timers = new TimersImpl(state.namespace());
-    }
-
-    @Override
-    public K key() {
-      return key;
-    }
-
-    @Override
-    public W window() {
-      return state.window();
-    }
-
-    @Override
-    public WindowingStrategy<?, W> windowingStrategy() {
-      return windowingStrategy;
-    }
-
-    @Override
-    public StateAccessor<K> state() {
-      return state;
-    }
-
-    @Override
-    public Timers timers() {
-      return timers;
-    }
-  }
-
-  private class ProcessValueContextImpl
-      extends ReduceFn<K, InputT, OutputT, W>.ProcessValueContext {
-    private final InputT value;
-    private final Instant timestamp;
-    private final StateAccessorImpl<K, W> state;
-    private final TimersImpl timers;
-
-    private ProcessValueContextImpl(StateAccessorImpl<K, W> state,
-        InputT value, Instant timestamp) {
-      reduceFn.super();
-      this.state = state;
-      this.value = value;
-      this.timestamp = timestamp;
-      this.timers = new TimersImpl(state.namespace());
-    }
-
-    @Override
-    public K key() {
-      return key;
-    }
-
-    @Override
-    public W window() {
-      return state.window();
-    }
-
-    @Override
-    public WindowingStrategy<?, W> windowingStrategy() {
-      return windowingStrategy;
-    }
-
-    @Override
-    public StateAccessor<K> state() {
-      return state;
-    }
-
-    @Override
-    public InputT value() {
-      return value;
-    }
-
-    @Override
-    public Instant timestamp() {
-      return timestamp;
-    }
-
-    @Override
-    public Timers timers() {
-      return timers;
-    }
-  }
-
-  private class OnTriggerContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnTriggerContext {
-    private final StateAccessorImpl<K, W> state;
-    private final ReadableState<PaneInfo> pane;
-    private final OnTriggerCallbacks<OutputT> callbacks;
-    private final TimersImpl timers;
-
-    private OnTriggerContextImpl(StateAccessorImpl<K, W> state, ReadableState<PaneInfo> pane,
-        OnTriggerCallbacks<OutputT> callbacks) {
-      reduceFn.super();
-      this.state = state;
-      this.pane = pane;
-      this.callbacks = callbacks;
-      this.timers = new TimersImpl(state.namespace());
-    }
-
-    @Override
-    public K key() {
-      return key;
-    }
-
-    @Override
-    public W window() {
-      return state.window();
-    }
-
-    @Override
-    public WindowingStrategy<?, W> windowingStrategy() {
-      return windowingStrategy;
-    }
-
-    @Override
-    public StateAccessor<K> state() {
-      return state;
-    }
-
-    @Override
-    public PaneInfo paneInfo() {
-      return pane.read();
-    }
-
-    @Override
-    public void output(OutputT value) {
-      callbacks.output(value);
-    }
-
-    @Override
-    public Timers timers() {
-      return timers;
-    }
-  }
-
-  private class OnMergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
-    private final MergingStateAccessorImpl<K, W> state;
-    private final TimersImpl timers;
-
-    private OnMergeContextImpl(MergingStateAccessorImpl<K, W> state) {
-      reduceFn.super();
-      this.state = state;
-      this.timers = new TimersImpl(state.namespace());
-    }
-
-    @Override
-    public K key() {
-      return key;
-    }
-
-    @Override
-    public WindowingStrategy<?, W> windowingStrategy() {
-      return windowingStrategy;
-    }
-
-    @Override
-    public MergingStateAccessor<K, W> state() {
-      return state;
-    }
-
-    @Override
-    public W window() {
-      return state.window();
-    }
-
-    @Override
-    public Timers timers() {
-      return timers;
-    }
-  }
-
-  private class OnPremergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
-    private final PremergingStateAccessorImpl<K, W> state;
-    private final TimersImpl timers;
-
-    private OnPremergeContextImpl(PremergingStateAccessorImpl<K, W> state) {
-      reduceFn.super();
-      this.state = state;
-      this.timers = new TimersImpl(state.namespace());
-    }
-
-    @Override
-    public K key() {
-      return key;
-    }
-
-    @Override
-    public WindowingStrategy<?, W> windowingStrategy() {
-      return windowingStrategy;
-    }
-
-    @Override
-    public MergingStateAccessor<K, W> state() {
-      return state;
-    }
-
-    @Override
-    public W window() {
-      return state.window();
-    }
-
-    @Override
-    public Timers timers() {
-      return timers;
-    }
-  }
-}


[33/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
new file mode 100644
index 0000000..3948d9e
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+
+/**
+ * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}.
+ *
+ * @param <K> The type of key being processed.
+ * @param <InputT> The type of values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W> The type of windows this operates on.
+ */
+public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends BoundedWindow>
+    extends ReduceFn<K, InputT, OutputT, W> {
+  private static final String BUFFER_NAME = "buf";
+
+  /**
+   * Create a factory that produces {@link SystemReduceFn} instances that that buffer all of the
+   * input values in persistent state and produces an {@code Iterable<T>}.
+   */
+  public static <K, T, W extends BoundedWindow> SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>
+      buffering(final Coder<T> inputCoder) {
+    final StateTag<Object, BagState<T>> bufferTag =
+        StateTags.makeSystemTagInternal(StateTags.bag(BUFFER_NAME, inputCoder));
+    return new SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>(bufferTag) {
+      @Override
+      public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
+        StateMerging.prefetchBags(state, bufferTag);
+      }
+
+      @Override
+      public void onMerge(OnMergeContext c) throws Exception {
+        StateMerging.mergeBags(c.state(), bufferTag);
+      }
+    };
+  }
+
+  /**
+   * Create a factory that produces {@link SystemReduceFn} instances that combine all of the input
+   * values using a {@link CombineFn}.
+   */
+  public static <K, InputT, AccumT, OutputT, W extends BoundedWindow> SystemReduceFn<K, InputT,
+      AccumT, OutputT, W>
+      combining(
+          final Coder<K> keyCoder, final AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
+    final StateTag<K, AccumulatorCombiningState<InputT, AccumT, OutputT>> bufferTag;
+    if (combineFn.getFn() instanceof KeyedCombineFnWithContext) {
+      bufferTag = StateTags.makeSystemTagInternal(
+          StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValueWithContext(
+              BUFFER_NAME, combineFn.getAccumulatorCoder(),
+              (KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>) combineFn.getFn()));
+
+    } else {
+      bufferTag = StateTags.makeSystemTagInternal(
+            StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValue(
+                BUFFER_NAME, combineFn.getAccumulatorCoder(),
+                (KeyedCombineFn<K, InputT, AccumT, OutputT>) combineFn.getFn()));
+    }
+    return new SystemReduceFn<K, InputT, AccumT, OutputT, W>(bufferTag) {
+      @Override
+      public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
+        StateMerging.prefetchCombiningValues(state, bufferTag);
+      }
+
+      @Override
+      public void onMerge(OnMergeContext c) throws Exception {
+        StateMerging.mergeCombiningValues(c.state(), bufferTag);
+      }
+    };
+  }
+
+  private StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag;
+
+  public SystemReduceFn(
+      StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag) {
+    this.bufferTag = bufferTag;
+  }
+
+  @Override
+  public void processValue(ProcessValueContext c) throws Exception {
+    c.state().access(bufferTag).add(c.value());
+  }
+
+  @Override
+  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+    justification = "prefetch side effect")
+  public void prefetchOnTrigger(StateAccessor<K> state) {
+    state.access(bufferTag).readLater();
+  }
+
+  @Override
+  public void onTrigger(OnTriggerContext c) throws Exception {
+    c.output(c.state().access(bufferTag).read());
+  }
+
+  @Override
+  public void clearState(Context c) throws Exception {
+    c.state().access(bufferTag).clear();
+  }
+
+  @Override
+  public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+    return state.access(bufferTag).isEmpty();
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
new file mode 100644
index 0000000..7d0b608
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowTracing;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Implements the logic to hold the output watermark for a computation back
+ * until it has seen all the elements it needs based on the input watermark for the
+ * computation.
+ *
+ * <p>The backend ensures the output watermark can never progress beyond the
+ * input watermark for a computation. GroupAlsoByWindows computations may add a 'hold'
+ * to the output watermark in order to prevent it progressing beyond a time within a window.
+ * The hold will be 'cleared' when the associated pane is emitted.
+ *
+ * <p>This class is only intended for use by {@link ReduceFnRunner}. The two evolve together and
+ * will likely break any other uses.
+ *
+ * @param <W> The kind of {@link BoundedWindow} the hold is for.
+ */
+class WatermarkHold<W extends BoundedWindow> implements Serializable {
+  /**
+   * Return tag for state containing the output watermark hold
+   * used for elements.
+   */
+  public static <W extends BoundedWindow>
+      StateTag<Object, WatermarkHoldState<W>> watermarkHoldTagForOutputTimeFn(
+          OutputTimeFn<? super W> outputTimeFn) {
+    return StateTags.<Object, WatermarkHoldState<W>>makeSystemTagInternal(
+        StateTags.<W>watermarkStateInternal("hold", outputTimeFn));
+  }
+
+  /**
+   * Tag for state containing end-of-window and garbage collection output watermark holds.
+   * (We can't piggy-back on the data hold state since the outputTimeFn may be
+   * {@link OutputTimeFns#outputAtLatestInputTimestamp()}, in which case every pane will
+   * would take the end-of-window time as its element time.)
+   */
+  @VisibleForTesting
+  public static final StateTag<Object, WatermarkHoldState<BoundedWindow>> EXTRA_HOLD_TAG =
+      StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal(
+          "extra", OutputTimeFns.outputAtEarliestInputTimestamp()));
+
+  private final TimerInternals timerInternals;
+  private final WindowingStrategy<?, W> windowingStrategy;
+  private final StateTag<Object, WatermarkHoldState<W>> elementHoldTag;
+
+  public WatermarkHold(TimerInternals timerInternals, WindowingStrategy<?, W> windowingStrategy) {
+    this.timerInternals = timerInternals;
+    this.windowingStrategy = windowingStrategy;
+    this.elementHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn());
+  }
+
+  /**
+   * Add a hold to prevent the output watermark progressing beyond the (possibly adjusted) timestamp
+   * of the element in {@code context}. We allow the actual hold time to be shifted later by
+   * {@link OutputTimeFn#assignOutputTime}, but no further than the end of the window. The hold will
+   * remain until cleared by {@link #extractAndRelease}. Return the timestamp at which the hold
+   * was placed, or {@literal null} if no hold was placed.
+   *
+   * <p>In the following we'll write {@code E} to represent an element's timestamp after passing
+   * through the window strategy's output time function, {@code IWM} for the local input watermark,
+   * {@code OWM} for the local output watermark, and {@code GCWM} for the garbage collection
+   * watermark (which is at {@code IWM - getAllowedLateness}). Time progresses from left to right,
+   * and we write {@code [ ... ]} to denote a bounded window with implied lower bound.
+   *
+   * <p>Note that the GCWM will be the same as the IWM if {@code getAllowedLateness}
+   * is {@code ZERO}.
+   *
+   * <p>Here are the cases we need to handle. They are conceptually considered in the
+   * sequence written since if getAllowedLateness is ZERO the GCWM is the same as the IWM.
+   * <ol>
+   * <li>(Normal)
+   * <pre>
+   *          |
+   *      [   | E        ]
+   *          |
+   *         IWM
+   * </pre>
+   * This is, hopefully, the common and happy case. The element is locally on-time and can
+   * definitely make it to an {@code ON_TIME} pane which we can still set an end-of-window timer
+   * for. We place an element hold at E, which may contribute to the {@code ON_TIME} pane's
+   * timestamp (depending on the output time function). Thus the OWM will not proceed past E
+   * until the next pane fires.
+   *
+   * <li>(Discard - no target window)
+   * <pre>
+   *                       |                            |
+   *      [     E        ] |                            |
+   *                       |                            |
+   *                     GCWM  <-getAllowedLateness->  IWM
+   * </pre>
+   * The element is very locally late. The window has been garbage collected, thus there
+   * is no target pane E could be assigned to. We discard E.
+   *
+   * <li>(Unobservably late)
+   * <pre>
+   *          |    |
+   *      [   | E  |     ]
+   *          |    |
+   *         OWM  IWM
+   * </pre>
+   * The element is locally late, however we can still treat this case as for 'Normal' above
+   * since the IWM has not yet passed the end of the window and the element is ahead of the
+   * OWM. In effect, we get to 'launder' the locally late element and consider it as locally
+   * on-time because no downstream computation can observe the difference.
+   *
+   * <li>(Maybe late 1)
+   * <pre>
+   *          |            |
+   *      [   | E        ] |
+   *          |            |
+   *         OWM          IWM
+   * </pre>
+   * The end-of-window timer may have already fired for this window, and thus an {@code ON_TIME}
+   * pane may have already been emitted. However, if timer firings have been delayed then it
+   * is possible the {@code ON_TIME} pane has not yet been emitted. We can't place an element
+   * hold since we can't be sure if it will be cleared promptly. Thus this element *may* find
+   * its way into an {@code ON_TIME} pane, but if so it will *not* contribute to that pane's
+   * timestamp. We may however set a garbage collection hold if required.
+   *
+   * <li>(Maybe late 2)
+   * <pre>
+   *               |   |
+   *      [     E  |   | ]
+   *               |   |
+   *              OWM IWM
+   * </pre>
+   * The end-of-window timer has not yet fired, so this element may still appear in an
+   * {@code ON_TIME} pane. However the element is too late to contribute to the output
+   * watermark hold, and thus won't contribute to the pane's timestamp. We can still place an
+   * end-of-window hold.
+   *
+   * <li>(Maybe late 3)
+   * <pre>
+   *               |       |
+   *      [     E  |     ] |
+   *               |       |
+   *              OWM     IWM
+   * </pre>
+   * As for the (Maybe late 2) case, however we don't even know if the end-of-window timer
+   * has already fired, or it is about to fire. We can place only the garbage collection hold,
+   * if required.
+   *
+   * <li>(Definitely late)
+   * <pre>
+   *                       |   |
+   *      [     E        ] |   |
+   *                       |   |
+   *                      OWM IWM
+   * </pre>
+   * The element is definitely too late to make an {@code ON_TIME} pane. We are too late to
+   * place an end-of-window hold. We can still place a garbage collection hold if required.
+   *
+   * </ol>
+   */
+  @Nullable
+  public Instant addHolds(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
+    Instant hold = addElementHold(context);
+    if (hold == null) {
+      hold = addEndOfWindowOrGarbageCollectionHolds(context, false/*paneIsEmpty*/);
+    }
+    return hold;
+  }
+
+  /**
+   * Return {@code timestamp}, possibly shifted forward in time according to the window
+   * strategy's output time function.
+   */
+  private Instant shift(Instant timestamp, W window) {
+    Instant shifted = windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window);
+    checkState(!shifted.isBefore(timestamp),
+        "OutputTimeFn moved element from %s to earlier time %s for window %s",
+        timestamp, shifted, window);
+    checkState(timestamp.isAfter(window.maxTimestamp())
+            || !shifted.isAfter(window.maxTimestamp()),
+        "OutputTimeFn moved element from %s to %s which is beyond end of "
+            + "window %s",
+        timestamp, shifted, window);
+
+    return shifted;
+  }
+
+  /**
+   * Attempt to add an 'element hold'. Return the {@link Instant} at which the hold was
+   * added (ie the element timestamp plus any forward shift requested by the
+   * {@link WindowingStrategy#getOutputTimeFn}), or {@literal null} if no hold was added.
+   * The hold is only added if both:
+   * <ol>
+   * <li>The backend will be able to respect it. In other words the output watermark cannot
+   * be ahead of the proposed hold time.
+   * <li>A timer will be set (by {@link ReduceFnRunner}) to clear the hold by the end of the
+   * window. In other words the input watermark cannot be ahead of the end of the window.
+   * </ol>
+   * The hold ensures the pane which incorporates the element is will not be considered late by
+   * any downstream computation when it is eventually emitted.
+   */
+  @Nullable
+  private Instant addElementHold(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
+    // Give the window function a chance to move the hold timestamp forward to encourage progress.
+    // (A later hold implies less impediment to the output watermark making progress, which in
+    // turn encourages end-of-window triggers to fire earlier in following computations.)
+    Instant elementHold = shift(context.timestamp(), context.window());
+
+    Instant outputWM = timerInternals.currentOutputWatermarkTime();
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+    String which;
+    boolean tooLate;
+    // TODO: These case labels could be tightened.
+    // See the case analysis in addHolds above for the motivation.
+    if (outputWM != null && elementHold.isBefore(outputWM)) {
+      which = "too late to effect output watermark";
+      tooLate = true;
+    } else if (context.window().maxTimestamp().isBefore(inputWM)) {
+      which = "too late for end-of-window timer";
+      tooLate = true;
+    } else {
+      which = "on time";
+      tooLate = false;
+      checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+          "Element hold %s is beyond end-of-time", elementHold);
+      context.state().access(elementHoldTag).add(elementHold);
+    }
+    WindowTracing.trace(
+        "WatermarkHold.addHolds: element hold at {} is {} for "
+        + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+        elementHold, which, context.key(), context.window(), inputWM,
+        outputWM);
+
+    return tooLate ? null : elementHold;
+  }
+
+  /**
+   * Add an end-of-window hold or, if too late for that, a garbage collection hold (if required).
+   * Return the {@link Instant} at which hold was added, or {@literal null} if no hold was added.
+   */
+  @Nullable
+  private Instant addEndOfWindowOrGarbageCollectionHolds(
+      ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+    Instant hold = addEndOfWindowHold(context, paneIsEmpty);
+    if (hold == null) {
+      hold = addGarbageCollectionHold(context, paneIsEmpty);
+    }
+    return hold;
+  }
+
+  /**
+   * Attempt to add an 'end-of-window hold'. Return the {@link Instant} at which the hold was added
+   * (ie the end of window time), or {@literal null} if no end of window hold is possible and we
+   * should fallback to a garbage collection hold.
+   *
+   * <p>We only add the hold if we can be sure a timer will be set (by {@link ReduceFnRunner})
+   * to clear it. In other words, the input watermark cannot be ahead of the end of window time.
+   *
+   * <p>An end-of-window hold is added in two situations:
+   * <ol>
+   * <li>An incoming element came in behind the output watermark (so we are too late for placing
+   * the usual element hold), but it may still be possible to include the element in an
+   * {@link Timing#ON_TIME} pane. We place the end of window hold to ensure that pane will
+   * not be considered late by any downstream computation.
+   * <li>We guarantee an {@link Timing#ON_TIME} pane will be emitted for all windows which saw at
+   * least one element, even if that {@link Timing#ON_TIME} pane is empty. Thus when elements in
+   * a pane are processed due to a fired trigger we must set both an end of window timer and an end
+   * of window hold. Again, the hold ensures the {@link Timing#ON_TIME} pane will not be considered
+   * late by any downstream computation.
+   * </ol>
+   */
+  @Nullable
+  private Instant addEndOfWindowHold(ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+    Instant outputWM = timerInternals.currentOutputWatermarkTime();
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+    Instant eowHold = context.window().maxTimestamp();
+
+    if (eowHold.isBefore(inputWM)) {
+      WindowTracing.trace(
+          "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is too late for "
+              + "end-of-window timer for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+          eowHold, context.key(), context.window(), inputWM, outputWM);
+      return null;
+    }
+
+    checkState(outputWM == null || !eowHold.isBefore(outputWM),
+        "End-of-window hold %s cannot be before output watermark %s",
+        eowHold, outputWM);
+    checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+        "End-of-window hold %s is beyond end-of-time", eowHold);
+    // If paneIsEmpty then this hold is just for empty ON_TIME panes, so we want to keep
+    // the hold away from the combining function in elementHoldTag.
+    // However if !paneIsEmpty then it could make sense  to use the elementHoldTag here.
+    // Alas, onMerge is forced to add an end of window or garbage collection hold without
+    // knowing whether an element hold is already in place (stopping to check is too expensive).
+    // This it would end up adding an element hold at the end of the window which could
+    // upset the elementHoldTag combining function.
+    context.state().access(EXTRA_HOLD_TAG).add(eowHold);
+    WindowTracing.trace(
+        "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is on time for "
+            + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+        eowHold, context.key(), context.window(), inputWM, outputWM);
+    return eowHold;
+  }
+
+  /**
+   * Attempt to add a 'garbage collection hold' if it is required. Return the {@link Instant} at
+   * which the hold was added (ie the end of window time plus allowed lateness),
+   * or {@literal null} if no hold was added.
+   *
+   * <p>We only add the hold if it is distinct from what would be added by
+   * {@link #addEndOfWindowHold}. In other words, {@link WindowingStrategy#getAllowedLateness}
+   * must be non-zero.
+   *
+   * <p>A garbage collection hold is added in two situations:
+   * <ol>
+   * <li>An incoming element came in behind the output watermark, and was too late for placing
+   * the usual element hold or an end of window hold. Place the garbage collection hold so that
+   * we can guarantee when the pane is finally triggered its output will not be dropped due to
+   * excessive lateness by any downstream computation.
+   * <li>The {@link WindowingStrategy#getClosingBehavior()} is
+   * {@link ClosingBehavior#FIRE_ALWAYS}, and thus we guarantee a final pane will be emitted
+   * for all windows which saw at least one element. Again, the garbage collection hold guarantees
+   * that any empty final pane can be given a timestamp which will not be considered beyond
+   * allowed lateness by any downstream computation.
+   * </ol>
+   *
+   * <p>We use {@code paneIsEmpty} to distinguish cases 1 and 2.
+   */
+  @Nullable
+  private Instant addGarbageCollectionHold(
+      ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+    Instant outputWM = timerInternals.currentOutputWatermarkTime();
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+    Instant eow = context.window().maxTimestamp();
+    Instant gcHold = eow.plus(windowingStrategy.getAllowedLateness());
+
+    if (!windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) {
+      WindowTracing.trace(
+          "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
+              + "since no allowed lateness for key:{}; window:{}; inputWatermark:{}; "
+              + "outputWatermark:{}",
+          gcHold, context.key(), context.window(), inputWM, outputWM);
+      return null;
+    }
+
+    if (paneIsEmpty && context.windowingStrategy().getClosingBehavior()
+        == ClosingBehavior.FIRE_IF_NON_EMPTY) {
+      WindowTracing.trace(
+          "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
+              + "since empty pane and FIRE_IF_NON_EMPTY for key:{}; window:{}; inputWatermark:{}; "
+              + "outputWatermark:{}",
+          gcHold, context.key(), context.window(), inputWM, outputWM);
+      return null;
+    }
+
+    checkState(!gcHold.isBefore(inputWM),
+        "Garbage collection hold %s cannot be before input watermark %s",
+        gcHold, inputWM);
+    checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+        "Garbage collection hold %s is beyond end-of-time", gcHold);
+    // Same EXTRA_HOLD_TAG vs elementHoldTag discussion as in addEndOfWindowHold above.
+    context.state().access(EXTRA_HOLD_TAG).add(gcHold);
+
+    WindowTracing.trace(
+        "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is on time for "
+            + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+        gcHold, context.key(), context.window(), inputWM, outputWM);
+    return gcHold;
+  }
+
+  /**
+   * Prefetch watermark holds in preparation for merging.
+   */
+  public void prefetchOnMerge(MergingStateAccessor<?, W> state) {
+    StateMerging.prefetchWatermarks(state, elementHoldTag);
+  }
+
+  /**
+   * Updates the watermark hold when windows merge if it is possible the merged value does
+   * not equal all of the existing holds. For example, if the new window implies a later
+   * watermark hold, then earlier holds may be released.
+   */
+  public void onMerge(ReduceFn<?, ?, ?, W>.OnMergeContext context) {
+    WindowTracing.debug("WatermarkHold.onMerge: for key:{}; window:{}; inputWatermark:{}; "
+            + "outputWatermark:{}",
+        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+        timerInternals.currentOutputWatermarkTime());
+    StateMerging.mergeWatermarks(context.state(), elementHoldTag, context.window());
+    // If we had a cheap way to determine if we have an element hold then we could
+    // avoid adding an unnecessary end-of-window or garbage collection hold.
+    // Simply reading the above merged watermark would impose an additional read for the
+    // common case that the active window has just one underlying state address window and
+    // the hold depends on the min of the element timestamps.
+    // At least one merged window must be non-empty for the merge to have been triggered.
+    StateMerging.clear(context.state(), EXTRA_HOLD_TAG);
+    addEndOfWindowOrGarbageCollectionHolds(context, false /*paneIsEmpty*/);
+  }
+
+  /**
+   * Result of {@link #extractAndRelease}.
+   */
+  public static class OldAndNewHolds {
+    public final Instant oldHold;
+    @Nullable
+    public final Instant newHold;
+
+    public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) {
+      this.oldHold = oldHold;
+      this.newHold = newHold;
+    }
+  }
+
+  /**
+   * Return (a future for) the earliest hold for {@code context}. Clear all the holds after
+   * reading, but add/restore an end-of-window or garbage collection hold if required.
+   *
+   * <p>The returned timestamp is the output timestamp according to the {@link OutputTimeFn}
+   * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late
+   * elements in the current pane. If there is no such value the timestamp is the end
+   * of the window.
+   */
+  public ReadableState<OldAndNewHolds> extractAndRelease(
+      final ReduceFn<?, ?, ?, W>.Context context, final boolean isFinished) {
+    WindowTracing.debug(
+        "WatermarkHold.extractAndRelease: for key:{}; window:{}; inputWatermark:{}; "
+            + "outputWatermark:{}",
+        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+        timerInternals.currentOutputWatermarkTime());
+    final WatermarkHoldState<W> elementHoldState = context.state().access(elementHoldTag);
+    final WatermarkHoldState<BoundedWindow> extraHoldState = context.state().access(EXTRA_HOLD_TAG);
+    return new ReadableState<OldAndNewHolds>() {
+      @Override
+      @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+        justification = "")
+      public ReadableState<OldAndNewHolds> readLater() {
+        elementHoldState.readLater();
+        extraHoldState.readLater();
+        return this;
+      }
+
+      @Override
+      public OldAndNewHolds read() {
+        // Read both the element and extra holds.
+        Instant elementHold = elementHoldState.read();
+        Instant extraHold = extraHoldState.read();
+        Instant oldHold;
+        // Find the minimum, accounting for null.
+        if (elementHold == null) {
+          oldHold = extraHold;
+        } else if (extraHold == null) {
+          oldHold = elementHold;
+        } else if (elementHold.isBefore(extraHold)) {
+          oldHold = elementHold;
+        } else {
+          oldHold = extraHold;
+        }
+        if (oldHold == null || oldHold.isAfter(context.window().maxTimestamp())) {
+          // If no hold (eg because all elements came in behind the output watermark), or
+          // the hold was for garbage collection, take the end of window as the result.
+          WindowTracing.debug(
+              "WatermarkHold.extractAndRelease.read: clipping from {} to end of window "
+              + "for key:{}; window:{}",
+              oldHold, context.key(), context.window());
+          oldHold = context.window().maxTimestamp();
+        }
+        WindowTracing.debug("WatermarkHold.extractAndRelease.read: clearing for key:{}; window:{}",
+            context.key(), context.window());
+
+        // Clear the underlying state to allow the output watermark to progress.
+        elementHoldState.clear();
+        extraHoldState.clear();
+
+        @Nullable Instant newHold = null;
+        if (!isFinished) {
+          // Only need to leave behind an end-of-window or garbage collection hold
+          // if future elements will be processed.
+          newHold = addEndOfWindowOrGarbageCollectionHolds(context, true /*paneIsEmpty*/);
+        }
+
+        return new OldAndNewHolds(oldHold, newHold);
+      }
+    };
+  }
+
+  /**
+   * Clear any remaining holds.
+   */
+  public void clearHolds(ReduceFn<?, ?, ?, W>.Context context) {
+    WindowTracing.debug(
+        "WatermarkHold.clearHolds: For key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+        timerInternals.currentOutputWatermarkTime());
+    context.state().access(elementHoldTag).clear();
+    context.state().access(EXTRA_HOLD_TAG).clear();
+  }
+
+  /**
+   * Return the current data hold, or null if none. Does not clear. For debugging only.
+   */
+  @Nullable
+  public Instant getDataCurrent(ReduceFn<?, ?, ?, W>.Context context) {
+    return context.state().access(elementHoldTag).read();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
deleted file mode 100644
index af28052..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a
- * {@link PCollection} to windows according to the provided {@link WindowFn}.
- *
- * @param <T> Type of elements being windowed
- * @param <W> Window type
- */
-public class AssignWindows<T, W extends BoundedWindow>
-    extends PTransform<PCollection<T>, PCollection<T>> {
-
-  private WindowFn<? super T, W> fn;
-
-  public AssignWindows(WindowFn<? super T, W> fn) {
-    this.fn = fn;
-  }
-
-  @Override
-  public PCollection<T> apply(PCollection<T> input) {
-    return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
deleted file mode 100644
index 7e26253..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Iterables;
-import java.util.Collection;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.joda.time.Instant;
-
-/**
- * {@link OldDoFn} that tags elements of a {@link PCollection} with windows, according to the
- * provided {@link WindowFn}.
- *
- * @param <T> Type of elements being windowed
- * @param <W> Window type
- */
-@SystemDoFnInternal
-public class AssignWindowsDoFn<T, W extends BoundedWindow> extends OldDoFn<T, T>
-    implements RequiresWindowAccess {
-  private WindowFn<? super T, W> fn;
-
-  public AssignWindowsDoFn(WindowFn<? super T, W> fn) {
-    this.fn =
-        checkNotNull(
-            fn,
-            "%s provided to %s cannot be null",
-            WindowFn.class.getSimpleName(),
-            AssignWindowsDoFn.class.getSimpleName());
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void processElement(final ProcessContext c) throws Exception {
-    Collection<W> windows =
-        ((WindowFn<T, W>) fn).assignWindows(
-            ((WindowFn<T, W>) fn).new AssignContext() {
-                @Override
-                public T element() {
-                  return c.element();
-                }
-
-                @Override
-                public Instant timestamp() {
-                  return c.timestamp();
-                }
-
-                @Override
-                public BoundedWindow window() {
-                  return Iterables.getOnlyElement(c.windowingInternals().windows());
-                }
-              });
-
-    c.windowingInternals()
-        .outputWindowedValue(c.element(), c.timestamp(), windows, PaneInfo.NO_FIRING);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
deleted file mode 100644
index f3e84a6..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
+++ /dev/null
@@ -1,137 +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.checkState;
-
-import com.google.common.base.MoreObjects;
-import java.util.HashSet;
-import java.util.PriorityQueue;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.joda.time.Instant;
-
-/**
- * TimerInternals that uses priority queues to manage the timers that are ready to fire.
- */
-public class BatchTimerInternals implements TimerInternals {
-  /** Set of timers that are scheduled used for deduplicating timers. */
-  private Set<TimerData> existingTimers = new HashSet<>();
-
-  // Keep these queues separate so we can advance over them separately.
-  private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
-  private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
-
-  private Instant inputWatermarkTime;
-  private Instant processingTime;
-
-  private PriorityQueue<TimerData> queue(TimeDomain domain) {
-    return TimeDomain.EVENT_TIME.equals(domain) ? watermarkTimers : processingTimers;
-  }
-
-  public BatchTimerInternals(Instant processingTime) {
-    this.processingTime = processingTime;
-    this.inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-  }
-
-  @Override
-  public void setTimer(TimerData timer) {
-    if (existingTimers.add(timer)) {
-      queue(timer.getDomain()).add(timer);
-    }
-  }
-
-  @Override
-  public void deleteTimer(TimerData timer) {
-    existingTimers.remove(timer);
-    queue(timer.getDomain()).remove(timer);
-  }
-
-  @Override
-  public Instant currentProcessingTime() {
-    return processingTime;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * @return {@link BoundedWindow#TIMESTAMP_MAX_VALUE}: in batch mode, upstream processing
-   * is already complete.
-   */
-  @Override
-  @Nullable
-  public Instant currentSynchronizedProcessingTime() {
-    return BoundedWindow.TIMESTAMP_MAX_VALUE;
-  }
-
-  @Override
-  public Instant currentInputWatermarkTime() {
-    return inputWatermarkTime;
-  }
-
-  @Override
-  @Nullable
-  public Instant currentOutputWatermarkTime() {
-    // The output watermark is always undefined in batch mode.
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(getClass())
-        .add("watermarkTimers", watermarkTimers)
-        .add("processingTimers", processingTimers)
-        .toString();
-  }
-
-  public void advanceInputWatermark(ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark)
-      throws Exception {
-    checkState(!newInputWatermark.isBefore(inputWatermarkTime),
-        "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
-        newInputWatermark);
-    inputWatermarkTime = newInputWatermark;
-    advance(runner, newInputWatermark, TimeDomain.EVENT_TIME);
-  }
-
-  public void advanceProcessingTime(ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime)
-      throws Exception {
-    checkState(!newProcessingTime.isBefore(processingTime),
-        "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
-    processingTime = newProcessingTime;
-    advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
-  }
-
-  private void advance(ReduceFnRunner<?, ?, ?, ?> runner, Instant newTime, TimeDomain domain)
-      throws Exception {
-    PriorityQueue<TimerData> timers = queue(domain);
-    boolean shouldFire = false;
-
-    do {
-      TimerData timer = timers.peek();
-      // Timers fire if the new time is ahead of the timer
-      shouldFire = timer != null && newTime.isAfter(timer.getTimestamp());
-      if (shouldFire) {
-        // Remove before firing, so that if the trigger adds another identical
-        // timer we don't remove it.
-        timers.remove();
-        runner.onTimer(timer);
-      }
-    } while (shouldFire);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
deleted file mode 100644
index 49206d1..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * An wrapper interface that represents the execution of a {@link OldDoFn}.
- */
-public interface DoFnRunner<InputT, OutputT> {
-  /**
-   * Prepares and calls {@link OldDoFn#startBundle}.
-   */
-  public void startBundle();
-
-  /**
-   * Calls {@link OldDoFn#processElement} with a {@link ProcessContext} containing the current
-   * element.
-   */
-  public void processElement(WindowedValue<InputT> elem);
-
-  /**
-   * Calls {@link OldDoFn#finishBundle} and performs additional tasks, such as
-   * flushing in-memory states.
-   */
-  public void finishBundle();
-
-  /**
-   * An internal interface for signaling that a {@link OldDoFn} requires late data dropping.
-   */
-  public interface ReduceFnExecutor<K, InputT, OutputT, W> {
-    /**
-     * Gets this object as a {@link OldDoFn}.
-     *
-     * Most implementors of this interface are expected to be {@link OldDoFn} instances, and will
-     * return themselves.
-     */
-    OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn();
-
-    /**
-     * Returns an aggregator that tracks elements that are dropped due to being late.
-     */
-    Aggregator<Long, Long> getDroppedDueToLatenessAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
deleted file mode 100644
index f0cfd74..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ /dev/null
@@ -1,551 +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 com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-
-/**
- * A base implementation of {@link DoFnRunner}.
- *
- * <p> Sub-classes should override {@link #invokeProcessElement}.
- */
-public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-
-  /** The {@link OldDoFn} being run. */
-  public final OldDoFn<InputT, OutputT> fn;
-
-  /** The context used for running the {@link OldDoFn}. */
-  public final DoFnContext<InputT, OutputT> context;
-
-  protected DoFnRunnerBase(
-      PipelineOptions options,
-      OldDoFn<InputT, OutputT> fn,
-      SideInputReader sideInputReader,
-      OutputManager outputManager,
-      TupleTag<OutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      StepContext stepContext,
-      AggregatorFactory aggregatorFactory,
-      WindowingStrategy<?, ?> windowingStrategy) {
-    this.fn = fn;
-    this.context = new DoFnContext<>(
-        options,
-        fn,
-        sideInputReader,
-        outputManager,
-        mainOutputTag,
-        sideOutputTags,
-        stepContext,
-        aggregatorFactory,
-        windowingStrategy == null ? null : windowingStrategy.getWindowFn());
-  }
-
-  /**
-   * An implementation of {@code OutputManager} using simple lists, for testing and in-memory
-   * contexts such as the {@code DirectRunner}.
-   */
-  public static class ListOutputManager implements OutputManager {
-
-    private Map<TupleTag<?>, List<WindowedValue<?>>> outputLists = Maps.newHashMap();
-
-    @Override
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
-
-      if (outputList == null) {
-        outputList = Lists.newArrayList();
-        @SuppressWarnings({"rawtypes", "unchecked"})
-        List<WindowedValue<?>> untypedList = (List) outputList;
-        outputLists.put(tag, untypedList);
-      }
-
-      outputList.add(output);
-    }
-
-    public <T> List<WindowedValue<T>> getOutput(TupleTag<T> tag) {
-      // Safe cast by design, inexpressible in Java without rawtypes
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
-      return (outputList != null) ? outputList : Collections.<WindowedValue<T>>emptyList();
-    }
-  }
-
-  @Override
-  public void startBundle() {
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.startBundle(context);
-    } catch (Throwable t) {
-      // Exception in user code.
-      throw wrapUserCodeException(t);
-    }
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> elem) {
-    if (elem.getWindows().size() <= 1
-        || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
-            && context.sideInputReader.isEmpty())) {
-      invokeProcessElement(elem);
-    } else {
-      // We could modify the windowed value (and the processContext) to
-      // avoid repeated allocations, but this is more straightforward.
-      for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
-        invokeProcessElement(windowedValue);
-      }
-    }
-  }
-
-  /**
-   * Invokes {@link OldDoFn#processElement} after certain pre-processings has been done in
-   * {@link DoFnRunnerBase#processElement}.
-   */
-  protected abstract void invokeProcessElement(WindowedValue<InputT> elem);
-
-  @Override
-  public void finishBundle() {
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.finishBundle(context);
-    } catch (Throwable t) {
-      // Exception in user code.
-      throw wrapUserCodeException(t);
-    }
-  }
-
-  /**
-   * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}.
-   *
-   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
-   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
-   */
-  private static class DoFnContext<InputT, OutputT>
-      extends OldDoFn<InputT, OutputT>.Context {
-    private static final int MAX_SIDE_OUTPUTS = 1000;
-
-    final PipelineOptions options;
-    final OldDoFn<InputT, OutputT> fn;
-    final SideInputReader sideInputReader;
-    final OutputManager outputManager;
-    final TupleTag<OutputT> mainOutputTag;
-    final StepContext stepContext;
-    final AggregatorFactory aggregatorFactory;
-    final WindowFn<?, ?> windowFn;
-
-    /**
-     * The set of known output tags, some of which may be undeclared, so we can throw an
-     * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
-     */
-    private Set<TupleTag<?>> outputTags;
-
-    public DoFnContext(PipelineOptions options,
-                       OldDoFn<InputT, OutputT> fn,
-                       SideInputReader sideInputReader,
-                       OutputManager outputManager,
-                       TupleTag<OutputT> mainOutputTag,
-                       List<TupleTag<?>> sideOutputTags,
-                       StepContext stepContext,
-                       AggregatorFactory aggregatorFactory,
-                       WindowFn<?, ?> windowFn) {
-      fn.super();
-      this.options = options;
-      this.fn = fn;
-      this.sideInputReader = sideInputReader;
-      this.outputManager = outputManager;
-      this.mainOutputTag = mainOutputTag;
-      this.outputTags = Sets.newHashSet();
-
-      outputTags.add(mainOutputTag);
-      for (TupleTag<?> sideOutputTag : sideOutputTags) {
-        outputTags.add(sideOutputTag);
-      }
-
-      this.stepContext = stepContext;
-      this.aggregatorFactory = aggregatorFactory;
-      this.windowFn = windowFn;
-      super.setupDelegateAggregators();
-    }
-
-    //////////////////////////////////////////////////////////////////////////////
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return options;
-    }
-
-    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
-        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
-      final Instant inputTimestamp = timestamp;
-
-      if (timestamp == null) {
-        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      }
-
-      if (windows == null) {
-        try {
-          // The windowFn can never succeed at accessing the element, so its type does not
-          // matter here
-          @SuppressWarnings("unchecked")
-          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
-          windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
-            @Override
-            public Object element() {
-              throw new UnsupportedOperationException(
-                  "WindowFn attempted to access input element when none was available");
-            }
-
-            @Override
-            public Instant timestamp() {
-              if (inputTimestamp == null) {
-                throw new UnsupportedOperationException(
-                    "WindowFn attempted to access input timestamp when none was available");
-              }
-              return inputTimestamp;
-            }
-
-            @Override
-            public W window() {
-              throw new UnsupportedOperationException(
-                  "WindowFn attempted to access input windows when none were available");
-            }
-          });
-        } catch (Exception e) {
-          throw UserCodeException.wrap(e);
-        }
-      }
-
-      return WindowedValue.of(output, timestamp, windows, pane);
-    }
-
-    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-      if (!sideInputReader.contains(view)) {
-        throw new IllegalArgumentException("calling sideInput() with unknown view");
-      }
-      BoundedWindow sideInputWindow =
-          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
-      return sideInputReader.get(view, sideInputWindow);
-    }
-
-    void outputWindowedValue(
-        OutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
-    }
-
-    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
-      outputManager.output(mainOutputTag, windowedElem);
-      if (stepContext != null) {
-        stepContext.noteOutput(windowedElem);
-      }
-    }
-
-    protected <T> void sideOutputWindowedValue(TupleTag<T> tag,
-                                               T output,
-                                               Instant timestamp,
-                                               Collection<? extends BoundedWindow> windows,
-                                               PaneInfo pane) {
-      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
-    }
-
-    protected <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
-      if (!outputTags.contains(tag)) {
-        // This tag wasn't declared nor was it seen before during this execution.
-        // Thus, this must be a new, undeclared and unconsumed output.
-        // To prevent likely user errors, enforce the limit on the number of side
-        // outputs.
-        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
-          throw new IllegalArgumentException(
-              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
-        }
-        outputTags.add(tag);
-      }
-
-      outputManager.output(tag, windowedElem);
-      if (stepContext != null) {
-        stepContext.noteSideOutput(tag, windowedElem);
-      }
-    }
-
-    // Following implementations of output, outputWithTimestamp, and sideOutput
-    // are only accessible in OldDoFn.startBundle and OldDoFn.finishBundle, and will be shadowed by
-    // ProcessContext's versions in OldDoFn.processElement.
-    @Override
-    public void output(OutputT output) {
-      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
-      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
-      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
-        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null");
-      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
-    }
-  }
-
-  /**
-   * Returns a new {@link OldDoFn.ProcessContext} for the given element.
-   */
-  protected OldDoFn<InputT, OutputT>.ProcessContext createProcessContext(
-      WindowedValue<InputT> elem) {
-    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
-  }
-
-  protected RuntimeException wrapUserCodeException(Throwable t) {
-    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
-  }
-
-  private boolean isSystemDoFn() {
-    return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
-  }
-
-  /**
-   * A concrete implementation of {@link OldDoFn.ProcessContext} used for
-   * running a {@link OldDoFn} over a single element.
-   *
-   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
-   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
-   */
-  static class DoFnProcessContext<InputT, OutputT>
-      extends OldDoFn<InputT, OutputT>.ProcessContext {
-
-
-    final OldDoFn<InputT, OutputT> fn;
-    final DoFnContext<InputT, OutputT> context;
-    final WindowedValue<InputT> windowedValue;
-
-    public DoFnProcessContext(OldDoFn<InputT, OutputT> fn,
-                              DoFnContext<InputT, OutputT> context,
-                              WindowedValue<InputT> windowedValue) {
-      fn.super();
-      this.fn = fn;
-      this.context = context;
-      this.windowedValue = windowedValue;
-    }
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return context.getPipelineOptions();
-    }
-
-    @Override
-    public InputT element() {
-      return windowedValue.getValue();
-    }
-
-    @Override
-    public <T> T sideInput(PCollectionView<T> view) {
-      checkNotNull(view, "View passed to sideInput cannot be null");
-      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
-      BoundedWindow window;
-      if (!windowIter.hasNext()) {
-        if (context.windowFn instanceof GlobalWindows) {
-          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
-          // without windows
-          window = GlobalWindow.INSTANCE;
-        } else {
-          throw new IllegalStateException(
-              "sideInput called when main input element is not in any windows");
-        }
-      } else {
-        window = windowIter.next();
-        if (windowIter.hasNext()) {
-          throw new IllegalStateException(
-              "sideInput called when main input element is in multiple windows");
-        }
-      }
-      return context.sideInput(view, window);
-    }
-
-    @Override
-    public BoundedWindow window() {
-      if (!(fn instanceof RequiresWindowAccess)) {
-        throw new UnsupportedOperationException(
-            "window() is only available in the context of a OldDoFn marked as"
-                + "RequiresWindowAccess.");
-      }
-      return Iterables.getOnlyElement(windows());
-    }
-
-    @Override
-    public PaneInfo pane() {
-      return windowedValue.getPane();
-    }
-
-    @Override
-    public void output(OutputT output) {
-      context.outputWindowedValue(windowedValue.withValue(output));
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      checkTimestamp(timestamp);
-      context.outputWindowedValue(output, timestamp,
-          windowedValue.getWindows(), windowedValue.getPane());
-    }
-
-    void outputWindowedValue(
-        OutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      context.outputWindowedValue(output, timestamp, windows, pane);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
-      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
-      checkTimestamp(timestamp);
-      context.sideOutputWindowedValue(
-          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
-    }
-
-    @Override
-    public Instant timestamp() {
-      return windowedValue.getTimestamp();
-    }
-
-    public Collection<? extends BoundedWindow> windows() {
-      return windowedValue.getWindows();
-    }
-
-    private void checkTimestamp(Instant timestamp) {
-      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
-        throw new IllegalArgumentException(String.format(
-            "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
-            + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
-            + "OldDoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
-            timestamp, windowedValue.getTimestamp(),
-            PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
-      }
-    }
-
-    @Override
-    public WindowingInternals<InputT, OutputT> windowingInternals() {
-      return new WindowingInternals<InputT, OutputT>() {
-        @Override
-        public void outputWindowedValue(OutputT output, Instant timestamp,
-            Collection<? extends BoundedWindow> windows, PaneInfo pane) {
-          context.outputWindowedValue(output, timestamp, windows, pane);
-        }
-
-        @Override
-        public Collection<? extends BoundedWindow> windows() {
-          return windowedValue.getWindows();
-        }
-
-        @Override
-        public PaneInfo pane() {
-          return windowedValue.getPane();
-        }
-
-        @Override
-        public TimerInternals timerInternals() {
-          return context.stepContext.timerInternals();
-        }
-
-        @Override
-        public <T> void writePCollectionViewData(
-            TupleTag<?> tag,
-            Iterable<WindowedValue<T>> data,
-            Coder<T> elemCoder) throws IOException {
-          @SuppressWarnings("unchecked")
-          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
-
-          context.stepContext.writePCollectionViewData(
-              tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
-              window(), windowCoder);
-        }
-
-        @Override
-        public StateInternals<?> stateInternals() {
-          return context.stepContext.stateInternals();
-        }
-
-        @Override
-        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-          return context.sideInput(view, mainInputWindow);
-        }
-      };
-    }
-
-    @Override
-    protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
-        createAggregatorInternal(
-            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
-      return context.createAggregatorInternal(name, combiner);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
deleted file mode 100644
index c4df7b2..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import java.util.List;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Static utility methods that provide {@link DoFnRunner} implementations.
- */
-public class DoFnRunners {
-  /**
-   * Information about how to create output receivers and output to them.
-   */
-  public interface OutputManager {
-    /**
-     * Outputs a single element to the receiver indicated by the given {@link TupleTag}.
-     */
-    public <T> void output(TupleTag<T> tag, WindowedValue<T> output);
-  }
-
-  /**
-   * Returns a basic implementation of {@link DoFnRunner} that works for most {@link OldDoFn DoFns}.
-   *
-   * <p>It invokes {@link OldDoFn#processElement} for each input.
-   */
-  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> simpleRunner(
-      PipelineOptions options,
-      OldDoFn<InputT, OutputT> fn,
-      SideInputReader sideInputReader,
-      OutputManager outputManager,
-      TupleTag<OutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      StepContext stepContext,
-      AggregatorFactory aggregatorFactory,
-      WindowingStrategy<?, ?> windowingStrategy) {
-    return new SimpleDoFnRunner<>(
-        options,
-        fn,
-        sideInputReader,
-        outputManager,
-        mainOutputTag,
-        sideOutputTags,
-        stepContext,
-        aggregatorFactory,
-        windowingStrategy);
-  }
-
-  /**
-   * Returns an implementation of {@link DoFnRunner} that handles late data dropping.
-   *
-   * <p>It drops elements from expired windows before they reach the underlying {@link OldDoFn}.
-   */
-  public static <K, InputT, OutputT, W extends BoundedWindow>
-      DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> lateDataDroppingRunner(
-          PipelineOptions options,
-          ReduceFnExecutor<K, InputT, OutputT, W> reduceFnExecutor,
-          SideInputReader sideInputReader,
-          OutputManager outputManager,
-          TupleTag<KV<K, OutputT>> mainOutputTag,
-          List<TupleTag<?>> sideOutputTags,
-          StepContext stepContext,
-          AggregatorFactory aggregatorFactory,
-          WindowingStrategy<?, W> windowingStrategy) {
-    DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> simpleDoFnRunner =
-        simpleRunner(
-            options,
-            reduceFnExecutor.asDoFn(),
-            sideInputReader,
-            outputManager,
-            mainOutputTag,
-            sideOutputTags,
-            stepContext,
-            aggregatorFactory,
-            windowingStrategy);
-    return new LateDataDroppingDoFnRunner<>(
-        simpleDoFnRunner,
-        windowingStrategy,
-        stepContext.timerInternals(),
-        reduceFnExecutor.getDroppedDueToLatenessAggregator());
-  }
-
-
-  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
-      PipelineOptions options,
-      OldDoFn<InputT, OutputT> doFn,
-      SideInputReader sideInputReader,
-      OutputManager outputManager,
-      TupleTag<OutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      StepContext stepContext,
-      AggregatorFactory aggregatorFactory,
-      WindowingStrategy<?, ?> windowingStrategy) {
-    if (doFn instanceof ReduceFnExecutor) {
-      @SuppressWarnings("rawtypes")
-      ReduceFnExecutor fn = (ReduceFnExecutor) doFn;
-      @SuppressWarnings({"unchecked", "cast", "rawtypes"})
-      DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
-          options,
-          fn,
-          sideInputReader,
-          outputManager,
-          (TupleTag) mainOutputTag,
-          sideOutputTags,
-          stepContext,
-          aggregatorFactory,
-          (WindowingStrategy) windowingStrategy);
-      return runner;
-    }
-    return simpleRunner(
-        options,
-        doFn,
-        sideInputReader,
-        outputManager,
-        mainOutputTag,
-        sideOutputTags,
-        stepContext,
-        aggregatorFactory,
-        windowingStrategy);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
deleted file mode 100644
index f386dfb..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * {@link OldDoFn} that merges windows and groups elements in those windows, optionally
- * combining values.
- *
- * @param <K> key type
- * @param <InputT> input value element type
- * @param <OutputT> output value element type
- * @param <W> window type
- */
-@SystemDoFnInternal
-public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends BoundedWindow>
-    extends OldDoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> {
-  public static final String DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER = "DroppedDueToClosedWindow";
-  public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness";
-
-  protected final Aggregator<Long, Long> droppedDueToClosedWindow =
-      createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
-  protected final Aggregator<Long, Long> droppedDueToLateness =
-      createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
-
-  /**
-   * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
-   * grouping.
-   *
-   * @param windowingStrategy The window function and trigger to use for grouping
-   * @param inputCoder the input coder to use
-   */
-  public static <K, V, W extends BoundedWindow>
-      GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W> createDefault(
-          WindowingStrategy<?, W> windowingStrategy,
-          StateInternalsFactory<K> stateInternalsFactory,
-          Coder<V> inputCoder) {
-    return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
-        windowingStrategy, stateInternalsFactory, SystemReduceFn.<K, V, W>buffering(inputCoder));
-  }
-}


[28/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
new file mode 100644
index 0000000..45062fb
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
@@ -0,0 +1,796 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Function;
+import com.google.common.base.MoreObjects;
+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.Sets;
+import java.io.IOException;
+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.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.PriorityQueue;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.coders.Coder;
+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.StringUtf8Coder;
+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.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+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.AppliedCombineFn;
+import org.apache.beam.sdk.util.ExecutableTrigger;
+import org.apache.beam.sdk.util.NullSideInputReader;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.SideInputReader;
+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.WindowTracing;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub
+ * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to
+ * run {@code Trigger}s and {@code ReduceFn}s.
+ *
+ * @param <InputT> The element types.
+ * @param <OutputT> The final type for elements in the window (for instance,
+ *     {@code Iterable<InputT>})
+ * @param <W> The type of windows being used.
+ */
+public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
+  private static final String KEY = "TEST_KEY";
+
+  private final TestInMemoryStateInternals<String> stateInternals =
+      new TestInMemoryStateInternals<>(KEY);
+  private final TestTimerInternals timerInternals = new TestTimerInternals();
+
+  private final WindowFn<Object, W> windowFn;
+  private final TestWindowingInternals windowingInternals;
+  private final Coder<OutputT> outputCoder;
+  private final WindowingStrategy<Object, W> objectStrategy;
+  private final ReduceFn<String, InputT, OutputT, W> reduceFn;
+  private final PipelineOptions options;
+
+  /**
+   * If true, the output watermark is automatically advanced to the latest possible
+   * point when the input watermark is advanced. This is the default for most tests.
+   * If false, the output watermark must be explicitly advanced by the test, which can
+   * be used to exercise some of the more subtle behavior of WatermarkHold.
+   */
+  private boolean autoAdvanceOutputWatermark;
+
+  private ExecutableTrigger executableTrigger;
+
+  private final InMemoryLongSumAggregator droppedDueToClosedWindow =
+      new InMemoryLongSumAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER);
+
+  public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
+      nonCombining(WindowingStrategy<?, W> windowingStrategy) throws Exception {
+    return new ReduceFnTester<Integer, Iterable<Integer>, W>(
+        windowingStrategy,
+        SystemReduceFn.<String, Integer, W>buffering(VarIntCoder.of()),
+        IterableCoder.of(VarIntCoder.of()),
+        PipelineOptionsFactory.create(),
+        NullSideInputReader.empty());
+  }
+
+  public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
+      nonCombining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
+          Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception {
+    WindowingStrategy<?, W> strategy =
+        WindowingStrategy.of(windowFn)
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withTrigger(trigger)
+            .withMode(mode)
+            .withAllowedLateness(allowedDataLateness)
+            .withClosingBehavior(closingBehavior);
+    return nonCombining(strategy);
+  }
+
+  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+      combining(WindowingStrategy<?, W> strategy,
+          KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn,
+          Coder<OutputT> outputCoder) throws Exception {
+
+    CoderRegistry registry = new CoderRegistry();
+    registry.registerStandardCoders();
+    AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
+        AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
+            combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+
+    return new ReduceFnTester<Integer, OutputT, W>(
+        strategy,
+        SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
+        outputCoder,
+        PipelineOptionsFactory.create(),
+        NullSideInputReader.empty());
+  }
+
+  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+  combining(WindowingStrategy<?, W> strategy,
+      KeyedCombineFnWithContext<String, Integer, AccumT, OutputT> combineFn,
+      Coder<OutputT> outputCoder,
+      PipelineOptions options,
+      SideInputReader sideInputReader) throws Exception {
+    CoderRegistry registry = new CoderRegistry();
+    registry.registerStandardCoders();
+    AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
+        AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
+            combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+
+    return new ReduceFnTester<Integer, OutputT, W>(
+        strategy,
+        SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
+        outputCoder,
+        options,
+        sideInputReader);
+  }
+  public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+      combining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
+          KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder,
+          Duration allowedDataLateness) throws Exception {
+
+    WindowingStrategy<?, W> strategy =
+        WindowingStrategy.of(windowFn)
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withTrigger(trigger)
+            .withMode(mode)
+            .withAllowedLateness(allowedDataLateness);
+
+    return combining(strategy, combineFn, outputCoder);
+  }
+
+  private ReduceFnTester(WindowingStrategy<?, W> wildcardStrategy,
+      ReduceFn<String, InputT, OutputT, W> reduceFn, Coder<OutputT> outputCoder,
+      PipelineOptions options, SideInputReader sideInputReader) throws Exception {
+    @SuppressWarnings("unchecked")
+    WindowingStrategy<Object, W> objectStrategy = (WindowingStrategy<Object, W>) wildcardStrategy;
+
+    this.objectStrategy = objectStrategy;
+    this.reduceFn = reduceFn;
+    this.windowFn = objectStrategy.getWindowFn();
+    this.windowingInternals = new TestWindowingInternals(sideInputReader);
+    this.outputCoder = outputCoder;
+    this.autoAdvanceOutputWatermark = true;
+    this.executableTrigger = wildcardStrategy.getTrigger();
+    this.options = options;
+  }
+
+  public void setAutoAdvanceOutputWatermark(boolean autoAdvanceOutputWatermark) {
+    this.autoAdvanceOutputWatermark = autoAdvanceOutputWatermark;
+  }
+
+  @Nullable
+  public Instant getNextTimer(TimeDomain domain) {
+    return timerInternals.getNextTimer(domain);
+  }
+
+  ReduceFnRunner<String, InputT, OutputT, W> createRunner() {
+    return new ReduceFnRunner<>(
+        KEY,
+        objectStrategy,
+        stateInternals,
+        timerInternals,
+        windowingInternals,
+        droppedDueToClosedWindow,
+        reduceFn,
+        options);
+  }
+
+  public ExecutableTrigger getTrigger() {
+    return executableTrigger;
+  }
+
+  public boolean isMarkedFinished(W window) {
+    return createRunner().isFinished(window);
+  }
+
+  public boolean hasNoActiveWindows() {
+    return createRunner().hasNoActiveWindows();
+  }
+
+  @SafeVarargs
+  public final void assertHasOnlyGlobalAndFinishedSetsFor(W... expectedWindows) {
+    assertHasOnlyGlobalAndAllowedTags(
+        ImmutableSet.copyOf(expectedWindows),
+        ImmutableSet.<StateTag<? super String, ?>>of(TriggerRunner.FINISHED_BITS_TAG));
+  }
+
+  @SafeVarargs
+  public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expectedWindows) {
+    assertHasOnlyGlobalAndAllowedTags(
+        ImmutableSet.copyOf(expectedWindows),
+        ImmutableSet.<StateTag<? super String, ?>>of(
+            TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG,
+            WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
+            WatermarkHold.EXTRA_HOLD_TAG));
+  }
+
+  public final void assertHasOnlyGlobalState() {
+    assertHasOnlyGlobalAndAllowedTags(
+        Collections.<W>emptySet(), Collections.<StateTag<? super String, ?>>emptySet());
+  }
+
+  @SafeVarargs
+  public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) {
+    assertHasOnlyGlobalAndAllowedTags(
+        ImmutableSet.copyOf(expectedWindows),
+        ImmutableSet.<StateTag<? super String, ?>>of(
+            PaneInfoTracker.PANE_INFO_TAG,
+            WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
+            WatermarkHold.EXTRA_HOLD_TAG));
+  }
+
+  /**
+   * Verifies that the the set of windows that have any state stored is exactly
+   * {@code expectedWindows} and that each of these windows has only tags from {@code allowedTags}.
+   */
+  private void assertHasOnlyGlobalAndAllowedTags(
+      Set<W> expectedWindows, Set<StateTag<? super String, ?>> allowedTags) {
+    Set<StateNamespace> expectedWindowsSet = new HashSet<>();
+    for (W expectedWindow : expectedWindows) {
+      expectedWindowsSet.add(windowNamespace(expectedWindow));
+    }
+    Map<StateNamespace, Set<StateTag<? super String, ?>>> actualWindows = new HashMap<>();
+
+    for (StateNamespace namespace : stateInternals.getNamespacesInUse()) {
+      if (namespace instanceof StateNamespaces.GlobalNamespace) {
+        continue;
+      } else if (namespace instanceof StateNamespaces.WindowNamespace) {
+        Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
+        if (tagsInUse.isEmpty()) {
+          continue;
+        }
+        actualWindows.put(namespace, tagsInUse);
+        Set<StateTag<? super String, ?>> unexpected = Sets.difference(tagsInUse, allowedTags);
+        if (unexpected.isEmpty()) {
+          continue;
+        } else {
+          fail(namespace + " has unexpected states: " + tagsInUse);
+        }
+      } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) {
+        Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
+        assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty());
+      } else {
+        fail("Unrecognized namespace " + namespace);
+      }
+    }
+
+    assertEquals("Still in use: " + actualWindows.toString(), expectedWindowsSet,
+        actualWindows.keySet());
+  }
+
+  private StateNamespace windowNamespace(W window) {
+    return StateNamespaces.window(windowFn.windowCoder(), window);
+  }
+
+  public Instant getWatermarkHold() {
+    return stateInternals.earliestWatermarkHold();
+  }
+
+  public Instant getOutputWatermark() {
+    return timerInternals.currentOutputWatermarkTime();
+  }
+
+  public long getElementsDroppedDueToClosedWindow() {
+    return droppedDueToClosedWindow.getSum();
+  }
+
+  /**
+   * How many panes do we have in the output?
+   */
+  public int getOutputSize() {
+    return windowingInternals.outputs.size();
+  }
+
+  /**
+   * Retrieve the values that have been output to this time, and clear out the output accumulator.
+   */
+  public List<WindowedValue<OutputT>> extractOutput() {
+    ImmutableList<WindowedValue<OutputT>> result =
+        FluentIterable.from(windowingInternals.outputs)
+            .transform(new Function<WindowedValue<KV<String, OutputT>>, WindowedValue<OutputT>>() {
+              @Override
+              public WindowedValue<OutputT> apply(WindowedValue<KV<String, OutputT>> input) {
+                return input.withValue(input.getValue().getValue());
+              }
+            })
+            .toList();
+    windowingInternals.outputs.clear();
+    return result;
+  }
+
+  /**
+   * Advance the input watermark to the specified time, firing any timers that should
+   * fire. Then advance the output watermark as far as possible.
+   */
+  public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
+    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+    timerInternals.advanceInputWatermark(runner, newInputWatermark);
+    runner.persist();
+  }
+
+  /**
+   * If {@link #autoAdvanceOutputWatermark} is {@literal false}, advance the output watermark
+   * to the given value. Otherwise throw.
+   */
+  public void advanceOutputWatermark(Instant newOutputWatermark) throws Exception {
+    timerInternals.advanceOutputWatermark(newOutputWatermark);
+  }
+
+  /** Advance the processing time to the specified time, firing any timers that should fire. */
+  public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
+    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+    timerInternals.advanceProcessingTime(runner, newProcessingTime);
+    runner.persist();
+  }
+
+  /**
+   * Advance the synchronized processing time to the specified time,
+   * firing any timers that should fire.
+   */
+  public void advanceSynchronizedProcessingTime(Instant newProcessingTime) throws Exception {
+    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+    timerInternals.advanceSynchronizedProcessingTime(runner, newProcessingTime);
+    runner.persist();
+  }
+
+  /**
+   * Inject all the timestamped values (after passing through the window function) as if they
+   * arrived in a single chunk of a bundle (or work-unit).
+   */
+  @SafeVarargs
+  public final void injectElements(TimestampedValue<InputT>... values) throws Exception {
+    for (TimestampedValue<InputT> value : values) {
+      WindowTracing.trace("TriggerTester.injectElements: {}", value);
+    }
+    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+    runner.processElements(
+        Iterables.transform(
+            Arrays.asList(values),
+            new Function<TimestampedValue<InputT>, WindowedValue<InputT>>() {
+              @Override
+              public WindowedValue<InputT> apply(TimestampedValue<InputT> input) {
+                try {
+                  InputT value = input.getValue();
+                  Instant timestamp = input.getTimestamp();
+                  Collection<W> windows =
+                      windowFn.assignWindows(
+                          new TestAssignContext<W>(
+                              windowFn, value, timestamp, GlobalWindow.INSTANCE));
+                  return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING);
+                } catch (Exception e) {
+                  throw new RuntimeException(e);
+                }
+              }
+            }));
+
+    // Persist after each bundle.
+    runner.persist();
+  }
+
+  public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception {
+    ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+    runner.onTimer(
+        TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain));
+    runner.persist();
+  }
+
+  /**
+   * Simulate state.
+   */
+  private static class TestInMemoryStateInternals<K> extends InMemoryStateInternals<K> {
+
+    public TestInMemoryStateInternals(K key) {
+      super(key);
+    }
+
+    public Set<StateTag<? super K, ?>> getTagsInUse(StateNamespace namespace) {
+      Set<StateTag<? super K, ?>> inUse = new HashSet<>();
+      for (Entry<StateTag<? super K, ?>, State> entry :
+        inMemoryState.getTagsInUse(namespace).entrySet()) {
+        if (!isEmptyForTesting(entry.getValue())) {
+          inUse.add(entry.getKey());
+        }
+      }
+      return inUse;
+    }
+
+    public Set<StateNamespace> getNamespacesInUse() {
+      return inMemoryState.getNamespacesInUse();
+    }
+
+    /** Return the earliest output watermark hold in state, or null if none. */
+    public Instant earliestWatermarkHold() {
+      Instant minimum = null;
+      for (State storage : inMemoryState.values()) {
+        if (storage instanceof WatermarkHoldState) {
+          Instant hold = ((WatermarkHoldState<?>) storage).read();
+          if (minimum == null || (hold != null && hold.isBefore(minimum))) {
+            minimum = hold;
+          }
+        }
+      }
+      return minimum;
+    }
+  }
+
+  /**
+   * Convey the simulated state and implement {@link #outputWindowedValue} to capture all output
+   * elements.
+   */
+  private class TestWindowingInternals implements WindowingInternals<InputT, KV<String, OutputT>> {
+    private List<WindowedValue<KV<String, OutputT>>> outputs = new ArrayList<>();
+    private SideInputReader sideInputReader;
+
+    private TestWindowingInternals(SideInputReader sideInputReader) {
+      this.sideInputReader = sideInputReader;
+    }
+
+    @Override
+    public void outputWindowedValue(KV<String, OutputT> output, Instant timestamp,
+        Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+      // Copy the output value (using coders) before capturing it.
+      KV<String, OutputT> copy = SerializableUtils.<KV<String, OutputT>>ensureSerializableByCoder(
+          KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow");
+      WindowedValue<KV<String, OutputT>> value = WindowedValue.of(copy, timestamp, windows, pane);
+      outputs.add(value);
+    }
+
+    @Override
+    public TimerInternals timerInternals() {
+      throw new UnsupportedOperationException(
+          "Testing triggers should not use timers from WindowingInternals.");
+    }
+
+    @Override
+    public Collection<? extends BoundedWindow> windows() {
+      throw new UnsupportedOperationException(
+          "Testing triggers should not use windows from WindowingInternals.");
+    }
+
+    @Override
+    public PaneInfo pane() {
+      throw new UnsupportedOperationException(
+          "Testing triggers should not use pane from WindowingInternals.");
+    }
+
+    @Override
+    public <T> void writePCollectionViewData(
+        TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+      throw new UnsupportedOperationException(
+          "Testing triggers should not use writePCollectionViewData from WindowingInternals.");
+    }
+
+    @Override
+    public StateInternals<Object> stateInternals() {
+      // Safe for testing only
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      TestInMemoryStateInternals<Object> untypedStateInternals =
+          (TestInMemoryStateInternals) stateInternals;
+      return untypedStateInternals;
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+      if (!sideInputReader.contains(view)) {
+        throw new IllegalArgumentException("calling sideInput() with unknown view");
+      }
+      BoundedWindow sideInputWindow =
+          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+      return sideInputReader.get(view, sideInputWindow);
+    }
+  }
+
+  private static class TestAssignContext<W extends BoundedWindow>
+      extends WindowFn<Object, W>.AssignContext {
+    private Object element;
+    private Instant timestamp;
+    private BoundedWindow window;
+
+    public TestAssignContext(
+        WindowFn<Object, W> windowFn, Object element, Instant timestamp, BoundedWindow window) {
+      windowFn.super();
+      this.element = element;
+      this.timestamp = timestamp;
+      this.window = window;
+    }
+
+    @Override
+    public Object element() {
+      return element;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return window;
+    }
+  }
+
+  private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
+    private final String name;
+    private long sum = 0;
+
+    public InMemoryLongSumAggregator(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public void addValue(Long value) {
+      sum += value;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public CombineFn<Long, ?, Long> getCombineFn() {
+      return new Sum.SumLongFn();
+    }
+
+    public long getSum() {
+      return sum;
+    }
+  }
+
+  /**
+   * Simulate the firing of timers and progression of input and output watermarks for a
+   * single computation and key in a Windmill-like streaming environment. Similar to
+   * {@link BatchTimerInternals}, but also tracks the output watermark.
+   */
+  private class TestTimerInternals implements TimerInternals {
+    /** At most one timer per timestamp is kept. */
+    private Set<TimerData> existingTimers = new HashSet<>();
+
+    /** Pending input watermark timers, in timestamp order. */
+    private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
+
+    /** Pending processing time timers, in timestamp order. */
+    private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
+
+    /** Current input watermark. */
+    @Nullable
+    private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+    /** Current output watermark. */
+    @Nullable
+    private Instant outputWatermarkTime = null;
+
+    /** Current processing time. */
+    private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+    /** Current synchronized processing time. */
+    @Nullable
+    private Instant synchronizedProcessingTime = null;
+
+    @Nullable
+    public Instant getNextTimer(TimeDomain domain) {
+      TimerData data = null;
+      switch (domain) {
+        case EVENT_TIME:
+           data = watermarkTimers.peek();
+           break;
+        case PROCESSING_TIME:
+        case SYNCHRONIZED_PROCESSING_TIME:
+          data = processingTimers.peek();
+          break;
+      }
+      checkNotNull(data); // cases exhaustive
+      return data == null ? null : data.getTimestamp();
+    }
+
+    private PriorityQueue<TimerData> queue(TimeDomain domain) {
+      switch (domain) {
+        case EVENT_TIME:
+          return watermarkTimers;
+        case PROCESSING_TIME:
+        case SYNCHRONIZED_PROCESSING_TIME:
+          return processingTimers;
+      }
+      throw new RuntimeException(); // cases exhaustive
+    }
+
+    @Override
+    public void setTimer(TimerData timer) {
+      WindowTracing.trace("TestTimerInternals.setTimer: {}", timer);
+      if (existingTimers.add(timer)) {
+        queue(timer.getDomain()).add(timer);
+      }
+    }
+
+    @Override
+    public void deleteTimer(TimerData timer) {
+      WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer);
+      existingTimers.remove(timer);
+      queue(timer.getDomain()).remove(timer);
+    }
+
+    @Override
+    public Instant currentProcessingTime() {
+      return processingTime;
+    }
+
+    @Override
+    @Nullable
+    public Instant currentSynchronizedProcessingTime() {
+      return synchronizedProcessingTime;
+    }
+
+    @Override
+    public Instant currentInputWatermarkTime() {
+      return checkNotNull(inputWatermarkTime);
+    }
+
+    @Override
+    @Nullable
+    public Instant currentOutputWatermarkTime() {
+      return outputWatermarkTime;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(getClass())
+          .add("watermarkTimers", watermarkTimers)
+          .add("processingTimers", processingTimers)
+          .add("inputWatermarkTime", inputWatermarkTime)
+          .add("outputWatermarkTime", outputWatermarkTime)
+          .add("processingTime", processingTime)
+          .toString();
+    }
+
+    public void advanceInputWatermark(
+        ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark) throws Exception {
+      checkNotNull(newInputWatermark);
+      checkState(
+          !newInputWatermark.isBefore(inputWatermarkTime),
+          "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
+          newInputWatermark);
+      WindowTracing.trace("TestTimerInternals.advanceInputWatermark: from {} to {}",
+          inputWatermarkTime, newInputWatermark);
+      inputWatermarkTime = newInputWatermark;
+      advanceAndFire(runner, newInputWatermark, TimeDomain.EVENT_TIME);
+
+      Instant hold = stateInternals.earliestWatermarkHold();
+      if (hold == null) {
+        WindowTracing.trace("TestTimerInternals.advanceInputWatermark: no holds, "
+            + "so output watermark = input watermark");
+        hold = inputWatermarkTime;
+      }
+      if (autoAdvanceOutputWatermark) {
+        advanceOutputWatermark(hold);
+      }
+    }
+
+    public void advanceOutputWatermark(Instant newOutputWatermark) {
+      checkNotNull(newOutputWatermark);
+      if (newOutputWatermark.isAfter(inputWatermarkTime)) {
+        WindowTracing.trace(
+            "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}",
+            newOutputWatermark, inputWatermarkTime);
+        newOutputWatermark = inputWatermarkTime;
+      }
+      checkState(
+          outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime),
+          "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime,
+          newOutputWatermark);
+      WindowTracing.trace("TestTimerInternals.advanceOutputWatermark: from {} to {}",
+          outputWatermarkTime, newOutputWatermark);
+      outputWatermarkTime = newOutputWatermark;
+    }
+
+    public void advanceProcessingTime(
+        ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime) throws Exception {
+      checkState(!newProcessingTime.isBefore(processingTime),
+          "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
+      WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime,
+          newProcessingTime);
+      processingTime = newProcessingTime;
+      advanceAndFire(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
+    }
+
+    public void advanceSynchronizedProcessingTime(
+        ReduceFnRunner<?, ?, ?, ?> runner, Instant newSynchronizedProcessingTime) throws Exception {
+      checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
+          "Cannot move processing time backwards from %s to %s", processingTime,
+          newSynchronizedProcessingTime);
+      WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}",
+          synchronizedProcessingTime, newSynchronizedProcessingTime);
+      synchronizedProcessingTime = newSynchronizedProcessingTime;
+      advanceAndFire(
+          runner, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    }
+
+    private void advanceAndFire(
+        ReduceFnRunner<?, ?, ?, ?> runner, Instant currentTime, TimeDomain domain)
+            throws Exception {
+      PriorityQueue<TimerData> queue = queue(domain);
+      boolean shouldFire = false;
+
+      do {
+        TimerData timer = queue.peek();
+        // Timers fire when the current time progresses past the timer time.
+        shouldFire = timer != null && currentTime.isAfter(timer.getTimestamp());
+        if (shouldFire) {
+          WindowTracing.trace(
+              "TestTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime);
+          // Remove before firing, so that if the trigger adds another identical
+          // timer we don't remove it.
+          queue.remove();
+
+          runner.onTimer(timer);
+        }
+      } while (shouldFire);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
new file mode 100644
index 0000000..adb0aac
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static org.hamcrest.Matchers.is;
+import static org.mockito.Mockito.mock;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for base {@link DoFnRunnerBase} functionality.
+ */
+@RunWith(JUnit4.class)
+public class SimpleDoFnRunnerTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner = createRunner(fn);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+  }
+
+  @Test
+  public void testSystemDoFnInternalExceptionsNotWrapped() {
+    ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
+    DoFnRunner<String, String> runner = createRunner(fn);
+
+    thrown.expect(is(fn.exceptionToThrow));
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+  }
+
+  private DoFnRunner<String, String> createRunner(OldDoFn<String, String> fn) {
+    // Pass in only necessary parameters for the test
+    List<TupleTag<?>> sideOutputTags = Arrays.asList();
+    StepContext context = mock(StepContext.class);
+    return new SimpleDoFnRunner<>(
+          null, fn, null, null, null, sideOutputTags, context, null, null);
+  }
+
+  static class ThrowingDoFn extends OldDoFn<String, String> {
+    final Exception exceptionToThrow =
+        new UnsupportedOperationException("Expected exception");
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      throw exceptionToThrow;
+    }
+  }
+
+  @SystemDoFnInternal
+  static class ThrowingSystemDoFn extends ThrowingDoFn {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
deleted file mode 100644
index 20a9852..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
+++ /dev/null
@@ -1,117 +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 org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaceForTest;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-/**
- * Tests for {@link BatchTimerInternals}.
- */
-@RunWith(JUnit4.class)
-public class BatchTimerInternalsTest {
-
-  private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
-
-  @Mock
-  private ReduceFnRunner<?, ?, ?, ?> mockRunner;
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  @Test
-  public void testFiringTimers() throws Exception {
-    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
-    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
-
-    underTest.setTimer(processingTime1);
-    underTest.setTimer(processingTime2);
-
-    underTest.advanceProcessingTime(mockRunner, new Instant(20));
-    Mockito.verify(mockRunner).onTimer(processingTime1);
-    Mockito.verifyNoMoreInteractions(mockRunner);
-
-    // Advancing just a little shouldn't refire
-    underTest.advanceProcessingTime(mockRunner, new Instant(21));
-    Mockito.verifyNoMoreInteractions(mockRunner);
-
-    // Adding the timer and advancing a little should refire
-    underTest.setTimer(processingTime1);
-    Mockito.verify(mockRunner).onTimer(processingTime1);
-    underTest.advanceProcessingTime(mockRunner, new Instant(21));
-    Mockito.verifyNoMoreInteractions(mockRunner);
-
-    // And advancing the rest of the way should still have the other timer
-    underTest.advanceProcessingTime(mockRunner, new Instant(30));
-    Mockito.verify(mockRunner).onTimer(processingTime2);
-    Mockito.verifyNoMoreInteractions(mockRunner);
-  }
-
-  @Test
-  public void testTimerOrdering() throws Exception {
-    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
-    TimerData watermarkTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
-    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    TimerData watermarkTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
-    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
-
-    underTest.setTimer(processingTime1);
-    underTest.setTimer(watermarkTime1);
-    underTest.setTimer(processingTime2);
-    underTest.setTimer(watermarkTime2);
-
-    underTest.advanceInputWatermark(mockRunner, new Instant(30));
-    Mockito.verify(mockRunner).onTimer(watermarkTime1);
-    Mockito.verify(mockRunner).onTimer(watermarkTime2);
-    Mockito.verifyNoMoreInteractions(mockRunner);
-
-    underTest.advanceProcessingTime(mockRunner, new Instant(30));
-    Mockito.verify(mockRunner).onTimer(processingTime1);
-    Mockito.verify(mockRunner).onTimer(processingTime2);
-    Mockito.verifyNoMoreInteractions(mockRunner);
-  }
-
-  @Test
-  public void testDeduplicate() throws Exception {
-    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
-    TimerData watermarkTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
-    TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    underTest.setTimer(watermarkTime);
-    underTest.setTimer(watermarkTime);
-    underTest.setTimer(processingTime);
-    underTest.setTimer(processingTime);
-    underTest.advanceProcessingTime(mockRunner, new Instant(20));
-    underTest.advanceInputWatermark(mockRunner, new Instant(20));
-
-    Mockito.verify(mockRunner).onTimer(processingTime);
-    Mockito.verify(mockRunner).onTimer(watermarkTime);
-    Mockito.verifyNoMoreInteractions(mockRunner);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
deleted file mode 100644
index 215cd4c..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
+++ /dev/null
@@ -1,658 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasSize;
-import static org.junit.Assert.assertThat;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFnTester;
-import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.util.state.InMemoryStateInternals;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Properties of {@link GroupAlsoByWindowsDoFn}.
- *
- * <p>Some properties may not hold of some implementations, due to restrictions on the context
- * in which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not
- * support merging windows.
- */
-public class GroupAlsoByWindowsProperties {
-
-  /**
-   * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide
-   * the appropriate windowing strategy under test.
-   */
-  public interface GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> {
-    <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, OutputT, W>
-    forStrategy(WindowingStrategy<?, W> strategy, StateInternalsFactory<K> stateInternalsFactory);
-  }
-
-  /**
-   * Tests that for empty input and the given {@link WindowingStrategy}, the provided GABW
-   * implementation produces no output.
-   *
-   * <p>The input type is deliberately left as a wildcard, since it is not relevant.
-   */
-  public static <K, InputT, OutputT> void emptyInputEmptyOutput(
-      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
-    // This key should never actually be used, though it is eagerly passed to the
-    // StateInternalsFactory so must be non-null
-    @SuppressWarnings("unchecked")
-    K fakeKey = (K) "this key should never be used";
-
-    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> result = runGABW(
-        gabwFactory,
-        windowingStrategy,
-        fakeKey,
-        Collections.<WindowedValue<InputT>>emptyList());
-
-    assertThat(result.peekOutputElements(), hasSize(0));
-  }
-
-  /**
-   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
-   * correctly groups them according to fixed windows.
-   */
-  public static void groupsElementsIntoFixedWindows(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
-            WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(2),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(13),
-                Arrays.asList(window(10, 20)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
-  }
-
-  /**
-   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
-   * correctly groups them into sliding windows.
-   *
-   * <p>In the input here, each element occurs in multiple windows.
-   */
-  public static void groupsElementsIntoSlidingWindowsWithMinTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(
-        SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
-            WindowedValue.of(
-                "v1",
-                new Instant(5),
-                Arrays.asList(window(-10, 10), window(0, 20)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(15),
-                Arrays.asList(window(0, 20), window(10, 30)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(3));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
-    assertThat(item0.getValue().getValue(), contains("v1"));
-    assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
-    assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
-    assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
-
-    TimestampedValue<KV<String, Iterable<String>>> item2 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
-    assertThat(item2.getValue().getValue(), contains("v2"));
-    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
-    assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
-  }
-
-  /**
-   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
-   * correctly groups and combines them according to sliding windows.
-   *
-   * <p>In the input here, each element occurs in multiple windows.
-   */
-  public static void combinesElementsInSlidingWindows(
-      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
-      CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
-            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
-
-    DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                1L,
-                new Instant(5),
-                Arrays.asList(window(-10, 10), window(0, 20)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                2L,
-                new Instant(15),
-                Arrays.asList(window(0, 20), window(10, 30)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                4L,
-                new Instant(18),
-                Arrays.asList(window(0, 20), window(10, 30)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(3));
-
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
-    assertThat(item0.getValue().getKey(), equalTo("k"));
-    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L))));
-    assertThat(item0.getTimestamp(), equalTo(new Instant(5L)));
-
-    TimestampedValue<KV<String, Long>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
-    assertThat(item1.getValue().getKey(), equalTo("k"));
-    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
-    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
-    assertThat(item1.getTimestamp(), equalTo(new Instant(10L)));
-
-    TimestampedValue<KV<String, Long>> item2 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
-    assertThat(item2.getValue().getKey(), equalTo("k"));
-    assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
-    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
-    assertThat(item2.getTimestamp(), equalTo(new Instant(20L)));
-  }
-
-  /**
-   * Tests that the given GABW implementation correctly groups elements that fall into overlapping
-   * windows that are not merged.
-   */
-  public static void groupsIntoOverlappingNonmergingWindows(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
-            WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 5)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(4),
-                Arrays.asList(window(1, 5)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(4),
-                Arrays.asList(window(0, 5)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3"));
-    assertThat(item0.getTimestamp(), equalTo(window(1, 5).maxTimestamp()));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(1, 5)));
-    assertThat(item1.getValue().getValue(), contains("v2"));
-    assertThat(item1.getTimestamp(), equalTo(window(0, 5).maxTimestamp()));
-  }
-
-  /**
-   * Tests that the given GABW implementation correctly groups elements into merged sessions.
-   */
-  public static void groupsElementsInMergedSessions(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
-            WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(5),
-                Arrays.asList(window(5, 15)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(15),
-                Arrays.asList(window(15, 25)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
-  }
-
-  /**
-   * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
-   * session window correctly according to the provided {@link CombineFn}.
-   */
-  public static void combinesElementsPerSession(
-      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
-      CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
-
-    DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                1L,
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                2L,
-                new Instant(5),
-                Arrays.asList(window(5, 15)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                4L,
-                new Instant(15),
-                Arrays.asList(window(15, 25)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
-    assertThat(item0.getValue().getKey(), equalTo("k"));
-    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
-    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
-    TimestampedValue<KV<String, Long>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
-    assertThat(item1.getValue().getKey(), equalTo("k"));
-    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
-    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
-  }
-
-  /**
-   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
-   * correctly groups them according to fixed windows and also sets the output timestamp
-   * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}.
-   */
-  public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
-            WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(2),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(13),
-                Arrays.asList(window(10, 20)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
-  }
-
-  /**
-   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
-   * correctly groups them according to fixed windows and also sets the output timestamp
-   * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}.
-   */
-  public static void groupsElementsIntoFixedWindowsWithLatestTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(2),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(13),
-                Arrays.asList(window(10, 20)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(new Instant(2)));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(new Instant(13)));
-  }
-
-  /**
-   * Tests that the given GABW implementation correctly groups elements into merged sessions
-   * with output timestamps at the end of the merged window.
-   */
-  public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
-            .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(5),
-                Arrays.asList(window(5, 15)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(15),
-                Arrays.asList(window(15, 25)),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
-  }
-
-  /**
-   * Tests that the given GABW implementation correctly groups elements into merged sessions
-   * with output timestamps at the end of the merged window.
-   */
-  public static void groupsElementsInMergedSessionsWithLatestTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
-            .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
-
-    BoundedWindow unmergedWindow = window(15, 25);
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v2",
-                new Instant(5),
-                Arrays.asList(window(5, 15)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                "v3",
-                new Instant(15),
-                Arrays.asList(unmergedWindow),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    BoundedWindow mergedWindow = window(0, 15);
-    TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow));
-    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
-    assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
-
-    TimestampedValue<KV<String, Iterable<String>>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(unmergedWindow));
-    assertThat(item1.getValue().getValue(), contains("v3"));
-    assertThat(item1.getTimestamp(), equalTo(new Instant(15)));
-  }
-
-  /**
-   * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
-   * session window correctly according to the provided {@link CombineFn}.
-   */
-  public static void combinesElementsPerSessionWithEndOfWindowTimestamp(
-      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
-      CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
-
-    WindowingStrategy<?, IntervalWindow> windowingStrategy =
-        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
-    BoundedWindow secondWindow = window(15, 25);
-    DoFnTester<?, KV<String, Long>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
-            WindowedValue.of(
-                1L,
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                2L,
-                new Instant(5),
-                Arrays.asList(window(5, 15)),
-                PaneInfo.NO_FIRING),
-            WindowedValue.of(
-                4L,
-                new Instant(15),
-                Arrays.asList(secondWindow),
-                PaneInfo.NO_FIRING));
-
-    assertThat(result.peekOutputElements(), hasSize(2));
-
-    BoundedWindow firstResultWindow = window(0, 15);
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow));
-    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
-    assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp()));
-
-    TimestampedValue<KV<String, Long>> item1 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(secondWindow));
-    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
-    assertThat(item1.getTimestamp(),
-        equalTo(secondWindow.maxTimestamp()));
-  }
-
-  @SafeVarargs
-  private static <K, InputT, OutputT, W extends BoundedWindow>
-  DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
-      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
-      WindowingStrategy<?, W> windowingStrategy,
-      K key,
-      WindowedValue<InputT>... values) throws Exception {
-    return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values));
-  }
-
-  private static <K, InputT, OutputT, W extends BoundedWindow>
-  DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
-      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
-      WindowingStrategy<?, W> windowingStrategy,
-      K key,
-      Collection<WindowedValue<InputT>> values) throws Exception {
-
-    final StateInternalsFactory<K> stateInternalsCache = new CachingStateInternalsFactory<K>();
-
-    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> tester =
-        DoFnTester.of(gabwFactory.forStrategy(windowingStrategy, stateInternalsCache));
-
-    // Though we use a DoFnTester, the function itself is instantiated directly by the
-    // runner and should not be serialized; it may not even be serializable.
-    tester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE);
-    tester.startBundle();
-    tester.processElement(KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
-    tester.finishBundle();
-
-    // Sanity check for corruption
-    for (KV<K, OutputT> elem : tester.peekOutputElements()) {
-      assertThat(elem.getKey(), equalTo(key));
-    }
-
-    return tester;
-  }
-
-  private static BoundedWindow window(long start, long end) {
-    return new IntervalWindow(new Instant(start), new Instant(end));
-  }
-
-  private static final class CachingStateInternalsFactory<K> implements StateInternalsFactory<K> {
-    private final LoadingCache<K, StateInternals<K>> stateInternalsCache;
-
-    private CachingStateInternalsFactory() {
-      this.stateInternalsCache = CacheBuilder.newBuilder().build(new StateInternalsLoader<K>());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public StateInternals<K> stateInternalsForKey(K key) {
-      try {
-        return stateInternalsCache.get(key);
-      } catch (Exception exc) {
-        throw new RuntimeException(exc);
-      }
-    }
-  }
-
-  private static class StateInternalsLoader<K> extends CacheLoader<K, StateInternals<K>> {
-    @Override
-    public StateInternals<K> load(K key) throws Exception {
-      return InMemoryStateInternals.forKey(key);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
deleted file mode 100644
index a1586c8..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
+++ /dev/null
@@ -1,109 +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 org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Unit tests for {@link GroupAlsoByWindowsViaOutputBufferDoFn}.
- */
-@RunWith(JUnit4.class)
-public class GroupAlsoByWindowsViaOutputBufferDoFnTest {
-
-  private class BufferingGABWViaOutputBufferDoFnFactory<K, InputT>
-  implements GroupAlsoByWindowsDoFnFactory<K, InputT, Iterable<InputT>> {
-
-    private final Coder<InputT> inputCoder;
-
-    public BufferingGABWViaOutputBufferDoFnFactory(Coder<InputT> inputCoder) {
-      this.inputCoder = inputCoder;
-    }
-
-    @Override
-    public <W extends BoundedWindow>
-        GroupAlsoByWindowsDoFn<K, InputT, Iterable<InputT>, W> forStrategy(
-            WindowingStrategy<?, W> windowingStrategy,
-            StateInternalsFactory<K> stateInternalsFactory) {
-      return new GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, Iterable<InputT>, W>(
-          windowingStrategy,
-          stateInternalsFactory,
-          SystemReduceFn.<K, InputT, W>buffering(inputCoder));
-    }
-  }
-
-  @Test
-  public void testEmptyInputEmptyOutput() throws Exception {
-    GroupAlsoByWindowsProperties.emptyInputEmptyOutput(
-        new BufferingGABWViaOutputBufferDoFnFactory<>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoFixedWindows() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindows(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoSlidingWindows() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsIntoSlidingWindowsWithMinTimestamp(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsIntoOverlappingNonmergingWindows() throws Exception {
-    GroupAlsoByWindowsProperties.groupsIntoOverlappingNonmergingWindows(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsIntoSessions() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsInMergedSessions(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoFixedWindowsWithLatestTimestamp() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoSessionsWithEndOfWindowTimestamp() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-
-  @Test
-  public void testGroupsElementsIntoSessionsWithLatestTimestamp() throws Exception {
-    GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithLatestTimestamp(
-        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
deleted file mode 100644
index c63e43e..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import java.util.Arrays;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter;
-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;
-
-/**
- * Unit tests for {@link LateDataDroppingDoFnRunner}.
- */
-@RunWith(JUnit4.class)
-public class LateDataDroppingDoFnRunnerTest {
-  private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
-
-  @Mock private TimerInternals mockTimerInternals;
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-  }
-
-  @Test
-  public void testLateDataFilter() throws Exception {
-    when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L));
-
-    InMemoryLongSumAggregator droppedDueToLateness =
-        new InMemoryLongSumAggregator("droppedDueToLateness");
-    LateDataFilter lateDataFilter = new LateDataFilter(
-        WindowingStrategy.of(WINDOW_FN), mockTimerInternals, droppedDueToLateness);
-
-    Iterable<WindowedValue<Integer>> actual = lateDataFilter.filter(
-        "a",
-        ImmutableList.of(
-            createDatum(13, 13L),
-            createDatum(5, 5L), // late element, earlier than 4L.
-            createDatum(16, 16L),
-            createDatum(18, 18L)));
-
-    Iterable<WindowedValue<Integer>> expected =  ImmutableList.of(
-        createDatum(13, 13L),
-        createDatum(16, 16L),
-        createDatum(18, 18L));
-    assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class)));
-    assertEquals(1, droppedDueToLateness.sum);
-  }
-
-  private <T> WindowedValue<T> createDatum(T element, long timestampMillis) {
-    Instant timestamp = new Instant(timestampMillis);
-    return WindowedValue.of(
-        element,
-        timestamp,
-        Arrays.asList(WINDOW_FN.assignWindow(timestamp)),
-        PaneInfo.NO_FIRING);
-  }
-
-  private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
-    private final String name;
-    private long sum = 0;
-
-    public InMemoryLongSumAggregator(String name) {
-      this.name = name;
-    }
-
-    @Override
-    public void addValue(Long value) {
-      sum += value;
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public CombineFn<Long, ?, Long> getCombineFn() {
-      return new Sum.SumLongFn();
-    }
-  }
-}


[40/50] [abbrv] incubator-beam git commit: BigQuery: limit max job polling time to 1 minute

Posted by dh...@apache.org.
BigQuery: limit max job polling time to 1 minute

Before the backoff would grow unboundedly, so we could in principle wait
1.5x to 2x the actual job time. For long running jobs this is hours.
Now, we just back off at most 1 minute between checking the job state.
Note there should be no danger of QPS overload here because we should
have very few concurrent outstanding jobs


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

Branch: refs/heads/gearpump-runner
Commit: 958f3fecb0dcfae48b237f1a2f84c5d24c868752
Parents: 5eb44aa
Author: Dan Halperin <dh...@google.com>
Authored: Sun Sep 4 14:54:42 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/958f3fec/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 8b5e8c2..20dadff 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -232,11 +232,12 @@ class BigQueryServicesImpl implements BigQueryServices {
     }
 
     @Override
-    public Job pollJob(JobReference jobRef, int maxAttempts)
-        throws InterruptedException {
+    public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException {
       BackOff backoff =
           FluentBackoff.DEFAULT
-              .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF)
+              .withMaxRetries(maxAttempts)
+              .withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF)
+              .withMaxBackoff(Duration.standardMinutes(1))
               .backoff();
       return pollJob(jobRef, Sleeper.DEFAULT, backoff);
     }


[17/50] [abbrv] incubator-beam git commit: Returned KafkaIO getWatermark log line in debug mode

Posted by dh...@apache.org.
Returned KafkaIO getWatermark log line in debug 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/a58afd3d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/a58afd3d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/a58afd3d

Branch: refs/heads/gearpump-runner
Commit: a58afd3db8cdce4f5135b78f3785c8b3ff7fb0ab
Parents: cf9ce2f
Author: Aviem Zur <av...@gmail.com>
Authored: Sun Aug 28 11:41:58 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a58afd3d/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index f639422..f16a5ab 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -1072,6 +1072,7 @@ public class KafkaIO {
     @Override
     public Instant getWatermark() {
       if (curRecord == null) {
+        LOG.debug("{}: getWatermark() : no records have been read yet.", name);
         return initialWatermark;
       }
 


[13/50] [abbrv] incubator-beam git commit: Fix condition in FlinkStreamingPipelineTranslator

Posted by dh...@apache.org.
Fix condition in FlinkStreamingPipelineTranslator


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

Branch: refs/heads/gearpump-runner
Commit: 1524494f77ca8667d57b00651b39005524c71d64
Parents: 6ae4b6a
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Aug 31 13:42:30 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../flink/translation/FlinkStreamingPipelineTranslator.java        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1524494f/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index b127455..284cd23 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -84,7 +84,7 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
     StreamTransformTranslator<?> translator =
         FlinkStreamingTransformTranslators.getTranslator(transform);
 
-    if (translator == null && applyCanTranslate(transform, node, translator)) {
+    if (translator == null || !applyCanTranslate(transform, node, translator)) {
       LOG.info(node.getTransform().getClass().toString());
       throw new UnsupportedOperationException(
           "The transform " + transform + " is currently not supported.");


[29/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java
new file mode 100644
index 0000000..59a7c92
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.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.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+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;
+
+/**
+ * Tests for {@link PushbackSideInputDoFnRunner}.
+ */
+@RunWith(JUnit4.class)
+public class PushbackSideInputDoFnRunnerTest {
+  @Mock private ReadyCheckingSideInputReader reader;
+  private TestDoFnRunner<Integer, Integer> underlying;
+  private PCollectionView<Integer> singletonView;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
+    singletonView =
+        created
+            .apply(Window.into(new IdentitySideInputWindowFn()))
+            .apply(Sum.integersGlobally().asSingletonView());
+
+    underlying = new TestDoFnRunner<>();
+  }
+
+  private PushbackSideInputDoFnRunner<Integer, Integer> createRunner(
+      ImmutableList<PCollectionView<?>> views) {
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        PushbackSideInputDoFnRunner.create(underlying, views, reader);
+    runner.startBundle();
+    return runner;
+  }
+
+  @Test
+  public void startFinishBundleDelegates() {
+    PushbackSideInputDoFnRunner runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+    assertThat(underlying.started, is(true));
+    assertThat(underlying.finished, is(false));
+    runner.finishBundle();
+    assertThat(underlying.finished, is(true));
+  }
+
+  @Test
+  public void processElementSideInputNotReady() {
+    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+        .thenReturn(false);
+
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+    WindowedValue<Integer> oneWindow =
+        WindowedValue.of(
+            2,
+            new Instant(-2),
+            new IntervalWindow(new Instant(-500L), new Instant(0L)),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    Iterable<WindowedValue<Integer>> oneWindowPushback =
+        runner.processElementInReadyWindows(oneWindow);
+    assertThat(oneWindowPushback, containsInAnyOrder(oneWindow));
+    assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
+  }
+
+  @Test
+  public void processElementSideInputNotReadyMultipleWindows() {
+    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+        .thenReturn(false);
+
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+    WindowedValue<Integer> multiWindow =
+        WindowedValue.of(
+            2,
+            new Instant(-2),
+            ImmutableList.of(
+                new IntervalWindow(new Instant(-500L), new Instant(0L)),
+                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+                GlobalWindow.INSTANCE),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    Iterable<WindowedValue<Integer>> multiWindowPushback =
+        runner.processElementInReadyWindows(multiWindow);
+    assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows()));
+    assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
+  }
+
+  @Test
+  public void processElementSideInputNotReadySomeWindows() {
+    when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE)))
+        .thenReturn(false);
+    when(
+            reader.isReady(
+                Mockito.eq(singletonView),
+                org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE))))
+        .thenReturn(true);
+
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+    IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L));
+    IntervalWindow bigWindow =
+        new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L));
+    WindowedValue<Integer> multiWindow =
+        WindowedValue.of(
+            2,
+            new Instant(-2),
+            ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE),
+            PaneInfo.NO_FIRING);
+    Iterable<WindowedValue<Integer>> multiWindowPushback =
+        runner.processElementInReadyWindows(multiWindow);
+    assertThat(
+        multiWindowPushback,
+        containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L))));
+    assertThat(underlying.inputElems,
+        containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING),
+            WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING)));
+  }
+
+  @Test
+  public void processElementSideInputReadyAllWindows() {
+    when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+        .thenReturn(true);
+
+    ImmutableList<PCollectionView<?>> views = ImmutableList.<PCollectionView<?>>of(singletonView);
+    PushbackSideInputDoFnRunner<Integer, Integer> runner = createRunner(views);
+
+    WindowedValue<Integer> multiWindow =
+        WindowedValue.of(
+            2,
+            new Instant(-2),
+            ImmutableList.of(
+                new IntervalWindow(new Instant(-500L), new Instant(0L)),
+                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+                GlobalWindow.INSTANCE),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    Iterable<WindowedValue<Integer>> multiWindowPushback =
+        runner.processElementInReadyWindows(multiWindow);
+    assertThat(multiWindowPushback, emptyIterable());
+    assertThat(underlying.inputElems,
+        containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray()));
+  }
+
+  @Test
+  public void processElementNoSideInputs() {
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of());
+
+    WindowedValue<Integer> multiWindow =
+        WindowedValue.of(
+            2,
+            new Instant(-2),
+            ImmutableList.of(
+                new IntervalWindow(new Instant(-500L), new Instant(0L)),
+                new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+                GlobalWindow.INSTANCE),
+            PaneInfo.ON_TIME_AND_ONLY_FIRING);
+    Iterable<WindowedValue<Integer>> multiWindowPushback =
+        runner.processElementInReadyWindows(multiWindow);
+    assertThat(multiWindowPushback, emptyIterable());
+    assertThat(underlying.inputElems, containsInAnyOrder(multiWindow));
+  }
+
+  private static class TestDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+    List<WindowedValue<InputT>> inputElems;
+    private boolean started = false;
+    private boolean finished = false;
+
+    @Override
+    public void startBundle() {
+      started = true;
+      inputElems = new ArrayList<>();
+    }
+
+    @Override
+    public void processElement(WindowedValue<InputT> elem) {
+      inputElems.add(elem);
+    }
+
+    @Override
+    public void finishBundle() {
+      finished = true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
new file mode 100644
index 0000000..4d5680c
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
@@ -0,0 +1,1446 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkArgument;
+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;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.withSettings;
+
+import com.google.common.collect.Iterables;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.beam.sdk.WindowMatchers;
+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.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.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;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+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.SideInputReader;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of
+ * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger}
+ * (sometimes mocked). They proceed by injecting elements and advancing watermark and
+ * processing time, then verifying produced panes and counters.
+ */
+@RunWith(JUnit4.class)
+public class ReduceFnRunnerTest {
+  @Mock private SideInputReader mockSideInputReader;
+  private Trigger mockTrigger;
+  private PCollectionView<Integer> mockView;
+
+  private IntervalWindow firstWindow;
+
+  private static Trigger.TriggerContext anyTriggerContext() {
+    return Mockito.<Trigger.TriggerContext>any();
+  }
+  private static Trigger.OnElementContext anyElementContext() {
+    return Mockito.<Trigger.OnElementContext>any();
+  }
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+
+    mockTrigger = mock(Trigger.class, withSettings().serializable());
+
+    @SuppressWarnings("unchecked")
+    PCollectionView<Integer> mockViewUnchecked =
+        mock(PCollectionView.class, withSettings().serializable());
+    mockView = mockViewUnchecked;
+    firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
+  }
+
+  private void injectElement(ReduceFnTester<Integer, ?, IntervalWindow> tester, int element)
+      throws Exception {
+    doNothing().when(mockTrigger).onElement(anyElementContext());
+    tester.injectElements(TimestampedValue.of(element, new Instant(element)));
+  }
+
+  private void triggerShouldFinish(Trigger mockTrigger) throws Exception {
+    doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Exception {
+        @SuppressWarnings("unchecked")
+        Trigger.TriggerContext context =
+            (Trigger.TriggerContext) invocation.getArguments()[0];
+        context.trigger().setFinished(true);
+        return null;
+      }
+    })
+    .when(mockTrigger).onFire(anyTriggerContext());
+ }
+
+  @Test
+  public void testOnElementBufferingDiscarding() throws Exception {
+    // Test basic execution of a trigger using a non-combining window set and discarding mode.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Pane of {1, 2}
+    injectElement(tester, 1);
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 2);
+    assertThat(tester.extractOutput(),
+        contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+    // Pane of just 3, and finish
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 3);
+    assertThat(tester.extractOutput(),
+            contains(isSingleWindowedValue(containsInAnyOrder(3), 3, 0, 10)));
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+    // This element shouldn't be seen, because the trigger has finished
+    injectElement(tester, 4);
+
+    assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
+  }
+
+  @Test
+  public void testOnElementBufferingAccumulating() throws Exception {
+    // Test basic execution of a trigger using a non-combining window set and accumulating mode.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    injectElement(tester, 1);
+
+    // Fires {1, 2}
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 2);
+
+    // Fires {1, 2, 3} because we are in accumulating mode
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 3);
+
+    // This element shouldn't be seen, because the trigger has finished
+    injectElement(tester, 4);
+
+    assertThat(
+        tester.extractOutput(),
+        contains(
+            isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+            isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 3, 0, 10)));
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+  }
+
+  @Test
+  public void testOnElementCombiningDiscarding() throws Exception {
+    // Test basic execution of a trigger using a non-combining window set and discarding mode.
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+        FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.DISCARDING_FIRED_PANES,
+        new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of(), Duration.millis(100));
+
+    injectElement(tester, 2);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 3);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 4);
+
+    // This element shouldn't be seen, because the trigger has finished
+    injectElement(tester, 6);
+
+    assertThat(
+        tester.extractOutput(),
+        contains(
+            isSingleWindowedValue(equalTo(5), 2, 0, 10),
+            isSingleWindowedValue(equalTo(4), 4, 0, 10)));
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    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 BoundedWindow window() {
+                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()),
+            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.
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+            VarIntCoder.of(), Duration.millis(100));
+
+    injectElement(tester, 1);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 2);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 3);
+
+    // This element shouldn't be seen, because the trigger has finished
+    injectElement(tester, 4);
+
+    assertThat(
+        tester.extractOutput(),
+        contains(
+            isSingleWindowedValue(equalTo(3), 1, 0, 10),
+            isSingleWindowedValue(equalTo(6), 3, 0, 10)));
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+  }
+
+  @Test
+  public void testOnElementCombiningWithContext() throws Exception {
+    Integer expectedValue = 5;
+    WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy
+        .of(FixedWindows.of(Duration.millis(10)))
+        .withTrigger(mockTrigger)
+        .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+        .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+        .withAllowedLateness(Duration.millis(100));
+
+    TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
+    options.setValue(5);
+
+    when(mockSideInputReader.contains(Matchers.<PCollectionView<Integer>>any())).thenReturn(true);
+    when(mockSideInputReader.get(
+        Matchers.<PCollectionView<Integer>>any(), any(BoundedWindow.class))).thenReturn(5);
+
+    @SuppressWarnings({"rawtypes", "unchecked", "unused"})
+    Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal())
+        .thenReturn((WindowingStrategy) windowingStrategy);
+
+    SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue);
+    // Test basic execution of a trigger using a non-combining window set and discarding mode.
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+        windowingStrategy, combineFn.<String>asKeyedFn(),
+        VarIntCoder.of(), options, mockSideInputReader);
+
+    injectElement(tester, 2);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 3);
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 4);
+
+    // This element shouldn't be seen, because the trigger has finished
+    injectElement(tester, 6);
+
+    assertThat(
+        tester.extractOutput(),
+        contains(
+            isSingleWindowedValue(equalTo(5), 2, 0, 10),
+            isSingleWindowedValue(equalTo(4), 4, 0, 10)));
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+  }
+
+  @Test
+  public void testWatermarkHoldAndLateData() throws Exception {
+    // Test handling of late data. Specifically, ensure the watermark hold is correct.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Input watermark -> null
+    assertEquals(null, tester.getWatermarkHold());
+    assertEquals(null, tester.getOutputWatermark());
+
+    // All on time data, verify watermark hold.
+    injectElement(tester, 1);
+    injectElement(tester, 3);
+    assertEquals(new Instant(1), tester.getWatermarkHold());
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 2);
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output, contains(
+        isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
+            1, // timestamp
+            0, // window start
+            10))); // window end
+    assertThat(output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+
+    // Holding for the end-of-window transition.
+    assertEquals(new Instant(9), tester.getWatermarkHold());
+    // Nothing dropped.
+    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+    // Input watermark -> 4, output watermark should advance that far as well
+    tester.advanceInputWatermark(new Instant(4));
+    assertEquals(new Instant(4), tester.getOutputWatermark());
+
+    // Some late, some on time. Verify that we only hold to the minimum of on-time.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    tester.advanceInputWatermark(new Instant(4));
+    injectElement(tester, 2);
+    injectElement(tester, 3);
+    assertEquals(new Instant(9), tester.getWatermarkHold());
+    injectElement(tester, 5);
+    assertEquals(new Instant(5), tester.getWatermarkHold());
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 4);
+    output = tester.extractOutput();
+    assertThat(output,
+        contains(
+            isSingleWindowedValue(containsInAnyOrder(
+                1, 2, 3, // earlier firing
+                2, 3, 4, 5), // new elements
+            4, // timestamp
+            0, // window start
+            10))); // window end
+    assertThat(output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)));
+
+    // All late -- output at end of window timestamp.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    tester.advanceInputWatermark(new Instant(8));
+    injectElement(tester, 6);
+    injectElement(tester, 5);
+    assertEquals(new Instant(9), tester.getWatermarkHold());
+    injectElement(tester, 4);
+
+    // Fire the ON_TIME pane
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.advanceInputWatermark(new Instant(10));
+
+    // Output time is end of the window, because all the new data was late, but the pane
+    // is the ON_TIME pane.
+    output = tester.extractOutput();
+    assertThat(output,
+        contains(isSingleWindowedValue(
+            containsInAnyOrder(1, 2, 3, // earlier firing
+                2, 3, 4, 5, // earlier firing
+                4, 5, 6), // new elements
+            9, // timestamp
+            0, // window start
+            10))); // window end
+    assertThat(output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0)));
+
+    // This is "pending" at the time the watermark makes it way-late.
+    // Because we're about to expire the window, we output it.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    injectElement(tester, 8);
+    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+    // Exceed the GC limit, triggering the last pane to be fired
+    tester.advanceInputWatermark(new Instant(50));
+    output = tester.extractOutput();
+    // Output time is still end of the window, because the new data (8) was behind
+    // the output watermark.
+    assertThat(output,
+        contains(isSingleWindowedValue(
+            containsInAnyOrder(1, 2, 3, // earlier firing
+                2, 3, 4, 5, // earlier firing
+                4, 5, 6, // earlier firing
+                8), // new element prior to window becoming expired
+            9, // timestamp
+            0, // window start
+            10))); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1)));
+    assertEquals(new Instant(50), tester.getOutputWatermark());
+    assertEquals(null, tester.getWatermarkHold());
+
+    // Late timers are ignored
+    tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), new Instant(12),
+        TimeDomain.EVENT_TIME);
+
+    // And because we're past the end of window + allowed lateness, everything should be cleaned up.
+    assertFalse(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor();
+  }
+
+  @Test
+  public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception {
+    // Make sure holds are only set if they are accompanied by an end-of-window timer.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
+            ClosingBehavior.FIRE_ALWAYS);
+    tester.setAutoAdvanceOutputWatermark(false);
+
+    // Case: Unobservably late
+    tester.advanceInputWatermark(new Instant(15));
+    tester.advanceOutputWatermark(new Instant(11));
+    injectElement(tester, 14);
+    // Hold was applied, waiting for end-of-window timer.
+    assertEquals(new Instant(14), tester.getWatermarkHold());
+    assertEquals(new Instant(19), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+    // Trigger the end-of-window timer.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.advanceInputWatermark(new Instant(20));
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    // Hold has been replaced with garbage collection hold. Waiting for garbage collection.
+    assertEquals(new Instant(29), tester.getWatermarkHold());
+    assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+    // Case: Maybe late 1
+    injectElement(tester, 13);
+    // No change to hold or timers.
+    assertEquals(new Instant(29), tester.getWatermarkHold());
+    assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+    // Trigger the garbage collection timer.
+    tester.advanceInputWatermark(new Instant(30));
+
+    // Everything should be cleaned up.
+    assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20))));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor();
+  }
+
+  @Test
+  public void testPaneInfoAllStates() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    tester.advanceInputWatermark(new Instant(0));
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 1);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY))));
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 2);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))));
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    tester.advanceInputWatermark(new Instant(15));
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 3);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))));
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    injectElement(tester, 4);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(false, false, Timing.LATE, 3, 1))));
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 5);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 4, 2))));
+  }
+
+  @Test
+  public void testPaneInfoAllStatesAfterWatermark() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+            .withTrigger(Repeatedly.forever(AfterFirst.of(
+                AfterPane.elementCountAtLeast(2),
+                AfterWatermark.pastEndOfWindow())))
+            .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .withAllowedLateness(Duration.millis(100))
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+    tester.advanceInputWatermark(new Instant(50));
+
+    // We should get the ON_TIME pane even though it is empty,
+    // because we have an AfterWatermark.pastEndOfWindow() trigger.
+    output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
+
+    // We should get the final pane even though it is empty.
+    tester.advanceInputWatermark(new Instant(150));
+    output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
+  }
+
+  @Test
+  public void noEmptyPanesFinalIfNonEmpty() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+            .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
+                AfterPane.elementCountAtLeast(2),
+                AfterWatermark.pastEndOfWindow())))
+            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+            .withAllowedLateness(Duration.millis(100))
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)),
+        TimestampedValue.of(2, new Instant(2)));
+    tester.advanceInputWatermark(new Instant(20));
+    tester.advanceInputWatermark(new Instant(250));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output, contains(
+        // Trigger with 2 elements
+        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+        // Trigger for the empty on time pane
+        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+  }
+
+  @Test
+  public void noEmptyPanesFinalAlways() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+            .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
+                AfterPane.elementCountAtLeast(2),
+                AfterWatermark.pastEndOfWindow())))
+            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+            .withAllowedLateness(Duration.millis(100))
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)),
+        TimestampedValue.of(2, new Instant(2)));
+    tester.advanceInputWatermark(new Instant(20));
+    tester.advanceInputWatermark(new Instant(250));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output, contains(
+        // Trigger with 2 elements
+        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+        // Trigger for the empty on time pane
+        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10),
+        // Trigger for the final pane
+        WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+  }
+
+  @Test
+  public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+            .withTrigger(Repeatedly.forever(AfterFirst.of(
+                AfterPane.elementCountAtLeast(2),
+                AfterWatermark.pastEndOfWindow())))
+            .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+            .withAllowedLateness(Duration.millis(100))
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+    tester.advanceInputWatermark(new Instant(50));
+
+    // We should get the ON_TIME pane even though it is empty,
+    // because we have an AfterWatermark.pastEndOfWindow() trigger.
+    output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+
+    // We should get the final pane even though it is empty.
+    tester.advanceInputWatermark(new Instant(150));
+    output = tester.extractOutput();
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
+    assertThat(
+        output,
+        contains(
+            WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+  }
+
+  @Test
+  public void testPaneInfoFinalAndOnTime() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+            .withTrigger(
+                Repeatedly.forever(AfterPane.elementCountAtLeast(2))
+                    .orFinally(AfterWatermark.pastEndOfWindow()))
+            .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .withAllowedLateness(Duration.millis(100))
+            .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+    tester.advanceInputWatermark(new Instant(0));
+
+    // Should trigger due to element count
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+    assertThat(
+        tester.extractOutput(),
+        contains(WindowMatchers.valueWithPaneInfo(
+            PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+
+    tester.advanceInputWatermark(new Instant(150));
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0))));
+  }
+
+  @Test
+  public void testPaneInfoSkipToFinish() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    tester.advanceInputWatermark(new Instant(0));
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 1);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY))));
+  }
+
+  @Test
+  public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    tester.advanceInputWatermark(new Instant(15));
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 1);
+    assertThat(tester.extractOutput(), contains(
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE))));
+  }
+
+  @Test
+  public void testMergeBeforeFinalizing() throws Exception {
+    // Verify that we merge windows before producing output so users don't see undesired
+    // unmerged windows.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(0),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // All on time data, verify watermark hold.
+    // These two windows should pre-merge immediately to [1, 20)
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)), // in [1, 11)
+        TimestampedValue.of(10, new Instant(10))); // in [10, 20)
+
+    // And this should fire the end-of-window timer
+    tester.advanceInputWatermark(new Instant(100));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(1));
+    assertThat(output.get(0),
+        isSingleWindowedValue(containsInAnyOrder(1, 10),
+            1, // timestamp
+            1, // window start
+            20)); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+  }
+
+  /**
+   * It is possible for a session window's trigger to be closed at the point at which
+   * the (merged) session window is garbage collected. Make sure we don't accidentally
+   * assume the window is still active.
+   */
+  @Test
+  public void testMergingWithCloseBeforeGC() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Two elements in two overlapping session windows.
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)), // in [1, 11)
+        TimestampedValue.of(10, new Instant(10))); // in [10, 20)
+
+    // Close the trigger, but the gargbage collection timer is still pending.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    tester.advanceInputWatermark(new Instant(30));
+
+    // Now the garbage collection timer will fire, finding the trigger already closed.
+    tester.advanceInputWatermark(new Instant(100));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(1));
+    assertThat(output.get(0),
+        isSingleWindowedValue(containsInAnyOrder(1, 10),
+            1, // timestamp
+            1, // window start
+            20)); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+  }
+
+  /**
+   * Ensure a closed trigger has its state recorded in the merge result window.
+   */
+  @Test
+  public void testMergingWithCloseTrigger() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Create a new merged session window.
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+                          TimestampedValue.of(2, new Instant(2)));
+
+    // Force the trigger to be closed for the merged window.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    tester.advanceInputWatermark(new Instant(13));
+
+    // Trigger is now closed.
+    assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
+
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+
+    // Revisit the same session window.
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+                          TimestampedValue.of(2, new Instant(2)));
+
+    // Trigger is still closed.
+    assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
+  }
+
+  /**
+   * If a later event tries to reuse an earlier session window which has been closed, we
+   * should reject that element and not fail due to the window no longer being active.
+   */
+  @Test
+  public void testMergingWithReusedWindow() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // One elements in one session window.
+    tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
+
+    // Close the trigger, but the gargbage collection timer is still pending.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    tester.advanceInputWatermark(new Instant(15));
+
+    // Another element in the same session window.
+    // Should be discarded with 'window closed'.
+    tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
+
+    // And nothing should be left in the active window state.
+    assertTrue(tester.hasNoActiveWindows());
+
+    // Now the garbage collection timer will fire, finding the trigger already closed.
+    tester.advanceInputWatermark(new Instant(100));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(1));
+    assertThat(output.get(0),
+               isSingleWindowedValue(containsInAnyOrder(1),
+                                     1, // timestamp
+                                     1, // window start
+                                     11)); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+  }
+
+  /**
+   * When a merged window's trigger is closed we record that state using the merged window rather
+   * than the original windows.
+   */
+  @Test
+  public void testMergingWithClosedRepresentative() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+                                    AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+                                    ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // 2 elements into merged session window.
+    // Close the trigger, but the garbage collection timer is still pending.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),       // in [1, 11), gc at 21.
+                          TimestampedValue.of(8, new Instant(8)));      // in [8, 18), gc at 28.
+
+    // More elements into the same merged session window.
+    // It has not yet been gced.
+    // Should be discarded with 'window closed'.
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),      // in [1, 11), gc at 21.
+                          TimestampedValue.of(2, new Instant(2)),      // in [2, 12), gc at 22.
+                          TimestampedValue.of(8, new Instant(8)));     // in [8, 18), gc at 28.
+
+    // Now the garbage collection timer will fire, finding the trigger already closed.
+    tester.advanceInputWatermark(new Instant(100));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+
+    assertThat(output.size(), equalTo(1));
+    assertThat(output.get(0),
+               isSingleWindowedValue(containsInAnyOrder(1, 8),
+                                     1, // timestamp
+                                     1, // window start
+                                     18)); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
+  }
+
+  /**
+   * If an element for a closed session window ends up being merged into other still-open
+   * session windows, the resulting session window is not 'poisoned'.
+   */
+  @Test
+  public void testMergingWithClosedDoesNotPoison() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // 1 element, force its trigger to close.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    tester.injectElements(TimestampedValue.of(2, new Instant(2)));
+
+    // 3 elements, one already closed.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+        TimestampedValue.of(2, new Instant(2)),
+        TimestampedValue.of(3, new Instant(3)));
+
+    tester.advanceInputWatermark(new Instant(100));
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(2));
+    assertThat(output.get(0),
+        isSingleWindowedValue(containsInAnyOrder(2),
+            2, // timestamp
+            2, // window start
+            12)); // window end
+    assertThat(
+        output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
+    assertThat(output.get(1),
+        isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
+            1, // timestamp
+            1, // window start
+            13)); // window end
+    assertThat(
+        output.get(1).getPane(),
+        equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+  }
+
+  /**
+   * Tests that when data is assigned to multiple windows but some of those windows have
+   * had their triggers finish, then the data is dropped and counted accurately.
+   */
+  @Test
+  public void testDropDataMultipleWindowsFinishedTrigger() throws Exception {
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+        WindowingStrategy.of(
+            SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
+        .withTrigger(AfterWatermark.pastEndOfWindow())
+        .withAllowedLateness(Duration.millis(1000)),
+        new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of());
+
+    tester.injectElements(
+        // assigned to [-60, 40), [-30, 70), [0, 100)
+        TimestampedValue.of(10, new Instant(23)),
+        // assigned to [-30, 70), [0, 100), [30, 130)
+        TimestampedValue.of(12, new Instant(40)));
+
+    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+    tester.advanceInputWatermark(new Instant(70));
+    tester.injectElements(
+        // assigned to [-30, 70), [0, 100), [30, 130)
+        // but [-30, 70) is closed by the trigger
+        TimestampedValue.of(14, new Instant(60)));
+
+    assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
+
+    tester.advanceInputWatermark(new Instant(130));
+    // assigned to [-30, 70), [0, 100), [30, 130)
+    // but they are all closed
+    tester.injectElements(TimestampedValue.of(16, new Instant(40)));
+
+    assertEquals(4, tester.getElementsDroppedDueToClosedWindow());
+  }
+
+  @Test
+  public void testIdempotentEmptyPanesDiscarding() throws Exception {
+    // Test uninteresting (empty) panes don't increment the index or otherwise
+    // modify PaneInfo.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Inject a couple of on-time elements and fire at the window end.
+    injectElement(tester, 1);
+    injectElement(tester, 2);
+    tester.advanceInputWatermark(new Instant(12));
+
+    // Fire the on-time pane
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+
+    // Fire another timer (with no data, so it's an uninteresting pane that should not be output).
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+
+    // Finish it off with another datum.
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 3);
+
+    // The intermediate trigger firing shouldn't result in any output.
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(2));
+
+    // The on-time pane is as expected.
+    assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
+
+    // The late pane has the correct indices.
+    assertThat(output.get(1).getValue(), contains(3));
+    assertThat(
+        output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
+
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+  }
+
+  @Test
+  public void testIdempotentEmptyPanesAccumulating() throws Exception {
+    // Test uninteresting (empty) panes don't increment the index or otherwise
+    // modify PaneInfo.
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+            AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    // Inject a couple of on-time elements and fire at the window end.
+    injectElement(tester, 1);
+    injectElement(tester, 2);
+    tester.advanceInputWatermark(new Instant(12));
+
+    // Trigger the on-time pane
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertThat(output.size(), equalTo(1));
+    assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
+    assertThat(output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0)));
+
+    // Fire another timer with no data; the empty pane should not be output even though the
+    // trigger is ready to fire
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+    assertThat(tester.extractOutput().size(), equalTo(0));
+
+    // Finish it off with another datum, which is late
+    when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+    triggerShouldFinish(mockTrigger);
+    injectElement(tester, 3);
+    output = tester.extractOutput();
+    assertThat(output.size(), equalTo(1));
+
+    // The late pane has the correct indices.
+    assertThat(output.get(0).getValue(), containsInAnyOrder(1, 2, 3));
+    assertThat(output.get(0).getPane(),
+        equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
+
+    assertTrue(tester.isMarkedFinished(firstWindow));
+    tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+  }
+
+  /**
+   * Test that we receive an empty on-time pane when an or-finally waiting for the watermark fires.
+   * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
+   * when the on-time pane is empty.
+   */
+  @Test
+  public void testEmptyOnTimeFromOrFinally() throws Exception {
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
+            AfterEach.<IntervalWindow>inOrder(
+                Repeatedly
+                    .forever(
+                        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+                            new Duration(5)))
+                    .orFinally(AfterWatermark.pastEndOfWindow()),
+                Repeatedly.forever(
+                    AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+                        new Duration(25)))),
+            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+            VarIntCoder.of(), Duration.millis(100));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.advanceProcessingTime(new Instant(0));
+
+    // Processing time timer for 5
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(1)),
+        TimestampedValue.of(1, new Instant(3)),
+        TimestampedValue.of(1, new Instant(7)),
+        TimestampedValue.of(1, new Instant(5)));
+
+    // Should fire early pane
+    tester.advanceProcessingTime(new Instant(6));
+
+    // Should fire empty on time pane
+    tester.advanceInputWatermark(new Instant(11));
+    List<WindowedValue<Integer>> output = tester.extractOutput();
+    assertEquals(2, output.size());
+
+    assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
+    assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(4, 9, 0, 10));
+
+    assertThat(
+        output.get(0),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+    assertThat(
+        output.get(1),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
+  }
+
+  /**
+   * Tests for processing time firings after the watermark passes the end of the window.
+   * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
+   * when the on-time pane is non-empty.
+   */
+  @Test
+  public void testProcessingTime() throws Exception {
+    ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+        ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
+            AfterEach.<IntervalWindow>inOrder(
+                Repeatedly
+                    .forever(
+                        AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+                            new Duration(5)))
+                    .orFinally(AfterWatermark.pastEndOfWindow()),
+                Repeatedly.forever(
+                    AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+                        new Duration(25)))),
+            AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+            VarIntCoder.of(), Duration.millis(100));
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.advanceProcessingTime(new Instant(0));
+
+    tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+        TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)),
+        TimestampedValue.of(1, new Instant(5)));
+    // 4 elements all at processing time 0
+
+    tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(8)),
+        TimestampedValue.of(1, new Instant(4)));
+    // 6 elements
+
+    tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(8)),
+        TimestampedValue.of(1, new Instant(4)),
+        TimestampedValue.of(1, new Instant(5)));
+    // 9 elements
+
+    tester.advanceInputWatermark(new Instant(12));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(3)));
+    // 10 elements
+
+    tester.advanceProcessingTime(new Instant(15));
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(5)));
+    // 11 elements
+    tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25
+
+    tester.injectElements(
+        TimestampedValue.of(1, new Instant(3)));
+    // 12 elements
+    // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25
+    tester.advanceInputWatermark(new Instant(125));
+
+    List<WindowedValue<Integer>> output = tester.extractOutput();
+    assertEquals(4, output.size());
+
+    assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
+    assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(6, 4, 0, 10));
+    assertThat(output.get(2), WindowMatchers.isSingleWindowedValue(11, 9, 0, 10));
+    assertThat(output.get(3), WindowMatchers.isSingleWindowedValue(12, 9, 0, 10));
+
+    assertThat(
+        output.get(0),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+    assertThat(
+        output.get(1),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
+    assertThat(
+        output.get(2),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 1)));
+    assertThat(
+        output.get(3),
+        WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 3, 2)));
+  }
+
+  /**
+   * We should fire a non-empty ON_TIME pane in the GlobalWindow when the watermark moves to
+   * end-of-time.
+   */
+  @Test
+  public void fireNonEmptyOnDrainInGlobalWindow() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
+        ReduceFnTester.nonCombining(
+            WindowingStrategy.of(new GlobalWindows())
+                             .withTrigger(Repeatedly.<GlobalWindow>forever(
+                                 AfterPane.elementCountAtLeast(3)))
+                             .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
+
+    tester.advanceInputWatermark(new Instant(0));
+
+    final int n = 20;
+    for (int i = 0; i < n; i++) {
+      tester.injectElements(TimestampedValue.of(i, new Instant(i)));
+    }
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertEquals(n / 3, output.size());
+    for (int i = 0; i < output.size(); i++) {
+      assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
+      assertEquals(i, output.get(i).getPane().getIndex());
+      assertEquals(3, Iterables.size(output.get(i).getValue()));
+    }
+
+    tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    output = tester.extractOutput();
+    assertEquals(1, output.size());
+    assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
+    assertEquals(n / 3, output.get(0).getPane().getIndex());
+    assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue()));
+  }
+
+  /**
+   * We should fire an empty ON_TIME pane in the GlobalWindow when the watermark moves to
+   * end-of-time.
+   */
+  @Test
+  public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
+        ReduceFnTester.nonCombining(
+            WindowingStrategy.of(new GlobalWindows())
+                             .withTrigger(Repeatedly.<GlobalWindow>forever(
+                                 AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+                                     new Duration(3))))
+                             .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
+
+    final int n = 20;
+    for (int i = 0; i < n; i++) {
+      tester.advanceProcessingTime(new Instant(i));
+      tester.injectElements(TimestampedValue.of(i, new Instant(i)));
+    }
+    tester.advanceProcessingTime(new Instant(n + 4));
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertEquals((n + 3) / 4, output.size());
+    for (int i = 0; i < output.size(); i++) {
+      assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
+      assertEquals(i, output.get(i).getPane().getIndex());
+      assertEquals(4, Iterables.size(output.get(i).getValue()));
+    }
+
+    tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    output = tester.extractOutput();
+    assertEquals(1, output.size());
+    assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
+    assertEquals((n + 3) / 4, output.get(0).getPane().getIndex());
+    assertEquals(0, Iterables.size(output.get(0).getValue()));
+  }
+
+  /**
+   * Late elements should still have a garbage collection hold set so that they
+   * can make a late pane rather than be dropped due to lateness.
+   */
+  @Test
+  public void setGarbageCollectionHoldOnLateElements() throws Exception {
+    ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+        ReduceFnTester.nonCombining(
+            FixedWindows.of(Duration.millis(10)),
+            AfterWatermark.pastEndOfWindow().withLateFirings(AfterPane.elementCountAtLeast(2)),
+            AccumulationMode.DISCARDING_FIRED_PANES,
+            Duration.millis(100),
+            ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+    tester.advanceInputWatermark(new Instant(0));
+    tester.advanceOutputWatermark(new Instant(0));
+    tester.injectElements(TimestampedValue.of(1,  new Instant(1)));
+
+    // Fire ON_TIME pane @ 9 with 1
+
+    tester.advanceInputWatermark(new Instant(109));
+    tester.advanceOutputWatermark(new Instant(109));
+    tester.injectElements(TimestampedValue.of(2,  new Instant(2)));
+    // We should have set a garbage collection hold for the final pane.
+    Instant hold = tester.getWatermarkHold();
+    assertEquals(new Instant(109), hold);
+
+    tester.advanceInputWatermark(new Instant(110));
+    tester.advanceOutputWatermark(new Instant(110));
+
+    // Fire final LATE pane @ 9 with 2
+
+    List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+    assertEquals(2, output.size());
+  }
+
+  private static class SumAndVerifyContextFn extends CombineFnWithContext<Integer, int[], Integer> {
+
+    private final PCollectionView<Integer> view;
+    private final int expectedValue;
+
+    private SumAndVerifyContextFn(PCollectionView<Integer> view, int expectedValue) {
+      this.view = view;
+      this.expectedValue = expectedValue;
+    }
+    @Override
+    public int[] createAccumulator(Context c) {
+      checkArgument(
+          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
+      return wrap(0);
+    }
+
+    @Override
+    public int[] addInput(int[] accumulator, Integer input, Context c) {
+      checkArgument(
+          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
+      accumulator[0] += input.intValue();
+      return accumulator;
+    }
+
+    @Override
+    public int[] mergeAccumulators(Iterable<int[]> accumulators, Context c) {
+      checkArgument(
+          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
+      Iterator<int[]> iter = accumulators.iterator();
+      if (!iter.hasNext()) {
+        return createAccumulator(c);
+      } else {
+        int[] running = iter.next();
+        while (iter.hasNext()) {
+          running[0] += iter.next()[0];
+        }
+        return running;
+      }
+    }
+
+    @Override
+    public Integer extractOutput(int[] accumulator, Context c) {
+      checkArgument(
+          c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
+      return accumulator[0];
+    }
+
+    private int[] wrap(int value) {
+      return new int[] { value };
+    }
+  }
+
+  /**
+   * A {@link PipelineOptions} to test combining with context.
+   */
+  public interface TestOptions extends PipelineOptions {
+    Integer getValue();
+    void setValue(Integer value);
+  }
+}


[12/50] [abbrv] incubator-beam git commit: Correct some accidental renames

Posted by dh...@apache.org.
Correct some accidental renames

IDE over-eagerly replaced some occurrences of createAggregator with
createAggregatorForDoFn. This corrects that.


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

Branch: refs/heads/gearpump-runner
Commit: 798566c3e0f9bbd535dc9336206ee3a8053911a8
Parents: 1524494
Author: bchambers <bc...@google.com>
Authored: Wed Aug 3 13:38:43 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java   | 2 +-
 .../main/java/org/apache/beam/sdk/transforms/Aggregator.java | 8 ++++----
 .../main/java/org/apache/beam/sdk/transforms/DoFnTester.java | 2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/798566c3/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
index 04a0978..f0cfd74 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -344,7 +344,7 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      checkNotNull(combiner, "Combiner passed to createAggregatorForDoFn cannot be null");
+      checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null");
       return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/798566c3/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 67d399f..e8f6247 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
@@ -25,7 +25,7 @@ import org.apache.beam.sdk.util.ExecutionContext;
  * to be combined across all bundles.
  *
  * <p>Aggregators are created by calling
- * {@link DoFn#createAggregator DoFn.createAggregatorForDoFn},
+ * {@link DoFn#createAggregator DoFn.createAggregator},
  * typically from the {@link DoFn} constructor. Elements can be added to the
  * {@code Aggregator} by calling {@link Aggregator#addValue}.
  *
@@ -41,7 +41,7 @@ import org.apache.beam.sdk.util.ExecutionContext;
  *   private Aggregator<Integer, Integer> myAggregator;
  *
  *   public MyDoFn() {
- *     myAggregator = createAggregatorForDoFn("myAggregator", new Sum.SumIntegerFn());
+ *     myAggregator = createAggregator("myAggregator", new Sum.SumIntegerFn());
  *   }
  *
  *   @ProcessElement
@@ -89,9 +89,9 @@ public interface Aggregator<InputT, OutputT> {
   }
 
   // TODO: Consider the following additional API conveniences:
-  // - In addition to createAggregatorForDoFn(), consider adding getAggregator() to
+  // - In addition to createAggregator(), consider adding getAggregator() to
   //   avoid the need to store the aggregator locally in a DoFn, i.e., create
   //   if not already present.
   // - Add a shortcut for the most common aggregator:
-  //   c.createAggregatorForDoFn("name", new Sum.SumIntegerFn()).
+  //   c.createAggregator("name", new Sum.SumIntegerFn()).
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/798566c3/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 6801768..b867a55 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
@@ -667,7 +667,7 @@ public class DoFnTester<InputT, OutputT> {
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
       throw new IllegalStateException("Aggregators should not be created within ProcessContext. "
           + "Instead, create an aggregator at OldDoFn construction time with"
-          + " createAggregatorForDoFn, and ensure they are set up by the time startBundle is"
+          + " createAggregator, and ensure they are set up by the time startBundle is"
           + " called with setupDelegateAggregators.");
     }
   }


[11/50] [abbrv] incubator-beam git commit: [BEAM-569] Define maxNumRecords default value to Long.MAX_VALUE in JmsIO

Posted by dh...@apache.org.
[BEAM-569] Define maxNumRecords default value to Long.MAX_VALUE in JmsIO


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

Branch: refs/heads/gearpump-runner
Commit: 8007bdf5c3e1d3ee0c2c697e049549e9742442aa
Parents: 00b4e95
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Sat Aug 27 14:01:34 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java  | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8007bdf5/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
index 29d0c5f..3107aab 100644
--- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
+++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
@@ -101,11 +101,11 @@ public class JmsIO {
   private static final Logger LOG = LoggerFactory.getLogger(JmsIO.class);
 
   public static Read read() {
-    return new Read();
+    return new Read(null, null, null, Long.MAX_VALUE, null);
   }
 
   public static Write write() {
-    return new Write();
+    return new Write(null, null, null);
   }
 
   /**
@@ -185,8 +185,6 @@ public class JmsIO {
     protected long maxNumRecords;
     protected Duration maxReadTime;
 
-    private Read() {}
-
     private Read(
         ConnectionFactory connectionFactory,
         String queue,
@@ -428,8 +426,6 @@ public class JmsIO {
       return new Write(connectionFactory, queue, topic);
     }
 
-    private Write() {}
-
     private Write(ConnectionFactory connectionFactory, String queue, String topic) {
       this.connectionFactory = connectionFactory;
       this.queue = queue;


[20/50] [abbrv] incubator-beam git commit: DatastoreIO SplitQueryFn integration test

Posted by dh...@apache.org.
DatastoreIO SplitQueryFn integration test


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

Branch: refs/heads/gearpump-runner
Commit: 0312f15e95898f2fbd4dd4a4accfa9529f5efeee
Parents: 6ee7b62
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Mon Aug 29 13:55:32 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  9 +-
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   |  6 +-
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    | 97 ++++++++++++++++++++
 3 files changed, 107 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0312f15e/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index 8456e02..e24bc80 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -24,6 +24,7 @@ import static com.google.common.base.Verify.verify;
 import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL;
 import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING;
 import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED;
+import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter;
 import static com.google.datastore.v1.client.DatastoreHelper.makeDelete;
 import static com.google.datastore.v1.client.DatastoreHelper.makeFilter;
 import static com.google.datastore.v1.client.DatastoreHelper.makeOrder;
@@ -290,7 +291,7 @@ public class DatastoreV1 {
         throws DatastoreException {
       String ourKind = query.getKind(0).getName();
       long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace);
-      LOG.info("Latest stats timestamp : {}", latestTimestamp);
+      LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp);
 
       Query.Builder queryBuilder = Query.newBuilder();
       if (namespace == null) {
@@ -298,8 +299,10 @@ public class DatastoreV1 {
       } else {
         queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__");
       }
-      queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build()));
-      queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()));
+
+      queryBuilder.setFilter(makeAndFilter(
+          makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(),
+          makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build()));
 
       RunQueryRequest request = makeRequest(queryBuilder.build(), namespace);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0312f15e/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
index 138671d..d96c320 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.io.gcp.datastore;
 
 import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL;
 import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING;
+import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter;
 import static com.google.datastore.v1.client.DatastoreHelper.makeDelete;
 import static com.google.datastore.v1.client.DatastoreHelper.makeFilter;
 import static com.google.datastore.v1.client.DatastoreHelper.makeKey;
@@ -805,8 +806,9 @@ public class DatastoreV1Test {
     } else {
       statQuery.addKindBuilder().setName("__Stat_Ns_Kind__");
     }
-    statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build());
-    statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build());
+    statQuery.setFilter(makeAndFilter(
+        makeFilter("kind_name", EQUAL, makeValue(KIND).build()).build(),
+        makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L).build()).build()));
     return statQuery.build();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0312f15e/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
new file mode 100644
index 0000000..72ab7c2
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.gcp.datastore;
+
+import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.NUM_QUERY_SPLITS_MIN;
+import static org.junit.Assert.assertEquals;
+
+import com.google.datastore.v1.Query;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn;
+import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Integration tests for {@link DatastoreV1.Read.SplitQueryFn}.
+ *
+ * <p> It is hard to mock the exact behavior of Cloud Datastore, especially for the statistics
+ * queries. Also the fact that DatastoreIO falls back gracefully when querying statistics fails,
+ * makes it hard to catch these issues in production. This test here ensures we interact with
+ * the Cloud Datastore directly, query the actual stats and verify that the SplitQueryFn generates
+ * the expected number of query splits.
+ *
+ * <p> These tests are brittle as they rely on statistics data in Cloud Datastore. If the data
+ * gets lost or changes then they will begin failing and this test should be disabled.
+ * At the time of writing, the Cloud Datastore has the following statistics,
+ * <ul>
+ *   <li>kind = sort_1G, entity_bytes = 2130000000, count = 10000000
+ *   <li>kind = shakespeare, entity_bytes = 26383451, count = 172948
+ * </ul>
+ */
+@RunWith(JUnit4.class)
+public class SplitQueryFnIT {
+  /**
+   * Tests {@link SplitQueryFn} to generate expected number of splits for a large dataset.
+   */
+  @Test
+  public void testSplitQueryFnWithLargeDataset() throws Exception {
+    String projectId = "apache-beam-testing";
+    String kind = "sort_1G";
+    String namespace = null;
+    // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by
+    // Datastore stats.
+    int expectedNumSplits = 32;
+    testSplitQueryFn(projectId, kind, namespace, expectedNumSplits);
+  }
+
+  /**
+   * Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset.
+   */
+  @Test
+  public void testSplitQueryFnWithSmallDataset() throws Exception {
+    String projectId = "apache-beam-testing";
+    String kind = "shakespeare";
+    String namespace = null;
+    int expectedNumSplits = NUM_QUERY_SPLITS_MIN;
+    testSplitQueryFn(projectId, kind, namespace, expectedNumSplits);
+  }
+
+  /**
+   * A helper method to test {@link SplitQueryFn} to generate the expected number of splits.
+   */
+  private void testSplitQueryFn(String projectId, String kind, @Nullable String namespace,
+      int expectedNumSplits) throws Exception {
+    Query.Builder query = Query.newBuilder();
+    query.addKindBuilder().setName(kind);
+
+    SplitQueryFn splitQueryFn = new SplitQueryFn(
+        V1Options.from(projectId, query.build(), namespace), 0);
+    DoFnTester<Query, KV<Integer, Query>> doFnTester = DoFnTester.of(splitQueryFn);
+
+    List<KV<Integer, Query>> queries = doFnTester.processBundle(query.build());
+    assertEquals(queries.size(), expectedNumSplits);
+  }
+
+  // TODO (vikasrk): Create datasets under a different namespace and add tests.
+}


[35/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
Put classes in runners-core package into runners.core namespace


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

Branch: refs/heads/gearpump-runner
Commit: 4bf3a3b345d94ecea4c77ebdfaed9dd7ef0f39e5
Parents: 60d8cd9
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Aug 25 14:57:26 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/AssignWindows.java |   46 +
 .../beam/runners/core/AssignWindowsDoFn.java    |   79 +
 .../beam/runners/core/BatchTimerInternals.java  |  140 ++
 .../apache/beam/runners/core/DoFnRunner.java    |   65 +
 .../beam/runners/core/DoFnRunnerBase.java       |  559 +++++++
 .../apache/beam/runners/core/DoFnRunners.java   |  148 ++
 .../runners/core/ElementByteSizeObservable.java |   44 +
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |    5 +-
 .../runners/core/GroupAlsoByWindowsDoFn.java    |   66 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 ++
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  271 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  151 ++
 .../apache/beam/runners/core/NonEmptyPanes.java |  151 ++
 .../beam/runners/core/PaneInfoTracker.java      |  158 ++
 .../beam/runners/core/PeekingReiterator.java    |  100 ++
 .../core/PushbackSideInputDoFnRunner.java       |  116 ++
 .../org/apache/beam/runners/core/ReduceFn.java  |  130 ++
 .../runners/core/ReduceFnContextFactory.java    |  499 ++++++
 .../beam/runners/core/ReduceFnRunner.java       |  993 ++++++++++++
 .../beam/runners/core/SimpleDoFnRunner.java     |   58 +
 .../beam/runners/core/SystemReduceFn.java       |  139 ++
 .../apache/beam/runners/core/TriggerRunner.java |  247 +++
 .../apache/beam/runners/core/WatermarkHold.java |  539 +++++++
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 -
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   77 -
 .../beam/sdk/util/BatchTimerInternals.java      |  137 --
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   63 -
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  551 -------
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  143 --
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   63 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   97 --
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  268 ----
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  145 --
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 --
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  156 --
 .../sdk/util/PushbackSideInputDoFnRunner.java   |  113 --
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  128 --
 .../beam/sdk/util/ReduceFnContextFactory.java   |  493 ------
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  983 ------------
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   55 -
 .../apache/beam/sdk/util/SystemReduceFn.java    |  138 --
 .../org/apache/beam/sdk/util/TriggerRunner.java |  241 ---
 .../org/apache/beam/sdk/util/WatermarkHold.java |  536 -------
 .../util/common/ElementByteSizeObservable.java  |   42 -
 .../beam/sdk/util/common/PeekingReiterator.java |   99 --
 .../beam/sdk/util/common/package-info.java      |   20 -
 .../org/apache/beam/sdk/util/package-info.java  |   20 -
 .../runners/core/BatchTimerInternalsTest.java   |  118 ++
 .../core/GroupAlsoByWindowsProperties.java      |  660 ++++++++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  110 ++
 .../core/LateDataDroppingDoFnRunnerTest.java    |  117 ++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  235 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   | 1446 ++++++++++++++++++
 .../beam/runners/core/ReduceFnTester.java       |  796 ++++++++++
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   88 ++
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  117 --
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  658 --------
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  109 --
 .../util/LateDataDroppingDoFnRunnerTest.java    |  114 --
 .../util/PushbackSideInputDoFnRunnerTest.java   |  231 ---
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1442 -----------------
 .../apache/beam/sdk/util/ReduceFnTester.java    |  784 ----------
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   84 -
 .../GroupAlsoByWindowEvaluatorFactory.java      |    8 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |    4 +-
 .../beam/runners/direct/ParDoEvaluator.java     |    8 +-
 .../direct/UncommittedBundleOutputManager.java  |    2 +-
 .../FlinkStreamingTransformTranslators.java     |    2 +-
 .../wrappers/streaming/DoFnOperator.java        |    6 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |    2 +-
 .../apache/beam/runners/spark/SparkRunner.java  |    2 +-
 .../spark/translation/TransformTranslator.java  |   10 +-
 .../streaming/StreamingTransformTranslator.java |    2 +-
 .../src/main/resources/beam/findbugs-filter.xml |    2 +-
 .../org/apache/beam/sdk/util/BitSetCoder.java   |    2 +-
 75 files changed, 8395 insertions(+), 8332 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java
new file mode 100644
index 0000000..f2387f5
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a
+ * {@link PCollection} to windows according to the provided {@link WindowFn}.
+ *
+ * @param <T> Type of elements being windowed
+ * @param <W> Window type
+ */
+public class AssignWindows<T, W extends BoundedWindow>
+    extends PTransform<PCollection<T>, PCollection<T>> {
+
+  private WindowFn<? super T, W> fn;
+
+  public AssignWindows(WindowFn<? super T, W> fn) {
+    this.fn = fn;
+  }
+
+  @Override
+  public PCollection<T> apply(PCollection<T> input) {
+    return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java
new file mode 100644
index 0000000..0eb1667
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collection;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+
+/**
+ * {@link OldDoFn} that tags elements of a {@link PCollection} with windows, according to the
+ * provided {@link WindowFn}.
+ *
+ * @param <T> Type of elements being windowed
+ * @param <W> Window type
+ */
+@SystemDoFnInternal
+public class AssignWindowsDoFn<T, W extends BoundedWindow> extends OldDoFn<T, T>
+    implements RequiresWindowAccess {
+  private WindowFn<? super T, W> fn;
+
+  public AssignWindowsDoFn(WindowFn<? super T, W> fn) {
+    this.fn =
+        checkNotNull(
+            fn,
+            "%s provided to %s cannot be null",
+            WindowFn.class.getSimpleName(),
+            AssignWindowsDoFn.class.getSimpleName());
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void processElement(final ProcessContext c) throws Exception {
+    Collection<W> windows =
+        ((WindowFn<T, W>) fn).assignWindows(
+            ((WindowFn<T, W>) fn).new AssignContext() {
+                @Override
+                public T element() {
+                  return c.element();
+                }
+
+                @Override
+                public Instant timestamp() {
+                  return c.timestamp();
+                }
+
+                @Override
+                public BoundedWindow window() {
+                  return Iterables.getOnlyElement(c.windowingInternals().windows());
+                }
+              });
+
+    c.windowingInternals()
+        .outputWindowedValue(c.element(), c.timestamp(), windows, PaneInfo.NO_FIRING);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java
new file mode 100644
index 0000000..829dbde
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.base.MoreObjects;
+import java.util.HashSet;
+import java.util.PriorityQueue;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+
+import org.joda.time.Instant;
+
+/**
+ * TimerInternals that uses priority queues to manage the timers that are ready to fire.
+ */
+public class BatchTimerInternals implements TimerInternals {
+  /** Set of timers that are scheduled used for deduplicating timers. */
+  private Set<TimerData> existingTimers = new HashSet<>();
+
+  // Keep these queues separate so we can advance over them separately.
+  private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
+  private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
+
+  private Instant inputWatermarkTime;
+  private Instant processingTime;
+
+  private PriorityQueue<TimerData> queue(TimeDomain domain) {
+    return TimeDomain.EVENT_TIME.equals(domain) ? watermarkTimers : processingTimers;
+  }
+
+  public BatchTimerInternals(Instant processingTime) {
+    this.processingTime = processingTime;
+    this.inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+  }
+
+  @Override
+  public void setTimer(TimerData timer) {
+    if (existingTimers.add(timer)) {
+      queue(timer.getDomain()).add(timer);
+    }
+  }
+
+  @Override
+  public void deleteTimer(TimerData timer) {
+    existingTimers.remove(timer);
+    queue(timer.getDomain()).remove(timer);
+  }
+
+  @Override
+  public Instant currentProcessingTime() {
+    return processingTime;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return {@link BoundedWindow#TIMESTAMP_MAX_VALUE}: in batch mode, upstream processing
+   * is already complete.
+   */
+  @Override
+  @Nullable
+  public Instant currentSynchronizedProcessingTime() {
+    return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  }
+
+  @Override
+  public Instant currentInputWatermarkTime() {
+    return inputWatermarkTime;
+  }
+
+  @Override
+  @Nullable
+  public Instant currentOutputWatermarkTime() {
+    // The output watermark is always undefined in batch mode.
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(getClass())
+        .add("watermarkTimers", watermarkTimers)
+        .add("processingTimers", processingTimers)
+        .toString();
+  }
+
+  public void advanceInputWatermark(ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark)
+      throws Exception {
+    checkState(!newInputWatermark.isBefore(inputWatermarkTime),
+        "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
+        newInputWatermark);
+    inputWatermarkTime = newInputWatermark;
+    advance(runner, newInputWatermark, TimeDomain.EVENT_TIME);
+  }
+
+  public void advanceProcessingTime(ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime)
+      throws Exception {
+    checkState(!newProcessingTime.isBefore(processingTime),
+        "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
+    processingTime = newProcessingTime;
+    advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
+  }
+
+  private void advance(ReduceFnRunner<?, ?, ?, ?> runner, Instant newTime, TimeDomain domain)
+      throws Exception {
+    PriorityQueue<TimerData> timers = queue(domain);
+    boolean shouldFire = false;
+
+    do {
+      TimerData timer = timers.peek();
+      // Timers fire if the new time is ahead of the timer
+      shouldFire = timer != null && newTime.isAfter(timer.getTimestamp());
+      if (shouldFire) {
+        // Remove before firing, so that if the trigger adds another identical
+        // timer we don't remove it.
+        timers.remove();
+        runner.onTimer(timer);
+      }
+    } while (shouldFire);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java
new file mode 100644
index 0000000..f4c8eea
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * An wrapper interface that represents the execution of a {@link OldDoFn}.
+ */
+public interface DoFnRunner<InputT, OutputT> {
+  /**
+   * Prepares and calls {@link OldDoFn#startBundle}.
+   */
+  public void startBundle();
+
+  /**
+   * Calls {@link OldDoFn#processElement} with a {@link ProcessContext} containing the current
+   * element.
+   */
+  public void processElement(WindowedValue<InputT> elem);
+
+  /**
+   * Calls {@link OldDoFn#finishBundle} and performs additional tasks, such as
+   * flushing in-memory states.
+   */
+  public void finishBundle();
+
+  /**
+   * An internal interface for signaling that a {@link OldDoFn} requires late data dropping.
+   */
+  public interface ReduceFnExecutor<K, InputT, OutputT, W> {
+    /**
+     * Gets this object as a {@link OldDoFn}.
+     *
+     * Most implementors of this interface are expected to be {@link OldDoFn} instances, and will
+     * return themselves.
+     */
+    OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn();
+
+    /**
+     * Returns an aggregator that tracks elements that are dropped due to being late.
+     */
+    Aggregator<Long, Long> getDroppedDueToLatenessAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java
new file mode 100644
index 0000000..71472da
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java
@@ -0,0 +1,559 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+/**
+ * A base implementation of {@link DoFnRunner}.
+ *
+ * <p> Sub-classes should override {@link #invokeProcessElement}.
+ */
+public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+  /** The {@link OldDoFn} being run. */
+  public final OldDoFn<InputT, OutputT> fn;
+
+  /** The context used for running the {@link OldDoFn}. */
+  public final DoFnContext<InputT, OutputT> context;
+
+  protected DoFnRunnerBase(
+      PipelineOptions options,
+      OldDoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    this.fn = fn;
+    this.context = new DoFnContext<>(
+        options,
+        fn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy == null ? null : windowingStrategy.getWindowFn());
+  }
+
+  /**
+   * An implementation of {@code OutputManager} using simple lists, for testing and in-memory
+   * contexts such as the {@code DirectRunner}.
+   */
+  public static class ListOutputManager implements OutputManager {
+
+    private Map<TupleTag<?>, List<WindowedValue<?>>> outputLists = Maps.newHashMap();
+
+    @Override
+    public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
+
+      if (outputList == null) {
+        outputList = Lists.newArrayList();
+        @SuppressWarnings({"rawtypes", "unchecked"})
+        List<WindowedValue<?>> untypedList = (List) outputList;
+        outputLists.put(tag, untypedList);
+      }
+
+      outputList.add(output);
+    }
+
+    public <T> List<WindowedValue<T>> getOutput(TupleTag<T> tag) {
+      // Safe cast by design, inexpressible in Java without rawtypes
+      @SuppressWarnings({"rawtypes", "unchecked"})
+      List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
+      return (outputList != null) ? outputList : Collections.<WindowedValue<T>>emptyList();
+    }
+  }
+
+  @Override
+  public void startBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.startBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> elem) {
+    if (elem.getWindows().size() <= 1
+        || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
+            && context.sideInputReader.isEmpty())) {
+      invokeProcessElement(elem);
+    } else {
+      // We could modify the windowed value (and the processContext) to
+      // avoid repeated allocations, but this is more straightforward.
+      for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
+        invokeProcessElement(windowedValue);
+      }
+    }
+  }
+
+  /**
+   * Invokes {@link OldDoFn#processElement} after certain pre-processings has been done in
+   * {@link DoFnRunnerBase#processElement}.
+   */
+  protected abstract void invokeProcessElement(WindowedValue<InputT> elem);
+
+  @Override
+  public void finishBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.finishBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  /**
+   * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}.
+   *
+   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+   */
+  private static class DoFnContext<InputT, OutputT>
+      extends OldDoFn<InputT, OutputT>.Context {
+    private static final int MAX_SIDE_OUTPUTS = 1000;
+
+    final PipelineOptions options;
+    final OldDoFn<InputT, OutputT> fn;
+    final SideInputReader sideInputReader;
+    final OutputManager outputManager;
+    final TupleTag<OutputT> mainOutputTag;
+    final StepContext stepContext;
+    final AggregatorFactory aggregatorFactory;
+    final WindowFn<?, ?> windowFn;
+
+    /**
+     * The set of known output tags, some of which may be undeclared, so we can throw an
+     * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
+     */
+    private Set<TupleTag<?>> outputTags;
+
+    public DoFnContext(PipelineOptions options,
+                       OldDoFn<InputT, OutputT> fn,
+                       SideInputReader sideInputReader,
+                       OutputManager outputManager,
+                       TupleTag<OutputT> mainOutputTag,
+                       List<TupleTag<?>> sideOutputTags,
+                       StepContext stepContext,
+                       AggregatorFactory aggregatorFactory,
+                       WindowFn<?, ?> windowFn) {
+      fn.super();
+      this.options = options;
+      this.fn = fn;
+      this.sideInputReader = sideInputReader;
+      this.outputManager = outputManager;
+      this.mainOutputTag = mainOutputTag;
+      this.outputTags = Sets.newHashSet();
+
+      outputTags.add(mainOutputTag);
+      for (TupleTag<?> sideOutputTag : sideOutputTags) {
+        outputTags.add(sideOutputTag);
+      }
+
+      this.stepContext = stepContext;
+      this.aggregatorFactory = aggregatorFactory;
+      this.windowFn = windowFn;
+      super.setupDelegateAggregators();
+    }
+
+    //////////////////////////////////////////////////////////////////////////////
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
+        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
+      final Instant inputTimestamp = timestamp;
+
+      if (timestamp == null) {
+        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      }
+
+      if (windows == null) {
+        try {
+          // The windowFn can never succeed at accessing the element, so its type does not
+          // matter here
+          @SuppressWarnings("unchecked")
+          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
+          windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
+            @Override
+            public Object element() {
+              throw new UnsupportedOperationException(
+                  "WindowFn attempted to access input element when none was available");
+            }
+
+            @Override
+            public Instant timestamp() {
+              if (inputTimestamp == null) {
+                throw new UnsupportedOperationException(
+                    "WindowFn attempted to access input timestamp when none was available");
+              }
+              return inputTimestamp;
+            }
+
+            @Override
+            public W window() {
+              throw new UnsupportedOperationException(
+                  "WindowFn attempted to access input windows when none were available");
+            }
+          });
+        } catch (Exception e) {
+          throw UserCodeException.wrap(e);
+        }
+      }
+
+      return WindowedValue.of(output, timestamp, windows, pane);
+    }
+
+    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+      if (!sideInputReader.contains(view)) {
+        throw new IllegalArgumentException("calling sideInput() with unknown view");
+      }
+      BoundedWindow sideInputWindow =
+          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+      return sideInputReader.get(view, sideInputWindow);
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
+      outputManager.output(mainOutputTag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteOutput(windowedElem);
+      }
+    }
+
+    protected <T> void sideOutputWindowedValue(TupleTag<T> tag,
+                                               T output,
+                                               Instant timestamp,
+                                               Collection<? extends BoundedWindow> windows,
+                                               PaneInfo pane) {
+      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    protected <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
+      if (!outputTags.contains(tag)) {
+        // This tag wasn't declared nor was it seen before during this execution.
+        // Thus, this must be a new, undeclared and unconsumed output.
+        // To prevent likely user errors, enforce the limit on the number of side
+        // outputs.
+        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
+          throw new IllegalArgumentException(
+              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
+        }
+        outputTags.add(tag);
+      }
+
+      outputManager.output(tag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteSideOutput(tag, windowedElem);
+      }
+    }
+
+    // Following implementations of output, outputWithTimestamp, and sideOutput
+    // are only accessible in OldDoFn.startBundle and OldDoFn.finishBundle, and will be shadowed by
+    // ProcessContext's versions in OldDoFn.processElement.
+    @Override
+    public void output(OutputT output) {
+      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null");
+      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
+    }
+  }
+
+  /**
+   * Returns a new {@link OldDoFn.ProcessContext} for the given element.
+   */
+  protected OldDoFn<InputT, OutputT>.ProcessContext createProcessContext(
+      WindowedValue<InputT> elem) {
+    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
+  }
+
+  protected RuntimeException wrapUserCodeException(Throwable t) {
+    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
+  }
+
+  private boolean isSystemDoFn() {
+    return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+  }
+
+  /**
+   * A concrete implementation of {@link OldDoFn.ProcessContext} used for
+   * running a {@link OldDoFn} over a single element.
+   *
+   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+   */
+  static class DoFnProcessContext<InputT, OutputT>
+      extends OldDoFn<InputT, OutputT>.ProcessContext {
+
+
+    final OldDoFn<InputT, OutputT> fn;
+    final DoFnContext<InputT, OutputT> context;
+    final WindowedValue<InputT> windowedValue;
+
+    public DoFnProcessContext(OldDoFn<InputT, OutputT> fn,
+                              DoFnContext<InputT, OutputT> context,
+                              WindowedValue<InputT> windowedValue) {
+      fn.super();
+      this.fn = fn;
+      this.context = context;
+      this.windowedValue = windowedValue;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public InputT element() {
+      return windowedValue.getValue();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      checkNotNull(view, "View passed to sideInput cannot be null");
+      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
+      BoundedWindow window;
+      if (!windowIter.hasNext()) {
+        if (context.windowFn instanceof GlobalWindows) {
+          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
+          // without windows
+          window = GlobalWindow.INSTANCE;
+        } else {
+          throw new IllegalStateException(
+              "sideInput called when main input element is not in any windows");
+        }
+      } else {
+        window = windowIter.next();
+        if (windowIter.hasNext()) {
+          throw new IllegalStateException(
+              "sideInput called when main input element is in multiple windows");
+        }
+      }
+      return context.sideInput(view, window);
+    }
+
+    @Override
+    public BoundedWindow window() {
+      if (!(fn instanceof RequiresWindowAccess)) {
+        throw new UnsupportedOperationException(
+            "window() is only available in the context of a OldDoFn marked as"
+                + "RequiresWindowAccess.");
+      }
+      return Iterables.getOnlyElement(windows());
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return windowedValue.getPane();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.outputWindowedValue(windowedValue.withValue(output));
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      checkTimestamp(timestamp);
+      context.outputWindowedValue(output, timestamp,
+          windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      context.outputWindowedValue(output, timestamp, windows, pane);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkTimestamp(timestamp);
+      context.sideOutputWindowedValue(
+          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    @Override
+    public Instant timestamp() {
+      return windowedValue.getTimestamp();
+    }
+
+    public Collection<? extends BoundedWindow> windows() {
+      return windowedValue.getWindows();
+    }
+
+    private void checkTimestamp(Instant timestamp) {
+      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
+        throw new IllegalArgumentException(String.format(
+            "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+            + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+            + "OldDoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
+            timestamp, windowedValue.getTimestamp(),
+            PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
+      }
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return new WindowingInternals<InputT, OutputT>() {
+        @Override
+        public void outputWindowedValue(OutputT output, Instant timestamp,
+            Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+          context.outputWindowedValue(output, timestamp, windows, pane);
+        }
+
+        @Override
+        public Collection<? extends BoundedWindow> windows() {
+          return windowedValue.getWindows();
+        }
+
+        @Override
+        public PaneInfo pane() {
+          return windowedValue.getPane();
+        }
+
+        @Override
+        public TimerInternals timerInternals() {
+          return context.stepContext.timerInternals();
+        }
+
+        @Override
+        public <T> void writePCollectionViewData(
+            TupleTag<?> tag,
+            Iterable<WindowedValue<T>> data,
+            Coder<T> elemCoder) throws IOException {
+          @SuppressWarnings("unchecked")
+          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
+
+          context.stepContext.writePCollectionViewData(
+              tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
+              window(), windowCoder);
+        }
+
+        @Override
+        public StateInternals<?> stateInternals() {
+          return context.stepContext.stateInternals();
+        }
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+          return context.sideInput(view, mainInputWindow);
+        }
+      };
+    }
+
+    @Override
+    protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
+        createAggregatorInternal(
+            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
+      return context.createAggregatorInternal(name, combiner);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
new file mode 100644
index 0000000..7726374
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.util.List;
+
+import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+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.TupleTag;
+
+/**
+ * Static utility methods that provide {@link DoFnRunner} implementations.
+ */
+public class DoFnRunners {
+  /**
+   * Information about how to create output receivers and output to them.
+   */
+  public interface OutputManager {
+    /**
+     * Outputs a single element to the receiver indicated by the given {@link TupleTag}.
+     */
+    public <T> void output(TupleTag<T> tag, WindowedValue<T> output);
+  }
+
+  /**
+   * Returns a basic implementation of {@link DoFnRunner} that works for most {@link OldDoFn DoFns}.
+   *
+   * <p>It invokes {@link OldDoFn#processElement} for each input.
+   */
+  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> simpleRunner(
+      PipelineOptions options,
+      OldDoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    return new SimpleDoFnRunner<>(
+        options,
+        fn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy);
+  }
+
+  /**
+   * Returns an implementation of {@link DoFnRunner} that handles late data dropping.
+   *
+   * <p>It drops elements from expired windows before they reach the underlying {@link OldDoFn}.
+   */
+  public static <K, InputT, OutputT, W extends BoundedWindow>
+      DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> lateDataDroppingRunner(
+          PipelineOptions options,
+          ReduceFnExecutor<K, InputT, OutputT, W> reduceFnExecutor,
+          SideInputReader sideInputReader,
+          OutputManager outputManager,
+          TupleTag<KV<K, OutputT>> mainOutputTag,
+          List<TupleTag<?>> sideOutputTags,
+          StepContext stepContext,
+          AggregatorFactory aggregatorFactory,
+          WindowingStrategy<?, W> windowingStrategy) {
+    DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> simpleDoFnRunner =
+        simpleRunner(
+            options,
+            reduceFnExecutor.asDoFn(),
+            sideInputReader,
+            outputManager,
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            aggregatorFactory,
+            windowingStrategy);
+    return new LateDataDroppingDoFnRunner<>(
+        simpleDoFnRunner,
+        windowingStrategy,
+        stepContext.timerInternals(),
+        reduceFnExecutor.getDroppedDueToLatenessAggregator());
+  }
+
+
+  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
+      PipelineOptions options,
+      OldDoFn<InputT, OutputT> doFn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    if (doFn instanceof ReduceFnExecutor) {
+      @SuppressWarnings("rawtypes")
+      ReduceFnExecutor fn = (ReduceFnExecutor) doFn;
+      @SuppressWarnings({"unchecked", "cast", "rawtypes"})
+      DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
+          options,
+          fn,
+          sideInputReader,
+          outputManager,
+          (TupleTag) mainOutputTag,
+          sideOutputTags,
+          stepContext,
+          aggregatorFactory,
+          (WindowingStrategy) windowingStrategy);
+      return runner;
+    }
+    return simpleRunner(
+        options,
+        doFn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java
new file mode 100644
index 0000000..2380ba9
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
+/**
+ * An interface for things that allow observing the size in bytes of
+ * encoded values of type {@code T}.
+ *
+ * @param <T> the type of the values being observed
+ */
+public interface ElementByteSizeObservable<T> {
+  /**
+   * Returns whether {@link #registerByteSizeObserver} is cheap enough
+   * to call for every element, that is, if this
+   * {@code ElementByteSizeObservable} can calculate the byte size of
+   * the element to be coded in roughly constant time (or lazily).
+   */
+  public boolean isRegisterByteSizeObserverCheap(T value);
+
+  /**
+   * Notifies the {@code ElementByteSizeObserver} about the byte size
+   * of the encoded value using this {@code ElementByteSizeObservable}.
+   */
+  public void registerByteSizeObserver(T value,
+                                       ElementByteSizeObserver observer)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
index 7cdab00..b427037 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
@@ -17,16 +17,13 @@
  */
 package org.apache.beam.runners.core;
 
+import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
-import org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn;
 import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.ReduceFnRunner;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
-import org.apache.beam.sdk.util.SystemReduceFn;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
 import org.apache.beam.sdk.util.WindowingStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
new file mode 100644
index 0000000..9851449
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * {@link OldDoFn} that merges windows and groups elements in those windows, optionally
+ * combining values.
+ *
+ * @param <K> key type
+ * @param <InputT> input value element type
+ * @param <OutputT> output value element type
+ * @param <W> window type
+ */
+@SystemDoFnInternal
+public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends BoundedWindow>
+    extends OldDoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> {
+  public static final String DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER = "DroppedDueToClosedWindow";
+  public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness";
+
+  protected final Aggregator<Long, Long> droppedDueToClosedWindow =
+      createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
+  protected final Aggregator<Long, Long> droppedDueToLateness =
+      createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
+
+  /**
+   * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
+   * grouping.
+   *
+   * @param windowingStrategy The window function and trigger to use for grouping
+   * @param inputCoder the input coder to use
+   */
+  public static <K, V, W extends BoundedWindow>
+      GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W> createDefault(
+          WindowingStrategy<?, W> windowingStrategy,
+          StateInternalsFactory<K> stateInternalsFactory,
+          Coder<V> inputCoder) {
+    return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
+        windowingStrategy, stateInternalsFactory, SystemReduceFn.<K, V, W>buffering(inputCoder));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
new file mode 100644
index 0000000..091ad33
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import com.google.common.collect.Iterables;
+import java.util.List;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.values.KV;
+import org.joda.time.Instant;
+
+/**
+ * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path"
+ * implementation is applicable.
+ */
+@SystemDoFnInternal
+public class GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, OutputT, W extends BoundedWindow>
+   extends GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> {
+
+  private final WindowingStrategy<?, W> strategy;
+  private final StateInternalsFactory<K> stateInternalsFactory;
+  private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
+
+  public GroupAlsoByWindowsViaOutputBufferDoFn(
+      WindowingStrategy<?, W> windowingStrategy,
+      StateInternalsFactory<K> stateInternalsFactory,
+      SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
+    this.strategy = windowingStrategy;
+    this.reduceFn = reduceFn;
+    this.stateInternalsFactory = stateInternalsFactory;
+  }
+
+  @Override
+  public void processElement(
+      OldDoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>.ProcessContext c)
+          throws Exception {
+    K key = c.element().getKey();
+    // Used with Batch, we know that all the data is available for this key. We can't use the
+    // timer manager from the context because it doesn't exist. So we create one and emulate the
+    // watermark, knowing that we have all data and it is in timestamp order.
+    BatchTimerInternals timerInternals = new BatchTimerInternals(Instant.now());
+    StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);
+
+    ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
+        new ReduceFnRunner<K, InputT, OutputT, W>(
+            key,
+            strategy,
+            stateInternals,
+            timerInternals,
+            c.windowingInternals(),
+            droppedDueToClosedWindow,
+            reduceFn,
+            c.getPipelineOptions());
+
+    Iterable<List<WindowedValue<InputT>>> chunks =
+        Iterables.partition(c.element().getValue(), 1000);
+    for (Iterable<WindowedValue<InputT>> chunk : chunks) {
+      // Process the chunk of elements.
+      reduceFnRunner.processElements(chunk);
+
+      // Then, since elements are sorted by their timestamp, advance the input watermark
+      // to the first element, and fire any timers that may have been scheduled.
+      timerInternals.advanceInputWatermark(reduceFnRunner, chunk.iterator().next().getTimestamp());
+
+      // Fire any processing timers that need to fire
+      timerInternals.advanceProcessingTime(reduceFnRunner, Instant.now());
+
+      // Leave the output watermark undefined. Since there's no late data in batch mode
+      // there's really no need to track it as we do for streaming.
+    }
+
+    // Finish any pending windows by advancing the input watermark to infinity.
+    timerInternals.advanceInputWatermark(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    // Finally, advance the processing time to infinity to fire any timers.
+    timerInternals.advanceProcessingTime(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+    reduceFnRunner.persist();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
new file mode 100644
index 0000000..b521425
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.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.core;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+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.OldDoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ReifyTimestampsAndWindows;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly}
+ * primitive.
+ *
+ * <p>This implementation of {@link GroupByKey} proceeds via the following steps:
+ * <ol>
+ *   <li>{@code ReifyTimestampsAndWindowsDoFn ParDo(ReifyTimestampsAndWindows)}: This embeds
+ *       the previously-implicit timestamp and window into the elements themselves, so a
+ *       window-and-timestamp-unaware transform can operate on them.</li>
+ *   <li>{@code GroupByKeyOnly}: This lower-level primitive groups by keys, ignoring windows
+ *       and timestamps. Many window-unaware runners have such a primitive already.</li>
+ *   <li>{@code SortValuesByTimestamp ParDo(SortValuesByTimestamp)}: The values in the iterables
+ *       output by {@link GroupByKeyOnly} are sorted by timestamp.</li>
+ *   <li>{@code GroupAlsoByWindow}: This primitive processes the sorted values. Today it is
+ *       implemented as a {@link ParDo} that calls reserved internal methods.</li>
+ * </ol>
+ *
+ * <p>This implementation of {@link GroupByKey} has severe limitations unless its component
+ * transforms are replaced. As-is, it is only applicable for in-memory runners using a batch-style
+ * execution strategy. Specifically:
+ *
+ * <ul>
+ *   <li>Every iterable output by {@link GroupByKeyOnly} must contain all elements for that key.
+ *       A streaming-style partition, with multiple elements for the same key, will not yield
+ *       correct results.</li>
+ *   <li>Sorting of values by timestamp is performed on an in-memory list. It will not succeed
+ *       for large iterables.</li>
+ *   <li>The implementation of {@code GroupAlsoByWindow} does not support timers. This is only
+ *       appropriate for runners which also do not support timers.</li>
+ * </ul>
+ */
+public class GroupByKeyViaGroupByKeyOnly<K, V>
+    extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+
+  private final GroupByKey<K, V> gbkTransform;
+
+  public GroupByKeyViaGroupByKeyOnly(GroupByKey<K, V> originalTransform) {
+    this.gbkTransform = originalTransform;
+  }
+
+  @Override
+  public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+    WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
+
+    return input
+        // Make each input element's timestamp and assigned windows
+        // explicit, in the value part.
+        .apply(new ReifyTimestampsAndWindows<K, V>())
+
+        // Group by just the key.
+        // Combiner lifting will not happen regardless of the disallowCombinerLifting value.
+        // There will be no combiners right after the GroupByKeyOnly because of the two ParDos
+        // introduced in here.
+        .apply(new GroupByKeyOnly<K, WindowedValue<V>>())
+
+        // Sort each key's values by timestamp. GroupAlsoByWindow requires
+        // its input to be sorted by timestamp.
+        .apply(new SortValuesByTimestamp<K, V>())
+
+        // Group each key's values by window, merging windows as needed.
+        .apply(new GroupAlsoByWindow<K, V>(windowingStrategy))
+
+        // And update the windowing strategy as appropriate.
+        .setWindowingStrategyInternal(
+            gbkTransform.updateWindowingStrategy(windowingStrategy));
+  }
+
+  /**
+   * Runner-specific primitive that groups by key only, ignoring any window assignments. A
+   * runner that uses {@link GroupByKeyViaGroupByKeyOnly} should have a primitive way to translate
+   * or evaluate this class.
+   */
+  public static class GroupByKeyOnly<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    @Override
+    public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
+          input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
+    }
+
+    @Override
+    public Coder<KV<K, Iterable<V>>> getDefaultOutputCoder(PCollection<KV<K, V>> input) {
+      return GroupByKey.getOutputKvCoder(input.getCoder());
+    }
+  }
+
+  /**
+   * Helper transform that sorts the values associated with each key by timestamp.
+   */
+  private static class SortValuesByTimestamp<K, V>
+      extends PTransform<
+          PCollection<KV<K, Iterable<WindowedValue<V>>>>,
+          PCollection<KV<K, Iterable<WindowedValue<V>>>>> {
+    @Override
+    public PCollection<KV<K, Iterable<WindowedValue<V>>>> apply(
+        PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
+      return input
+          .apply(
+              ParDo.of(
+                  new OldDoFn<
+                      KV<K, Iterable<WindowedValue<V>>>,
+                      KV<K, Iterable<WindowedValue<V>>>>() {
+                    @Override
+                    public void processElement(ProcessContext c) {
+                      KV<K, Iterable<WindowedValue<V>>> kvs = c.element();
+                      K key = kvs.getKey();
+                      Iterable<WindowedValue<V>> unsortedValues = kvs.getValue();
+                      List<WindowedValue<V>> sortedValues = new ArrayList<>();
+                      for (WindowedValue<V> value : unsortedValues) {
+                        sortedValues.add(value);
+                      }
+                      Collections.sort(
+                          sortedValues,
+                          new Comparator<WindowedValue<V>>() {
+                            @Override
+                            public int compare(WindowedValue<V> e1, WindowedValue<V> e2) {
+                              return e1.getTimestamp().compareTo(e2.getTimestamp());
+                            }
+                          });
+                      c.output(KV.<K, Iterable<WindowedValue<V>>>of(key, sortedValues));
+                    }
+                  }))
+          .setCoder(input.getCoder());
+    }
+  }
+
+  /**
+   * Runner-specific primitive that takes a collection of timestamp-ordered values associated with
+   * each key, groups the values by window, merges windows as needed, and for each window in each
+   * key, outputs a collection of key/value-list pairs implicitly assigned to the window and with
+   * the timestamp derived from that window.
+   */
+  public static class GroupAlsoByWindow<K, V>
+      extends PTransform<
+          PCollection<KV<K, Iterable<WindowedValue<V>>>>, PCollection<KV<K, Iterable<V>>>> {
+    private final WindowingStrategy<?, ?> windowingStrategy;
+
+    public GroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
+      this.windowingStrategy = windowingStrategy;
+    }
+
+    public WindowingStrategy<?, ?> getWindowingStrategy() {
+      return windowingStrategy;
+    }
+
+    private KvCoder<K, Iterable<WindowedValue<V>>> getKvCoder(
+        Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
+      // Coder<KV<...>> --> KvCoder<...>
+      checkArgument(inputCoder instanceof KvCoder,
+          "%s requires a %s<...> but got %s",
+          getClass().getSimpleName(),
+          KvCoder.class.getSimpleName(),
+          inputCoder);
+      @SuppressWarnings("unchecked")
+      KvCoder<K, Iterable<WindowedValue<V>>> kvCoder =
+          (KvCoder<K, Iterable<WindowedValue<V>>>) inputCoder;
+      return kvCoder;
+    }
+
+    public Coder<K> getKeyCoder(Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
+      return getKvCoder(inputCoder).getKeyCoder();
+    }
+
+    public Coder<V> getValueCoder(Coder<KV<K, Iterable<WindowedValue<V>>>> inputCoder) {
+      // Coder<Iterable<...>> --> IterableCoder<...>
+      Coder<Iterable<WindowedValue<V>>> iterableWindowedValueCoder =
+          getKvCoder(inputCoder).getValueCoder();
+      checkArgument(iterableWindowedValueCoder instanceof IterableCoder,
+          "%s requires a %s<..., %s> but got a %s",
+          getClass().getSimpleName(),
+          KvCoder.class.getSimpleName(),
+          IterableCoder.class.getSimpleName(),
+          iterableWindowedValueCoder);
+      IterableCoder<WindowedValue<V>> iterableCoder =
+          (IterableCoder<WindowedValue<V>>) iterableWindowedValueCoder;
+
+      // Coder<WindowedValue<...>> --> WindowedValueCoder<...>
+      Coder<WindowedValue<V>> iterableElementCoder = iterableCoder.getElemCoder();
+      checkArgument(iterableElementCoder instanceof WindowedValueCoder,
+          "%s requires a %s<..., %s<%s>> but got a %s",
+          getClass().getSimpleName(),
+          KvCoder.class.getSimpleName(),
+          IterableCoder.class.getSimpleName(),
+          WindowedValueCoder.class.getSimpleName(),
+          iterableElementCoder);
+      WindowedValueCoder<V> windowedValueCoder =
+          (WindowedValueCoder<V>) iterableElementCoder;
+
+      return windowedValueCoder.getValueCoder();
+    }
+
+    @Override
+    public PCollection<KV<K, Iterable<V>>> apply(
+        PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
+      @SuppressWarnings("unchecked")
+      KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
+          (KvCoder<K, Iterable<WindowedValue<V>>>) input.getCoder();
+
+      Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+      Coder<Iterable<WindowedValue<V>>> inputValueCoder = inputKvCoder.getValueCoder();
+
+      IterableCoder<WindowedValue<V>> inputIterableValueCoder =
+          (IterableCoder<WindowedValue<V>>) inputValueCoder;
+      Coder<WindowedValue<V>> inputIterableElementCoder = inputIterableValueCoder.getElemCoder();
+      WindowedValueCoder<V> inputIterableWindowedValueCoder =
+          (WindowedValueCoder<V>) inputIterableElementCoder;
+
+      Coder<V> inputIterableElementValueCoder = inputIterableWindowedValueCoder.getValueCoder();
+      Coder<Iterable<V>> outputValueCoder = IterableCoder.of(inputIterableElementValueCoder);
+      Coder<KV<K, Iterable<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
+
+      return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
+          input.getPipeline(), windowingStrategy, input.isBounded())
+          .setCoder(outputKvCoder);
+    }
+
+    private <W extends BoundedWindow>
+        GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W> groupAlsoByWindowsFn(
+            WindowingStrategy<?, W> strategy,
+            StateInternalsFactory<K> stateInternalsFactory,
+            Coder<V> inputIterableElementValueCoder) {
+      return new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
+          strategy,
+          stateInternalsFactory,
+          SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
new file mode 100644
index 0000000..63a80d2
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowTracing;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.joda.time.Instant;
+
+/**
+ * A customized {@link DoFnRunner} that handles late data dropping for
+ * a {@link KeyedWorkItem} input {@link OldDoFn}.
+ *
+ * <p>It expands windows before checking data lateness.
+ *
+ * <p>{@link KeyedWorkItem KeyedWorkItems} are always in empty windows.
+ *
+ * @param <K> key type
+ * @param <InputT> input value element type
+ * @param <OutputT> output value element type
+ * @param <W> window type
+ */
+public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
+    implements DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
+  private final DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner;
+  private final LateDataFilter lateDataFilter;
+
+  public LateDataDroppingDoFnRunner(
+      DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner,
+      WindowingStrategy<?, ?> windowingStrategy,
+      TimerInternals timerInternals,
+      Aggregator<Long, Long> droppedDueToLateness) {
+    this.doFnRunner = doFnRunner;
+    lateDataFilter = new LateDataFilter(windowingStrategy, timerInternals, droppedDueToLateness);
+  }
+
+  @Override
+  public void startBundle() {
+    doFnRunner.startBundle();
+  }
+
+  @Override
+  public void processElement(WindowedValue<KeyedWorkItem<K, InputT>> elem) {
+    Iterable<WindowedValue<InputT>> nonLateElements = lateDataFilter.filter(
+        elem.getValue().key(), elem.getValue().elementsIterable());
+    KeyedWorkItem<K, InputT> keyedWorkItem = KeyedWorkItems.workItem(
+        elem.getValue().key(), elem.getValue().timersIterable(), nonLateElements);
+    doFnRunner.processElement(elem.withValue(keyedWorkItem));
+  }
+
+  @Override
+  public void finishBundle() {
+    doFnRunner.finishBundle();
+  }
+
+  /**
+   * It filters late data in a {@link KeyedWorkItem}.
+   */
+  @VisibleForTesting
+  static class LateDataFilter {
+    private final WindowingStrategy<?, ?> windowingStrategy;
+    private final TimerInternals timerInternals;
+    private final Aggregator<Long, Long> droppedDueToLateness;
+
+    public LateDataFilter(
+        WindowingStrategy<?, ?> windowingStrategy,
+        TimerInternals timerInternals,
+        Aggregator<Long, Long> droppedDueToLateness) {
+      this.windowingStrategy = windowingStrategy;
+      this.timerInternals = timerInternals;
+      this.droppedDueToLateness = droppedDueToLateness;
+    }
+
+    /**
+     * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains
+     * non-late input elements.
+     */
+    public <K, InputT> Iterable<WindowedValue<InputT>> filter(
+        final K key, Iterable<WindowedValue<InputT>> elements) {
+      Iterable<Iterable<WindowedValue<InputT>>> windowsExpandedElements = Iterables.transform(
+          elements,
+          new Function<WindowedValue<InputT>, Iterable<WindowedValue<InputT>>>() {
+            @Override
+            public Iterable<WindowedValue<InputT>> apply(final WindowedValue<InputT> input) {
+              return Iterables.transform(
+                  input.getWindows(),
+                  new Function<BoundedWindow, WindowedValue<InputT>>() {
+                    @Override
+                    public WindowedValue<InputT> apply(BoundedWindow window) {
+                      return WindowedValue.of(
+                          input.getValue(), input.getTimestamp(), window, input.getPane());
+                    }
+                  });
+            }});
+
+      Iterable<WindowedValue<InputT>> nonLateElements = Iterables.filter(
+          Iterables.concat(windowsExpandedElements),
+          new Predicate<WindowedValue<InputT>>() {
+            @Override
+            public boolean apply(WindowedValue<InputT> input) {
+              BoundedWindow window = Iterables.getOnlyElement(input.getWindows());
+              if (canDropDueToExpiredWindow(window)) {
+                // The element is too late for this window.
+                droppedDueToLateness.addValue(1L);
+                WindowTracing.debug(
+                    "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
+                    + "since too far behind inputWatermark:{}; outputWatermark:{}",
+                    input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
+                    timerInternals.currentOutputWatermarkTime());
+                return false;
+              } else {
+                return true;
+              }
+            }
+          });
+      return nonLateElements;
+    }
+
+    /** Is {@code window} expired w.r.t. the garbage collection watermark? */
+    private boolean canDropDueToExpiredWindow(BoundedWindow window) {
+      Instant inputWM = timerInternals.currentInputWatermarkTime();
+      return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
new file mode 100644
index 0000000..3e51dfb
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+
+/**
+ * Tracks which windows have non-empty panes. Specifically, which windows have new elements since
+ * their last triggering.
+ *
+ * @param <W> The kind of windows being tracked.
+ */
+public abstract class NonEmptyPanes<K, W extends BoundedWindow> {
+
+  static <K, W extends BoundedWindow> NonEmptyPanes<K, W> create(
+      WindowingStrategy<?, W> strategy, ReduceFn<K, ?, ?, W> reduceFn) {
+    if (strategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
+      return new DiscardingModeNonEmptyPanes<>(reduceFn);
+    } else {
+      return new GeneralNonEmptyPanes<>();
+    }
+  }
+
+  /**
+   * Record that some content has been added to the window in {@code context}, and therefore the
+   * current pane is not empty.
+   */
+  public abstract void recordContent(StateAccessor<K> context);
+
+  /**
+   * Record that the given pane is empty.
+   */
+  public abstract void clearPane(StateAccessor<K> state);
+
+  /**
+   * Return true if the current pane for the window in {@code context} is empty.
+   */
+  public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
+
+  /**
+   * Prefetch in preparation for merging.
+   */
+  public abstract void prefetchOnMerge(MergingStateAccessor<K, W> state);
+
+  /**
+   * Eagerly merge backing state.
+   */
+  public abstract void onMerge(MergingStateAccessor<K, W> context);
+
+  /**
+   * An implementation of {@code NonEmptyPanes} optimized for use with discarding mode. Uses the
+   * presence of data in the accumulation buffer to record non-empty panes.
+   */
+  private static class DiscardingModeNonEmptyPanes<K, W extends BoundedWindow>
+      extends NonEmptyPanes<K, W> {
+
+    private ReduceFn<K, ?, ?, W> reduceFn;
+
+    private DiscardingModeNonEmptyPanes(ReduceFn<K, ?, ?, W> reduceFn) {
+      this.reduceFn = reduceFn;
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+      return reduceFn.isEmpty(state);
+    }
+
+    @Override
+    public void recordContent(StateAccessor<K> state) {
+      // Nothing to do -- the reduceFn is tracking contents
+    }
+
+    @Override
+    public void clearPane(StateAccessor<K> state) {
+      // Nothing to do -- the reduceFn is tracking contents
+    }
+
+    @Override
+    public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
+      // Nothing to do -- the reduceFn is tracking contents
+    }
+
+    @Override
+    public void onMerge(MergingStateAccessor<K, W> context) {
+      // Nothing to do -- the reduceFn is tracking contents
+    }
+  }
+
+  /**
+   * An implementation of {@code NonEmptyPanes} for general use.
+   */
+  private static class GeneralNonEmptyPanes<K, W extends BoundedWindow>
+      extends NonEmptyPanes<K, W> {
+
+    private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
+        PANE_ADDITIONS_TAG =
+        StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+            "count", VarLongCoder.of(), new Sum.SumLongFn()));
+
+    @Override
+    public void recordContent(StateAccessor<K> state) {
+      state.access(PANE_ADDITIONS_TAG).add(1L);
+    }
+
+    @Override
+    public void clearPane(StateAccessor<K> state) {
+      state.access(PANE_ADDITIONS_TAG).clear();
+    }
+
+    @Override
+    public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+      return state.access(PANE_ADDITIONS_TAG).isEmpty();
+    }
+
+    @Override
+    public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
+      StateMerging.prefetchCombiningValues(state, PANE_ADDITIONS_TAG);
+    }
+
+    @Override
+    public void onMerge(MergingStateAccessor<K, W> context) {
+      StateMerging.mergeCombiningValues(context, PANE_ADDITIONS_TAG);
+    }
+  }
+}


[06/50] [abbrv] incubator-beam git commit: Fix javadoc in Kinesis

Posted by dh...@apache.org.
Fix javadoc in Kinesis


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

Branch: refs/heads/gearpump-runner
Commit: 973081e63378bcc2cfcc8f7ecee519ebe8addb8c
Parents: aee5fbf
Author: Dan Halperin <dh...@google.com>
Authored: Fri Aug 26 15:20:44 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/kinesis/TransientKinesisException.java  | 2 +-
 .../java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java     | 2 +-
 .../java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java    | 2 +-
 .../java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java   | 2 +-
 .../java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java     | 2 +-
 .../org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java     | 2 +-
 .../test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java   | 2 +-
 .../org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java   | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java
index a1a974b..57ad8a8 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.io.kinesis;
 import com.amazonaws.AmazonServiceException;
 
 /**
- * Created by p.pastuszka on 21.06.2016.
+ * A transient exception thrown by Kinesis.
  */
 class TransientKinesisException extends Exception {
     public TransientKinesisException(String s, AmazonServiceException e) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
index b007fa4..046c9d9 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
@@ -74,7 +74,7 @@ import org.apache.commons.lang.builder.EqualsBuilder;
 import org.joda.time.Instant;
 
 /**
- * Created by p.pastuszka on 21.07.2016.
+ * Mock implemenation of {@link AmazonKinesis} for testing.
  */
 class AmazonKinesisMock implements AmazonKinesis {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
index cb0d0e2..20e8372 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
@@ -21,7 +21,7 @@ import java.util.NoSuchElementException;
 import org.junit.Test;
 
 /**
- * Created by ppastuszka on 12.12.15.
+ * Tests {@link CustomOptional}.
  */
 public class CustomOptionalTest {
     @Test(expected = NoSuchElementException.class)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
index 304220b..f0ab46c 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
@@ -32,7 +32,7 @@ import org.joda.time.DateTime;
 import org.junit.Test;
 
 /**
- * Created by p.pastuszka on 22.07.2016.
+ * Tests {@link AmazonKinesisMock}.
  */
 public class KinesisMockReadTest {
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
index 29a24821..3111029 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
@@ -30,7 +30,7 @@ import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
 
 /**
- * Created by ppastuszka on 12.12.15.
+ * Tests {@link KinesisReader}.
  */
 @RunWith(MockitoJUnitRunner.class)
 public class KinesisReaderTest {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
index d301f25..8771c86 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
@@ -23,7 +23,7 @@ import org.joda.time.Instant;
 import org.junit.Test;
 
 /**
- * Created by p.pastuszka on 20.07.2016.
+ * Tests {@link KinesisRecordCoder}.
  */
 public class KinesisRecordCoderTest {
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
index aedc89e..f032eea 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.junit.Test;
 
 /**
- * Created by ppastuszka on 12.12.15.
+ * Tests {@link RoundRobin}.
  */
 public class RoundRobinTest {
     @Test(expected = IllegalArgumentException.class)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/973081e6/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
index 585b884..49e806d 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
@@ -36,7 +36,7 @@ import org.mockito.runners.MockitoJUnitRunner;
 import org.mockito.stubbing.Answer;
 
 /**
- * Created by ppastuszka on 12.12.15.
+ * Tests {@link ShardRecordsIterator}.
  */
 @RunWith(MockitoJUnitRunner.class)
 public class ShardRecordsIteratorTest {


[34/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
new file mode 100644
index 0000000..114f5e6
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowTracing;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.joda.time.Instant;
+
+/**
+ * Determine the timing and other properties of a new pane for a given computation, key and window.
+ * Incorporates any previous pane, whether the pane has been produced because an
+ * on-time {@link AfterWatermark} trigger firing, and the relation between the element's timestamp
+ * and the current output watermark.
+ */
+public class PaneInfoTracker {
+  private TimerInternals timerInternals;
+
+  public PaneInfoTracker(TimerInternals timerInternals) {
+    this.timerInternals = timerInternals;
+  }
+
+  @VisibleForTesting
+  static final StateTag<Object, ValueState<PaneInfo>> PANE_INFO_TAG =
+      StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE));
+
+  public void clear(StateAccessor<?> state) {
+    state.access(PANE_INFO_TAG).clear();
+  }
+
+  /**
+   * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane
+   * info includes the timing for the pane, who's calculation is quite subtle.
+   *
+   * @param isFinal should be {@code true} only if the triggering machinery can guarantee
+   * no further firings for the
+   */
+  public ReadableState<PaneInfo> getNextPaneInfo(
+      ReduceFn<?, ?, ?, ?>.Context context, final boolean isFinal) {
+    final Object key = context.key();
+    final ReadableState<PaneInfo> previousPaneFuture =
+        context.state().access(PaneInfoTracker.PANE_INFO_TAG);
+    final Instant windowMaxTimestamp = context.window().maxTimestamp();
+
+    return new ReadableState<PaneInfo>() {
+      @Override
+      @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
+          justification = "prefetch side effect")
+      public ReadableState<PaneInfo> readLater() {
+        previousPaneFuture.readLater();
+        return this;
+      }
+
+      @Override
+      public PaneInfo read() {
+        PaneInfo previousPane = previousPaneFuture.read();
+        return describePane(key, windowMaxTimestamp, previousPane, isFinal);
+      }
+    };
+  }
+
+  public void storeCurrentPaneInfo(ReduceFn<?, ?, ?, ?>.Context context, PaneInfo currentPane) {
+    context.state().access(PANE_INFO_TAG).write(currentPane);
+  }
+
+  private <W> PaneInfo describePane(
+      Object key, Instant windowMaxTimestamp, PaneInfo previousPane, boolean isFinal) {
+    boolean isFirst = previousPane == null;
+    Timing previousTiming = isFirst ? null : previousPane.getTiming();
+    long index = isFirst ? 0 : previousPane.getIndex() + 1;
+    long nonSpeculativeIndex = isFirst ? 0 : previousPane.getNonSpeculativeIndex() + 1;
+    Instant outputWM = timerInternals.currentOutputWatermarkTime();
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+    // True if it is not possible to assign the element representing this pane a timestamp
+    // which will make an ON_TIME pane for any following computation.
+    // Ie true if the element's latest possible timestamp is before the current output watermark.
+    boolean isLateForOutput = outputWM != null && windowMaxTimestamp.isBefore(outputWM);
+
+    // True if all emitted panes (if any) were EARLY panes.
+    // Once the ON_TIME pane has fired, all following panes must be considered LATE even
+    // if the output watermark is behind the end of the window.
+    boolean onlyEarlyPanesSoFar = previousTiming == null || previousTiming == Timing.EARLY;
+
+    // True is the input watermark hasn't passed the window's max timestamp.
+    boolean isEarlyForInput = !inputWM.isAfter(windowMaxTimestamp);
+
+    Timing timing;
+    if (isLateForOutput || !onlyEarlyPanesSoFar) {
+      // The output watermark has already passed the end of this window, or we have already
+      // emitted a non-EARLY pane. Irrespective of how this pane was triggered we must
+      // consider this pane LATE.
+      timing = Timing.LATE;
+    } else if (isEarlyForInput) {
+      // This is an EARLY firing.
+      timing = Timing.EARLY;
+      nonSpeculativeIndex = -1;
+    } else {
+      // This is the unique ON_TIME firing for the window.
+      timing = Timing.ON_TIME;
+    }
+
+    WindowTracing.debug(
+        "describePane: {} pane (prev was {}) for key:{}; windowMaxTimestamp:{}; "
+        + "inputWatermark:{}; outputWatermark:{}; isLateForOutput:{}",
+        timing, previousTiming, key, windowMaxTimestamp, inputWM, outputWM, isLateForOutput);
+
+    if (previousPane != null) {
+      // Timing transitions should follow EARLY* ON_TIME? LATE*
+      switch (previousTiming) {
+        case EARLY:
+          checkState(
+              timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE,
+              "EARLY cannot transition to %s", timing);
+          break;
+        case ON_TIME:
+          checkState(
+              timing == Timing.LATE, "ON_TIME cannot transition to %s", timing);
+          break;
+        case LATE:
+          checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
+          break;
+        case UNKNOWN:
+          break;
+      }
+      checkState(!previousPane.isLast(), "Last pane was not last after all.");
+    }
+
+    return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
new file mode 100644
index 0000000..fcdff3b
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.NoSuchElementException;
+import org.apache.beam.sdk.util.common.Reiterator;
+
+/**
+ * A {@link Reiterator} that supports one-element lookahead during iteration.
+ *
+ * @param <T> the type of elements returned by this iterator
+ */
+public final class PeekingReiterator<T> implements Reiterator<T> {
+  private T nextElement;
+  private boolean nextElementComputed;
+  private final Reiterator<T> iterator;
+
+  public PeekingReiterator(Reiterator<T> iterator) {
+    this.iterator = checkNotNull(iterator);
+  }
+
+  PeekingReiterator(PeekingReiterator<T> it) {
+    this.iterator = checkNotNull(checkNotNull(it).iterator.copy());
+    this.nextElement = it.nextElement;
+    this.nextElementComputed = it.nextElementComputed;
+  }
+
+  @Override
+  public boolean hasNext() {
+    computeNext();
+    return nextElementComputed;
+  }
+
+  @Override
+  public T next() {
+    T result = peek();
+    nextElementComputed = false;
+    return result;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>If {@link #peek} is called, {@code remove} is disallowed until
+   * {@link #next} has been subsequently called.
+   */
+  @Override
+  public void remove() {
+    checkState(!nextElementComputed,
+        "After peek(), remove() is disallowed until next() is called");
+    iterator.remove();
+  }
+
+  @Override
+  public PeekingReiterator<T> copy() {
+    return new PeekingReiterator<>(this);
+  }
+
+  /**
+   * Returns the element that would be returned by {@link #next}, without
+   * actually consuming the element.
+   * @throws NoSuchElementException if there is no next element
+   */
+  public T peek() {
+    computeNext();
+    if (!nextElementComputed) {
+      throw new NoSuchElementException();
+    }
+    return nextElement;
+  }
+
+  private void computeNext() {
+    if (nextElementComputed) {
+      return;
+    }
+    if (!iterator.hasNext()) {
+      return;
+    }
+    nextElement = iterator.next();
+    nextElementComputed = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java
new file mode 100644
index 0000000..deeac3c
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollectionView;
+
+/**
+ * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
+ * them via the {@link #processElementInReadyWindows(WindowedValue)}.
+ */
+public class PushbackSideInputDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+  private final DoFnRunner<InputT, OutputT> underlying;
+  private final Collection<PCollectionView<?>> views;
+  private final ReadyCheckingSideInputReader sideInputReader;
+
+  private Set<BoundedWindow> notReadyWindows;
+
+  public static <InputT, OutputT> PushbackSideInputDoFnRunner<InputT, OutputT> create(
+      DoFnRunner<InputT, OutputT> underlying,
+      Collection<PCollectionView<?>> views,
+      ReadyCheckingSideInputReader sideInputReader) {
+    return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader);
+  }
+
+  private PushbackSideInputDoFnRunner(
+      DoFnRunner<InputT, OutputT> underlying,
+      Collection<PCollectionView<?>> views,
+      ReadyCheckingSideInputReader sideInputReader) {
+    this.underlying = underlying;
+    this.views = views;
+    this.sideInputReader = sideInputReader;
+  }
+
+  @Override
+  public void startBundle() {
+    notReadyWindows = new HashSet<>();
+    underlying.startBundle();
+  }
+
+  /**
+   * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element
+   * for each window the element is in that is ready.
+   *
+   * @param elem the element to process in all ready windows
+   * @return each element that could not be processed because it requires a side input window
+   * that is not ready.
+   */
+  public Iterable<WindowedValue<InputT>> processElementInReadyWindows(WindowedValue<InputT> elem) {
+    if (views.isEmpty()) {
+      processElement(elem);
+      return Collections.emptyList();
+    }
+    ImmutableList.Builder<WindowedValue<InputT>> pushedBack = ImmutableList.builder();
+    for (WindowedValue<InputT> windowElem : elem.explodeWindows()) {
+      BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows());
+      boolean isReady = !notReadyWindows.contains(mainInputWindow);
+      for (PCollectionView<?> view : views) {
+        BoundedWindow sideInputWindow =
+            view.getWindowingStrategyInternal()
+                .getWindowFn()
+                .getSideInputWindow(mainInputWindow);
+        if (!sideInputReader.isReady(view, sideInputWindow)) {
+          isReady = false;
+          break;
+        }
+      }
+      if (isReady) {
+        processElement(windowElem);
+      } else {
+        notReadyWindows.add(mainInputWindow);
+        pushedBack.add(windowElem);
+      }
+    }
+    return pushedBack.build();
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> elem) {
+    underlying.processElement(elem);
+  }
+
+  /**
+   * Call the underlying {@link DoFnRunner#finishBundle()}.
+   */
+  @Override
+  public void finishBundle() {
+    notReadyWindows = null;
+    underlying.finishBundle();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java
new file mode 100644
index 0000000..bb20226
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.Timers;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.joda.time.Instant;
+
+/**
+ * Specification for processing to happen after elements have been grouped by key.
+ *
+ * @param <K> The type of key being processed.
+ * @param <InputT> The type of input values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W> The type of windows this operates on.
+ */
+public abstract class ReduceFn<K, InputT, OutputT, W extends BoundedWindow>
+    implements Serializable {
+
+  /** Information accessible to all the processing methods in this {@code ReduceFn}. */
+  public abstract class Context {
+    /** Return the key that is being processed. */
+    public abstract K key();
+
+    /** The window that is being processed. */
+    public abstract W window();
+
+    /** Access the current {@link WindowingStrategy}. */
+    public abstract WindowingStrategy<?, W> windowingStrategy();
+
+    /** Return the interface for accessing state. */
+    public abstract StateAccessor<K> state();
+
+    /** Return the interface for accessing timers. */
+    public abstract Timers timers();
+  }
+
+  /** Information accessible within {@link #processValue}. */
+  public abstract class ProcessValueContext extends Context {
+    /** Return the actual value being processed. */
+    public abstract InputT value();
+
+    /** Return the timestamp associated with the value. */
+    public abstract Instant timestamp();
+  }
+
+  /** Information accessible within {@link #onMerge}. */
+  public abstract class OnMergeContext extends Context {
+    /** Return the interface for accessing state. */
+    @Override
+    public abstract MergingStateAccessor<K, W> state();
+  }
+
+  /** Information accessible within {@link #onTrigger}. */
+  public abstract class OnTriggerContext extends Context {
+    /** Returns the {@link PaneInfo} for the trigger firing being processed. */
+    public abstract PaneInfo paneInfo();
+
+    /** Output the given value in the current window. */
+    public abstract void output(OutputT value);
+  }
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Called for each value of type {@code InputT} associated with the current key.
+   */
+  public abstract void processValue(ProcessValueContext c) throws Exception;
+
+  /**
+   * Called when windows are merged.
+   */
+  public abstract void onMerge(OnMergeContext context) throws Exception;
+
+  /**
+   * Called when triggers fire.
+   *
+   * <p>Implementations of {@link ReduceFn} should call {@link OnTriggerContext#output} to emit
+   * any results that should be included in the pane produced by this trigger firing.
+   */
+  public abstract void onTrigger(OnTriggerContext context) throws Exception;
+
+  /**
+   * Called before {@link #onMerge} is invoked to provide an opportunity to prefetch any needed
+   * state.
+   *
+   * @param c Context to use prefetch from.
+   */
+  public void prefetchOnMerge(MergingStateAccessor<K, W> c) throws Exception {}
+
+  /**
+   * Called before {@link #onTrigger} is invoked to provide an opportunity to prefetch any needed
+   * state.
+   *
+   * @param context Context to use prefetch from.
+   */
+  public void prefetchOnTrigger(StateAccessor<K> context) {}
+
+  /**
+   * Called to clear any persisted state that the {@link ReduceFn} may be holding. This will be
+   * called when the windowing is closing and will receive no future interactions.
+   */
+  public abstract void clearState(Context context) throws Exception;
+
+  /**
+   * Returns true if the there is no buffered state.
+   */
+  public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
new file mode 100644
index 0000000..2043f14
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
@@ -0,0 +1,499 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Collection;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.ActiveWindowSet;
+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.Timers;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.joda.time.Instant;
+
+/**
+ * Factory for creating instances of the various {@link ReduceFn} contexts.
+ */
+class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
+  public interface OnTriggerCallbacks<OutputT> {
+    void output(OutputT toOutput);
+  }
+
+  private final K key;
+  private final ReduceFn<K, InputT, OutputT, W> reduceFn;
+  private final WindowingStrategy<?, W> windowingStrategy;
+  private final StateInternals<K> stateInternals;
+  private final ActiveWindowSet<W> activeWindows;
+  private final TimerInternals timerInternals;
+  private final WindowingInternals<?, ?> windowingInternals;
+  private final PipelineOptions options;
+
+  ReduceFnContextFactory(K key, ReduceFn<K, InputT, OutputT, W> reduceFn,
+      WindowingStrategy<?, W> windowingStrategy, StateInternals<K> stateInternals,
+      ActiveWindowSet<W> activeWindows, TimerInternals timerInternals,
+      WindowingInternals<?, ?> windowingInternals, PipelineOptions options) {
+    this.key = key;
+    this.reduceFn = reduceFn;
+    this.windowingStrategy = windowingStrategy;
+    this.stateInternals = stateInternals;
+    this.activeWindows = activeWindows;
+    this.timerInternals = timerInternals;
+    this.windowingInternals = windowingInternals;
+    this.options = options;
+  }
+
+  /** Where should we look for state associated with a given window? */
+  public static enum StateStyle {
+    /** All state is associated with the window itself. */
+    DIRECT,
+    /** State is associated with the 'state address' windows tracked by the active window set. */
+    RENAMED
+  }
+
+  private StateAccessorImpl<K, W> stateAccessor(W window, StateStyle style) {
+    return new StateAccessorImpl<K, W>(
+        activeWindows, windowingStrategy.getWindowFn().windowCoder(),
+        stateInternals, StateContexts.createFromComponents(options, windowingInternals, window),
+        style);
+  }
+
+  public ReduceFn<K, InputT, OutputT, W>.Context base(W window, StateStyle style) {
+    return new ContextImpl(stateAccessor(window, style));
+  }
+
+  public ReduceFn<K, InputT, OutputT, W>.ProcessValueContext forValue(
+      W window, InputT value, Instant timestamp, StateStyle style) {
+    return new ProcessValueContextImpl(stateAccessor(window, style), value, timestamp);
+  }
+
+  public ReduceFn<K, InputT, OutputT, W>.OnTriggerContext forTrigger(W window,
+      ReadableState<PaneInfo> pane, StateStyle style, OnTriggerCallbacks<OutputT> callbacks) {
+    return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks);
+  }
+
+  public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forMerge(
+      Collection<W> activeToBeMerged, W mergeResult, StateStyle style) {
+    return new OnMergeContextImpl(
+        new MergingStateAccessorImpl<K, W>(activeWindows,
+            windowingStrategy.getWindowFn().windowCoder(),
+            stateInternals, style, activeToBeMerged, mergeResult));
+  }
+
+  public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forPremerge(W window) {
+    return new OnPremergeContextImpl(new PremergingStateAccessorImpl<K, W>(
+        activeWindows, windowingStrategy.getWindowFn().windowCoder(), stateInternals, window));
+  }
+
+  private class TimersImpl implements Timers {
+    private final StateNamespace namespace;
+
+    public TimersImpl(StateNamespace namespace) {
+      checkArgument(namespace instanceof WindowNamespace);
+      this.namespace = namespace;
+    }
+
+    @Override
+    public void setTimer(Instant timestamp, TimeDomain timeDomain) {
+      timerInternals.setTimer(TimerData.of(namespace, timestamp, timeDomain));
+    }
+
+    @Override
+    public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
+      timerInternals.deleteTimer(TimerData.of(namespace, timestamp, timeDomain));
+    }
+
+    @Override
+    public Instant currentProcessingTime() {
+      return timerInternals.currentProcessingTime();
+    }
+
+    @Override
+    @Nullable
+    public Instant currentSynchronizedProcessingTime() {
+      return timerInternals.currentSynchronizedProcessingTime();
+    }
+
+    @Override
+    public Instant currentEventTime() {
+      return timerInternals.currentInputWatermarkTime();
+    }
+  }
+
+  // ======================================================================
+  // StateAccessors
+  // ======================================================================
+  static class StateAccessorImpl<K, W extends BoundedWindow> implements StateAccessor<K> {
+
+
+    protected final ActiveWindowSet<W> activeWindows;
+    protected final StateContext<W> context;
+    protected final StateNamespace windowNamespace;
+    protected final Coder<W> windowCoder;
+    protected final StateInternals<K> stateInternals;
+    protected final StateStyle style;
+
+    public StateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+        StateInternals<K> stateInternals, StateContext<W> context, StateStyle style) {
+
+      this.activeWindows = activeWindows;
+      this.windowCoder = windowCoder;
+      this.stateInternals = stateInternals;
+      this.context = checkNotNull(context);
+      this.windowNamespace = namespaceFor(context.window());
+      this.style = style;
+    }
+
+    protected StateNamespace namespaceFor(W window) {
+      return StateNamespaces.window(windowCoder, window);
+    }
+
+    protected StateNamespace windowNamespace() {
+      return windowNamespace;
+    }
+
+    W window() {
+      return context.window();
+    }
+
+    StateNamespace namespace() {
+      return windowNamespace();
+    }
+
+    @Override
+    public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
+      switch (style) {
+        case DIRECT:
+          return stateInternals.state(windowNamespace(), address, context);
+        case RENAMED:
+          return stateInternals.state(
+              namespaceFor(activeWindows.writeStateAddress(context.window())), address, context);
+      }
+      throw new RuntimeException(); // cases are exhaustive.
+    }
+  }
+
+  static class MergingStateAccessorImpl<K, W extends BoundedWindow>
+      extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
+    private final Collection<W> activeToBeMerged;
+
+    public MergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+        StateInternals<K> stateInternals, StateStyle style, Collection<W> activeToBeMerged,
+        W mergeResult) {
+      super(activeWindows, windowCoder, stateInternals,
+          StateContexts.windowOnly(mergeResult), style);
+      this.activeToBeMerged = activeToBeMerged;
+    }
+
+    @Override
+    public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
+      switch (style) {
+        case DIRECT:
+          return stateInternals.state(windowNamespace(), address, context);
+        case RENAMED:
+          return stateInternals.state(
+              namespaceFor(activeWindows.mergedWriteStateAddress(
+                  activeToBeMerged, context.window())),
+              address,
+              context);
+      }
+      throw new RuntimeException(); // cases are exhaustive.
+    }
+
+    @Override
+    public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
+        StateTag<? super K, StateT> address) {
+      ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
+      for (W mergingWindow : activeToBeMerged) {
+        StateNamespace namespace = null;
+        switch (style) {
+          case DIRECT:
+            namespace = namespaceFor(mergingWindow);
+            break;
+          case RENAMED:
+            namespace = namespaceFor(activeWindows.writeStateAddress(mergingWindow));
+            break;
+        }
+        checkNotNull(namespace); // cases are exhaustive.
+        builder.put(mergingWindow, stateInternals.state(namespace, address, context));
+      }
+      return builder.build();
+    }
+  }
+
+  static class PremergingStateAccessorImpl<K, W extends BoundedWindow>
+      extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
+    public PremergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+        StateInternals<K> stateInternals, W window) {
+      super(activeWindows, windowCoder, stateInternals,
+          StateContexts.windowOnly(window), StateStyle.RENAMED);
+    }
+
+    Collection<W> mergingWindows() {
+      return activeWindows.readStateAddresses(context.window());
+    }
+
+    @Override
+    public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
+        StateTag<? super K, StateT> address) {
+      ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
+      for (W stateAddressWindow : activeWindows.readStateAddresses(context.window())) {
+        StateT stateForWindow =
+            stateInternals.state(namespaceFor(stateAddressWindow), address, context);
+        builder.put(stateAddressWindow, stateForWindow);
+      }
+      return builder.build();
+    }
+  }
+
+  // ======================================================================
+  // Contexts
+  // ======================================================================
+
+  private class ContextImpl extends ReduceFn<K, InputT, OutputT, W>.Context {
+    private final StateAccessorImpl<K, W> state;
+    private final TimersImpl timers;
+
+    private ContextImpl(StateAccessorImpl<K, W> state) {
+      reduceFn.super();
+      this.state = state;
+      this.timers = new TimersImpl(state.namespace());
+    }
+
+    @Override
+    public K key() {
+      return key;
+    }
+
+    @Override
+    public W window() {
+      return state.window();
+    }
+
+    @Override
+    public WindowingStrategy<?, W> windowingStrategy() {
+      return windowingStrategy;
+    }
+
+    @Override
+    public StateAccessor<K> state() {
+      return state;
+    }
+
+    @Override
+    public Timers timers() {
+      return timers;
+    }
+  }
+
+  private class ProcessValueContextImpl
+      extends ReduceFn<K, InputT, OutputT, W>.ProcessValueContext {
+    private final InputT value;
+    private final Instant timestamp;
+    private final StateAccessorImpl<K, W> state;
+    private final TimersImpl timers;
+
+    private ProcessValueContextImpl(StateAccessorImpl<K, W> state,
+        InputT value, Instant timestamp) {
+      reduceFn.super();
+      this.state = state;
+      this.value = value;
+      this.timestamp = timestamp;
+      this.timers = new TimersImpl(state.namespace());
+    }
+
+    @Override
+    public K key() {
+      return key;
+    }
+
+    @Override
+    public W window() {
+      return state.window();
+    }
+
+    @Override
+    public WindowingStrategy<?, W> windowingStrategy() {
+      return windowingStrategy;
+    }
+
+    @Override
+    public StateAccessor<K> state() {
+      return state;
+    }
+
+    @Override
+    public InputT value() {
+      return value;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public Timers timers() {
+      return timers;
+    }
+  }
+
+  private class OnTriggerContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnTriggerContext {
+    private final StateAccessorImpl<K, W> state;
+    private final ReadableState<PaneInfo> pane;
+    private final OnTriggerCallbacks<OutputT> callbacks;
+    private final TimersImpl timers;
+
+    private OnTriggerContextImpl(StateAccessorImpl<K, W> state, ReadableState<PaneInfo> pane,
+        OnTriggerCallbacks<OutputT> callbacks) {
+      reduceFn.super();
+      this.state = state;
+      this.pane = pane;
+      this.callbacks = callbacks;
+      this.timers = new TimersImpl(state.namespace());
+    }
+
+    @Override
+    public K key() {
+      return key;
+    }
+
+    @Override
+    public W window() {
+      return state.window();
+    }
+
+    @Override
+    public WindowingStrategy<?, W> windowingStrategy() {
+      return windowingStrategy;
+    }
+
+    @Override
+    public StateAccessor<K> state() {
+      return state;
+    }
+
+    @Override
+    public PaneInfo paneInfo() {
+      return pane.read();
+    }
+
+    @Override
+    public void output(OutputT value) {
+      callbacks.output(value);
+    }
+
+    @Override
+    public Timers timers() {
+      return timers;
+    }
+  }
+
+  private class OnMergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
+    private final MergingStateAccessorImpl<K, W> state;
+    private final TimersImpl timers;
+
+    private OnMergeContextImpl(MergingStateAccessorImpl<K, W> state) {
+      reduceFn.super();
+      this.state = state;
+      this.timers = new TimersImpl(state.namespace());
+    }
+
+    @Override
+    public K key() {
+      return key;
+    }
+
+    @Override
+    public WindowingStrategy<?, W> windowingStrategy() {
+      return windowingStrategy;
+    }
+
+    @Override
+    public MergingStateAccessor<K, W> state() {
+      return state;
+    }
+
+    @Override
+    public W window() {
+      return state.window();
+    }
+
+    @Override
+    public Timers timers() {
+      return timers;
+    }
+  }
+
+  private class OnPremergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
+    private final PremergingStateAccessorImpl<K, W> state;
+    private final TimersImpl timers;
+
+    private OnPremergeContextImpl(PremergingStateAccessorImpl<K, W> state) {
+      reduceFn.super();
+      this.state = state;
+      this.timers = new TimersImpl(state.namespace());
+    }
+
+    @Override
+    public K key() {
+      return key;
+    }
+
+    @Override
+    public WindowingStrategy<?, W> windowingStrategy() {
+      return windowingStrategy;
+    }
+
+    @Override
+    public MergingStateAccessor<K, W> state() {
+      return state;
+    }
+
+    @Override
+    public W window() {
+      return state.window();
+    }
+
+    @Override
+    public Timers timers() {
+      return timers;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
new file mode 100644
index 0000000..96d764a
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.runners.core.ReduceFnContextFactory.OnTriggerCallbacks;
+import org.apache.beam.runners.core.ReduceFnContextFactory.StateStyle;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.ActiveWindowSet;
+import org.apache.beam.sdk.util.MergingActiveWindowSet;
+import org.apache.beam.sdk.util.NonMergingActiveWindowSet;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TriggerContextFactory;
+import org.apache.beam.sdk.util.WindowTracing;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the
+ * {@link PCollection} by key.
+ *
+ * <p>The {@link #onTrigger} relies on a {@link TriggerRunner} to manage the execution of
+ * the triggering logic. The {@code ReduceFnRunner}s responsibilities are:
+ *
+ * <ul>
+ * <li>Tracking the windows that are active (have buffered data) as elements arrive and
+ * triggers are fired.
+ * <li>Holding the watermark based on the timestamps of elements in a pane and releasing it
+ * when the trigger fires.
+ * <li>Calling the appropriate callbacks on {@link ReduceFn} based on trigger execution, timer
+ * firings, etc, and providing appropriate contexts to the {@link ReduceFn} for actions
+ * such as output.
+ * <li>Scheduling garbage collection of state associated with a specific window, and making that
+ * happen when the appropriate timer fires.
+ * </ul>
+ *
+ * @param <K>       The type of key being processed.
+ * @param <InputT>  The type of values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W>       The type of windows this operates on.
+ */
+public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
+
+  /**
+   * The {@link ReduceFnRunner} depends on most aspects of the {@link WindowingStrategy}.
+   *
+   * <ul>
+   * <li>It runs the trigger from the {@link WindowingStrategy}.</li>
+   * <li>It merges windows according to the {@link WindowingStrategy}.</li>
+   * <li>It chooses how to track active windows and clear out expired windows
+   * according to the {@link WindowingStrategy}, based on the allowed lateness and
+   * whether windows can merge.</li>
+   * <li>It decides whether to emit empty final panes according to whether the
+   * {@link WindowingStrategy} requires it.<li>
+   * <li>It uses discarding or accumulation mode according to the {@link WindowingStrategy}.</li>
+   * </ul>
+   */
+  private final WindowingStrategy<Object, W> windowingStrategy;
+
+  private final OutputWindowedValue<KV<K, OutputT>> outputter;
+
+  private final StateInternals<K> stateInternals;
+
+  private final Aggregator<Long, Long> droppedDueToClosedWindow;
+
+  private final K key;
+
+  /**
+   * Track which windows are still active and the 'state address' windows which hold their state.
+   *
+   * <ul>
+   * <li>State: Global map for all active windows for this computation and key.
+   * <li>Lifetime: Cleared when no active windows need to be tracked. A window lives within
+   * the active window set until its trigger is closed or the window is garbage collected.
+   * </ul>
+   */
+  private final ActiveWindowSet<W> activeWindows;
+
+  /**
+   * Always a {@link SystemReduceFn}.
+   *
+   * <ul>
+   * <li>State: A bag of accumulated values, or the intermediate result of a combiner.
+   * <li>State style: RENAMED
+   * <li>Merging: Concatenate or otherwise combine the state from each merged window.
+   * <li>Lifetime: Cleared when a pane fires if DISCARDING_FIRED_PANES. Otherwise cleared
+   * when trigger is finished or when the window is garbage collected.
+   * </ul>
+   */
+  private final ReduceFn<K, InputT, OutputT, W> reduceFn;
+
+  /**
+   * Manage the setting and firing of timer events.
+   *
+   * <ul>
+   * <li>Merging: End-of-window and garbage collection timers are cancelled when windows are
+   * merged away. Timers created by triggers are never garbage collected and are left to
+   * fire and be ignored.
+   * <li>Lifetime: Timers automatically disappear after they fire.
+   * </ul>
+   */
+  private final TimerInternals timerInternals;
+
+  /**
+   * Manage the execution and state for triggers.
+   *
+   * <ul>
+   * <li>State: Tracks which sub-triggers have finished, and any additional state needed to
+   * determine when the trigger should fire.
+   * <li>State style: DIRECT
+   * <li>Merging: Finished bits are explicitly managed. Other state is eagerly merged as
+   * needed.
+   * <li>Lifetime: Most trigger state is cleared when the final pane is emitted. However
+   * the finished bits are left behind and must be cleared when the window is
+   * garbage collected.
+   * </ul>
+   */
+  private final TriggerRunner<W> triggerRunner;
+
+  /**
+   * Store the output watermark holds for each window.
+   *
+   * <ul>
+   * <li>State: Bag of hold timestamps.
+   * <li>State style: RENAMED
+   * <li>Merging: Depending on {@link OutputTimeFn}, may need to be recalculated on merging.
+   * When a pane fires it may be necessary to add (back) an end-of-window or garbage collection
+   * hold.
+   * <li>Lifetime: Cleared when a pane fires or when the window is garbage collected.
+   * </ul>
+   */
+  private final WatermarkHold<W> watermarkHold;
+
+  private final ReduceFnContextFactory<K, InputT, OutputT, W> contextFactory;
+
+  /**
+   * Store the previously emitted pane (if any) for each window.
+   *
+   * <ul>
+   * <li>State: The previous {@link PaneInfo} passed to the user's {@code DoFn.ProcessElement}
+   * method, if any.
+   * <li>Style style: DIRECT
+   * <li>Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}.
+   * Cleared when window is merged away.
+   * <li>Lifetime: Cleared when trigger is closed or window is garbage collected.
+   * </ul>
+   */
+  private final PaneInfoTracker paneInfoTracker;
+
+  /**
+   * Store whether we've seen any elements for a window since the last pane was emitted.
+   *
+   * <ul>
+   * <li>State: Unless DISCARDING_FIRED_PANES, a count of number of elements added so far.
+   * <li>State style: RENAMED.
+   * <li>Merging: Counts are summed when windows are merged.
+   * <li>Lifetime: Cleared when pane fires or window is garbage collected.
+   * </ul>
+   */
+  private final NonEmptyPanes<K, W> nonEmptyPanes;
+
+  public ReduceFnRunner(
+      K key,
+      WindowingStrategy<?, W> windowingStrategy,
+      StateInternals<K> stateInternals,
+      TimerInternals timerInternals,
+      WindowingInternals<?, KV<K, OutputT>> windowingInternals,
+      Aggregator<Long, Long> droppedDueToClosedWindow,
+      ReduceFn<K, InputT, OutputT, W> reduceFn,
+      PipelineOptions options) {
+    this.key = key;
+    this.timerInternals = timerInternals;
+    this.paneInfoTracker = new PaneInfoTracker(timerInternals);
+    this.stateInternals = stateInternals;
+    this.outputter = new OutputViaWindowingInternals<>(windowingInternals);
+    this.droppedDueToClosedWindow = droppedDueToClosedWindow;
+    this.reduceFn = reduceFn;
+
+    @SuppressWarnings("unchecked")
+    WindowingStrategy<Object, W> objectWindowingStrategy =
+        (WindowingStrategy<Object, W>) windowingStrategy;
+    this.windowingStrategy = objectWindowingStrategy;
+
+    this.nonEmptyPanes = NonEmptyPanes.create(this.windowingStrategy, this.reduceFn);
+
+    // Note this may incur I/O to load persisted window set data.
+    this.activeWindows = createActiveWindowSet();
+
+    this.contextFactory =
+        new ReduceFnContextFactory<K, InputT, OutputT, W>(key, reduceFn, this.windowingStrategy,
+            stateInternals, this.activeWindows, timerInternals, windowingInternals, options);
+
+    this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy);
+    this.triggerRunner =
+        new TriggerRunner<>(
+            windowingStrategy.getTrigger(),
+            new TriggerContextFactory<>(
+                windowingStrategy.getWindowFn(), stateInternals, activeWindows));
+  }
+
+  private ActiveWindowSet<W> createActiveWindowSet() {
+    return windowingStrategy.getWindowFn().isNonMerging()
+        ? new NonMergingActiveWindowSet<W>()
+        : new MergingActiveWindowSet<W>(windowingStrategy.getWindowFn(), stateInternals);
+  }
+
+  @VisibleForTesting
+  boolean isFinished(W window) {
+    return triggerRunner.isClosed(contextFactory.base(window, StateStyle.DIRECT).state());
+  }
+
+  @VisibleForTesting
+  boolean hasNoActiveWindows() {
+    return activeWindows.getActiveAndNewWindows().isEmpty();
+  }
+
+  /**
+   * Incorporate {@code values} into the underlying reduce function, and manage holds, timers,
+   * triggers, and window merging.
+   *
+   * <p>The general strategy is:
+   * <ol>
+   * <li>Use {@link WindowedValue#getWindows} (itself determined using
+   * {@link WindowFn#assignWindows}) to determine which windows each element belongs to. Some
+   * of those windows will already have state associated with them. The rest are considered
+   * NEW.
+   * <li>Use {@link WindowFn#mergeWindows} to attempt to merge currently ACTIVE and NEW windows.
+   * Each NEW window will become either ACTIVE or be discardedL.
+   * (See {@link ActiveWindowSet} for definitions of these terms.)
+   * <li>If at all possible, eagerly substitute NEW windows with their ACTIVE state address
+   * windows before any state is associated with the NEW window. In the common case that
+   * windows for new elements are merged into existing ACTIVE windows then no additional
+   * storage or merging overhead will be incurred.
+   * <li>Otherwise, keep track of the state address windows for ACTIVE windows so that their
+   * states can be merged on-demand when a pane fires.
+   * <li>Process the element for each of the windows it's windows have been merged into according
+   * to {@link ActiveWindowSet}. Processing may require running triggers, setting timers,
+   * setting holds, and invoking {@link ReduceFn#onTrigger}.
+   * </ol>
+   */
+  public void processElements(Iterable<WindowedValue<InputT>> values) throws Exception {
+    // If an incoming element introduces a new window, attempt to merge it into an existing
+    // window eagerly.
+    Map<W, W> windowToMergeResult = collectAndMergeWindows(values);
+
+    Set<W> windowsToConsider = new HashSet<>();
+
+    // Process each element, using the updated activeWindows determined by collectAndMergeWindows.
+    for (WindowedValue<InputT> value : values) {
+      windowsToConsider.addAll(processElement(windowToMergeResult, value));
+    }
+
+    // Trigger output from any window for which the trigger is ready
+    for (W mergedWindow : windowsToConsider) {
+      ReduceFn<K, InputT, OutputT, W>.Context directContext =
+          contextFactory.base(mergedWindow, StateStyle.DIRECT);
+      ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
+          contextFactory.base(mergedWindow, StateStyle.RENAMED);
+      triggerRunner.prefetchShouldFire(mergedWindow, directContext.state());
+      emitIfAppropriate(directContext, renamedContext);
+    }
+
+    // We're all done with merging and emitting elements so can compress the activeWindow state.
+    // Any windows which are still NEW must have come in on a new element which was then discarded
+    // due to the window's trigger being closed. We can thus delete them.
+    activeWindows.cleanupTemporaryWindows();
+  }
+
+  public void persist() {
+    activeWindows.persist();
+  }
+
+  /**
+   * Extract the windows associated with the values, and invoke merge. Return a map
+   * from windows to the merge result window. If a window is not in the domain of
+   * the result map then it did not get merged into a different window.
+   */
+  private Map<W, W> collectAndMergeWindows(Iterable<WindowedValue<InputT>> values)
+      throws Exception {
+    // No-op if no merging can take place
+    if (windowingStrategy.getWindowFn().isNonMerging()) {
+      return ImmutableMap.of();
+    }
+
+    // Collect the windows from all elements (except those which are too late) and
+    // make sure they are already in the active window set or are added as NEW windows.
+    for (WindowedValue<?> value : values) {
+      for (BoundedWindow untypedWindow : value.getWindows()) {
+        @SuppressWarnings("unchecked")
+        W window = (W) untypedWindow;
+
+        // For backwards compat with pre 1.4 only.
+        // We may still have ACTIVE windows with multiple state addresses, representing
+        // a window who's state has not yet been eagerly merged.
+        // We'll go ahead and merge that state now so that we don't have to worry about
+        // this legacy case anywhere else.
+        if (activeWindows.isActive(window)) {
+          Set<W> stateAddressWindows = activeWindows.readStateAddresses(window);
+          if (stateAddressWindows.size() > 1) {
+            // This is a legacy window who's state has not been eagerly merged.
+            // Do that now.
+            ReduceFn<K, InputT, OutputT, W>.OnMergeContext premergeContext =
+                contextFactory.forPremerge(window);
+            reduceFn.onMerge(premergeContext);
+            watermarkHold.onMerge(premergeContext);
+            activeWindows.merged(window);
+          }
+        }
+
+        // Add this window as NEW if it is not currently ACTIVE.
+        // If we had already seen this window and closed its trigger, then the
+        // window will not be currently ACTIVE. It will then be added as NEW here,
+        // and fall into the merging logic as usual.
+        activeWindows.ensureWindowExists(window);
+      }
+    }
+
+    // Merge all of the active windows and retain a mapping from source windows to result windows.
+    Map<W, W> windowToMergeResult = new HashMap<>();
+    activeWindows.merge(new OnMergeCallback(windowToMergeResult));
+    return windowToMergeResult;
+  }
+
+  private class OnMergeCallback implements ActiveWindowSet.MergeCallback<W> {
+    private final Map<W, W> windowToMergeResult;
+
+    OnMergeCallback(Map<W, W> windowToMergeResult) {
+      this.windowToMergeResult = windowToMergeResult;
+    }
+
+    /**
+     * Return the subset of {@code windows} which are currently ACTIVE. We only need to worry
+     * about merging state from ACTIVE windows. NEW windows by definition have no existing state.
+     */
+    private List<W> activeWindows(Iterable<W> windows) {
+      List<W> active = new ArrayList<>();
+      for (W window : windows) {
+        if (activeWindows.isActive(window)) {
+          active.add(window);
+        }
+      }
+      return active;
+    }
+
+    /**
+     * Called from the active window set to indicate {@code toBeMerged} (of which only
+     * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) will later
+     * be merged into {@code mergeResult}.
+     */
+    @Override
+    public void prefetchOnMerge(
+        Collection<W> toBeMerged, W mergeResult) throws Exception {
+      List<W> activeToBeMerged = activeWindows(toBeMerged);
+      ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
+          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
+      ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
+          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
+
+      // Prefetch various state.
+      triggerRunner.prefetchForMerge(mergeResult, activeToBeMerged, directMergeContext.state());
+      reduceFn.prefetchOnMerge(renamedMergeContext.state());
+      watermarkHold.prefetchOnMerge(renamedMergeContext.state());
+      nonEmptyPanes.prefetchOnMerge(renamedMergeContext.state());
+    }
+
+    /**
+     * Called from the active window set to indicate {@code toBeMerged} (of which only
+     * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) are about
+     * to be merged into {@code mergeResult}.
+     */
+    @Override
+    public void onMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {
+      // Remember we have merged these windows.
+      for (W window : toBeMerged) {
+        windowToMergeResult.put(window, mergeResult);
+      }
+
+      // At this point activeWindows has NOT incorporated the results of the merge.
+      List<W> activeToBeMerged = activeWindows(toBeMerged);
+      ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
+          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
+      ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
+          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
+
+      // Run the reduceFn to perform any needed merging.
+      reduceFn.onMerge(renamedMergeContext);
+
+      // Merge the watermark holds.
+      watermarkHold.onMerge(renamedMergeContext);
+
+      // Merge non-empty pane state.
+      nonEmptyPanes.onMerge(renamedMergeContext.state());
+
+      // Have the trigger merge state as needed.
+      triggerRunner.onMerge(
+          directMergeContext.window(), directMergeContext.timers(), directMergeContext.state());
+
+      for (W active : activeToBeMerged) {
+        if (active.equals(mergeResult)) {
+          // Not merged away.
+          continue;
+        }
+        // Cleanup flavor A: Currently ACTIVE window is about to be merged away.
+        // Clear any state not already cleared by the onMerge calls above.
+        WindowTracing.debug("ReduceFnRunner.onMerge: Merging {} into {}", active, mergeResult);
+        ReduceFn<K, InputT, OutputT, W>.Context directClearContext =
+            contextFactory.base(active, StateStyle.DIRECT);
+        // No need for the end-of-window or garbage collection timers.
+        // We will establish a new end-of-window or garbage collection timer for the mergeResult
+        // window in processElement below. There must be at least one element for the mergeResult
+        // window since a new element with a new window must have triggered this onMerge.
+        cancelEndOfWindowAndGarbageCollectionTimers(directClearContext);
+        // We no longer care about any previous panes of merged away windows. The
+        // merge result window gets to start fresh if it is new.
+        paneInfoTracker.clear(directClearContext.state());
+      }
+    }
+  }
+
+  /**
+   * Process an element.
+   *
+   * @param value the value being processed
+   * @return the set of windows in which the element was actually processed
+   */
+  private Collection<W> processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT> value)
+      throws Exception {
+    // Redirect element windows to the ACTIVE windows they have been merged into.
+    // The compressed representation (value, {window1, window2, ...}) actually represents
+    // distinct elements (value, window1), (value, window2), ...
+    // so if window1 and window2 merge, the resulting window will contain both copies
+    // of the value.
+    Collection<W> windows = new ArrayList<>();
+    for (BoundedWindow untypedWindow : value.getWindows()) {
+      @SuppressWarnings("unchecked")
+      W window = (W) untypedWindow;
+      W mergeResult = windowToMergeResult.get(window);
+      if (mergeResult == null) {
+        mergeResult = window;
+      }
+      windows.add(mergeResult);
+    }
+
+    // Prefetch in each of the windows if we're going to need to process triggers
+    for (W window : windows) {
+      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
+          window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
+      triggerRunner.prefetchForValue(window, directContext.state());
+    }
+
+    // Process the element for each (mergeResultWindow, not closed) window it belongs to.
+    List<W> triggerableWindows = new ArrayList<>(windows.size());
+    for (W window : windows) {
+      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
+          window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
+      if (triggerRunner.isClosed(directContext.state())) {
+        // This window has already been closed.
+        droppedDueToClosedWindow.addValue(1L);
+        WindowTracing.debug(
+            "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
+            + "since window is no longer active at inputWatermark:{}; outputWatermark:{}",
+            value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
+            timerInternals.currentOutputWatermarkTime());
+        continue;
+      }
+
+      triggerableWindows.add(window);
+      activeWindows.ensureWindowIsActive(window);
+      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext renamedContext = contextFactory.forValue(
+          window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED);
+
+      nonEmptyPanes.recordContent(renamedContext.state());
+
+      // Make sure we've scheduled the end-of-window or garbage collection timer for this window.
+      Instant timer = scheduleEndOfWindowOrGarbageCollectionTimer(directContext);
+
+      // Hold back progress of the output watermark until we have processed the pane this
+      // element will be included within. If the element is too late for that, place a hold at
+      // the end-of-window or garbage collection time to allow empty panes to contribute elements
+      // which won't be dropped due to lateness by a following computation (assuming the following
+      // computation uses the same allowed lateness value...)
+      @Nullable Instant hold = watermarkHold.addHolds(renamedContext);
+
+      if (hold != null) {
+        // Assert that holds have a proximate timer.
+        boolean holdInWindow = !hold.isAfter(window.maxTimestamp());
+        boolean timerInWindow = !timer.isAfter(window.maxTimestamp());
+        checkState(
+            holdInWindow == timerInWindow,
+            "set a hold at %s, a timer at %s, which disagree as to whether they are in window %s",
+            hold,
+            timer,
+            directContext.window());
+      }
+
+      // Execute the reduceFn, which will buffer the value as appropriate
+      reduceFn.processValue(renamedContext);
+
+      // Run the trigger to update its state
+      triggerRunner.processValue(
+          directContext.window(),
+          directContext.timestamp(),
+          directContext.timers(),
+          directContext.state());
+
+      // At this point, if triggerRunner.shouldFire before the processValue then
+      // triggerRunner.shouldFire after the processValue. In other words adding values
+      // cannot take a trigger state from firing to non-firing.
+      // (We don't actually assert this since it is too slow.)
+    }
+
+    return triggerableWindows;
+  }
+
+  /**
+   * Called when an end-of-window, garbage collection, or trigger-specific timer fires.
+   */
+  public void onTimer(TimerData timer) throws Exception {
+    // Which window is the timer for?
+    checkArgument(timer.getNamespace() instanceof WindowNamespace,
+        "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace());
+    @SuppressWarnings("unchecked")
+    WindowNamespace<W> windowNamespace = (WindowNamespace<W>) timer.getNamespace();
+    W window = windowNamespace.getWindow();
+    ReduceFn<K, InputT, OutputT, W>.Context directContext =
+        contextFactory.base(window, StateStyle.DIRECT);
+    ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
+        contextFactory.base(window, StateStyle.RENAMED);
+
+    // Has this window had its trigger finish?
+    // - The trigger may implement isClosed as constant false.
+    // - If the window function does not support windowing then all windows will be considered
+    // active.
+    // So we must take conjunction of activeWindows and triggerRunner state.
+    boolean windowIsActiveAndOpen =
+        activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state());
+
+    if (!windowIsActiveAndOpen) {
+      WindowTracing.debug(
+          "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window);
+    }
+
+    // If this is an end-of-window timer then we may need to set a garbage collection timer
+    // if allowed lateness is non-zero.
+    boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain()
+        && timer.getTimestamp().equals(window.maxTimestamp());
+
+    // If this is a garbage collection timer then we should trigger and garbage collect the window.
+    // We'll consider any timer at or after the end-of-window time to be a signal to garbage
+    // collect.
+    Instant cleanupTime = garbageCollectionTime(window);
+    boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain()
+        && !timer.getTimestamp().isBefore(cleanupTime);
+
+    if (isGarbageCollection) {
+      WindowTracing.debug(
+          "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with "
+          + "inputWatermark:{}; outputWatermark:{}",
+          key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
+          timerInternals.currentOutputWatermarkTime());
+
+      if (windowIsActiveAndOpen) {
+        // We need to call onTrigger to emit the final pane if required.
+        // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted,
+        // and the watermark has passed the end of the window.
+        @Nullable Instant newHold =
+            onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow);
+        checkState(newHold == null,
+            "Hold placed at %s despite isFinished being true.", newHold);
+      }
+
+      // Cleanup flavor B: Clear all the remaining state for this window since we'll never
+      // see elements for it again.
+      clearAllState(directContext, renamedContext, windowIsActiveAndOpen);
+    } else {
+      WindowTracing.debug(
+          "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with "
+          + "inputWatermark:{}; outputWatermark:{}",
+          key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
+          timerInternals.currentOutputWatermarkTime());
+      if (windowIsActiveAndOpen) {
+        emitIfAppropriate(directContext, renamedContext);
+      }
+
+      if (isEndOfWindow) {
+        // If the window strategy trigger includes a watermark trigger then at this point
+        // there should be no data holds, either because we'd already cleared them on an
+        // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate.
+        // We could assert this but it is very expensive.
+
+        // Since we are processing an on-time firing we should schedule the garbage collection
+        // timer. (If getAllowedLateness is zero then the timer event will be considered a
+        // cleanup event and handled by the above).
+        // Note we must do this even if the trigger is finished so that we are sure to cleanup
+        // any final trigger finished bits.
+        checkState(
+            windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO),
+            "Unexpected zero getAllowedLateness");
+        WindowTracing.debug(
+            "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with "
+            + "inputWatermark:{}; outputWatermark:{}",
+            key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(),
+            timerInternals.currentOutputWatermarkTime());
+        checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+                                 "Cleanup time %s is beyond end-of-time", cleanupTime);
+        directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME);
+      }
+    }
+  }
+
+  /**
+   * Clear all the state associated with {@code context}'s window.
+   * Should only be invoked if we know all future elements for this window will be considered
+   * beyond allowed lateness.
+   * This is a superset of the clearing done by {@link #emitIfAppropriate} below since:
+   * <ol>
+   * <li>We can clear the trigger finished bits since we'll never need to ask if the trigger is
+   * closed again.
+   * <li>We can clear any remaining garbage collection hold.
+   * </ol>
+   */
+  private void clearAllState(
+      ReduceFn<K, InputT, OutputT, W>.Context directContext,
+      ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
+      boolean windowIsActiveAndOpen)
+      throws Exception {
+    if (windowIsActiveAndOpen) {
+      // Since both the window is in the active window set AND the trigger was not yet closed,
+      // it is possible we still have state.
+      reduceFn.clearState(renamedContext);
+      watermarkHold.clearHolds(renamedContext);
+      nonEmptyPanes.clearPane(renamedContext.state());
+      // These calls work irrespective of whether the window is active or not, but
+      // are unnecessary if the window is not active.
+      triggerRunner.clearState(
+          directContext.window(), directContext.timers(), directContext.state());
+      paneInfoTracker.clear(directContext.state());
+    } else {
+      // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2).
+      // For (1), if !activeWindows.isActive then the window must be merging and has been
+      // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired
+      // and been closed, so this case reduces to (2).
+      // For (2), if triggerRunner.isClosed then the trigger was fired and entered the
+      // closed state. In that case emitIfAppropriate will have cleared all state in
+      // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows.
+      // We also know nonEmptyPanes must have been unconditionally cleared by the trigger.
+      // Since the trigger fired the existing watermark holds must have been cleared, and since
+      // the trigger closed no new end of window or garbage collection hold will have been
+      // placed by WatermarkHold.extractAndRelease.
+      // Thus all the state clearing above is unnecessary.
+      //
+      // But(!) for backwards compatibility we must allow a pipeline to be updated from
+      // an sdk version <= 1.3. In that case it is possible we have an end-of-window or
+      // garbage collection hold keyed by the current window (reached via directContext) rather
+      // than the state address window (reached via renamedContext).
+      // However this can only happen if:
+      // - We have merging windows.
+      // - We are DISCARDING_FIRED_PANES.
+      // - A pane has fired.
+      // - But the trigger is not (yet) closed.
+      if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES
+          && !windowingStrategy.getWindowFn().isNonMerging()) {
+        watermarkHold.clearHolds(directContext);
+      }
+    }
+
+    // Don't need to track address state windows anymore.
+    activeWindows.remove(directContext.window());
+    // We'll never need to test for the trigger being closed again.
+    triggerRunner.clearFinished(directContext.state());
+  }
+
+  /** Should the reduce function state be cleared? */
+  private boolean shouldDiscardAfterFiring(boolean isFinished) {
+    if (isFinished) {
+      // This is the last firing for trigger.
+      return true;
+    }
+    if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
+      // Nothing should be accumulated between panes.
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state.
+   */
+  private void emitIfAppropriate(ReduceFn<K, InputT, OutputT, W>.Context directContext,
+      ReduceFn<K, InputT, OutputT, W>.Context renamedContext)
+      throws Exception {
+    if (!triggerRunner.shouldFire(
+        directContext.window(), directContext.timers(), directContext.state())) {
+      // Ignore unless trigger is ready to fire
+      return;
+    }
+
+    // Inform the trigger of the transition to see if it is finished
+    triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state());
+    boolean isFinished = triggerRunner.isClosed(directContext.state());
+
+    // Will be able to clear all element state after triggering?
+    boolean shouldDiscard = shouldDiscardAfterFiring(isFinished);
+
+    // Run onTrigger to produce the actual pane contents.
+    // As a side effect it will clear all element holds, but not necessarily any
+    // end-of-window or garbage collection holds.
+    onTrigger(directContext, renamedContext, isFinished, false /*isEndOfWindow*/);
+
+    // Now that we've triggered, the pane is empty.
+    nonEmptyPanes.clearPane(renamedContext.state());
+
+    // Cleanup buffered data if appropriate
+    if (shouldDiscard) {
+      // Cleanup flavor C: The user does not want any buffered data to persist between panes.
+      reduceFn.clearState(renamedContext);
+    }
+
+    if (isFinished) {
+      // Cleanup flavor D: If trigger is closed we will ignore all new incoming elements.
+      // Clear state not otherwise cleared by onTrigger and clearPane above.
+      // Remember the trigger is, indeed, closed until the window is garbage collected.
+      triggerRunner.clearState(
+          directContext.window(), directContext.timers(), directContext.state());
+      paneInfoTracker.clear(directContext.state());
+      activeWindows.remove(directContext.window());
+    }
+  }
+
+  /**
+   * Do we need to emit a pane?
+   */
+  private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) {
+    if (!isEmpty) {
+      // The pane has elements.
+      return true;
+    }
+    if (timing == Timing.ON_TIME) {
+      // This is the unique ON_TIME pane.
+      return true;
+    }
+    if (isFinished && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) {
+      // This is known to be the final pane, and the user has requested it even when empty.
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Run the {@link ReduceFn#onTrigger} method and produce any necessary output.
+   *
+   * @return output watermark hold added, or {@literal null} if none.
+   */
+  @Nullable
+  private Instant onTrigger(
+      final ReduceFn<K, InputT, OutputT, W>.Context directContext,
+      ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
+      boolean isFinished, boolean isEndOfWindow)
+          throws Exception {
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+    // Prefetch necessary states
+    ReadableState<WatermarkHold.OldAndNewHolds> outputTimestampFuture =
+        watermarkHold.extractAndRelease(renamedContext, isFinished).readLater();
+    ReadableState<PaneInfo> paneFuture =
+        paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater();
+    ReadableState<Boolean> isEmptyFuture =
+        nonEmptyPanes.isEmpty(renamedContext.state()).readLater();
+
+    reduceFn.prefetchOnTrigger(directContext.state());
+    triggerRunner.prefetchOnFire(directContext.window(), directContext.state());
+
+    // Calculate the pane info.
+    final PaneInfo pane = paneFuture.read();
+    // Extract the window hold, and as a side effect clear it.
+
+    WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read();
+    final Instant outputTimestamp = pair.oldHold;
+    @Nullable Instant newHold = pair.newHold;
+
+    if (newHold != null) {
+      // We can't be finished yet.
+      checkState(
+        !isFinished, "new hold at %s but finished %s", newHold, directContext.window());
+      // The hold cannot be behind the input watermark.
+      checkState(
+        !newHold.isBefore(inputWM), "new hold %s is before input watermark %s", newHold, inputWM);
+      if (newHold.isAfter(directContext.window().maxTimestamp())) {
+        // The hold must be for garbage collection, which can't have happened yet.
+        checkState(
+          newHold.isEqual(garbageCollectionTime(directContext.window())),
+          "new hold %s should be at garbage collection for window %s plus %s",
+          newHold,
+          directContext.window(),
+          windowingStrategy.getAllowedLateness());
+      } else {
+        // The hold must be for the end-of-window, which can't have happened yet.
+        checkState(
+          newHold.isEqual(directContext.window().maxTimestamp()),
+          "new hold %s should be at end of window %s",
+          newHold,
+          directContext.window());
+        checkState(
+          !isEndOfWindow,
+          "new hold at %s for %s but this is the watermark trigger",
+          newHold,
+          directContext.window());
+      }
+    }
+
+    // Only emit a pane if it has data or empty panes are observable.
+    if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) {
+      // Run reduceFn.onTrigger method.
+      final List<W> windows = Collections.singletonList(directContext.window());
+      ReduceFn<K, InputT, OutputT, W>.OnTriggerContext renamedTriggerContext =
+          contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED,
+              new OnTriggerCallbacks<OutputT>() {
+                @Override
+                public void output(OutputT toOutput) {
+                  // We're going to output panes, so commit the (now used) PaneInfo.
+                  // TODO: This is unnecessary if the trigger isFinished since the saved
+                  // state will be immediately deleted.
+                  paneInfoTracker.storeCurrentPaneInfo(directContext, pane);
+
+                  // Output the actual value.
+                  outputter.outputWindowedValue(
+                      KV.of(key, toOutput), outputTimestamp, windows, pane);
+                }
+              });
+
+      reduceFn.onTrigger(renamedTriggerContext);
+    }
+
+    return newHold;
+  }
+
+  /**
+   * Make sure we'll eventually have a timer fire which will tell us to garbage collect
+   * the window state. For efficiency we may need to do this in two steps rather
+   * than one. Return the time at which the timer will fire.
+   *
+   * <ul>
+   * <li>If allowedLateness is zero then we'll garbage collect at the end of the window.
+   * For simplicity we'll set our own timer for this situation even though an
+   * {@link AfterWatermark} trigger may have also set an end-of-window timer.
+   * ({@code setTimer} is idempotent.)
+   * <li>If allowedLateness is non-zero then we could just always set a timer for the garbage
+   * collection time. However if the windows are large (eg hourly) and the allowedLateness is small
+   * (eg seconds) then we'll end up with nearly twice the number of timers in-flight. So we
+   * instead set an end-of-window timer and then roll that forward to a garbage collection timer
+   * when it fires. We use the input watermark to distinguish those cases.
+   * </ul>
+   */
+  private Instant scheduleEndOfWindowOrGarbageCollectionTimer(
+      ReduceFn<?, ?, ?, W>.Context directContext) {
+    Instant inputWM = timerInternals.currentInputWatermarkTime();
+    Instant endOfWindow = directContext.window().maxTimestamp();
+    String which;
+    Instant timer;
+    if (endOfWindow.isBefore(inputWM)) {
+      timer = garbageCollectionTime(directContext.window());
+      which = "garbage collection";
+    } else {
+      timer = endOfWindow;
+      which = "end-of-window";
+    }
+    WindowTracing.trace(
+        "ReduceFnRunner.scheduleEndOfWindowOrGarbageCollectionTimer: Scheduling {} timer at {} for "
+        + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
+        which,
+        timer,
+        key,
+        directContext.window(),
+        inputWM,
+        timerInternals.currentOutputWatermarkTime());
+    checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+                             "Timer %s is beyond end-of-time", timer);
+    directContext.timers().setTimer(timer, TimeDomain.EVENT_TIME);
+    return timer;
+  }
+
+  private void cancelEndOfWindowAndGarbageCollectionTimers(
+      ReduceFn<?, ?, ?, W>.Context directContext) {
+    WindowTracing.debug(
+        "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for "
+        + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
+        key, directContext.window(), timerInternals.currentInputWatermarkTime(),
+        timerInternals.currentOutputWatermarkTime());
+    Instant eow = directContext.window().maxTimestamp();
+    directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+    Instant gc = garbageCollectionTime(directContext.window());
+    if (gc.isAfter(eow)) {
+      directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+    }
+  }
+
+  /**
+   * 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) {
+
+    // 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 window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
+    }
+  }
+
+  /**
+   * An object that can output a value with all of its windowing information. This is a deliberately
+   * restricted subinterface of {@link WindowingInternals} to express how it is used here.
+   */
+  private interface OutputWindowedValue<OutputT> {
+    void outputWindowedValue(OutputT output, Instant timestamp,
+        Collection<? extends BoundedWindow> windows, PaneInfo pane);
+  }
+
+  private static class OutputViaWindowingInternals<OutputT>
+      implements OutputWindowedValue<OutputT> {
+
+    private final WindowingInternals<?, OutputT> windowingInternals;
+
+    public OutputViaWindowingInternals(WindowingInternals<?, OutputT> windowingInternals) {
+      this.windowingInternals = windowingInternals;
+    }
+
+    @Override
+    public void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      windowingInternals.outputWindowedValue(output, timestamp, windows, pane);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
new file mode 100644
index 0000000..df74ed3
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.util.List;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+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.TupleTag;
+
+/**
+ * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+ * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+ */
+public class SimpleDoFnRunner<InputT, OutputT> extends DoFnRunnerBase<InputT, OutputT> {
+
+  protected SimpleDoFnRunner(PipelineOptions options, OldDoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
+      AggregatorFactory aggregatorFactory, WindowingStrategy<?, ?> windowingStrategy) {
+    super(options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext,
+        aggregatorFactory, windowingStrategy);
+  }
+
+  @Override
+  protected void invokeProcessElement(WindowedValue<InputT> elem) {
+    final OldDoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.processElement(processContext);
+    } catch (Exception ex) {
+      throw wrapUserCodeException(ex);
+    }
+  }
+}


[37/50] [abbrv] incubator-beam git commit: Remove the DataflowRunner instructions from examples

Posted by dh...@apache.org.
Remove the DataflowRunner instructions from examples


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

Branch: refs/heads/gearpump-runner
Commit: c92e45dd4019e613a7670e4bb0e1fcc4b7e2c462
Parents: 4bf3a3b
Author: Pei He <pe...@google.com>
Authored: Thu Aug 25 14:20:30 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/DebuggingWordCount.java       | 16 +++++++------
 .../apache/beam/examples/MinimalWordCount.java  |  7 +++---
 .../apache/beam/examples/WindowedWordCount.java | 22 +++++++----------
 .../org/apache/beam/examples/WordCount.java     | 22 +++++------------
 .../beam/examples/complete/AutoComplete.java    | 25 +++++++-------------
 .../examples/complete/StreamingWordExtract.java |  4 ++--
 .../apache/beam/examples/complete/TfIdf.java    | 18 +++++---------
 .../examples/complete/TopWikipediaSessions.java | 12 ++++------
 .../examples/complete/TrafficMaxLaneFlow.java   |  4 ++--
 .../beam/examples/complete/TrafficRoutes.java   |  4 ++--
 .../examples/cookbook/BigQueryTornadoes.java    | 18 ++++----------
 .../cookbook/CombinePerKeyExamples.java         | 18 ++++----------
 .../examples/cookbook/DatastoreWordCount.java   | 17 ++++++-------
 .../beam/examples/cookbook/DeDupExample.java    | 16 ++++++-------
 .../beam/examples/cookbook/FilterExamples.java  | 21 ++++------------
 .../beam/examples/cookbook/JoinExamples.java    | 18 ++++----------
 .../examples/cookbook/MaxPerKeyExamples.java    | 19 ++++-----------
 .../beam/examples/cookbook/TriggerExample.java  | 16 ++++++-------
 18 files changed, 92 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 be3aa41..eb38227 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
@@ -46,12 +46,12 @@ import org.slf4j.LoggerFactory;
  *
  * <p>Basic concepts, also in the MinimalWordCount and WordCount examples:
  * Reading text files; counting a PCollection; executing a Pipeline both locally
- * and using the Dataflow service; defining DoFns.
+ * and using a selected runner; defining DoFns.
  *
  * <p>New Concepts:
  * <pre>
  *   1. Logging to Cloud Logging
- *   2. Controlling Dataflow worker log levels
+ *   2. Controlling worker log levels
  *   3. Creating a custom aggregator
  *   4. Testing your Pipeline via PAssert
  * </pre>
@@ -62,12 +62,14 @@ import org.slf4j.LoggerFactory;
  * }
  * </pre>
  *
- * <p>To execute this pipeline using the Dataflow service and the additional logging discussed
- * below, specify pipeline configuration:
+ * <p>To change the runner, specify:
+ * <pre>{@code
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ *
+ * <p>To use the additional logging discussed below, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
  *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
  * </pre>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 f28a20c..f772dd5 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
@@ -66,12 +66,11 @@ public class MinimalWordCount {
 
     // In order to run your pipeline, you need to make following runner specific changes:
     //
-    // CHANGE 1/3: Select a Beam runner, such as BlockingDataflowRunner
-    // or FlinkRunner.
+    // CHANGE 1/3: Select a Beam runner, such as DataflowRunner or FlinkRunner.
     // CHANGE 2/3: Specify runner-required options.
-    // For BlockingDataflowRunner, set project and temp location as follows:
+    // For DataflowRunner, set project and temp location as follows:
     //   DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-    //   dataflowOptions.setRunner(BlockingDataflowRunner.class);
+    //   dataflowOptions.setRunner(DataflowRunner.class);
     //   dataflowOptions.setProject("SET_YOUR_PROJECT_ID_HERE");
     //   dataflowOptions.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY");
     // For FlinkRunner, set the runner as follows. See {@code FlinkPipelineOptions}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 7af354c..c8bd9d3 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
@@ -54,7 +54,7 @@ import org.joda.time.Instant;
  *
  * <p>Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples:
  * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally
- * and using the Dataflow service; defining DoFns; creating a custom aggregator;
+ * and using a selected runner; defining DoFns; creating a custom aggregator;
  * user-defined PTransforms; defining PipelineOptions.
  *
  * <p>New Concepts:
@@ -66,19 +66,13 @@ import org.joda.time.Instant;
  *   5. Writing to BigQuery
  * </pre>
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
+ * <p>By default, the examples will run with the {@code DirectRunner}.
+ * To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>Optionally specify the input file path via:
  * {@code --inputFile=gs://INPUT_PATH},
@@ -86,7 +80,7 @@ import org.joda.time.Instant;
  *
  * <p>Specify an output BigQuery dataset and optionally, a table for the output. If you don't
  * specify the table, one will be created for you using the job name. If you don't specify the
- * dataset, a dataset called {@code dataflow-examples} must already exist in your project.
+ * dataset, a dataset called {@code beam_examples} must already exist in your project.
  * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}.
  *
  * <p>By default, the pipeline will do fixed windowing, on 1-minute windows.  You can
@@ -190,7 +184,7 @@ public class WindowedWordCount {
     Pipeline pipeline = Pipeline.create(options);
 
     /**
-     * Concept #1: the Dataflow SDK lets us run the same pipeline with either a bounded or
+     * Concept #1: the Beam SDK lets us run the same pipeline with either a bounded or
      * unbounded input source.
      */
     PCollection<String> input = pipeline
@@ -229,7 +223,7 @@ public class WindowedWordCount {
 
     PipelineResult result = pipeline.run();
 
-    // dataflowUtils will try to cancel the pipeline before the program exists.
+    // ExampleUtils will try to cancel the pipeline before the program exists.
     exampleUtils.waitToFinish(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 793ee4b..498b069 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
@@ -48,8 +48,8 @@ import org.apache.beam.sdk.values.PCollection;
  * pipeline, for introduction of additional concepts.
  *
  * <p>For a detailed walkthrough of this example, see
- *   <a href="https://cloud.google.com/dataflow/java-sdk/wordcount-example">
- *   https://cloud.google.com/dataflow/java-sdk/wordcount-example
+ *   <a href="http://beam.incubator.apache.org/use/walkthroughs/">
+ *   http://beam.incubator.apache.org/use/walkthroughs/
  *   </a>
  *
  * <p>Basic concepts, also in the MinimalWordCount example:
@@ -66,27 +66,17 @@ import org.apache.beam.sdk.values.PCollection;
  * <p>Concept #1: you can execute this pipeline either locally or using the selected runner.
  * These are now command-line options and not hard-coded as they were in the MinimalWordCount
  * example.
- * To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and a local output file or output prefix on GCS:
+ * To execute this pipeline locally, specify a local output file or output prefix on GCS:
  * <pre>{@code
  *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt}
  * and can be overridden with {@code --inputFile}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 2182e6d..c3ac614 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
@@ -77,24 +77,17 @@ import org.joda.time.Duration;
  * <p>Concepts: Using the same pipeline in both streaming and batch, combiners,
  *              composite transforms.
  *
- * <p>To execute this pipeline using the Dataflow service in batch mode,
- * specify pipeline configuration:
+ * <p>To execute this pipeline in streaming mode, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=DataflowRunner
- *   --inputFile=gs://path/to/input*.txt
+ *   --streaming
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service in streaming mode,
- * specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=DataflowRunner
- *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
- *   --streaming
- * }</pre>
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>This will update the Cloud Datastore every 10 seconds based on the last
  * 30 minutes of data received.
@@ -417,7 +410,7 @@ public class AutoComplete {
   /**
    * Options supported by this class.
    *
-   * <p>Inherits standard Dataflow configuration options.
+   * <p>Inherits standard Beam example configuration options.
    */
   private static interface Options
       extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions {
@@ -510,7 +503,7 @@ public class AutoComplete {
     // Run the pipeline.
     PipelineResult result = p.run();
 
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    // ExampleUtils will try to cancel the pipeline and the injector before the program exists.
     exampleUtils.waitToFinish(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
index 869ea69..e8d8950 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
@@ -44,7 +44,7 @@ import org.apache.beam.sdk.transforms.ParDo;
  * a BigQuery table.
  *
  * <p>The example is configured to use the default BigQuery table from the example common package
- * (there are no defaults for a general Dataflow pipeline).
+ * (there are no defaults for a general Beam pipeline).
  * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
  * options. If the BigQuery table do not exist, the example will try to create them.
  *
@@ -141,7 +141,7 @@ public class StreamingWordExtract {
 
     PipelineResult result = pipeline.run();
 
-    // dataflowUtils will try to cancel the pipeline before the program exists.
+    // ExampleUtils will try to cancel the pipeline before the program exists.
     exampleUtils.waitToFinish(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 6684553..59bbd49 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
@@ -65,23 +65,17 @@ import org.slf4j.LoggerFactory;
  *
  * <p>Concepts: joining data; side inputs; logging
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }</pre>
- * and a local output file or output prefix on GCS:
+ * <p>To execute this pipeline locally, specify a local output file or output prefix on GCS:
  * <pre>{@code
  *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
- * and an output prefix on GCS:
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The default input is {@code gs://apache-beam-samples/shakespeare/} and can be overridden with
  * {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 d597258..0f594d7 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
@@ -52,17 +52,13 @@ import org.joda.time.Instant;
  * <p>It is not recommended to execute this pipeline locally, given the size of the default input
  * data.
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To execute this pipeline using a selected runner and an output prefix on GCS, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
+ *   --output=gs://YOUR_OUTPUT_PREFIX
  * }
  * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The default input is {@code gs://apache-beam-samples/wikipedia_edits/*.json} and can be
  * overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
index e456960..0c367d4 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
@@ -66,7 +66,7 @@ import org.joda.time.format.DateTimeFormatter;
  * <p>The pipeline reads traffic sensor data from {@literal --inputFile}.
  *
  * <p>The example is configured to use the default BigQuery table from the example common package
- * (there are no defaults for a general Dataflow pipeline).
+ * (there are no defaults for a general Beam pipeline).
  * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
  * options. If the BigQuery table do not exist, the example will try to create them.
  *
@@ -354,7 +354,7 @@ public class TrafficMaxLaneFlow {
     // Run the pipeline.
     PipelineResult result = pipeline.run();
 
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    // ExampleUtils will try to cancel the pipeline and the injector before the program exists.
     exampleUtils.waitToFinish(result);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
index 95336c6..14cee4d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
@@ -69,7 +69,7 @@ import org.joda.time.format.DateTimeFormatter;
  * <p>The pipeline reads traffic sensor data from {@literal --inputFile}.
  *
  * <p>The example is configured to use the default BigQuery table from the example common package
- * (there are no defaults for a general Dataflow pipeline).
+ * (there are no defaults for a general Beam pipeline).
  * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
  * options. If the BigQuery table do not exist, the example will try to create them.
  *
@@ -365,7 +365,7 @@ public class TrafficRoutes {
     // Run the pipeline.
     PipelineResult result = pipeline.run();
 
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    // ExampleUtils will try to cancel the pipeline and the injector before the program exists.
     exampleUtils.waitToFinish(result);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 439cf02..1e4918d 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
@@ -45,27 +45,17 @@ import org.apache.beam.sdk.values.PCollection;
  * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
  * table.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output, with the form
+ * <p>To execute this pipeline locally, specify the BigQuery table for the output with the form:
  * <pre>{@code
  *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
  * and can be overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 1d280a6..fc11ac9 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
@@ -52,27 +52,17 @@ import org.apache.beam.sdk.values.PCollection;
  * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
  * table.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output:
+ * <p>To execute this pipeline locally, specify the BigQuery table for the output:
  * <pre>{@code
  *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://<STAGING DIRECTORY>
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The BigQuery input table defaults to {@code publicdata:samples.shakespeare} and can
  * be overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 434e9fb..c0066e6 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
@@ -58,14 +58,15 @@ import org.apache.beam.sdk.transforms.ParDo;
  *
  * <p>To run this pipeline locally, the following options must be provided:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
  *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PATH]
  * }</pre>
  *
- * <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=BlockingDataflowRunner}.
+ * <p>To change the runner, specify:
+ * <pre>{@code
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <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
@@ -239,13 +240,9 @@ 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 DataflowRunner} in the pipeline options.
+   * An example to demo how to use {@link DatastoreIO}.
    */
   public static void main(String args[]) {
-    // The options are used in two places, for Dataflow service, and
-    // building DatastoreIO.Read object
     Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
 
     if (!options.isReadOnly()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 5791710..594d52d 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
@@ -35,17 +35,15 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath;
  * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/
  * {@link RemoveDuplicates}/{@link org.apache.beam.sdk.io.TextIO.Write}.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- *   --project=YOUR_PROJECT_ID
- * and a local output file or output prefix on GCS:
+ * <p>To execute this pipeline locally, specify a local output file or output prefix on GCS:
  *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
- * and an output prefix on GCS:
- *   --output=gs://YOUR_OUTPUT_PREFIX
+ * <p>To change the runner, specify:
+ * <pre>{@code
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be
  * overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 6c42520..01d668b 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
@@ -54,12 +54,7 @@ import org.apache.beam.sdk.values.PCollectionView;
  * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
  * table.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output:
+ * <p>To execute this pipeline locally, specify the BigQuery table for the output:
  * <pre>{@code
  *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
  *   [--monthFilter=<month_number>]
@@ -67,20 +62,12 @@ import org.apache.beam.sdk.values.PCollectionView;
  * </pre>
  * where optional parameter {@code --monthFilter} is set to a number 1-12.
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
- * and the BigQuery table for the output:
- * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
- *   [--monthFilter=<month_number>]
- * }
- * </pre>
- * where optional parameter {@code --monthFilter} is set to a number 1-12.
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations}
  * and can be overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 1b91bf1..799cad3 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
@@ -41,27 +41,17 @@ import org.apache.beam.sdk.values.TupleTag;
  *
  * <p>Concepts: Join operation; multiple input sources.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and a local output file or output prefix on GCS:
+ * <p>To execute this pipeline locally, specify a local output file or output prefix on GCS:
  * <pre>{@code
  *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
+ *   --runner=YOUR_SELECTED_RUNNER
  * }
  * </pre>
- * and an output prefix on GCS:
- * <pre>{@code
- *   --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  */
 public class JoinExamples {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/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 3772a7b..3a4fa26 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
@@ -46,27 +46,16 @@ import org.apache.beam.sdk.values.PCollection;
  * <p>Note: Before running this example, you must create a BigQuery dataset to contain your output
  * table.
  *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- * and the BigQuery table for the output, with the form
+ * <p>To execute this pipeline locally, specify the BigQuery table for the output with the form:
  * <pre>{@code
  *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
  * }</pre>
  *
- * <p>To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
- *   --runner=BlockingDataflowRunner
- * }
- * </pre>
- * and the BigQuery table for the output:
+ * <p>To change the runner, specify:
  * <pre>{@code
- *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
+ *   --runner=YOUR_SELECTED_RUNNER
  * }</pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
  *
  * <p>The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations }
  * and can be overridden with {@code --input}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c92e45dd/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
index 2630541..68d4d32 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -73,15 +73,13 @@ import org.joda.time.Instant;
  *   4. Combining late data and speculative estimates
  * </pre>
  *
- * <p> Before running this example, it will be useful to familiarize yourself with Dataflow triggers
+ * <p> Before running this example, it will be useful to familiarize yourself with Beam triggers
  * and understand the concept of 'late data',
- * See:  <a href="https://cloud.google.com/dataflow/model/triggers">
- * https://cloud.google.com/dataflow/model/triggers </a> and
- * <a href="https://cloud.google.com/dataflow/model/windowing#Advanced">
- * https://cloud.google.com/dataflow/model/windowing#Advanced </a>
+ * See: <a href="http://beam.incubator.apache.org/use/walkthroughs/">
+ * http://beam.incubator.apache.org/use/walkthroughs/</a>
  *
  * <p> The example is configured to use the default BigQuery table from the example common package
- * (there are no defaults for a general Dataflow pipeline).
+ * (there are no defaults for a general Beam pipeline).
  * You can override them by using the {@code --bigQueryDataset}, and {@code --bigQueryTable}
  * options. If the BigQuery table do not exist, the example will try to create them.
  *
@@ -155,7 +153,7 @@ public class TriggerExample {
    * 5             | 60                 | 10:27:20   | 10:27:25
    * 5             | 60                 | 10:29:00   | 11:11:00
    *
-   * <p> Dataflow tracks a watermark which records up to what point in event time the data is
+   * <p> Beam tracks a watermark which records up to what point in event time the data is
    * complete. For the purposes of the example, we'll assume the watermark is approximately 15m
    * behind the current processing time. In practice, the actual value would vary over time based
    * on the systems knowledge of the current delay and contents of the backlog (data
@@ -176,7 +174,7 @@ public class TriggerExample {
     public PCollectionList<TableRow> apply(PCollection<KV<String, Integer>> flowInfo) {
 
       // Concept #1: The default triggering behavior
-      // By default Dataflow uses a trigger which fires when the watermark has passed the end of the
+      // By default Beam uses a trigger which fires when the watermark has passed the end of the
       // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
 
       // The system also defaults to dropping late data -- data which arrives after the watermark
@@ -459,7 +457,7 @@ public class TriggerExample {
 
     PipelineResult result = pipeline.run();
 
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exits.
+    // ExampleUtils will try to cancel the pipeline and the injector before the program exits.
     exampleUtils.waitToFinish(result);
   }
 



[02/50] [abbrv] incubator-beam git commit: Query latest timestamp

Posted by dh...@apache.org.
Query latest timestamp


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

Branch: refs/heads/gearpump-runner
Commit: 79491ebe04e54043ebdf5afa6be78718eae0fe15
Parents: 0fbd9c8
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Tue Aug 23 16:44:08 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  | 41 +++++++++++--
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   | 60 +++++++++++++++++---
 2 files changed, 88 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/79491ebe/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index c7433d3..8456e02 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -239,6 +239,7 @@ public class DatastoreV1 {
       int numSplits;
       try {
         long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace);
+        LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes);
         numSplits = (int) Math.min(NUM_QUERY_SPLITS_MAX,
             Math.round(((double) estimatedSizeBytes) / DEFAULT_BUNDLE_SIZE_BYTES));
       } catch (Exception e) {
@@ -250,6 +251,33 @@ public class DatastoreV1 {
     }
 
     /**
+     * Datastore system tables with statistics are periodically updated. This method fetches
+     * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__}
+     * table.
+     */
+    private static long queryLatestStatisticsTimestamp(Datastore datastore,
+        @Nullable String namespace)  throws DatastoreException {
+      Query.Builder query = Query.newBuilder();
+      if (namespace == null) {
+        query.addKindBuilder().setName("__Stat_Total__");
+      } else {
+        query.addKindBuilder().setName("__Stat_Ns_Total__");
+      }
+      query.addOrder(makeOrder("timestamp", DESCENDING));
+      query.setLimit(Int32Value.newBuilder().setValue(1));
+      RunQueryRequest request = makeRequest(query.build(), namespace);
+
+      RunQueryResponse response = datastore.runQuery(request);
+      QueryResultBatch batch = response.getBatch();
+      if (batch.getEntityResultsCount() == 0) {
+        throw new NoSuchElementException(
+            "Datastore total statistics unavailable");
+      }
+      Entity entity = batch.getEntityResults(0).getEntity();
+      return entity.getProperties().get("timestamp").getTimestampValue().getSeconds() * 1000000;
+    }
+
+    /**
      * Get the estimated size of the data returned by the given query.
      *
      * <p>Datastore provides no way to get a good estimate of how large the result of a query
@@ -261,17 +289,17 @@ public class DatastoreV1 {
     static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable String namespace)
         throws DatastoreException {
       String ourKind = query.getKind(0).getName();
+      long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace);
+      LOG.info("Latest stats timestamp : {}", latestTimestamp);
+
       Query.Builder queryBuilder = Query.newBuilder();
       if (namespace == null) {
         queryBuilder.addKindBuilder().setName("__Stat_Kind__");
       } else {
-        queryBuilder.addKindBuilder().setName("__Ns_Stat_Kind__");
+        queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__");
       }
       queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build()));
-
-      // Get the latest statistics
-      queryBuilder.addOrder(makeOrder("timestamp", DESCENDING));
-      queryBuilder.setLimit(Int32Value.newBuilder().setValue(1));
+      queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()));
 
       RunQueryRequest request = makeRequest(queryBuilder.build(), namespace);
 
@@ -547,6 +575,7 @@ public class DatastoreV1 {
           estimatedNumSplits = numSplits;
         }
 
+        LOG.info("Splitting the query into {} splits", estimatedNumSplits);
         List<Query> querySplits;
         try {
           querySplits = splitQuery(query, options.getNamespace(), datastore, querySplitter,
@@ -866,7 +895,7 @@ public class DatastoreV1 {
 
     @FinishBundle
     public void finishBundle(Context c) throws Exception {
-      if (mutations.size() > 0) {
+      if (!mutations.isEmpty()) {
         flushBatch();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/79491ebe/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
index ab1df2f..138671d 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
@@ -61,6 +61,7 @@ import com.google.datastore.v1.client.Datastore;
 import com.google.datastore.v1.client.QuerySplitter;
 import com.google.protobuf.Int32Value;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -561,14 +562,23 @@ public class DatastoreV1Test {
   @Test
   public void testEstimatedSizeBytes() throws Exception {
     long entityBytes = 100L;
+    // In seconds
+    long timestamp = 1234L;
+
+    RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE),
+        NAMESPACE);
+    RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp);
     // Per Kind statistics request and response
-    RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE);
+    RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE);
     RunQueryResponse statResponse = makeStatKindResponse(entityBytes);
 
+    when(mockDatastore.runQuery(latestTimestampRequest))
+        .thenReturn(latestTimestampResponse);
     when(mockDatastore.runQuery(statRequest))
         .thenReturn(statResponse);
 
     assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE));
+    verify(mockDatastore, times(1)).runQuery(latestTimestampRequest);
     verify(mockDatastore, times(1)).runQuery(statRequest);
   }
 
@@ -609,11 +619,19 @@ public class DatastoreV1Test {
     int numSplits = 0;
     int expectedNumSplits = 20;
     long entityBytes = expectedNumSplits * DEFAULT_BUNDLE_SIZE_BYTES;
+    // In seconds
+    long timestamp = 1234L;
+
+    RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE),
+        NAMESPACE);
+    RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp);
 
     // Per Kind statistics request and response
-    RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE);
+    RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE);
     RunQueryResponse statResponse = makeStatKindResponse(entityBytes);
 
+    when(mockDatastore.runQuery(latestTimestampRequest))
+        .thenReturn(latestTimestampResponse);
     when(mockDatastore.runQuery(statRequest))
         .thenReturn(statResponse);
     when(mockQuerySplitter.getSplits(
@@ -629,6 +647,7 @@ public class DatastoreV1Test {
     verifyUniqueKeys(queries);
     verify(mockQuerySplitter, times(1)).getSplits(
         eq(QUERY), any(PartitionId.class), eq(expectedNumSplits), any(Datastore.class));
+    verify(mockDatastore, times(1)).runQuery(latestTimestampRequest);
     verify(mockDatastore, times(1)).runQuery(statRequest);
   }
 
@@ -752,7 +771,7 @@ public class DatastoreV1Test {
 
   /** Builds a per-kind statistics response with the given entity size. */
   private static RunQueryResponse makeStatKindResponse(long entitySizeInBytes) {
-    RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder();
+    RunQueryResponse.Builder statKindResponse = RunQueryResponse.newBuilder();
     Entity.Builder entity = Entity.newBuilder();
     entity.setKey(makeKey("dummyKind", "dummyId"));
     entity.getMutableProperties().put("entity_bytes", makeValue(entitySizeInBytes).build());
@@ -760,24 +779,51 @@ public class DatastoreV1Test {
     entityResult.setEntity(entity);
     QueryResultBatch.Builder batch = QueryResultBatch.newBuilder();
     batch.addEntityResults(entityResult);
+    statKindResponse.setBatch(batch);
+    return statKindResponse.build();
+  }
+
+  /** Builds a response of the given timestamp. */
+  private static RunQueryResponse makeLatestTimestampResponse(long timestamp) {
+    RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder();
+    Entity.Builder entity = Entity.newBuilder();
+    entity.setKey(makeKey("dummyKind", "dummyId"));
+    entity.getMutableProperties().put("timestamp", makeValue(new Date(timestamp * 1000)).build());
+    EntityResult.Builder entityResult = EntityResult.newBuilder();
+    entityResult.setEntity(entity);
+    QueryResultBatch.Builder batch = QueryResultBatch.newBuilder();
+    batch.addEntityResults(entityResult);
     timestampResponse.setBatch(batch);
     return timestampResponse.build();
   }
 
   /** Builds a per-kind statistics query for the given timestamp and namespace. */
-  private static Query makeStatKindQuery(String namespace) {
+  private static Query makeStatKindQuery(String namespace, long timestamp) {
     Query.Builder statQuery = Query.newBuilder();
     if (namespace == null) {
       statQuery.addKindBuilder().setName("__Stat_Kind__");
     } else {
-      statQuery.addKindBuilder().setName("__Ns_Stat_Kind__");
+      statQuery.addKindBuilder().setName("__Stat_Ns_Kind__");
     }
     statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build());
-    statQuery.addOrder(makeOrder("timestamp", DESCENDING));
-    statQuery.setLimit(Int32Value.newBuilder().setValue(1));
+    statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build());
     return statQuery.build();
   }
 
+  /** Builds a latest timestamp statistics query. */
+  private static Query makeLatestTimestampQuery(String namespace) {
+    Query.Builder timestampQuery = Query.newBuilder();
+    if (namespace == null) {
+      timestampQuery.addKindBuilder().setName("__Stat_Total__");
+    } else {
+      timestampQuery.addKindBuilder().setName("__Stat_Ns_Total__");
+    }
+    timestampQuery.addOrder(makeOrder("timestamp", DESCENDING));
+    timestampQuery.setLimit(Int32Value.newBuilder().setValue(1));
+    return timestampQuery.build();
+  }
+
+
   /** Generate dummy query splits. */
   private List<Query> splitQuery(Query query, int numSplits) {
     List<Query> queries = new LinkedList<>();


[03/50] [abbrv] incubator-beam git commit: travis.yml: disable updating snapshots

Posted by dh...@apache.org.
travis.yml: disable updating snapshots

Will still update releases with the --update-snapshots version (I know, bad name)


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

Branch: refs/heads/gearpump-runner
Commit: 0fbd9c8c41b770b176886540a7083a166b2485b6
Parents: ef312e9
Author: Dan Halperin <dh...@google.com>
Authored: Tue Aug 23 20:21:30 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fbd9c8c/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 656aba0..2331282 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -52,7 +52,7 @@ install:
   - rm -rf "$HOME/.m2/repository/org/apache/beam"
 
 script:
-  - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots $MAVEN_OVERRIDE verify
+  - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify
   - travis_retry .travis/test_wordcount.sh
 
 cache:


[21/50] [abbrv] incubator-beam git commit: Fixed Combine display data

Posted by dh...@apache.org.
Fixed Combine display data


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

Branch: refs/heads/gearpump-runner
Commit: 9943fd7d47819d522cef248d23c8db8f42981ad3
Parents: f44fa2c
Author: Ian Zhou <ia...@google.com>
Authored: Thu Aug 18 13:50:52 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/Combine.java | 53 ++++++++++++++++++--
 .../apache/beam/sdk/transforms/CombineTest.java | 19 +++++++
 2 files changed, 68 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9943fd7d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index 26f0f66..d432e15 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -51,6 +51,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext.Context;
 import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
 import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
@@ -1815,7 +1816,14 @@ public class Combine {
      */
     public PerKeyWithHotKeyFanout<K, InputT, OutputT> withHotKeyFanout(final int hotKeyFanout) {
       return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData,
-          new SerializableFunction<K, Integer>() {
+          new SimpleFunction<K, Integer>() {
+            @Override
+            public void populateDisplayData(Builder builder) {
+              super.populateDisplayData(builder);
+              builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout)
+                  .withLabel("Key Fanout Size"), 0);
+            }
+
             @Override
             public Integer apply(K unused) {
               return hotKeyFanout;
@@ -1904,7 +1912,7 @@ public class Combine {
           new InputOrAccum.InputOrAccumCoder<InputT, AccumT>(
               inputCoder.getValueCoder(), accumCoder);
 
-      // A CombineFn's mergeAccumulator can be applied in a tree-like fashon.
+      // A CombineFn's mergeAccumulator can be applied in a tree-like fashion.
       // Here we shard the key using an integer nonce, combine on that partial
       // set of values, then drop the nonce and do a final combine of the
       // aggregates.  We do this by splitting the original CombineFn into two,
@@ -1944,6 +1952,16 @@ public class Combine {
                   throws CannotProvideCoderException {
                 return accumCoder;
               }
+
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                super.populateDisplayData(builder);
+                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
+                    .withLabel("Fanout Function"));
+                if (hotKeyFanout instanceof HasDisplayData) {
+                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
+                }
+              }
             };
         postCombine =
             new KeyedCombineFn<K, InputOrAccum<InputT, AccumT>, AccumT, OutputT>() {
@@ -1988,6 +2006,15 @@ public class Combine {
                       throws CannotProvideCoderException {
                 return accumCoder;
               }
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                super.populateDisplayData(builder);
+                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
+                    .withLabel("Fanout Function"));
+                if (hotKeyFanout instanceof HasDisplayData) {
+                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
+                }
+              }
             };
       } else {
         final KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> keyedFnWithContext =
@@ -2028,6 +2055,15 @@ public class Combine {
                   throws CannotProvideCoderException {
                 return accumCoder;
               }
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                super.populateDisplayData(builder);
+                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
+                    .withLabel("Fanout Function"));
+                if (hotKeyFanout instanceof HasDisplayData) {
+                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
+                }
+              }
             };
         postCombine =
             new KeyedCombineFnWithContext<K, InputOrAccum<InputT, AccumT>, AccumT, OutputT>() {
@@ -2073,6 +2109,15 @@ public class Combine {
                   throws CannotProvideCoderException {
                 return accumCoder;
               }
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                super.populateDisplayData(builder);
+                builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())
+                    .withLabel("Fanout Function"));
+                if (hotKeyFanout instanceof HasDisplayData) {
+                  ((HasDisplayData) hotKeyFanout).populateDisplayData(builder);
+                }
+              }
             };
       }
 
@@ -2117,7 +2162,7 @@ public class Combine {
           .setCoder(KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), VarIntCoder.of()),
                                inputCoder.getValueCoder()))
           .setWindowingStrategyInternal(preCombineStrategy)
-          .apply("PreCombineHot", Combine.perKey(hotPreCombine))
+          .apply("PreCombineHot", Combine.perKey(hotPreCombine, fnDisplayData))
           .apply("StripNonce", MapElements.via(
               new SimpleFunction<KV<KV<K, Integer>, AccumT>,
                        KV<K, InputOrAccum<InputT, AccumT>>>() {
@@ -2147,7 +2192,7 @@ public class Combine {
       // Combine the union of the pre-processed hot and cold key results.
       return PCollectionList.of(precombinedHot).and(preprocessedCold)
           .apply(Flatten.<KV<K, InputOrAccum<InputT, AccumT>>>pCollections())
-          .apply("PostCombine", Combine.perKey(postCombine));
+          .apply("PostCombine", Combine.perKey(postCombine, fnDisplayData));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9943fd7d/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 77a1d6b..be061af 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
@@ -731,6 +731,25 @@ public class CombineTest implements Serializable {
         displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass())));
   }
 
+  @Test
+  @Category(RunnableOnService.class)
+  public void testCombinePerKeyWithHotKeyFanoutPrimitiveDisplayData() {
+    int hotKeyFanout = 2;
+    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+
+    CombineTest.UniqueInts combineFn = new CombineTest.UniqueInts();
+    PTransform<PCollection<KV<Integer, Integer>>, PCollection<KV<Integer, Set<Integer>>>> combine =
+        Combine.<Integer, Integer, Set<Integer>>perKey(combineFn).withHotKeyFanout(hotKeyFanout);
+
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(combine,
+        KvCoder.of(VarIntCoder.of(), VarIntCoder.of()));
+
+    assertThat("Combine.perKey.withHotKeyFanout should include the combineFn in its primitive "
+        + "transform", displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass())));
+    assertThat("Combine.perKey.withHotKeyFanout(int) should include the fanout in its primitive "
+        + "transform", displayData, hasItem(hasDisplayItem("fanout", hotKeyFanout)));
+  }
+
   ////////////////////////////////////////////////////////////////////////////
   // Test classes, for different kinds of combining fns.
 


[10/50] [abbrv] incubator-beam git commit: take advantage of setup/teardown for KafkaWriter

Posted by dh...@apache.org.
take advantage of setup/teardown for KafkaWriter


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

Branch: refs/heads/gearpump-runner
Commit: 74d0195d15660727de80314b1658b498ceda00e1
Parents: a58afd3
Author: Raghu Angadi <ra...@google.com>
Authored: Mon Aug 29 13:09:58 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   | 25 ++++++++++----------
 1 file changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/74d0195d/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index f16a5ab..6769b31 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -1325,16 +1325,12 @@ public class KafkaIO {
 
   private static class KafkaWriter<K, V> extends DoFn<KV<K, V>, Void> {
 
-    @StartBundle
-    public void startBundle(Context c) throws Exception {
-      // Producer initialization is fairly costly. Move this to future initialization api to avoid
-      // creating a producer for each bundle.
-      if (producer == null) {
-        if (producerFactoryFnOpt.isPresent()) {
-           producer = producerFactoryFnOpt.get().apply(producerConfig);
-        } else {
-          producer = new KafkaProducer<K, V>(producerConfig);
-        }
+    @Setup
+    public void setup() {
+      if (producerFactoryFnOpt.isPresent()) {
+        producer = producerFactoryFnOpt.get().apply(producerConfig);
+      } else {
+        producer = new KafkaProducer<K, V>(producerConfig);
       }
     }
 
@@ -1349,13 +1345,16 @@ public class KafkaIO {
     }
 
     @FinishBundle
-    public void finishBundle(Context c) throws Exception {
+    public void finishBundle(Context c) throws IOException {
       producer.flush();
-      producer.close();
-      producer = null;
       checkForFailures();
     }
 
+    @Teardown
+    public void teardown() {
+      producer.close();
+    }
+
     ///////////////////////////////////////////////////////////////////////////////////
 
     private final String topic;


[47/50] [abbrv] incubator-beam git commit: Reverted header and footer to be of type String.

Posted by dh...@apache.org.
Reverted header and footer to be of type String.


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

Branch: refs/heads/gearpump-runner
Commit: 092a1870fc84067ae0e19a736a37160a9a55c653
Parents: 1b420db
Author: Stas Levin <st...@gmail.com>
Authored: Wed Sep 7 09:57:17 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 68 ++++++++------------
 .../java/org/apache/beam/sdk/io/TextIOTest.java | 68 +++++++++++---------
 2 files changed, 63 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/092a1870/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 eefa867..0895123 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
@@ -22,7 +22,6 @@ import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
-
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -36,14 +35,12 @@ import java.util.regex.Pattern;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.Context;
-import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.io.Read.Bounded;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
 import org.apache.beam.sdk.values.PBegin;
@@ -477,10 +474,10 @@ public class TextIO {
       private final String filenameSuffix;
 
       /** An optional header to add to each file. */
-      private final T header;
+      private final String header;
 
       /** An optional footer to add to each file. */
-      private final T footer;
+      private final String footer;
 
       /** The Coder to use to decode each line. */
       private final Coder<T> coder;
@@ -498,8 +495,8 @@ public class TextIO {
         this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true);
       }
 
-      private Bound(String name, String filenamePrefix, String filenameSuffix, T header,
-                    T footer, Coder<T> coder, int numShards, String shardTemplate,
+      private Bound(String name, String filenamePrefix, String filenameSuffix, String header,
+                    String footer, Coder<T> coder, int numShards, String shardTemplate,
                     boolean validate) {
         super(name);
         this.header = header;
@@ -512,14 +509,6 @@ public class TextIO {
         this.validate = validate;
       }
 
-      private String asString(T obj, Coder<T> coder) {
-        try {
-          return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj));
-        } catch (CoderException e) {
-          throw new RuntimeException(e);
-        }
-      }
-
       /**
        * Returns a transform for writing to text files that's like this one but
        * that writes to the file(s) with the given filename prefix.
@@ -605,8 +594,9 @@ public class TextIO {
        * the elements of the input {@link PCollection PCollection<X>} into an
        * output text line. Does not modify this object.
        *
+       * @param <X> the type of the elements of the input {@link PCollection}
        */
-      public Bound<?> withCoder(Coder<? super T> coder) {
+      public <X> Bound<X> withCoder(Coder<X> coder) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
@@ -626,12 +616,12 @@ public class TextIO {
             shardTemplate, false);
       }
 
-      public Bound<T> withHeader(T header) {
+      public Bound<T> withHeader(String header) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
 
-      public Bound<T> withFooter(T footer) {
+      public Bound<T> withFooter(String footer) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
@@ -669,9 +659,9 @@ public class TextIO {
               .withLabel("Validation Enabled"), true)
             .addIfNotDefault(DisplayData.item("numShards", numShards)
               .withLabel("Maximum Output Shards"), 0)
-            .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder))
+            .addIfNotNull(DisplayData.item("fileHeader", header)
               .withLabel("File Header"))
-            .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder))
+            .addIfNotNull(DisplayData.item("fileFooter", footer)
                 .withLabel("File Footer"));
       }
 
@@ -707,11 +697,11 @@ public class TextIO {
         return coder;
       }
 
-      public T getHeader() {
+      public String getHeader() {
         return header;
       }
 
-      public T getFooter() {
+      public String getFooter() {
         return footer;
       }
 
@@ -997,21 +987,17 @@ public class TextIO {
   @VisibleForTesting
   static class TextSink<T> extends FileBasedSink<T> {
     private final Coder<T> coder;
-    private final byte[] header;
-    private final byte[] footer;
+    private final String header;
+    private final String footer;
 
     @VisibleForTesting
     TextSink(
-        String baseOutputFilename, String extension, T header, T footer,
+        String baseOutputFilename, String extension, String header, String footer,
         String fileNameTemplate, Coder<T> coder) {
       super(baseOutputFilename, extension, fileNameTemplate);
       this.coder = coder;
-      try {
-        this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header);
-        this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer);
-      } catch (CoderException e) {
-        throw new RuntimeException(e);
-      }
+      this.header = header;
+      this.footer = footer;
     }
 
     @Override
@@ -1025,10 +1011,10 @@ public class TextIO {
      */
     private static class TextWriteOperation<T> extends FileBasedWriteOperation<T> {
       private final Coder<T> coder;
-      private final byte[] header;
-      private final byte[] footer;
+      private final String header;
+      private final String footer;
 
-      private TextWriteOperation(TextSink<T> sink, Coder<T> coder, byte[] header, byte[] footer) {
+      private TextWriteOperation(TextSink<T> sink, Coder<T> coder, String header, String footer) {
         super(sink);
         this.coder = coder;
         this.header = header;
@@ -1048,20 +1034,20 @@ public class TextIO {
     private static class TextWriter<T> extends FileBasedWriter<T> {
       private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
       private final Coder<T> coder;
-      private final byte[] header;
-      private final byte[] footer;
+      private final String header;
+      private final String footer;
       private OutputStream out;
 
       public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder) {
         this(writeOperation, coder, null, null);
       }
 
-      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, byte[] header) {
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header) {
         this(writeOperation, coder, header, null);
       }
 
-      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, byte[] header,
-                        byte[] footer) {
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header,
+                        String footer) {
         super(writeOperation);
         this.header = header;
         this.footer = footer;
@@ -1069,9 +1055,9 @@ public class TextIO {
         this.coder = coder;
       }
 
-      private void writeLine(byte[] line) throws IOException {
+      private void writeLine(String line) throws IOException {
         if (line != null) {
-          out.write(line);
+          out.write(line.getBytes(StandardCharsets.UTF_8));
           out.write(NEWLINE);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/092a1870/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 7028761..c60b735 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -233,11 +233,11 @@ public class TextIOTest {
     runTestWrite(elems, null, null, coder, numShards);
   }
 
-  <T> void runTestWrite(T[] elems, Coder<T> coder, T header, T footer) throws Exception {
+  <T> void runTestWrite(T[] elems, Coder<T> coder, String header, String footer) throws Exception {
     runTestWrite(elems, header, footer, coder, 1);
   }
 
-  <T> void runTestWrite(T[] elems, T header, T footer, Coder<T> coder, int numShards)
+  <T> void runTestWrite(T[] elems, String header, String footer, Coder<T> coder, int numShards)
       throws Exception {
     String outputName = "file.txt";
     String baseFilename = tmpFolder.newFile(outputName).getPath();
@@ -252,7 +252,7 @@ public class TextIOTest {
       // T==String
       write = (TextIO.Write.Bound<T>) writeStrings;
     } else {
-      write = TextIO.Write.withCoder(coder).to(baseFilename);
+      write = TextIO.Write.to(baseFilename).withCoder(coder);
     }
     write = write.withHeader(header).withFooter(footer);
 
@@ -271,9 +271,9 @@ public class TextIOTest {
 
   public static <T> void assertOutputFiles(
       T[] elems,
-      final T header,
-      final T footer,
-      final Coder<T> coder,
+      final String header,
+      final String footer,
+      Coder<T> coder,
       int numShards,
       TemporaryFolder rootLocation,
       String outputName,
@@ -320,44 +320,48 @@ public class TextIOTest {
       expectedElements.add(line);
     }
 
-    final String headerString =
-        header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header));
-
-    final String footerString =
-        footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer));
-
     ArrayList<String> actualElements =
         Lists.newArrayList(
             Iterables.concat(
                 FluentIterable
                     .from(actual)
-                    .transform(new Function<List<String>, List<String>>() {
-                      @Nullable
-                      @Override
-                      public List<String> apply(List<String> lines) {
-                        ArrayList<String> newLines = Lists.newArrayList(lines);
-                        if (headerString != null) {
-                          newLines.remove(0);
-                        }
-                        if (footerString != null) {
-                          int last = newLines.size() - 1;
-                          newLines.remove(last);
-                        }
-                        return newLines;
-                      }
-                    })
+                    .transform(removeHeaderAndFooter(header, footer))
                     .toList()));
 
     assertThat(actualElements, containsInAnyOrder(expectedElements.toArray()));
 
-    assertTrue(Iterables.all(actual, new Predicate<List<String>>() {
+    assertTrue(Iterables.all(actual, haveProperHeaderAndFooter(header, footer)));
+  }
+
+  private static Function<List<String>, List<String>> removeHeaderAndFooter(final String header,
+                                                                            final String footer) {
+    return new Function<List<String>, List<String>>() {
+      @Nullable
+      @Override
+      public List<String> apply(List<String> lines) {
+        ArrayList<String> newLines = Lists.newArrayList(lines);
+        if (header != null) {
+          newLines.remove(0);
+        }
+        if (footer != null) {
+          int last = newLines.size() - 1;
+          newLines.remove(last);
+        }
+        return newLines;
+      }
+    };
+  }
+
+  private static Predicate<List<String>> haveProperHeaderAndFooter(final String header,
+                                                                   final String footer) {
+    return new Predicate<List<String>>() {
       @Override
-      public boolean apply(@Nullable List<String> fileLines) {
+      public boolean apply(List<String> fileLines) {
         int last = fileLines.size() - 1;
-        return (headerString == null || fileLines.get(0).equals(headerString))
-              && (footerString == null || fileLines.get(last).equals(footerString));
+        return (header == null || fileLines.get(0).equals(header))
+            && (footer == null || fileLines.get(last).equals(footer));
       }
-    }));
+    };
   }
 
   @Test


[19/50] [abbrv] incubator-beam git commit: [BEAM-456] Add MongoDbIO

Posted by dh...@apache.org.
[BEAM-456] Add MongoDbIO


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

Branch: refs/heads/gearpump-runner
Commit: 9ae5cc7310137e928d353eba0104dfd7ae263a96
Parents: 3f48566
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Fri Jul 15 18:44:26 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 sdks/java/io/mongodb/pom.xml                    | 129 +++++
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   | 553 +++++++++++++++++++
 .../beam/sdk/io/mongodb/package-info.java       |  22 +
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      | 209 +++++++
 .../beam/sdk/io/mongodb/package-info.java       |  22 +
 sdks/java/io/pom.xml                            |   1 +
 6 files changed, 936 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml
new file mode 100644
index 0000000..60f1d1e
--- /dev/null
+++ b/sdks/java/io/mongodb/pom.xml
@@ -0,0 +1,129 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-io-parent</artifactId>
+    <version>0.3.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-io-mongodb</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: IO :: MongoDB</name>
+  <description>IO to read and write on MongoDB.</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <properties>
+    <mongo-java-driver.version>3.2.2</mongo-java-driver.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mongodb</groupId>
+      <artifactId>mongo-java-driver</artifactId>
+      <version>${mongo-java-driver.version}</version>
+    </dependency>
+
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>de.flapdoodle.embed</groupId>
+      <artifactId>de.flapdoodle.embed.mongo</artifactId>
+      <version>1.50.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>de.flapdoodle.embed</groupId>
+      <artifactId>de.flapdoodle.embed.process</artifactId>
+      <version>1.50.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
new file mode 100644
index 0000000..7724614
--- /dev/null
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
@@ -0,0 +1,553 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.mongodb;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import com.mongodb.BasicDBObject;
+import com.mongodb.MongoClient;
+import com.mongodb.MongoClientURI;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.MongoCursor;
+import com.mongodb.client.MongoDatabase;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import org.bson.Document;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * IO to read and write data on MongoDB.
+ * <p>
+ * <h3>Reading from MongoDB</h3>
+ * <p>
+ * <p>MongoDbIO source returns a bounded collection of String as {@code PCollection<String>}.
+ * The String is the JSON form of the MongoDB Document.</p>
+ * <p>
+ * <p>To configure the MongoDB source, you have to provide the connection URI, the database name
+ * and the collection name. The following example illustrates various options for configuring the
+ * source:</p>
+ * <p>
+ * <pre>{@code
+ *
+ * pipeline.apply(MongoDbIO.read()
+ *   .withUri("mongodb://localhost:27017")
+ *   .withDatabase("my-database")
+ *   .withCollection("my-collection"))
+ *   // above three are required configuration, returns PCollection<String>
+ *
+ *   // rest of the settings are optional
+ *
+ * }</pre>
+ * <p>
+ * <p>The source also accepts an optional configuration: {@code withFilter()} allows you to
+ * define a JSON filter to get subset of data.</p>
+ * <p>
+ * <h3>Writing to MongoDB</h3>
+ * <p>
+ * <p>MongoDB sink supports writing of Document (as JSON String) in a MongoDB.</p>
+ * <p>
+ * <p>To configure a MongoDB sink, you must specify a connection {@code URI}, a {@code Database}
+ * name, a {@code Collection} name. For instance:</p>
+ * <p>
+ * <pre>{@code
+ *
+ * pipeline
+ *   .apply(...)
+ *   .apply(MongoDbIO.write()
+ *     .withUri("mongodb://localhost:27017")
+ *     .withDatabase("my-database")
+ *     .withCollection("my-collection")
+ *     .withNumSplits(30))
+ *
+ * }</pre>
+ */
+// TODO instead of JSON String, does it make sense to populate the PCollection with BSON Document or
+//  DBObject ??
+public class MongoDbIO {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIO.class);
+
+  /** Read data from MongoDB. */
+  public static Read read() {
+    return new Read(new BoundedMongoDbSource(null, null, null, null, 0));
+  }
+
+  /** Write data to MongoDB. */
+  public static Write write() {
+    return new Write(new Write.MongoDbWriter(null, null, null, 1024L));
+  }
+
+  private MongoDbIO() {
+  }
+
+  /**
+   * A {@link PTransform} to read data from MongoDB.
+   */
+  public static class Read extends PTransform<PBegin, PCollection<String>> {
+
+    public Read withUri(String uri) {
+      return new Read(source.withUri(uri));
+    }
+
+    public Read withDatabase(String database) {
+      return new Read(source.withDatabase(database));
+    }
+
+    public Read withCollection(String collection) {
+      return new Read(source.withCollection(collection));
+    }
+
+    public Read withFilter(String filter) {
+      return new Read(source.withFilter(filter));
+    }
+
+    public Read withNumSplits(int numSplits) {
+      return new Read(source.withNumSplits(numSplits));
+    }
+
+    private final BoundedMongoDbSource source;
+
+    private Read(BoundedMongoDbSource source) {
+      this.source = source;
+    }
+
+    @Override
+    public PCollection<String> apply(PBegin input) {
+      return input.apply(org.apache.beam.sdk.io.Read.from(getSource()));
+    }
+
+    /**
+     * Creates a {@link BoundedSource} with the configuration in {@link Read}.
+     */
+    @VisibleForTesting
+    BoundedSource<String> getSource() {
+      return source;
+    }
+
+    @Override
+    public void validate(PBegin input) {
+      source.validate();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+
+      source.populateDisplayData(builder);
+    }
+
+  }
+
+  private static class BoundedMongoDbSource extends BoundedSource<String> {
+
+    public BoundedMongoDbSource withUri(String uri) {
+      return new BoundedMongoDbSource(uri, database, collection, filter, numSplits);
+    }
+
+    public BoundedMongoDbSource withDatabase(String database) {
+      return new BoundedMongoDbSource(uri, database, collection, filter, numSplits);
+    }
+
+    public BoundedMongoDbSource withCollection(String collection) {
+      return new BoundedMongoDbSource(uri, database, collection, filter, numSplits);
+    }
+
+    public BoundedMongoDbSource withFilter(String filter) {
+      return new BoundedMongoDbSource(uri, database, collection, filter, numSplits);
+    }
+
+    public BoundedMongoDbSource withNumSplits(int numSplits) {
+      return new BoundedMongoDbSource(uri, database, collection, filter, numSplits);
+    }
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+    @Nullable
+    private final String filter;
+    private final int numSplits;
+
+    public BoundedMongoDbSource(String uri, String database, String collection, String filter,
+                                int numSplits) {
+      this.uri = uri;
+      this.database = database;
+      this.collection = collection;
+      this.filter = filter;
+      this.numSplits = numSplits;
+    }
+
+    @Override
+    public Coder getDefaultOutputCoder() {
+      return SerializableCoder.of(String.class);
+    }
+
+    @Override
+    public void validate() {
+      Preconditions.checkNotNull(uri, "uri");
+      Preconditions.checkNotNull(database, "database");
+      Preconditions.checkNotNull(collection, "collection");
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      builder.add(DisplayData.item("uri", uri));
+      builder.add(DisplayData.item("database", database));
+      builder.add(DisplayData.item("collection", collection));
+      builder.addIfNotNull(DisplayData.item("filter", filter));
+      builder.add(DisplayData.item("numSplit", numSplits));
+    }
+
+    @Override
+    public boolean producesSortedKeys(PipelineOptions options) {
+      return false;
+    }
+
+    @Override
+    public BoundedReader createReader(PipelineOptions options) {
+      return new BoundedMongoDbReader(this);
+    }
+
+    @Override
+    public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) {
+      long estimatedSizeBytes = 0L;
+
+      MongoClient mongoClient = new MongoClient();
+      MongoDatabase mongoDatabase = mongoClient.getDatabase(database);
+      MongoCollection mongoCollection = mongoDatabase.getCollection(collection);
+
+      // get the Mongo collStats object
+      // it gives the size for the entire collection
+      BasicDBObject stat = new BasicDBObject();
+      stat.append("collStats", collection);
+      Document stats = mongoDatabase.runCommand(stat);
+      estimatedSizeBytes = Long.valueOf(stats.get("size").toString());
+      return estimatedSizeBytes;
+    }
+
+    @Override
+    public List<BoundedSource<String>> splitIntoBundles(long desiredBundleSizeBytes,
+                                                PipelineOptions options) {
+      MongoClient mongoClient = new MongoClient();
+      MongoDatabase mongoDatabase = mongoClient.getDatabase(database);
+
+      List<Document> splitKeys = null;
+      if (numSplits > 0) {
+        // the user defines his desired number of splits
+        // calculate the batch size
+        long estimatedSizeBytes = getEstimatedSizeBytes(options);
+        desiredBundleSizeBytes = estimatedSizeBytes / numSplits;
+      }
+
+      // the desired batch size is small, using default chunk size of 1MB
+      if (desiredBundleSizeBytes < 1024 * 1024) {
+        desiredBundleSizeBytes = 1 * 1024 * 1024;
+      }
+
+      // now we have the batch size (provided by user or provided by the runner)
+      // we use Mongo splitVector command to get the split keys
+      BasicDBObject splitVectorCommand = new BasicDBObject();
+      splitVectorCommand.append("splitVector", database + "." + collection);
+      splitVectorCommand.append("keyPattern", new BasicDBObject().append("_id", 1));
+      splitVectorCommand.append("force", false);
+      // maxChunkSize is the Mongo partition size in MB
+      LOGGER.debug("Splitting in chunk of {} MB", desiredBundleSizeBytes / 1024 / 1024);
+      splitVectorCommand.append("maxChunkSize", desiredBundleSizeBytes / 1024 / 1024);
+      Document splitVectorCommandResult = mongoDatabase.runCommand(splitVectorCommand);
+      splitKeys = (List<Document>) splitVectorCommandResult.get("splitKeys");
+
+      List<BoundedSource<String>> sources = new ArrayList<>();
+      if (splitKeys.size() < 1) {
+        LOGGER.debug("Split keys is low, using an unique source");
+        sources.add(this);
+        return sources;
+      }
+
+      LOGGER.debug("Number of splits is {}", splitKeys.size());
+      for (String shardFilter : splitKeysToFilters(splitKeys, filter)) {
+        sources.add(this.withFilter(shardFilter));
+      }
+
+      return sources;
+    }
+
+    /**
+     * Transform a list of split keys as a list of filters containing corresponding range.
+     *
+     * <p>The list of split keys contains BSon Document basically containing for example:
+     * <ul>
+     *   <li>_id: 56</li>
+     *   <li>_id: 109</li>
+     *   <li>_id: 256</li>
+     * </ul>
+     * </p>
+     *
+     * This method will generate a list of range filters performing the following splits:
+     * <ul>
+     *   <li>from the beginning of the collection up to _id 56, so basically data with
+     *   _id lower than 56</li>
+     *   <li>from _id 57 up to _id 109</li>
+     *   <li>from _id 110 up to _id 256</li>
+     *   <li>from _id 257 up to the end of the collection, so basically data with _id greater
+     *   than 257</li>
+     * </ul>
+     *
+     * @param splitKeys The list of split keys.
+     * @param additionalFilter A custom (user) additional filter to append to the range filters.
+     * @return A list of filters containing the ranges.
+     */
+    private static List<String> splitKeysToFilters(List<Document> splitKeys, String
+        additionalFilter) {
+      ArrayList<String> filters = new ArrayList<>();
+      String lowestBound = null; // lower boundary (previous split in the iteration)
+      for (int i = 0; i < splitKeys.size(); i++) {
+        String splitKey = splitKeys.get(i).toString();
+        String rangeFilter = null;
+        if (i == 0) {
+          // this is the first split in the list, the filter defines
+          // the range from the beginning up to this split
+          rangeFilter = String.format("{ $and: [ {\"_id\":{$lte:Objectd(\"%s\")}}",
+              splitKey);
+        } else if (i == splitKeys.size() - 1) {
+          // this is the last split in the list, the filter defines
+          // the range from the split up to the end
+          rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\")}}",
+              splitKey);
+        } else {
+          // we are between two splits
+          rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\"),"
+              + "$lte:ObjectId(\"%s\")}}", lowestBound, splitKey);
+        }
+        if (additionalFilter != null && !additionalFilter.isEmpty()) {
+          // user provided a filter, we append the user filter to the range filter
+          rangeFilter = String.format("%s,%s ]}", rangeFilter, additionalFilter);
+        } else {
+          // user didn't provide a filter, just cleany close the range filter
+          rangeFilter = String.format("%s ]}", rangeFilter);
+        }
+
+        filters.add(rangeFilter);
+
+        lowestBound = splitKey;
+      }
+      return filters;
+    }
+  }
+
+  private static class BoundedMongoDbReader extends BoundedSource.BoundedReader<String> {
+
+    private final BoundedMongoDbSource source;
+
+    private MongoClient client;
+    private MongoCursor<Document> cursor;
+    private String current;
+
+    public BoundedMongoDbReader(BoundedMongoDbSource source) {
+      this.source = source;
+    }
+
+    @Override
+    public boolean start() {
+      client = new MongoClient(new MongoClientURI(source.uri));
+
+      MongoDatabase mongoDatabase = client.getDatabase(source.database);
+
+      MongoCollection<Document> mongoCollection = mongoDatabase.getCollection(source.collection);
+
+      if (source.filter == null) {
+        cursor = mongoCollection.find().iterator();
+      } else {
+        Document bson = Document.parse(source.filter);
+        cursor = mongoCollection.find(bson).iterator();
+      }
+
+      return advance();
+    }
+
+    @Override
+    public boolean advance() {
+      if (cursor.hasNext()) {
+        current = cursor.next().toJson();
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public BoundedSource getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    public String getCurrent() {
+      return current;
+    }
+
+    @Override
+    public void close() {
+      try {
+        if (cursor != null) {
+          cursor.close();
+        }
+      } catch (Exception e) {
+        LOGGER.warn("Error closing MongoDB cursor", e);
+      }
+      try {
+        client.close();
+      } catch (Exception e) {
+        LOGGER.warn("Error closing MongoDB client", e);
+      }
+    }
+
+  }
+
+  /**
+   * A {@link PTransform} to write to a MongoDB database.
+   */
+  public static class Write extends PTransform<PCollection<String>, PDone> {
+
+    public Write withUri(String uri) {
+      return new Write(writer.withUri(uri));
+    }
+
+    public Write withDatabase(String database) {
+      return new Write(writer.withDatabase(database));
+    }
+
+    public Write withCollection(String collection) {
+      return new Write(writer.withCollection(collection));
+    }
+
+    public Write withBatchSize(long batchSize) {
+      return new Write(writer.withBatchSize(batchSize));
+    }
+
+    private final MongoDbWriter writer;
+
+    private Write(MongoDbWriter writer) {
+      this.writer = writer;
+    }
+
+    @Override
+    public PDone apply(PCollection<String> input) {
+      input.apply(ParDo.of(writer));
+      return PDone.in(input.getPipeline());
+    }
+
+    @Override
+    public void validate(PCollection<String> input) {
+      writer.validate();
+    }
+
+    private static class MongoDbWriter extends DoFn<String, Void> {
+
+      private final String uri;
+      private final String database;
+      private final String collection;
+      private final long batchSize;
+
+      private MongoClient client;
+      private List<Document> batch;
+
+      public MongoDbWriter(String uri, String database, String collection, long batchSize) {
+        this.uri = uri;
+        this.database = database;
+        this.collection = collection;
+        this.batchSize = batchSize;
+      }
+
+      public MongoDbWriter withUri(String uri) {
+        return new MongoDbWriter(uri, database, collection, batchSize);
+      }
+
+      public MongoDbWriter withDatabase(String database) {
+        return new MongoDbWriter(uri, database, collection, batchSize);
+      }
+
+      public MongoDbWriter withCollection(String collection) {
+        return new MongoDbWriter(uri, database, collection, batchSize);
+      }
+
+      public MongoDbWriter withBatchSize(long batchSize) {
+        return new MongoDbWriter(uri, database, collection, batchSize);
+      }
+
+      public void validate() {
+        Preconditions.checkNotNull(uri, "uri");
+        Preconditions.checkNotNull(database, "database");
+        Preconditions.checkNotNull(collection, "collection");
+        Preconditions.checkNotNull(batchSize, "batchSize");
+      }
+
+      @Setup
+      public void createMongoClient() throws Exception {
+        client = new MongoClient(new MongoClientURI(uri));
+      }
+
+      @StartBundle
+      public void startBundle(Context ctx) throws Exception {
+        batch = new ArrayList<>();
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext ctx) throws Exception {
+        String value = ctx.element();
+
+        batch.add(Document.parse(ctx.element()));
+        if (batch.size() >= batchSize) {
+          finishBundle(ctx);
+        }
+      }
+
+      @FinishBundle
+      public void finishBundle(Context ctx) throws Exception {
+        MongoDatabase mongoDatabase = client.getDatabase(database);
+        MongoCollection mongoCollection = mongoDatabase.getCollection(collection);
+
+        mongoCollection.insertMany(batch);
+
+        batch.clear();
+      }
+
+      @Teardown
+      public void closeMongoClient() throws Exception {
+        client.close();
+        client = null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java
new file mode 100644
index 0000000..fd08b58
--- /dev/null
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Transforms for reading and writing from MongoDB.
+ */
+package org.apache.beam.sdk.io.mongodb;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
new file mode 100644
index 0000000..308e071
--- /dev/null
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.mongodb;
+
+import static org.junit.Assert.assertEquals;
+
+import com.mongodb.MongoClient;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.MongoCursor;
+import com.mongodb.client.MongoDatabase;
+
+import de.flapdoodle.embed.mongo.MongodExecutable;
+import de.flapdoodle.embed.mongo.MongodStarter;
+import de.flapdoodle.embed.mongo.config.IMongodConfig;
+import de.flapdoodle.embed.mongo.config.MongoCmdOptionsBuilder;
+import de.flapdoodle.embed.mongo.config.MongodConfigBuilder;
+import de.flapdoodle.embed.mongo.config.Net;
+import de.flapdoodle.embed.mongo.config.Storage;
+import de.flapdoodle.embed.mongo.distribution.Version;
+import de.flapdoodle.embed.process.io.file.Files;
+import de.flapdoodle.embed.process.runtime.Network;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.ArrayList;
+
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.bson.Document;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test on the MongoDbIO.
+ */
+public class MongoDbIOTest implements Serializable {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIOTest.class);
+
+  private static final String MONGODB_LOCATION = "target/mongodb";
+  private static final int PORT = 27017;
+  private static final String DATABASE = "beam";
+  private static final String COLLECTION = "test";
+
+  private transient MongodExecutable mongodExecutable;
+
+  @Before
+  public void setup() throws Exception {
+    LOGGER.info("Starting MongoDB embedded instance");
+    try {
+      Files.forceDelete(new File(MONGODB_LOCATION));
+    } catch (Exception e) {
+
+    }
+    new File(MONGODB_LOCATION).mkdirs();
+    IMongodConfig mongodConfig = new MongodConfigBuilder()
+        .version(Version.Main.PRODUCTION)
+        .configServer(false)
+        .replication(new Storage(MONGODB_LOCATION, null, 0))
+        .net(new Net("localhost", PORT, Network.localhostIsIPv6()))
+        .cmdOptions(new MongoCmdOptionsBuilder()
+            .syncDelay(10)
+            .useNoPrealloc(true)
+            .useSmallFiles(true)
+            .useNoJournal(true)
+            .build())
+        .build();
+    mongodExecutable = MongodStarter.getDefaultInstance().prepare(mongodConfig);
+    mongodExecutable.start();
+
+    LOGGER.info("Insert test data");
+
+    MongoClient client = new MongoClient("localhost", PORT);
+    MongoDatabase database = client.getDatabase(DATABASE);
+
+    MongoCollection collection = database.getCollection(COLLECTION);
+
+    String[] scientists = {"Einstein", "Darwin", "Copernicus", "Pasteur", "Curie", "Faraday",
+        "Newton", "Bohr", "Galilei", "Maxwell"};
+    for (int i = 1; i <= 1000; i++) {
+      int index = i % scientists.length;
+      Document document = new Document();
+      document.append("_id", i);
+      document.append("scientist", scientists[index]);
+      collection.insertOne(document);
+    }
+
+  }
+
+  @After
+  public void stop() throws Exception {
+    LOGGER.info("Stopping MongoDB instance");
+    mongodExecutable.stop();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testFullRead() throws Exception {
+    TestPipeline pipeline = TestPipeline.create();
+
+    PCollection<String> output = pipeline.apply(
+        MongoDbIO.read()
+          .withUri("mongodb://localhost:" + PORT)
+          .withDatabase(DATABASE)
+          .withCollection(COLLECTION));
+
+    PAssert.thatSingleton(output.apply("Count All", Count.<String>globally()))
+        .isEqualTo(1000L);
+
+    PAssert.that(output
+        .apply("Map Scientist", MapElements.via(new SimpleFunction<String, KV<String, Void>>() {
+          public KV<String, Void> apply(String input) {
+            Document bson = Document.parse(input);
+            return KV.of(bson.getString("scientist"), null);
+          }
+        }))
+        .apply("Count Scientist", Count.<String, Void>perKey())
+    ).satisfies(new SerializableFunction<Iterable<KV<String, Long>>, Void>() {
+      @Override
+      public Void apply(Iterable<KV<String, Long>> input) {
+        for (KV<String, Long> element : input) {
+          assertEquals(100L, element.getValue().longValue());
+        }
+        return null;
+      }
+    });
+
+    pipeline.run();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testReadWithFilter() throws Exception {
+    TestPipeline pipeline = TestPipeline.create();
+
+    PCollection<String> output = pipeline.apply(
+        MongoDbIO.read()
+        .withUri("mongodb://localhost:" + PORT)
+        .withDatabase(DATABASE)
+        .withCollection(COLLECTION)
+        .withFilter("{\"scientist\":\"Einstein\"}"));
+
+    PAssert.thatSingleton(output.apply("Count", Count.<String>globally()))
+        .isEqualTo(100L);
+
+    pipeline.run();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testWrite() throws Exception {
+    TestPipeline pipeline = TestPipeline.create();
+
+    ArrayList<String> data = new ArrayList<>();
+    for (int i = 0; i < 10000; i++) {
+      data.add(String.format("{\"scientist\":\"Test %s\"}", i));
+    }
+    pipeline.apply(Create.of(data))
+        .apply(MongoDbIO.write().withUri("mongodb://localhost:" + PORT).withDatabase("test")
+            .withCollection("test"));
+
+    pipeline.run();
+
+    MongoClient client = new MongoClient("localhost", PORT);
+    MongoDatabase database = client.getDatabase("test");
+    MongoCollection collection = database.getCollection("test");
+
+    MongoCursor cursor = collection.find().iterator();
+
+    int count = 0;
+    while (cursor.hasNext()) {
+      count = count + 1;
+      cursor.next();
+    }
+
+    Assert.assertEquals(10000, count);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
new file mode 100644
index 0000000..fd08b58
--- /dev/null
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Transforms for reading and writing from MongoDB.
+ */
+package org.apache.beam.sdk.io.mongodb;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9ae5cc73/sdks/java/io/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml
index 6cbd615..c4c32ed 100644
--- a/sdks/java/io/pom.xml
+++ b/sdks/java/io/pom.xml
@@ -38,6 +38,7 @@
     <module>jms</module>
     <module>kafka</module>
     <module>kinesis</module>
+    <module>mongodb</module>
   </modules>
 
 </project>


[24/50] [abbrv] incubator-beam git commit: Cloud Datastore naming clean-up

Posted by dh...@apache.org.
Cloud Datastore naming clean-up


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

Branch: refs/heads/gearpump-runner
Commit: f44fa2cb78e9531399edc71ae1387c6c9038e25c
Parents: 0312f15
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Thu Sep 1 13:31:35 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/AutoComplete.java    | 12 ++---
 .../examples/cookbook/DatastoreWordCount.java   | 18 +++----
 .../beam/sdk/io/gcp/datastore/DatastoreIO.java  |  6 +--
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  | 52 ++++++++++----------
 .../beam/sdk/io/gcp/datastore/package-info.java |  2 +-
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   |  6 +--
 6 files changed, 48 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/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 56c7855..2182e6d 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
@@ -96,7 +96,7 @@ import org.joda.time.Duration;
  *   --streaming
  * }</pre>
  *
- * <p>This will update the datastore every 10 seconds based on the last
+ * <p>This will update the Cloud Datastore every 10 seconds based on the last
  * 30 minutes of data received.
  */
 public class AutoComplete {
@@ -380,7 +380,7 @@ public class AutoComplete {
 
   /**
    * Takes as input a the top candidates per prefix, and emits an entity
-   * suitable for writing to Datastore.
+   * suitable for writing to Cloud Datastore.
    *
    * <p>Note: We use ancestor keys for strong consistency. See the Cloud Datastore documentation on
    * <a href="https://cloud.google.com/datastore/docs/concepts/structuring_for_strong_consistency">
@@ -431,7 +431,7 @@ public class AutoComplete {
     Boolean getRecursive();
     void setRecursive(Boolean value);
 
-    @Description("Datastore entity kind")
+    @Description("Cloud Datastore entity kind")
     @Default.String("autocomplete-demo")
     String getKind();
     void setKind(String value);
@@ -441,17 +441,17 @@ public class AutoComplete {
     Boolean getOutputToBigQuery();
     void setOutputToBigQuery(Boolean value);
 
-    @Description("Whether output to Datastore")
+    @Description("Whether output to Cloud Datastore")
     @Default.Boolean(false)
     Boolean getOutputToDatastore();
     void setOutputToDatastore(Boolean value);
 
-    @Description("Datastore ancestor key")
+    @Description("Cloud Datastore ancestor key")
     @Default.String("root")
     String getDatastoreAncestorKey();
     void setDatastoreAncestorKey(String value);
 
-    @Description("Datastore output project ID, defaults to project ID")
+    @Description("Cloud Datastore output project ID, defaults to project ID")
     String getOutputProject();
     void setOutputProject(String value);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/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 eb2165f..434e9fb 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
@@ -47,11 +47,11 @@ import org.apache.beam.sdk.transforms.ParDo;
 /**
  * A WordCount example using DatastoreIO.
  *
- * <p>This example shows how to use DatastoreIO to read from Datastore and
+ * <p>This example shows how to use DatastoreIO to read from Cloud Datastore and
  * write the results to Cloud Storage.  Note that this example will write
- * data to Datastore, which may incur charge for Datastore operations.
+ * data to Cloud Datastore, which may incur charge for Cloud Datastore operations.
  *
- * <p>To run this example, users need to use gcloud to get credential for Datastore:
+ * <p>To run this example, users need to use gcloud to get credential for Cloud Datastore:
  * <pre>{@code
  * $ gcloud auth login
  * }</pre>
@@ -150,7 +150,7 @@ public class DatastoreWordCount {
    * <p>Inherits standard configuration options.
    */
   public static interface Options extends PipelineOptions {
-    @Description("Path of the file to read from and store to Datastore")
+    @Description("Path of the file to read from and store to Cloud Datastore")
     @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt")
     String getInput();
     void setInput(String value);
@@ -160,17 +160,17 @@ public class DatastoreWordCount {
     String getOutput();
     void setOutput(String value);
 
-    @Description("Project ID to read from datastore")
+    @Description("Project ID to read from Cloud Datastore")
     @Validation.Required
     String getProject();
     void setProject(String value);
 
-    @Description("Datastore Entity kind")
+    @Description("Cloud Datastore Entity kind")
     @Default.String("shakespeare-demo")
     String getKind();
     void setKind(String value);
 
-    @Description("Datastore Namespace")
+    @Description("Cloud Datastore Namespace")
     String getNamespace();
     void setNamespace(@Nullable String value);
 
@@ -186,7 +186,7 @@ public class DatastoreWordCount {
 
   /**
    * An example that creates a pipeline to populate DatastoreIO from a
-   * text input.  Forces use of DirectRunner 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);
@@ -217,7 +217,7 @@ public class DatastoreWordCount {
   }
 
   /**
-   * An example that creates a pipeline to do DatastoreIO.Read from Datastore.
+   * An example that creates a pipeline to do DatastoreIO.Read from Cloud Datastore.
    */
   public static void readDataFromDatastore(Options options) {
     Query query = makeAncestorKindQuery(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java
index 5abf015..c50c23a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java
@@ -22,7 +22,7 @@ import org.apache.beam.sdk.annotations.Experimental;
 /**
  * <p>{@link DatastoreIO} provides an API for reading from and writing to
  * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a> over different
- * versions of the Datastore Client libraries.
+ * versions of the Cloud Datastore Client libraries.
  *
  * <p>To use the v1 version see {@link DatastoreV1}.
  */
@@ -32,8 +32,8 @@ public class DatastoreIO {
   private DatastoreIO() {}
 
   /**
-   * Returns a {@link DatastoreV1} that provides an API for accessing Datastore through v1 version
-   * of Datastore Client library.
+   * Returns a {@link DatastoreV1} that provides an API for accessing Cloud Datastore through v1
+   * version of Datastore Client library.
    */
   public static DatastoreV1 v1() {
     return new DatastoreV1();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index e24bc80..6bd03b5 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -91,13 +91,13 @@ import org.slf4j.LoggerFactory;
  * {@link Entity} objects.
  *
  * <p>This API currently requires an authentication workaround. To use {@link DatastoreV1}, users
- * must use the {@code gcloud} command line tool to get credentials for Datastore:
+ * must use the {@code gcloud} command line tool to get credentials for Cloud Datastore:
  * <pre>
  * $ gcloud auth login
  * </pre>
  *
- * <p>To read a {@link PCollection} from a query to Datastore, use {@link DatastoreV1#read} and
- * its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to
+ * <p>To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read}
+ * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to
  * specify the project to query and the query to read from. You can optionally provide a namespace
  * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify
  * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}.
@@ -122,7 +122,7 @@ import org.slf4j.LoggerFactory;
  * {@link com.google.datastore.v1.Query.Builder#setLimit(Int32Value)}, then
  * all returned results will be read by a single Dataflow worker in order to ensure correct data.
  *
- * <p>To write a {@link PCollection} to a Datastore, use {@link DatastoreV1#write},
+ * <p>To write a {@link PCollection} to a Cloud Datastore, use {@link DatastoreV1#write},
  * specifying the Cloud Datastore project to write to:
  *
  * <pre> {@code
@@ -131,7 +131,7 @@ import org.slf4j.LoggerFactory;
  * p.run();
  * } </pre>
  *
- * <p>To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use
+ * <p>To delete a {@link PCollection} of {@link Entity Entities} from Cloud Datastore, use
  * {@link DatastoreV1#deleteEntity()}, specifying the Cloud Datastore project to write to:
  *
  * <pre> {@code
@@ -140,8 +140,8 @@ import org.slf4j.LoggerFactory;
  * p.run();
  * } </pre>
  *
- * <p>To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore,
- * use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to:
+ * <p>To delete entities associated with a {@link PCollection} of {@link Key Keys} from Cloud
+ * Datastore, use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to:
  *
  * <pre> {@code
  * PCollection<Entity> entities = ...;
@@ -169,7 +169,7 @@ import org.slf4j.LoggerFactory;
  * more details.
  *
  * <p>Please see <a href="https://cloud.google.com/datastore/docs/activate">Cloud Datastore Sign Up
- * </a>for security and permission related information specific to Datastore.
+ * </a>for security and permission related information specific to Cloud Datastore.
  *
  * @see org.apache.beam.sdk.runners.PipelineRunner
  */
@@ -180,7 +180,7 @@ public class DatastoreV1 {
   DatastoreV1() {}
 
   /**
-   * Datastore has a limit of 500 mutations per batch operation, so we flush
+   * Cloud Datastore has a limit of 500 mutations per batch operation, so we flush
    * changes to Datastore every 500 entities.
    */
   @VisibleForTesting
@@ -197,7 +197,7 @@ public class DatastoreV1 {
   }
 
   /**
-   * A {@link PTransform} that reads the result rows of a Datastore query as {@code Entity}
+   * A {@link PTransform} that reads the result rows of a Cloud Datastore query as {@code Entity}
    * objects.
    *
    * @see DatastoreIO
@@ -234,7 +234,7 @@ public class DatastoreV1 {
 
     /**
      * Computes the number of splits to be performed on the given query by querying the estimated
-     * size from Datastore.
+     * size from Cloud Datastore.
      */
     static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable String namespace) {
       int numSplits;
@@ -252,7 +252,7 @@ public class DatastoreV1 {
     }
 
     /**
-     * Datastore system tables with statistics are periodically updated. This method fetches
+     * Cloud Datastore system tables with statistics are periodically updated. This method fetches
      * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__}
      * table.
      */
@@ -281,7 +281,7 @@ public class DatastoreV1 {
     /**
      * Get the estimated size of the data returned by the given query.
      *
-     * <p>Datastore provides no way to get a good estimate of how large the result of a query
+     * <p>Cloud Datastore provides no way to get a good estimate of how large the result of a query
      * entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind
      * is specified in the query.
      *
@@ -357,7 +357,7 @@ public class DatastoreV1 {
     }
 
     /**
-     * Returns a new {@link DatastoreV1.Read} that reads from the Datastore for the specified
+     * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified
      * project.
      */
     public DatastoreV1.Read withProjectId(String projectId) {
@@ -400,7 +400,7 @@ public class DatastoreV1 {
      *   <li>If the {@code query} has a user limit set, then {@code numQuerySplits} will be
      *   ignored and no split will be performed.
      *   <li>Under certain cases Cloud Datastore is unable to split query to the requested number of
-     *   splits. In such cases we just use whatever the Datastore returns.
+     *   splits. In such cases we just use whatever the Cloud Datastore returns.
      * </ul>
      */
     public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) {
@@ -492,7 +492,7 @@ public class DatastoreV1 {
     }
 
     /**
-     * A class for v1 Datastore related options.
+     * A class for v1 Cloud Datastore related options.
      */
     @VisibleForTesting
     static class V1Options implements Serializable {
@@ -608,7 +608,7 @@ public class DatastoreV1 {
     }
 
     /**
-     * A {@link DoFn} that reads entities from Datastore for each query.
+     * A {@link DoFn} that reads entities from Cloud Datastore for each query.
      */
     @VisibleForTesting
     static class ReadFn extends DoFn<Query, Entity> {
@@ -908,8 +908,8 @@ public class DatastoreV1 {
      *
      * <p>If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES}
      * times). All mutations in the batch will be committed again, even if the commit was partially
-     * successful. If the retry limit is exceeded, the last exception from the Datastore will be
-     * thrown.
+     * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will
+     * be thrown.
      *
      * @throws DatastoreException if the commit fails or IOException or InterruptedException if
      * backing off between retries fails.
@@ -953,7 +953,7 @@ public class DatastoreV1 {
   }
 
   /**
-   * Returns true if a Datastore key is complete. A key is complete if its last element
+   * Returns true if a Cloud Datastore key is complete. A key is complete if its last element
    * has either an id or a name.
    */
   static boolean isValidKey(Key key) {
@@ -974,7 +974,7 @@ public class DatastoreV1 {
     public Mutation apply(Entity entity) {
       // Verify that the entity to write has a complete key.
       checkArgument(isValidKey(entity.getKey()),
-          "Entities to be written to the Datastore must have complete keys:\n%s", entity);
+          "Entities to be written to the Cloud Datastore must have complete keys:\n%s", entity);
 
       return makeUpsert(entity).build();
     }
@@ -995,7 +995,7 @@ public class DatastoreV1 {
     public Mutation apply(Entity entity) {
       // Verify that the entity to delete has a complete key.
       checkArgument(isValidKey(entity.getKey()),
-          "Entities to be deleted from the Datastore must have complete keys:\n%s", entity);
+          "Entities to be deleted from the Cloud Datastore must have complete keys:\n%s", entity);
 
       return makeDelete(entity.getKey()).build();
     }
@@ -1016,7 +1016,7 @@ public class DatastoreV1 {
     public Mutation apply(Key key) {
       // Verify that the entity to delete has a complete key.
       checkArgument(isValidKey(key),
-          "Keys to be deleted from the Datastore must be complete:\n%s", key);
+          "Keys to be deleted from the Cloud Datastore must be complete:\n%s", key);
 
       return makeDelete(key).build();
     }
@@ -1029,7 +1029,7 @@ public class DatastoreV1 {
   }
 
   /**
-   * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and
+   * A wrapper factory class for Cloud Datastore singleton classes {@link DatastoreFactory} and
    * {@link QuerySplitter}
    *
    * <p>{@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence
@@ -1038,7 +1038,7 @@ public class DatastoreV1 {
   @VisibleForTesting
   static class V1DatastoreFactory implements Serializable {
 
-    /** Builds a Datastore client for the given pipeline options and project. */
+    /** Builds a Cloud Datastore client for the given pipeline options and project. */
     public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) {
       DatastoreOptions.Builder builder =
           new DatastoreOptions.Builder()
@@ -1055,7 +1055,7 @@ public class DatastoreV1 {
       return DatastoreFactory.get().create(builder.build());
     }
 
-    /** Builds a Datastore {@link QuerySplitter}. */
+    /** Builds a Cloud Datastore {@link QuerySplitter}. */
     public QuerySplitter getQuerySplitter() {
       return DatastoreHelper.getQuerySplitter();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java
index 1ca0266..4e5fbc3 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java
@@ -19,6 +19,6 @@
 /**
  * <p>Provides an API for reading from and writing to
  * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a> over different
- * versions of the Datastore Client libraries.
+ * versions of the Cloud Datastore Client libraries.
  */
 package org.apache.beam.sdk.io.gcp.datastore;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f44fa2cb/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
index d96c320..dd1904a 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
@@ -430,7 +430,7 @@ public class DatastoreV1Test {
     UpsertFn upsertFn = new UpsertFn();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Entities to be written to the Datastore must have complete keys");
+    thrown.expectMessage("Entities to be written to the Cloud Datastore must have complete keys");
 
     upsertFn.apply(entity);
   }
@@ -458,7 +458,7 @@ public class DatastoreV1Test {
     DeleteEntityFn deleteEntityFn = new DeleteEntityFn();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Entities to be deleted from the Datastore must have complete keys");
+    thrown.expectMessage("Entities to be deleted from the Cloud Datastore must have complete keys");
 
     deleteEntityFn.apply(entity);
   }
@@ -485,7 +485,7 @@ public class DatastoreV1Test {
     DeleteKeyFn deleteKeyFn = new DeleteKeyFn();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Keys to be deleted from the Datastore must be complete");
+    thrown.expectMessage("Keys to be deleted from the Cloud Datastore must be complete");
 
     deleteKeyFn.apply(key);
   }


[38/50] [abbrv] incubator-beam git commit: Be more accepting in UnboundedReadDeduplicatorTest

Posted by dh...@apache.org.
Be more accepting in UnboundedReadDeduplicatorTest

Don't depend on all the threads failing. Instead, assert that at most
one success was encountered, and we saw at most numThreads - 1 failures.


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

Branch: refs/heads/gearpump-runner
Commit: c8052b6e1b65e37341b41e827f95b31e0df0be99
Parents: 958f3fe
Author: Thomas Groh <tg...@google.com>
Authored: Fri Sep 2 10:43:43 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/UnboundedReadDeduplicatorTest.java    | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c8052b6e/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java
index 7d2a95c..0aa2c49 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java
@@ -19,6 +19,7 @@ package org.apache.beam.runners.direct;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
 import static org.junit.Assert.assertThat;
 
 import java.util.concurrent.CountDownLatch;
@@ -60,18 +61,18 @@ public class UnboundedReadDeduplicatorTest {
     byte[] id = new byte[] {-1, 2, 4, 22};
     UnboundedReadDeduplicator dedupper = CachedIdDeduplicator.create();
     final CountDownLatch startSignal = new CountDownLatch(1);
-    int numThreads = 1000;
+    int numThreads = 50;
     final CountDownLatch readyLatch = new CountDownLatch(numThreads);
     final CountDownLatch finishLine = new CountDownLatch(numThreads);
 
     ExecutorService executor = Executors.newCachedThreadPool();
     AtomicInteger successCount = new AtomicInteger();
-    AtomicInteger failureCount = new AtomicInteger();
+    AtomicInteger noOutputCount = new AtomicInteger();
     for (int i = 0; i < numThreads; i++) {
       executor.submit(new TryOutputIdRunnable(dedupper,
           id,
           successCount,
-          failureCount,
+          noOutputCount,
           readyLatch,
           startSignal,
           finishLine));
@@ -82,8 +83,10 @@ public class UnboundedReadDeduplicatorTest {
     finishLine.await(10L, TimeUnit.SECONDS);
     executor.shutdownNow();
 
+    // The first thread to run will succeed, and no others will
     assertThat(successCount.get(), equalTo(1));
-    assertThat(failureCount.get(), equalTo(numThreads - 1));
+    // The threads may not all complete; all of the threads that do not succeed must not output
+    assertThat(noOutputCount.get(), lessThan(numThreads));
   }
 
   private static class TryOutputIdRunnable implements Runnable {


[16/50] [abbrv] incubator-beam git commit: Add Latest CombineFn and PTransforms

Posted by dh...@apache.org.
Add Latest CombineFn and PTransforms

Add DoFnTester support for specifying input timestamps


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

Branch: refs/heads/gearpump-runner
Commit: 6ee7b620bf8e2ee07c0f30e9ff20363e79765405
Parents: 28ad44d
Author: Scott Wegner <sw...@google.com>
Authored: Thu Aug 18 13:56:34 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/coders/NullableCoder.java   |   7 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  33 ++-
 .../org/apache/beam/sdk/transforms/Latest.java  | 203 ++++++++++++++++
 .../beam/sdk/values/TimestampedValue.java       |  14 ++
 .../beam/sdk/transforms/DoFnTesterTest.java     |  34 ++-
 .../beam/sdk/transforms/LatestFnTests.java      | 233 +++++++++++++++++++
 .../apache/beam/sdk/transforms/LatestTest.java  | 146 ++++++++++++
 .../beam/sdk/values/TimestampedValueTest.java   |  83 +++++++
 8 files changed, 747 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
index 44aadbd..9c6c7c0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
@@ -65,6 +65,13 @@ public class NullableCoder<T> extends StandardCoder<T> {
     this.valueCoder = valueCoder;
   }
 
+  /**
+   * Returns the inner {@link Coder} wrapped by this {@link NullableCoder} instance.
+   */
+  public Coder<T> getValueCoder() {
+    return valueCoder;
+  }
+
   @Override
   public void encode(@Nullable T value, OutputStream outStream, Context context)
       throws IOException, CoderException  {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/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 b867a55..0e018ba 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
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
@@ -221,9 +224,26 @@ public class DoFnTester<InputT, OutputT> {
    * been finished
    */
   public void processElement(InputT element) throws Exception {
-    if (state == State.FINISHED) {
-      throw new IllegalStateException("finishBundle() has already been called");
-    }
+    processTimestampedElement(TimestampedValue.atMinimumTimestamp(element));
+  }
+
+  /**
+   * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a
+   * context where {@link OldDoFn.ProcessContext#element} returns the
+   * given element and timestamp.
+   *
+   * <p>Will call {@link #startBundle} automatically, if it hasn't
+   * already been called.
+   *
+   * <p>If the input timestamp is {@literal null}, the minimum timestamp will be used.
+   *
+   * @throws IllegalStateException if the {@code OldDoFn} under test has already
+   * been finished
+   */
+  public void processTimestampedElement(TimestampedValue<InputT> element) throws Exception {
+    checkNotNull(element, "Timestamped element cannot be null");
+    checkState(state != State.FINISHED, "finishBundle() has already been called");
+
     if (state == State.UNSTARTED) {
       startBundle();
     }
@@ -522,10 +542,13 @@ public class DoFnTester<InputT, OutputT> {
 
   private TestProcessContext<InputT, OutputT> createProcessContext(
       OldDoFn<InputT, OutputT> fn,
-      InputT elem) {
+      TimestampedValue<InputT> elem) {
+    WindowedValue<InputT> windowedValue = WindowedValue.timestampedValueInGlobalWindow(
+        elem.getValue(), elem.getTimestamp());
+
     return new TestProcessContext<>(fn,
         createContext(fn),
-        WindowedValue.valueInGlobalWindow(elem),
+        windowedValue,
         mainOutputTag,
         sideInputs);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
new file mode 100644
index 0000000..7f13649
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * {@link PTransform} and {@link Combine.CombineFn} for computing the latest element
+ * in a {@link PCollection}.
+ *
+ * <p>Example 1: compute the latest value for each session:
+ * <pre><code>
+ * PCollection<Long> input = ...;
+ * PCollection<Long> sessioned = input
+ *    .apply(Window.<Long>into(Sessions.withGapDuration(Duration.standardMinutes(5)));
+ * PCollection<Long> latestValues = sessioned.apply(Latest.<Long>globally());
+ * </code></pre>
+ *
+ * <p>Example 2: track a latest computed value in an aggregator:
+ * <pre><code>
+ * class MyDoFn extends DoFn<String, String> {
+ *  private Aggregator<TimestampedValue<Double>, Double> latestValue =
+ *    createAggregator("latestValue", new Latest.LatestFn<Double>());
+ *
+ *  {@literal @}ProcessElement
+ *  public void processElement(ProcessContext c) {
+ *    double val = // ..
+ *    latestValue.addValue(TimestampedValue.of(val, c.timestamp()));
+ *    // ..
+ *  }
+ * }
+ * </code></pre>
+ *
+ * <p>For elements with the same timestamp, the element chosen for output is arbitrary.
+ */
+public class Latest {
+  // Do not instantiate
+  private Latest() {}
+
+  /**
+   * A {@link Combine.CombineFn} that computes the latest element from a set of inputs. This is
+   * particularly useful as an {@link Aggregator}.
+   *
+   * @param <T> Type of input element.
+   * @see Latest
+   */
+  public static class LatestFn<T>
+      extends Combine.CombineFn<TimestampedValue<T>, TimestampedValue<T>, T> {
+    /** Construct a new {@link LatestFn} instance. */
+    public LatestFn() {}
+
+    @Override
+    public TimestampedValue<T> createAccumulator() {
+      return TimestampedValue.atMinimumTimestamp(null);
+    }
+
+    @Override
+    public TimestampedValue<T> addInput(TimestampedValue<T> accumulator,
+        TimestampedValue<T> input) {
+      checkNotNull(accumulator, "accumulator must be non-null");
+      checkNotNull(input, "input must be non-null");
+
+      if (input.getTimestamp().isBefore(accumulator.getTimestamp())) {
+        return accumulator;
+      } else {
+        return input;
+      }
+    }
+
+    @Override
+    public Coder<TimestampedValue<T>> getAccumulatorCoder(CoderRegistry registry,
+        Coder<TimestampedValue<T>> inputCoder) throws CannotProvideCoderException {
+      return NullableCoder.of(inputCoder);
+    }
+
+    @Override
+    public Coder<T> getDefaultOutputCoder(CoderRegistry registry,
+        Coder<TimestampedValue<T>> inputCoder) throws CannotProvideCoderException {
+      checkState(inputCoder instanceof TimestampedValue.TimestampedValueCoder,
+          "inputCoder must be a TimestampedValueCoder, but was %s", inputCoder);
+
+      TimestampedValue.TimestampedValueCoder<T> inputTVCoder =
+          (TimestampedValue.TimestampedValueCoder<T>) inputCoder;
+      return NullableCoder.of(inputTVCoder.<T>getValueCoder());
+    }
+
+    @Override
+    public TimestampedValue<T> mergeAccumulators(Iterable<TimestampedValue<T>> accumulators) {
+      checkNotNull(accumulators, "accumulators must be non-null");
+
+      Iterator<TimestampedValue<T>> iter = accumulators.iterator();
+      if (!iter.hasNext()) {
+        return createAccumulator();
+      }
+
+      TimestampedValue<T> merged = iter.next();
+      while (iter.hasNext()) {
+        merged = addInput(merged, iter.next());
+      }
+
+      return merged;
+    }
+
+    @Override
+    public T extractOutput(TimestampedValue<T> accumulator) {
+      return accumulator.getValue();
+    }
+  }
+
+  /**
+   * Returns a {@link PTransform} that takes as input a {@link PCollection<T>} and returns a
+   * {@link PCollection<T>} whose contents is the latest element according to its event time, or
+   * {@literal null} if there are no elements.
+   *
+   * @param <T> The type of the elements being combined.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<T>> globally() {
+    return new Globally<>();
+  }
+
+  /**
+   * Returns a {@link PTransform} that takes as input a {@code PCollection<KV<K, V>>} and returns a
+   * {@code PCollection<KV<K, V>>} whose contents is the latest element per-key according to its
+   * event time.
+   *
+   * @param <K> The key type of the elements being combined.
+   * @param <V> The value type of the elements being combined.
+   */
+  public static <K, V> PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>> perKey() {
+    return new PerKey<>();
+  }
+
+  private static class Globally<T> extends PTransform<PCollection<T>, PCollection<T>> {
+    @Override
+    public PCollection<T> apply(PCollection<T> input) {
+      Coder<T> inputCoder = input.getCoder();
+
+      return input
+          .apply("Reify Timestamps", ParDo.of(
+            new DoFn<T, TimestampedValue<T>>() {
+              @ProcessElement
+              public void processElement(ProcessContext c) {
+                c.output(TimestampedValue.of(c.element(), c.timestamp()));
+              }
+            })).setCoder(TimestampedValue.TimestampedValueCoder.of(inputCoder))
+          .apply("Latest Value", Combine.globally(new LatestFn<T>()))
+            .setCoder(NullableCoder.of(inputCoder));
+    }
+  }
+
+  private static class PerKey<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>> {
+    @Override
+    public PCollection<KV<K, V>> apply(PCollection<KV<K, V>> input) {
+      checkNotNull(input);
+      checkArgument(input.getCoder() instanceof KvCoder,
+          "Input specifiedCoder must be an instance of KvCoder, but was %s", input.getCoder());
+
+      @SuppressWarnings("unchecked")
+      KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+      return input
+          .apply("Reify Timestamps", ParDo.of(
+            new DoFn<KV<K, V>, KV<K, TimestampedValue<V>>>() {
+              @ProcessElement
+              public void processElement(ProcessContext c) {
+                c.output(KV.of(c.element().getKey(), TimestampedValue.of(c.element().getValue(),
+                    c.timestamp())));
+              }
+            })).setCoder(KvCoder.of(
+                inputCoder.getKeyCoder(),
+                TimestampedValue.TimestampedValueCoder.of(inputCoder.getValueCoder())))
+          .apply("Latest Value", Combine.<K, TimestampedValue<V>, V>perKey(new LatestFn<V>()))
+            .setCoder(inputCoder);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
index f2ad616..dd80fb2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
@@ -31,6 +31,7 @@ import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.InstantCoder;
 import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.PropertyNames;
 import org.joda.time.Instant;
 
@@ -43,6 +44,13 @@ import org.joda.time.Instant;
  * @param <V> the type of the value
  */
 public class TimestampedValue<V> {
+  /**
+   * Returns a new {@link TimestampedValue} with the
+   * {@link BoundedWindow#TIMESTAMP_MIN_VALUE minimum timestamp}.
+   */
+  public static <V> TimestampedValue<V> atMinimumTimestamp(V value) {
+    return of(value, BoundedWindow.TIMESTAMP_MIN_VALUE);
+  }
 
   /**
    * Returns a new {@code TimestampedValue} with the given value and timestamp.
@@ -136,6 +144,10 @@ public class TimestampedValue<V> {
       return Arrays.<Coder<?>>asList(valueCoder);
     }
 
+    public Coder<T> getValueCoder() {
+      return valueCoder;
+    }
+
     public static <T> List<Object> getInstanceComponents(TimestampedValue<T> exampleValue) {
       return Arrays.<Object>asList(exampleValue.getValue());
     }
@@ -147,6 +159,8 @@ public class TimestampedValue<V> {
   private final Instant timestamp;
 
   protected TimestampedValue(V value, Instant timestamp) {
+    checkNotNull(timestamp, "timestamp must be non-null");
+
     this.value = value;
     this.timestamp = timestamp;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/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 2649be5..3ed30fd 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
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasItems;
@@ -35,7 +36,9 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -44,6 +47,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class DoFnTesterTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void processElement() throws Exception {
@@ -126,6 +130,16 @@ public class DoFnTesterTest {
   }
 
   @Test
+  public void processElementAfterFinish() throws Exception {
+    DoFnTester<Long, String> tester = DoFnTester.of(new CounterDoFn());
+    tester.finishBundle();
+
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("finishBundle() has already been called");
+    tester.processElement(1L);
+  }
+
+  @Test
   public void processBatch() throws Exception {
     CounterDoFn counterDoFn = new CounterDoFn();
     DoFnTester<Long, String> tester = DoFnTester.of(counterDoFn);
@@ -145,7 +159,25 @@ public class DoFnTesterTest {
   }
 
   @Test
-  public void processElementWithTimestamp() throws Exception {
+  public void processTimestampedElement() throws Exception {
+    DoFn<Long, TimestampedValue<Long>> reifyTimestamps = new ReifyTimestamps();
+
+    DoFnTester<Long, TimestampedValue<Long>> tester = DoFnTester.of(reifyTimestamps);
+
+    TimestampedValue<Long> input = TimestampedValue.of(1L, new Instant(100));
+    tester.processTimestampedElement(input);
+    assertThat(tester.takeOutputElements(), contains(input));
+  }
+
+  static class ReifyTimestamps extends DoFn<Long, TimestampedValue<Long>> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(TimestampedValue.of(c.element(), c.timestamp()));
+    }
+  }
+
+  @Test
+  public void processElementWithOutputTimestamp() throws Exception {
     CounterDoFn counterDoFn = new CounterDoFn();
     DoFnTester<Long, String> tester = DoFnTester.of(counterDoFn);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
new file mode 100644
index 0000000..84b5b68
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.isOneOf;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Objects;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link Latest.LatestFn}.
+ * */
+@RunWith(JUnit4.class)
+public class LatestFnTests {
+  private static final Instant INSTANT = new Instant(100);
+  private static final long VALUE = 100 * INSTANT.getMillis();
+
+  private static final TimestampedValue<Long> TV = TimestampedValue.of(VALUE, INSTANT);
+  private static final TimestampedValue<Long> TV_MINUS_TEN =
+      TimestampedValue.of(VALUE - 10, INSTANT.minus(10));
+  private static final TimestampedValue<Long> TV_PLUS_TEN =
+      TimestampedValue.of(VALUE + 10, INSTANT.plus(10));
+
+  @Rule
+  public final ExpectedException thrown = ExpectedException.none();
+
+  private final Latest.LatestFn<Long> fn = new Latest.LatestFn<>();
+  private final Instant baseTimestamp = Instant.now();
+
+  @Test
+  public void testDefaultValue() {
+    assertThat(fn.defaultValue(), nullValue());
+  }
+
+  @Test
+  public void testCreateAccumulator() {
+    assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.createAccumulator());
+  }
+
+  @Test
+  public void testAddInputInitialAdd() {
+    TimestampedValue<Long> input = TV;
+    assertEquals(input, fn.addInput(fn.createAccumulator(), input));
+  }
+
+  @Test
+  public void testAddInputMinTimestamp() {
+    TimestampedValue<Long> input = TimestampedValue.atMinimumTimestamp(1234L);
+    assertEquals(input, fn.addInput(fn.createAccumulator(), input));
+  }
+
+  @Test
+  public void testAddInputEarlierValue() {
+    assertEquals(TV, fn.addInput(TV, TV_MINUS_TEN));
+  }
+
+  @Test
+  public void testAddInputLaterValue() {
+    assertEquals(TV_PLUS_TEN, fn.addInput(TV, TV_PLUS_TEN));
+  }
+
+  @Test
+  public void testAddInputSameTimestamp() {
+    TimestampedValue<Long> accum = TimestampedValue.of(100L, INSTANT);
+    TimestampedValue<Long> input = TimestampedValue.of(200L, INSTANT);
+
+    assertThat("Latest for values with the same timestamp is chosen arbitrarily",
+        fn.addInput(accum, input), isOneOf(accum, input));
+  }
+
+  @Test
+  public void testAddInputNullAccumulator() {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("accumulators");
+    fn.addInput(null, TV);
+  }
+
+  @Test
+  public void testAddInputNullInput() {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("input");
+    fn.addInput(TV, null);
+  }
+
+  @Test
+  public void testAddInputNullValue() {
+    TimestampedValue<Long> input = TimestampedValue.of(null, INSTANT.plus(10));
+    assertEquals("Null values are allowed", input, fn.addInput(TV, input));
+  }
+
+  @Test
+  public void testMergeAccumulatorsMultipleValues() {
+    Iterable<TimestampedValue<Long>> accums = Lists.newArrayList(
+        TV,
+        TV_PLUS_TEN,
+        TV_MINUS_TEN
+    );
+
+    assertEquals(TV_PLUS_TEN, fn.mergeAccumulators(accums));
+  }
+
+  @Test
+  public void testMergeAccumulatorsSingleValue() {
+    assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV)));
+  }
+
+  @Test
+  public void testMergeAccumulatorsEmptyIterable() {
+    ArrayList<TimestampedValue<Long>> emptyAccums = Lists.newArrayList();
+    assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.mergeAccumulators(emptyAccums));
+  }
+
+  @Test
+  public void testMergeAccumulatorsDefaultAccumulator() {
+    TimestampedValue<Long> defaultAccum = fn.createAccumulator();
+    assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV, defaultAccum)));
+  }
+
+  @Test
+  public void testMergeAccumulatorsAllDefaultAccumulators() {
+    TimestampedValue<Long> defaultAccum = fn.createAccumulator();
+    assertEquals(defaultAccum, fn.mergeAccumulators(
+        Lists.newArrayList(defaultAccum, defaultAccum)));
+  }
+
+  @Test
+  public void testMergeAccumulatorsNullIterable() {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("accumulators");
+    fn.mergeAccumulators(null);
+  }
+
+  @Test
+  public void testExtractOutput() {
+    assertEquals(TV.getValue(), fn.extractOutput(TV));
+  }
+
+  @Test
+  public void testExtractOutputDefaultAggregator() {
+    TimestampedValue<Long> accum = fn.createAccumulator();
+    assertThat(fn.extractOutput(accum), nullValue());
+  }
+
+  @Test
+  public void testExtractOutputNullValue() {
+    TimestampedValue<Long> accum = TimestampedValue.of(null, baseTimestamp);
+    assertEquals(null, fn.extractOutput(accum));
+  }
+
+  @Test
+  public void testAggregator() throws Exception {
+    LatestAggregatorsFn<Long> doFn = new LatestAggregatorsFn<>(TV_MINUS_TEN.getValue());
+    DoFnTester<Long, Long> harness = DoFnTester.of(doFn);
+    for (TimestampedValue<Long> element : Arrays.asList(TV, TV_PLUS_TEN, TV_MINUS_TEN)) {
+      harness.processTimestampedElement(element);
+    }
+
+    assertEquals(TV_PLUS_TEN.getValue(), harness.getAggregatorValue(doFn.allValuesAgg));
+    assertEquals(TV_MINUS_TEN.getValue(), harness.getAggregatorValue(doFn.specialValueAgg));
+    assertThat(harness.getAggregatorValue(doFn.noValuesAgg), nullValue());
+  }
+
+  @Test
+  public void testDefaultCoderHandlesNull() throws CannotProvideCoderException {
+    Latest.LatestFn<Long> fn = new Latest.LatestFn<>();
+
+    CoderRegistry registry = new CoderRegistry();
+    TimestampedValue.TimestampedValueCoder<Long> inputCoder =
+        TimestampedValue.TimestampedValueCoder.of(VarLongCoder.of());
+
+    assertThat("Default output coder should handle null values",
+        fn.getDefaultOutputCoder(registry, inputCoder), instanceOf(NullableCoder.class));
+    assertThat("Default accumulator coder should handle null values",
+        fn.getAccumulatorCoder(registry, inputCoder), instanceOf(NullableCoder.class));
+  }
+
+  static class LatestAggregatorsFn<T> extends DoFn<T, T> {
+    private final T specialValue;
+    LatestAggregatorsFn(T specialValue) {
+      this.specialValue = specialValue;
+    }
+
+    Aggregator<TimestampedValue<T>, T> allValuesAgg =
+        createAggregator("allValues", new Latest.LatestFn<T>());
+
+    Aggregator<TimestampedValue<T>, T> specialValueAgg =
+        createAggregator("oneValue", new Latest.LatestFn<T>());
+
+    Aggregator<TimestampedValue<T>, T> noValuesAgg =
+        createAggregator("noValues", new Latest.LatestFn<T>());
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      TimestampedValue<T> val = TimestampedValue.of(c.element(), c.timestamp());
+      allValuesAgg.addValue(val);
+      if (Objects.equals(c.element(), specialValue)) {
+        specialValueAgg.addValue(val);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
new file mode 100644
index 0000000..ce9ae37
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link Latest} {@link PTransform} and {@link Combine.CombineFn}.
+ */
+@RunWith(JUnit4.class)
+public class LatestTest implements Serializable {
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testGloballyEventTimestamp() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<String> output =
+        p.apply(Create.timestamped(
+            TimestampedValue.of("foo", new Instant(100)),
+            TimestampedValue.of("bar", new Instant(300)),
+            TimestampedValue.of("baz", new Instant(200))
+        ))
+        .apply(Latest.<String>globally());
+
+    PAssert.that(output).containsInAnyOrder("bar");
+    p.run();
+  }
+
+  @Test
+  public void testGloballyOutputCoder() {
+    TestPipeline p = TestPipeline.create();
+    BigEndianLongCoder inputCoder = BigEndianLongCoder.of();
+
+    PCollection<Long> output =
+        p.apply(Create.of(1L, 2L).withCoder(inputCoder))
+            .apply(Latest.<Long>globally());
+
+    Coder<Long> outputCoder = output.getCoder();
+    assertThat(outputCoder, instanceOf(NullableCoder.class));
+    assertEquals(inputCoder, ((NullableCoder<?>) outputCoder).getValueCoder());
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testGloballyEmptyCollection() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Long> emptyInput = p.apply(Create.<Long>of()
+        // Explicitly set coder such that then runner enforces encodability.
+        .withCoder(VarLongCoder.of()));
+    PCollection<Long> output = emptyInput.apply(Latest.<Long>globally());
+
+    PAssert.that(output).containsInAnyOrder((Long) null);
+    p.run();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testPerKeyEventTimestamp() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<KV<String, String>> output =
+        p.apply(Create.timestamped(
+            TimestampedValue.of(KV.of("A", "foo"), new Instant(100)),
+            TimestampedValue.of(KV.of("B", "bar"), new Instant(300)),
+            TimestampedValue.of(KV.of("A", "baz"), new Instant(200))
+        ))
+            .apply(Latest.<String, String>perKey());
+
+    PAssert.that(output).containsInAnyOrder(KV.of("B", "bar"), KV.of("A", "baz"));
+    p.run();
+  }
+
+  @Test
+  public void testPerKeyOutputCoder() {
+    TestPipeline p = TestPipeline.create();
+    KvCoder<String, Long> inputCoder = KvCoder.of(
+        AvroCoder.of(String.class), AvroCoder.of(Long.class));
+
+    PCollection<KV<String, Long>> output =
+        p.apply(Create.of(KV.of("foo", 1L)).withCoder(inputCoder))
+            .apply(Latest.<String, Long>perKey());
+
+    assertEquals("Should use input coder for outputs", inputCoder, output.getCoder());
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testPerKeyEmptyCollection() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<KV<String, String>> output =
+        p.apply(Create.<KV<String, String>>of().withCoder(KvCoder.of(
+            StringUtf8Coder.of(), StringUtf8Coder.of())))
+         .apply(Latest.<String, String>perKey());
+
+    PAssert.that(output).empty();
+    p.run();
+  }
+
+  /** Helper method to easily create a timestamped value. */
+  private static TimestampedValue<Long> timestamped(Instant timestamp) {
+    return TimestampedValue.of(uniqueLong.incrementAndGet(), timestamp);
+  }
+  private static final AtomicLong uniqueLong = new AtomicLong();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ee7b620/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
new file mode 100644
index 0000000..a982f31
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
@@ -0,0 +1,83 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF 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.values;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.testing.EqualsTester;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link TimestampedValue}.
+ */
+@RunWith(JUnit4.class)
+public class TimestampedValueTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testValues() {
+    Instant now = Instant.now();
+    TimestampedValue<String> tsv = TimestampedValue.of("foobar", now);
+
+    assertEquals(now, tsv.getTimestamp());
+    assertEquals("foobar", tsv.getValue());
+  }
+
+  @Test
+  public void testAtMinimumTimestamp() {
+    TimestampedValue<String> tsv = TimestampedValue.atMinimumTimestamp("foobar");
+    assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, tsv.getTimestamp());
+  }
+
+  @Test
+  public void testNullTimestamp() {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("timestamp");
+    TimestampedValue.of("foobar", null);
+  }
+
+  @Test
+  public void testNullValue() {
+    TimestampedValue<String> tsv = TimestampedValue.atMinimumTimestamp(null);
+    assertEquals(null, tsv.getValue());
+  }
+
+  @Test
+  public void testEquality() {
+    new EqualsTester()
+        .addEqualityGroup(
+            TimestampedValue.of("foo", new Instant(1000)),
+            TimestampedValue.of("foo", new Instant(1000)))
+        .addEqualityGroup(TimestampedValue.of("foo", new Instant(2000)))
+        .addEqualityGroup(TimestampedValue.of("bar", new Instant(1000)))
+        .addEqualityGroup(
+            TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE),
+            TimestampedValue.atMinimumTimestamp("foo"))
+        .testEquals();
+  }
+}


[48/50] [abbrv] incubator-beam git commit: Added javadoc to TextIO#withHeader and TextIO#withFooter.

Posted by dh...@apache.org.
Added javadoc to TextIO#withHeader and TextIO#withFooter.


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

Branch: refs/heads/gearpump-runner
Commit: 5084580f0f11f97df00c1c972056acaae713c41f
Parents: 092a187
Author: Stas Levin <st...@gmail.com>
Authored: Wed Sep 7 18:33:48 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:14 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/io/TextIO.java    | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5084580f/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 0895123..e75aaa9 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
@@ -448,10 +448,22 @@ public class TextIO {
       return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation();
     }
 
+    /**
+     * Returns a transform for writing to text files that adds a header string to the files
+     * it writes.
+     *
+     * @param header the string to be added as file header
+     */
     public static Bound<String> withHeader(String header) {
       return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header);
     }
 
+    /**
+     * Returns a transform for writing to text files that adds a footer string to the files
+     * it writes.
+     *
+     * @param footer the string to be added as file footer
+     */
     public static Bound<String> withFooter(String footer) {
       return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer);
     }


[18/50] [abbrv] incubator-beam git commit: [BEAM-572] Remove Spark Reference in WordCount

Posted by dh...@apache.org.
[BEAM-572] Remove Spark Reference 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/cf9ce2ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/cf9ce2ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/cf9ce2ff

Branch: refs/heads/gearpump-runner
Commit: cf9ce2ff27dfe13715119c5a9c04900f0e5e6e93
Parents: 435054b
Author: Mark Liu <ma...@markliu-macbookpro.roam.corp.google.com>
Authored: Fri Aug 26 16:13:17 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/examples/WordCount.java     | 24 +-------------------
 .../org/apache/beam/examples/WordCountIT.java   |  9 ++++++++
 2 files changed, 10 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cf9ce2ff/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 0275651..793ee4b 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
@@ -18,7 +18,6 @@
 package org.apache.beam.examples;
 
 import com.google.common.base.Strings;
-import com.google.common.io.Resources;
 import java.io.IOException;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
@@ -165,7 +164,7 @@ public class WordCount {
    */
   public static interface WordCountOptions extends PipelineOptions {
     @Description("Path of the file to read from")
-    @Default.InstanceFactory(InputFactory.class)
+    @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt")
     String getInputFile();
     void setInputFile(String value);
 
@@ -194,27 +193,6 @@ public class WordCount {
         }
       }
     }
-
-    /**
-     * Return default input file path according to runner type.
-     *
-     * <p><ul>
-     *   <li>SparkRunner:
-     *   .../src/test/resources/LICENSE</li>
-     *   <li>other runners:
-     *   gs://apache-beam-samples/apache/LICENSE</li>
-     * </ul>
-     */
-    public static class InputFactory implements DefaultValueFactory<String> {
-      @Override
-      public String create(PipelineOptions options) {
-        if (options.getRunner().getName().contains("SparkRunner")) {
-          return Resources.getResource("LICENSE").getPath();
-        } else {
-          return "gs://apache-beam-samples/apache/LICENSE";
-        }
-      }
-    }
   }
 
   public static void main(String[] args) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cf9ce2ff/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
index ca0c9d6..b0e0fe0 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
@@ -18,6 +18,7 @@
 
 package org.apache.beam.examples;
 
+import com.google.common.io.Resources;
 import java.util.Date;
 import org.apache.beam.examples.WordCount.WordCountOptions;
 import org.apache.beam.sdk.options.Default;
@@ -61,6 +62,14 @@ public class WordCountIT {
     options.setOnSuccessMatcher(
         new FileChecksumMatcher(options.getOutputChecksum(), options.getOutput() + "*"));
 
+    String e2eTestInputPath = "gs://apache-beam-samples/apache/LICENSE";
+    // Spark runner currently doesn't support GCS I/O, change default input to:
+    // .../src/test/resources/LICENSE
+    if (options.getRunner().getName().contains("SparkRunner")) {
+      e2eTestInputPath = Resources.getResource("LICENSE").getPath();
+    }
+    options.setInputFile(e2eTestInputPath);
+
     WordCount.main(TestPipeline.convertToArgs(options));
   }
 }


[30/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
deleted file mode 100644
index 7404e1b..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
+++ /dev/null
@@ -1,536 +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.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.io.Serializable;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Implements the logic to hold the output watermark for a computation back
- * until it has seen all the elements it needs based on the input watermark for the
- * computation.
- *
- * <p>The backend ensures the output watermark can never progress beyond the
- * input watermark for a computation. GroupAlsoByWindows computations may add a 'hold'
- * to the output watermark in order to prevent it progressing beyond a time within a window.
- * The hold will be 'cleared' when the associated pane is emitted.
- *
- * <p>This class is only intended for use by {@link ReduceFnRunner}. The two evolve together and
- * will likely break any other uses.
- *
- * @param <W> The kind of {@link BoundedWindow} the hold is for.
- */
-class WatermarkHold<W extends BoundedWindow> implements Serializable {
-  /**
-   * Return tag for state containing the output watermark hold
-   * used for elements.
-   */
-  public static <W extends BoundedWindow>
-      StateTag<Object, WatermarkHoldState<W>> watermarkHoldTagForOutputTimeFn(
-          OutputTimeFn<? super W> outputTimeFn) {
-    return StateTags.<Object, WatermarkHoldState<W>>makeSystemTagInternal(
-        StateTags.<W>watermarkStateInternal("hold", outputTimeFn));
-  }
-
-  /**
-   * Tag for state containing end-of-window and garbage collection output watermark holds.
-   * (We can't piggy-back on the data hold state since the outputTimeFn may be
-   * {@link OutputTimeFns#outputAtLatestInputTimestamp()}, in which case every pane will
-   * would take the end-of-window time as its element time.)
-   */
-  @VisibleForTesting
-  public static final StateTag<Object, WatermarkHoldState<BoundedWindow>> EXTRA_HOLD_TAG =
-      StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal(
-          "extra", OutputTimeFns.outputAtEarliestInputTimestamp()));
-
-  private final TimerInternals timerInternals;
-  private final WindowingStrategy<?, W> windowingStrategy;
-  private final StateTag<Object, WatermarkHoldState<W>> elementHoldTag;
-
-  public WatermarkHold(TimerInternals timerInternals, WindowingStrategy<?, W> windowingStrategy) {
-    this.timerInternals = timerInternals;
-    this.windowingStrategy = windowingStrategy;
-    this.elementHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn());
-  }
-
-  /**
-   * Add a hold to prevent the output watermark progressing beyond the (possibly adjusted) timestamp
-   * of the element in {@code context}. We allow the actual hold time to be shifted later by
-   * {@link OutputTimeFn#assignOutputTime}, but no further than the end of the window. The hold will
-   * remain until cleared by {@link #extractAndRelease}. Return the timestamp at which the hold
-   * was placed, or {@literal null} if no hold was placed.
-   *
-   * <p>In the following we'll write {@code E} to represent an element's timestamp after passing
-   * through the window strategy's output time function, {@code IWM} for the local input watermark,
-   * {@code OWM} for the local output watermark, and {@code GCWM} for the garbage collection
-   * watermark (which is at {@code IWM - getAllowedLateness}). Time progresses from left to right,
-   * and we write {@code [ ... ]} to denote a bounded window with implied lower bound.
-   *
-   * <p>Note that the GCWM will be the same as the IWM if {@code getAllowedLateness}
-   * is {@code ZERO}.
-   *
-   * <p>Here are the cases we need to handle. They are conceptually considered in the
-   * sequence written since if getAllowedLateness is ZERO the GCWM is the same as the IWM.
-   * <ol>
-   * <li>(Normal)
-   * <pre>
-   *          |
-   *      [   | E        ]
-   *          |
-   *         IWM
-   * </pre>
-   * This is, hopefully, the common and happy case. The element is locally on-time and can
-   * definitely make it to an {@code ON_TIME} pane which we can still set an end-of-window timer
-   * for. We place an element hold at E, which may contribute to the {@code ON_TIME} pane's
-   * timestamp (depending on the output time function). Thus the OWM will not proceed past E
-   * until the next pane fires.
-   *
-   * <li>(Discard - no target window)
-   * <pre>
-   *                       |                            |
-   *      [     E        ] |                            |
-   *                       |                            |
-   *                     GCWM  <-getAllowedLateness->  IWM
-   * </pre>
-   * The element is very locally late. The window has been garbage collected, thus there
-   * is no target pane E could be assigned to. We discard E.
-   *
-   * <li>(Unobservably late)
-   * <pre>
-   *          |    |
-   *      [   | E  |     ]
-   *          |    |
-   *         OWM  IWM
-   * </pre>
-   * The element is locally late, however we can still treat this case as for 'Normal' above
-   * since the IWM has not yet passed the end of the window and the element is ahead of the
-   * OWM. In effect, we get to 'launder' the locally late element and consider it as locally
-   * on-time because no downstream computation can observe the difference.
-   *
-   * <li>(Maybe late 1)
-   * <pre>
-   *          |            |
-   *      [   | E        ] |
-   *          |            |
-   *         OWM          IWM
-   * </pre>
-   * The end-of-window timer may have already fired for this window, and thus an {@code ON_TIME}
-   * pane may have already been emitted. However, if timer firings have been delayed then it
-   * is possible the {@code ON_TIME} pane has not yet been emitted. We can't place an element
-   * hold since we can't be sure if it will be cleared promptly. Thus this element *may* find
-   * its way into an {@code ON_TIME} pane, but if so it will *not* contribute to that pane's
-   * timestamp. We may however set a garbage collection hold if required.
-   *
-   * <li>(Maybe late 2)
-   * <pre>
-   *               |   |
-   *      [     E  |   | ]
-   *               |   |
-   *              OWM IWM
-   * </pre>
-   * The end-of-window timer has not yet fired, so this element may still appear in an
-   * {@code ON_TIME} pane. However the element is too late to contribute to the output
-   * watermark hold, and thus won't contribute to the pane's timestamp. We can still place an
-   * end-of-window hold.
-   *
-   * <li>(Maybe late 3)
-   * <pre>
-   *               |       |
-   *      [     E  |     ] |
-   *               |       |
-   *              OWM     IWM
-   * </pre>
-   * As for the (Maybe late 2) case, however we don't even know if the end-of-window timer
-   * has already fired, or it is about to fire. We can place only the garbage collection hold,
-   * if required.
-   *
-   * <li>(Definitely late)
-   * <pre>
-   *                       |   |
-   *      [     E        ] |   |
-   *                       |   |
-   *                      OWM IWM
-   * </pre>
-   * The element is definitely too late to make an {@code ON_TIME} pane. We are too late to
-   * place an end-of-window hold. We can still place a garbage collection hold if required.
-   *
-   * </ol>
-   */
-  @Nullable
-  public Instant addHolds(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
-    Instant hold = addElementHold(context);
-    if (hold == null) {
-      hold = addEndOfWindowOrGarbageCollectionHolds(context, false/*paneIsEmpty*/);
-    }
-    return hold;
-  }
-
-  /**
-   * Return {@code timestamp}, possibly shifted forward in time according to the window
-   * strategy's output time function.
-   */
-  private Instant shift(Instant timestamp, W window) {
-    Instant shifted = windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window);
-    checkState(!shifted.isBefore(timestamp),
-        "OutputTimeFn moved element from %s to earlier time %s for window %s",
-        timestamp, shifted, window);
-    checkState(timestamp.isAfter(window.maxTimestamp())
-            || !shifted.isAfter(window.maxTimestamp()),
-        "OutputTimeFn moved element from %s to %s which is beyond end of "
-            + "window %s",
-        timestamp, shifted, window);
-
-    return shifted;
-  }
-
-  /**
-   * Attempt to add an 'element hold'. Return the {@link Instant} at which the hold was
-   * added (ie the element timestamp plus any forward shift requested by the
-   * {@link WindowingStrategy#getOutputTimeFn}), or {@literal null} if no hold was added.
-   * The hold is only added if both:
-   * <ol>
-   * <li>The backend will be able to respect it. In other words the output watermark cannot
-   * be ahead of the proposed hold time.
-   * <li>A timer will be set (by {@link ReduceFnRunner}) to clear the hold by the end of the
-   * window. In other words the input watermark cannot be ahead of the end of the window.
-   * </ol>
-   * The hold ensures the pane which incorporates the element is will not be considered late by
-   * any downstream computation when it is eventually emitted.
-   */
-  @Nullable
-  private Instant addElementHold(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
-    // Give the window function a chance to move the hold timestamp forward to encourage progress.
-    // (A later hold implies less impediment to the output watermark making progress, which in
-    // turn encourages end-of-window triggers to fire earlier in following computations.)
-    Instant elementHold = shift(context.timestamp(), context.window());
-
-    Instant outputWM = timerInternals.currentOutputWatermarkTime();
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-
-    String which;
-    boolean tooLate;
-    // TODO: These case labels could be tightened.
-    // See the case analysis in addHolds above for the motivation.
-    if (outputWM != null && elementHold.isBefore(outputWM)) {
-      which = "too late to effect output watermark";
-      tooLate = true;
-    } else if (context.window().maxTimestamp().isBefore(inputWM)) {
-      which = "too late for end-of-window timer";
-      tooLate = true;
-    } else {
-      which = "on time";
-      tooLate = false;
-      checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-          "Element hold %s is beyond end-of-time", elementHold);
-      context.state().access(elementHoldTag).add(elementHold);
-    }
-    WindowTracing.trace(
-        "WatermarkHold.addHolds: element hold at {} is {} for "
-        + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
-        elementHold, which, context.key(), context.window(), inputWM,
-        outputWM);
-
-    return tooLate ? null : elementHold;
-  }
-
-  /**
-   * Add an end-of-window hold or, if too late for that, a garbage collection hold (if required).
-   * Return the {@link Instant} at which hold was added, or {@literal null} if no hold was added.
-   */
-  @Nullable
-  private Instant addEndOfWindowOrGarbageCollectionHolds(
-      ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
-    Instant hold = addEndOfWindowHold(context, paneIsEmpty);
-    if (hold == null) {
-      hold = addGarbageCollectionHold(context, paneIsEmpty);
-    }
-    return hold;
-  }
-
-  /**
-   * Attempt to add an 'end-of-window hold'. Return the {@link Instant} at which the hold was added
-   * (ie the end of window time), or {@literal null} if no end of window hold is possible and we
-   * should fallback to a garbage collection hold.
-   *
-   * <p>We only add the hold if we can be sure a timer will be set (by {@link ReduceFnRunner})
-   * to clear it. In other words, the input watermark cannot be ahead of the end of window time.
-   *
-   * <p>An end-of-window hold is added in two situations:
-   * <ol>
-   * <li>An incoming element came in behind the output watermark (so we are too late for placing
-   * the usual element hold), but it may still be possible to include the element in an
-   * {@link Timing#ON_TIME} pane. We place the end of window hold to ensure that pane will
-   * not be considered late by any downstream computation.
-   * <li>We guarantee an {@link Timing#ON_TIME} pane will be emitted for all windows which saw at
-   * least one element, even if that {@link Timing#ON_TIME} pane is empty. Thus when elements in
-   * a pane are processed due to a fired trigger we must set both an end of window timer and an end
-   * of window hold. Again, the hold ensures the {@link Timing#ON_TIME} pane will not be considered
-   * late by any downstream computation.
-   * </ol>
-   */
-  @Nullable
-  private Instant addEndOfWindowHold(ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
-    Instant outputWM = timerInternals.currentOutputWatermarkTime();
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-    Instant eowHold = context.window().maxTimestamp();
-
-    if (eowHold.isBefore(inputWM)) {
-      WindowTracing.trace(
-          "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is too late for "
-              + "end-of-window timer for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
-          eowHold, context.key(), context.window(), inputWM, outputWM);
-      return null;
-    }
-
-    checkState(outputWM == null || !eowHold.isBefore(outputWM),
-        "End-of-window hold %s cannot be before output watermark %s",
-        eowHold, outputWM);
-    checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-        "End-of-window hold %s is beyond end-of-time", eowHold);
-    // If paneIsEmpty then this hold is just for empty ON_TIME panes, so we want to keep
-    // the hold away from the combining function in elementHoldTag.
-    // However if !paneIsEmpty then it could make sense  to use the elementHoldTag here.
-    // Alas, onMerge is forced to add an end of window or garbage collection hold without
-    // knowing whether an element hold is already in place (stopping to check is too expensive).
-    // This it would end up adding an element hold at the end of the window which could
-    // upset the elementHoldTag combining function.
-    context.state().access(EXTRA_HOLD_TAG).add(eowHold);
-    WindowTracing.trace(
-        "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is on time for "
-            + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
-        eowHold, context.key(), context.window(), inputWM, outputWM);
-    return eowHold;
-  }
-
-  /**
-   * Attempt to add a 'garbage collection hold' if it is required. Return the {@link Instant} at
-   * which the hold was added (ie the end of window time plus allowed lateness),
-   * or {@literal null} if no hold was added.
-   *
-   * <p>We only add the hold if it is distinct from what would be added by
-   * {@link #addEndOfWindowHold}. In other words, {@link WindowingStrategy#getAllowedLateness}
-   * must be non-zero.
-   *
-   * <p>A garbage collection hold is added in two situations:
-   * <ol>
-   * <li>An incoming element came in behind the output watermark, and was too late for placing
-   * the usual element hold or an end of window hold. Place the garbage collection hold so that
-   * we can guarantee when the pane is finally triggered its output will not be dropped due to
-   * excessive lateness by any downstream computation.
-   * <li>The {@link WindowingStrategy#getClosingBehavior()} is
-   * {@link ClosingBehavior#FIRE_ALWAYS}, and thus we guarantee a final pane will be emitted
-   * for all windows which saw at least one element. Again, the garbage collection hold guarantees
-   * that any empty final pane can be given a timestamp which will not be considered beyond
-   * allowed lateness by any downstream computation.
-   * </ol>
-   *
-   * <p>We use {@code paneIsEmpty} to distinguish cases 1 and 2.
-   */
-  @Nullable
-  private Instant addGarbageCollectionHold(
-      ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
-    Instant outputWM = timerInternals.currentOutputWatermarkTime();
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-    Instant eow = context.window().maxTimestamp();
-    Instant gcHold = eow.plus(windowingStrategy.getAllowedLateness());
-
-    if (!windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) {
-      WindowTracing.trace(
-          "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
-              + "since no allowed lateness for key:{}; window:{}; inputWatermark:{}; "
-              + "outputWatermark:{}",
-          gcHold, context.key(), context.window(), inputWM, outputWM);
-      return null;
-    }
-
-    if (paneIsEmpty && context.windowingStrategy().getClosingBehavior()
-        == ClosingBehavior.FIRE_IF_NON_EMPTY) {
-      WindowTracing.trace(
-          "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
-              + "since empty pane and FIRE_IF_NON_EMPTY for key:{}; window:{}; inputWatermark:{}; "
-              + "outputWatermark:{}",
-          gcHold, context.key(), context.window(), inputWM, outputWM);
-      return null;
-    }
-
-    checkState(!gcHold.isBefore(inputWM),
-        "Garbage collection hold %s cannot be before input watermark %s",
-        gcHold, inputWM);
-    checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-        "Garbage collection hold %s is beyond end-of-time", gcHold);
-    // Same EXTRA_HOLD_TAG vs elementHoldTag discussion as in addEndOfWindowHold above.
-    context.state().access(EXTRA_HOLD_TAG).add(gcHold);
-
-    WindowTracing.trace(
-        "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is on time for "
-            + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
-        gcHold, context.key(), context.window(), inputWM, outputWM);
-    return gcHold;
-  }
-
-  /**
-   * Prefetch watermark holds in preparation for merging.
-   */
-  public void prefetchOnMerge(MergingStateAccessor<?, W> state) {
-    StateMerging.prefetchWatermarks(state, elementHoldTag);
-  }
-
-  /**
-   * Updates the watermark hold when windows merge if it is possible the merged value does
-   * not equal all of the existing holds. For example, if the new window implies a later
-   * watermark hold, then earlier holds may be released.
-   */
-  public void onMerge(ReduceFn<?, ?, ?, W>.OnMergeContext context) {
-    WindowTracing.debug("WatermarkHold.onMerge: for key:{}; window:{}; inputWatermark:{}; "
-            + "outputWatermark:{}",
-        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
-        timerInternals.currentOutputWatermarkTime());
-    StateMerging.mergeWatermarks(context.state(), elementHoldTag, context.window());
-    // If we had a cheap way to determine if we have an element hold then we could
-    // avoid adding an unnecessary end-of-window or garbage collection hold.
-    // Simply reading the above merged watermark would impose an additional read for the
-    // common case that the active window has just one underlying state address window and
-    // the hold depends on the min of the element timestamps.
-    // At least one merged window must be non-empty for the merge to have been triggered.
-    StateMerging.clear(context.state(), EXTRA_HOLD_TAG);
-    addEndOfWindowOrGarbageCollectionHolds(context, false /*paneIsEmpty*/);
-  }
-
-  /**
-   * Result of {@link #extractAndRelease}.
-   */
-  public static class OldAndNewHolds {
-    public final Instant oldHold;
-    @Nullable
-    public final Instant newHold;
-
-    public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) {
-      this.oldHold = oldHold;
-      this.newHold = newHold;
-    }
-  }
-
-  /**
-   * Return (a future for) the earliest hold for {@code context}. Clear all the holds after
-   * reading, but add/restore an end-of-window or garbage collection hold if required.
-   *
-   * <p>The returned timestamp is the output timestamp according to the {@link OutputTimeFn}
-   * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late
-   * elements in the current pane. If there is no such value the timestamp is the end
-   * of the window.
-   */
-  public ReadableState<OldAndNewHolds> extractAndRelease(
-      final ReduceFn<?, ?, ?, W>.Context context, final boolean isFinished) {
-    WindowTracing.debug(
-        "WatermarkHold.extractAndRelease: for key:{}; window:{}; inputWatermark:{}; "
-            + "outputWatermark:{}",
-        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
-        timerInternals.currentOutputWatermarkTime());
-    final WatermarkHoldState<W> elementHoldState = context.state().access(elementHoldTag);
-    final WatermarkHoldState<BoundedWindow> extraHoldState = context.state().access(EXTRA_HOLD_TAG);
-    return new ReadableState<OldAndNewHolds>() {
-      @Override
-      @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-        justification = "")
-      public ReadableState<OldAndNewHolds> readLater() {
-        elementHoldState.readLater();
-        extraHoldState.readLater();
-        return this;
-      }
-
-      @Override
-      public OldAndNewHolds read() {
-        // Read both the element and extra holds.
-        Instant elementHold = elementHoldState.read();
-        Instant extraHold = extraHoldState.read();
-        Instant oldHold;
-        // Find the minimum, accounting for null.
-        if (elementHold == null) {
-          oldHold = extraHold;
-        } else if (extraHold == null) {
-          oldHold = elementHold;
-        } else if (elementHold.isBefore(extraHold)) {
-          oldHold = elementHold;
-        } else {
-          oldHold = extraHold;
-        }
-        if (oldHold == null || oldHold.isAfter(context.window().maxTimestamp())) {
-          // If no hold (eg because all elements came in behind the output watermark), or
-          // the hold was for garbage collection, take the end of window as the result.
-          WindowTracing.debug(
-              "WatermarkHold.extractAndRelease.read: clipping from {} to end of window "
-              + "for key:{}; window:{}",
-              oldHold, context.key(), context.window());
-          oldHold = context.window().maxTimestamp();
-        }
-        WindowTracing.debug("WatermarkHold.extractAndRelease.read: clearing for key:{}; window:{}",
-            context.key(), context.window());
-
-        // Clear the underlying state to allow the output watermark to progress.
-        elementHoldState.clear();
-        extraHoldState.clear();
-
-        @Nullable Instant newHold = null;
-        if (!isFinished) {
-          // Only need to leave behind an end-of-window or garbage collection hold
-          // if future elements will be processed.
-          newHold = addEndOfWindowOrGarbageCollectionHolds(context, true /*paneIsEmpty*/);
-        }
-
-        return new OldAndNewHolds(oldHold, newHold);
-      }
-    };
-  }
-
-  /**
-   * Clear any remaining holds.
-   */
-  public void clearHolds(ReduceFn<?, ?, ?, W>.Context context) {
-    WindowTracing.debug(
-        "WatermarkHold.clearHolds: For key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
-        context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
-        timerInternals.currentOutputWatermarkTime());
-    context.state().access(elementHoldTag).clear();
-    context.state().access(EXTRA_HOLD_TAG).clear();
-  }
-
-  /**
-   * Return the current data hold, or null if none. Does not clear. For debugging only.
-   */
-  @Nullable
-  public Instant getDataCurrent(ReduceFn<?, ?, ?, W>.Context context) {
-    return context.state().access(elementHoldTag).read();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
deleted file mode 100644
index 613aa4b..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
+++ /dev/null
@@ -1,42 +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.common;
-
-/**
- * An interface for things that allow observing the size in bytes of
- * encoded values of type {@code T}.
- *
- * @param <T> the type of the values being observed
- */
-public interface ElementByteSizeObservable<T> {
-  /**
-   * Returns whether {@link #registerByteSizeObserver} is cheap enough
-   * to call for every element, that is, if this
-   * {@code ElementByteSizeObservable} can calculate the byte size of
-   * the element to be coded in roughly constant time (or lazily).
-   */
-  public boolean isRegisterByteSizeObserverCheap(T value);
-
-  /**
-   * Notifies the {@code ElementByteSizeObserver} about the byte size
-   * of the encoded value using this {@code ElementByteSizeObservable}.
-   */
-  public void registerByteSizeObserver(T value,
-                                       ElementByteSizeObserver observer)
-      throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
deleted file mode 100644
index 1e3c17f..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
+++ /dev/null
@@ -1,99 +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.common;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.util.NoSuchElementException;
-
-/**
- * A {@link Reiterator} that supports one-element lookahead during iteration.
- *
- * @param <T> the type of elements returned by this iterator
- */
-public final class PeekingReiterator<T> implements Reiterator<T> {
-  private T nextElement;
-  private boolean nextElementComputed;
-  private final Reiterator<T> iterator;
-
-  public PeekingReiterator(Reiterator<T> iterator) {
-    this.iterator = checkNotNull(iterator);
-  }
-
-  PeekingReiterator(PeekingReiterator<T> it) {
-    this.iterator = checkNotNull(checkNotNull(it).iterator.copy());
-    this.nextElement = it.nextElement;
-    this.nextElementComputed = it.nextElementComputed;
-  }
-
-  @Override
-  public boolean hasNext() {
-    computeNext();
-    return nextElementComputed;
-  }
-
-  @Override
-  public T next() {
-    T result = peek();
-    nextElementComputed = false;
-    return result;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>If {@link #peek} is called, {@code remove} is disallowed until
-   * {@link #next} has been subsequently called.
-   */
-  @Override
-  public void remove() {
-    checkState(!nextElementComputed,
-        "After peek(), remove() is disallowed until next() is called");
-    iterator.remove();
-  }
-
-  @Override
-  public PeekingReiterator<T> copy() {
-    return new PeekingReiterator<>(this);
-  }
-
-  /**
-   * Returns the element that would be returned by {@link #next}, without
-   * actually consuming the element.
-   * @throws NoSuchElementException if there is no next element
-   */
-  public T peek() {
-    computeNext();
-    if (!nextElementComputed) {
-      throw new NoSuchElementException();
-    }
-    return nextElement;
-  }
-
-  private void computeNext() {
-    if (nextElementComputed) {
-      return;
-    }
-    if (!iterator.hasNext()) {
-      return;
-    }
-    nextElement = iterator.next();
-    nextElementComputed = true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java
deleted file mode 100644
index 1ea8af8..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java
+++ /dev/null
@@ -1,20 +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.
- */
-
-/** Defines utilities shared by multiple PipelineRunner implementations. */
-package org.apache.beam.sdk.util.common;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java
deleted file mode 100644
index b4772f3..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java
+++ /dev/null
@@ -1,20 +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.
- */
-
-/** Defines utilities that can be used by Beam runners. */
-package org.apache.beam.sdk.util;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java
new file mode 100644
index 0000000..122e60c
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaceForTest;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+/**
+ * Tests for {@link BatchTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class BatchTimerInternalsTest {
+
+  private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
+
+  @Mock
+  private ReduceFnRunner<?, ?, ?, ?> mockRunner;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testFiringTimers() throws Exception {
+    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+
+    underTest.setTimer(processingTime1);
+    underTest.setTimer(processingTime2);
+
+    underTest.advanceProcessingTime(mockRunner, new Instant(20));
+    Mockito.verify(mockRunner).onTimer(processingTime1);
+    Mockito.verifyNoMoreInteractions(mockRunner);
+
+    // Advancing just a little shouldn't refire
+    underTest.advanceProcessingTime(mockRunner, new Instant(21));
+    Mockito.verifyNoMoreInteractions(mockRunner);
+
+    // Adding the timer and advancing a little should refire
+    underTest.setTimer(processingTime1);
+    Mockito.verify(mockRunner).onTimer(processingTime1);
+    underTest.advanceProcessingTime(mockRunner, new Instant(21));
+    Mockito.verifyNoMoreInteractions(mockRunner);
+
+    // And advancing the rest of the way should still have the other timer
+    underTest.advanceProcessingTime(mockRunner, new Instant(30));
+    Mockito.verify(mockRunner).onTimer(processingTime2);
+    Mockito.verifyNoMoreInteractions(mockRunner);
+  }
+
+  @Test
+  public void testTimerOrdering() throws Exception {
+    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+    TimerData watermarkTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+    TimerData watermarkTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
+    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+
+    underTest.setTimer(processingTime1);
+    underTest.setTimer(watermarkTime1);
+    underTest.setTimer(processingTime2);
+    underTest.setTimer(watermarkTime2);
+
+    underTest.advanceInputWatermark(mockRunner, new Instant(30));
+    Mockito.verify(mockRunner).onTimer(watermarkTime1);
+    Mockito.verify(mockRunner).onTimer(watermarkTime2);
+    Mockito.verifyNoMoreInteractions(mockRunner);
+
+    underTest.advanceProcessingTime(mockRunner, new Instant(30));
+    Mockito.verify(mockRunner).onTimer(processingTime1);
+    Mockito.verify(mockRunner).onTimer(processingTime2);
+    Mockito.verifyNoMoreInteractions(mockRunner);
+  }
+
+  @Test
+  public void testDeduplicate() throws Exception {
+    BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+    TimerData watermarkTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+    TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+    underTest.setTimer(watermarkTime);
+    underTest.setTimer(watermarkTime);
+    underTest.setTimer(processingTime);
+    underTest.setTimer(processingTime);
+    underTest.advanceProcessingTime(mockRunner, new Instant(20));
+    underTest.advanceInputWatermark(mockRunner, new Instant(20));
+
+    Mockito.verify(mockRunner).onTimer(processingTime);
+    Mockito.verify(mockRunner).onTimer(watermarkTime);
+    Mockito.verifyNoMoreInteractions(mockRunner);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
new file mode 100644
index 0000000..d1e0c68
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
@@ -0,0 +1,660 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Properties of {@link GroupAlsoByWindowsDoFn}.
+ *
+ * <p>Some properties may not hold of some implementations, due to restrictions on the context
+ * in which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not
+ * support merging windows.
+ */
+public class GroupAlsoByWindowsProperties {
+
+  /**
+   * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide
+   * the appropriate windowing strategy under test.
+   */
+  public interface GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> {
+    <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, OutputT, W>
+    forStrategy(WindowingStrategy<?, W> strategy, StateInternalsFactory<K> stateInternalsFactory);
+  }
+
+  /**
+   * Tests that for empty input and the given {@link WindowingStrategy}, the provided GABW
+   * implementation produces no output.
+   *
+   * <p>The input type is deliberately left as a wildcard, since it is not relevant.
+   */
+  public static <K, InputT, OutputT> void emptyInputEmptyOutput(
+      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+    // This key should never actually be used, though it is eagerly passed to the
+    // StateInternalsFactory so must be non-null
+    @SuppressWarnings("unchecked")
+    K fakeKey = (K) "this key should never be used";
+
+    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> result = runGABW(
+        gabwFactory,
+        windowingStrategy,
+        fakeKey,
+        Collections.<WindowedValue<InputT>>emptyList());
+
+    assertThat(result.peekOutputElements(), hasSize(0));
+  }
+
+  /**
+   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+   * correctly groups them according to fixed windows.
+   */
+  public static void groupsElementsIntoFixedWindows(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "key",
+            WindowedValue.of(
+                "v1",
+                new Instant(1),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(2),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(13),
+                Arrays.asList(window(10, 20)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
+  }
+
+  /**
+   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+   * correctly groups them into sliding windows.
+   *
+   * <p>In the input here, each element occurs in multiple windows.
+   */
+  public static void groupsElementsIntoSlidingWindowsWithMinTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(
+        SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
+        .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "key",
+            WindowedValue.of(
+                "v1",
+                new Instant(5),
+                Arrays.asList(window(-10, 10), window(0, 20)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(15),
+                Arrays.asList(window(0, 20), window(10, 30)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(3));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+    assertThat(item0.getValue().getValue(), contains("v1"));
+    assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
+    assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
+
+    TimestampedValue<KV<String, Iterable<String>>> item2 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+    assertThat(item2.getValue().getValue(), contains("v2"));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
+  }
+
+  /**
+   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+   * correctly groups and combines them according to sliding windows.
+   *
+   * <p>In the input here, each element occurs in multiple windows.
+   */
+  public static void combinesElementsInSlidingWindows(
+      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+      CombineFn<Long, ?, Long> combineFn)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
+            .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
+
+    DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                1L,
+                new Instant(5),
+                Arrays.asList(window(-10, 10), window(0, 20)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                2L,
+                new Instant(15),
+                Arrays.asList(window(0, 20), window(10, 30)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                4L,
+                new Instant(18),
+                Arrays.asList(window(0, 20), window(10, 30)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(3));
+
+    TimestampedValue<KV<String, Long>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+    assertThat(item0.getValue().getKey(), equalTo("k"));
+    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L))));
+    assertThat(item0.getTimestamp(), equalTo(new Instant(5L)));
+
+    TimestampedValue<KV<String, Long>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
+    assertThat(item1.getValue().getKey(), equalTo("k"));
+    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item1.getTimestamp(), equalTo(new Instant(10L)));
+
+    TimestampedValue<KV<String, Long>> item2 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+    assertThat(item2.getValue().getKey(), equalTo("k"));
+    assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item2.getTimestamp(), equalTo(new Instant(20L)));
+  }
+
+  /**
+   * Tests that the given GABW implementation correctly groups elements that fall into overlapping
+   * windows that are not merged.
+   */
+  public static void groupsIntoOverlappingNonmergingWindows(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "key",
+            WindowedValue.of(
+                "v1",
+                new Instant(1),
+                Arrays.asList(window(0, 5)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(4),
+                Arrays.asList(window(1, 5)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(4),
+                Arrays.asList(window(0, 5)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3"));
+    assertThat(item0.getTimestamp(), equalTo(window(1, 5).maxTimestamp()));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(1, 5)));
+    assertThat(item1.getValue().getValue(), contains("v2"));
+    assertThat(item1.getTimestamp(), equalTo(window(0, 5).maxTimestamp()));
+  }
+
+  /**
+   * Tests that the given GABW implementation correctly groups elements into merged sessions.
+   */
+  public static void groupsElementsInMergedSessions(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "key",
+            WindowedValue.of(
+                "v1",
+                new Instant(0),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(5),
+                Arrays.asList(window(5, 15)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(15),
+                Arrays.asList(window(15, 25)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+  }
+
+  /**
+   * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
+   * session window correctly according to the provided {@link CombineFn}.
+   */
+  public static void combinesElementsPerSession(
+      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+      CombineFn<Long, ?, Long> combineFn)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
+
+    DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                1L,
+                new Instant(0),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                2L,
+                new Instant(5),
+                Arrays.asList(window(5, 15)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                4L,
+                new Instant(15),
+                Arrays.asList(window(15, 25)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Long>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+    assertThat(item0.getValue().getKey(), equalTo("k"));
+    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
+    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+    TimestampedValue<KV<String, Long>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+    assertThat(item1.getValue().getKey(), equalTo("k"));
+    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
+    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+  }
+
+  /**
+   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+   * correctly groups them according to fixed windows and also sets the output timestamp
+   * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}.
+   */
+  public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "key",
+            WindowedValue.of(
+                "v1",
+                new Instant(1),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(2),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(13),
+                Arrays.asList(window(10, 20)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
+  }
+
+  /**
+   * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+   * correctly groups them according to fixed windows and also sets the output timestamp
+   * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}.
+   */
+  public static void groupsElementsIntoFixedWindowsWithLatestTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+        .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                "v1",
+                new Instant(1),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(2),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(13),
+                Arrays.asList(window(10, 20)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(new Instant(2)));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(new Instant(13)));
+  }
+
+  /**
+   * Tests that the given GABW implementation correctly groups elements into merged sessions
+   * with output timestamps at the end of the merged window.
+   */
+  public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+            .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                "v1",
+                new Instant(0),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(5),
+                Arrays.asList(window(5, 15)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(15),
+                Arrays.asList(window(15, 25)),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+  }
+
+  /**
+   * Tests that the given GABW implementation correctly groups elements into merged sessions
+   * with output timestamps at the end of the merged window.
+   */
+  public static void groupsElementsInMergedSessionsWithLatestTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+            .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
+
+    BoundedWindow unmergedWindow = window(15, 25);
+    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                "v1",
+                new Instant(0),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v2",
+                new Instant(5),
+                Arrays.asList(window(5, 15)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                "v3",
+                new Instant(15),
+                Arrays.asList(unmergedWindow),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    BoundedWindow mergedWindow = window(0, 15);
+    TimestampedValue<KV<String, Iterable<String>>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow));
+    assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
+
+    TimestampedValue<KV<String, Iterable<String>>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(unmergedWindow));
+    assertThat(item1.getValue().getValue(), contains("v3"));
+    assertThat(item1.getTimestamp(), equalTo(new Instant(15)));
+  }
+
+  /**
+   * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
+   * session window correctly according to the provided {@link CombineFn}.
+   */
+  public static void combinesElementsPerSessionWithEndOfWindowTimestamp(
+      GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+      CombineFn<Long, ?, Long> combineFn)
+          throws Exception {
+
+    WindowingStrategy<?, IntervalWindow> windowingStrategy =
+        WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+    BoundedWindow secondWindow = window(15, 25);
+    DoFnTester<?, KV<String, Long>> result =
+        runGABW(gabwFactory, windowingStrategy, "k",
+            WindowedValue.of(
+                1L,
+                new Instant(0),
+                Arrays.asList(window(0, 10)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                2L,
+                new Instant(5),
+                Arrays.asList(window(5, 15)),
+                PaneInfo.NO_FIRING),
+            WindowedValue.of(
+                4L,
+                new Instant(15),
+                Arrays.asList(secondWindow),
+                PaneInfo.NO_FIRING));
+
+    assertThat(result.peekOutputElements(), hasSize(2));
+
+    BoundedWindow firstResultWindow = window(0, 15);
+    TimestampedValue<KV<String, Long>> item0 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow));
+    assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
+    assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp()));
+
+    TimestampedValue<KV<String, Long>> item1 =
+        Iterables.getOnlyElement(result.peekOutputElementsInWindow(secondWindow));
+    assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
+    assertThat(item1.getTimestamp(),
+        equalTo(secondWindow.maxTimestamp()));
+  }
+
+  @SafeVarargs
+  private static <K, InputT, OutputT, W extends BoundedWindow>
+  DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
+      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
+      WindowingStrategy<?, W> windowingStrategy,
+      K key,
+      WindowedValue<InputT>... values) throws Exception {
+    return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values));
+  }
+
+  private static <K, InputT, OutputT, W extends BoundedWindow>
+  DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
+      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
+      WindowingStrategy<?, W> windowingStrategy,
+      K key,
+      Collection<WindowedValue<InputT>> values) throws Exception {
+
+    final StateInternalsFactory<K> stateInternalsCache = new CachingStateInternalsFactory<K>();
+
+    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> tester =
+        DoFnTester.of(gabwFactory.forStrategy(windowingStrategy, stateInternalsCache));
+
+    // Though we use a DoFnTester, the function itself is instantiated directly by the
+    // runner and should not be serialized; it may not even be serializable.
+    tester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE);
+    tester.startBundle();
+    tester.processElement(KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
+    tester.finishBundle();
+
+    // Sanity check for corruption
+    for (KV<K, OutputT> elem : tester.peekOutputElements()) {
+      assertThat(elem.getKey(), equalTo(key));
+    }
+
+    return tester;
+  }
+
+  private static BoundedWindow window(long start, long end) {
+    return new IntervalWindow(new Instant(start), new Instant(end));
+  }
+
+  private static final class CachingStateInternalsFactory<K> implements StateInternalsFactory<K> {
+    private final LoadingCache<K, StateInternals<K>> stateInternalsCache;
+
+    private CachingStateInternalsFactory() {
+      this.stateInternalsCache = CacheBuilder.newBuilder().build(new StateInternalsLoader<K>());
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public StateInternals<K> stateInternalsForKey(K key) {
+      try {
+        return stateInternalsCache.get(key);
+      } catch (Exception exc) {
+        throw new RuntimeException(exc);
+      }
+    }
+  }
+
+  private static class StateInternalsLoader<K> extends CacheLoader<K, StateInternals<K>> {
+    @Override
+    public StateInternals<K> load(K key) throws Exception {
+      return InMemoryStateInternals.forKey(key);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
new file mode 100644
index 0000000..1fad1fb
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import org.apache.beam.runners.core.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link GroupAlsoByWindowsViaOutputBufferDoFn}.
+ */
+@RunWith(JUnit4.class)
+public class GroupAlsoByWindowsViaOutputBufferDoFnTest {
+
+  private class BufferingGABWViaOutputBufferDoFnFactory<K, InputT>
+  implements GroupAlsoByWindowsDoFnFactory<K, InputT, Iterable<InputT>> {
+
+    private final Coder<InputT> inputCoder;
+
+    public BufferingGABWViaOutputBufferDoFnFactory(Coder<InputT> inputCoder) {
+      this.inputCoder = inputCoder;
+    }
+
+    @Override
+    public <W extends BoundedWindow>
+    GroupAlsoByWindowsDoFn<K, InputT, Iterable<InputT>, W> forStrategy(
+            WindowingStrategy<?, W> windowingStrategy,
+            StateInternalsFactory<K> stateInternalsFactory) {
+      return new GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, Iterable<InputT>, W>(
+          windowingStrategy,
+          stateInternalsFactory,
+          SystemReduceFn.<K, InputT, W>buffering(inputCoder));
+    }
+  }
+
+  @Test
+  public void testEmptyInputEmptyOutput() throws Exception {
+    GroupAlsoByWindowsProperties.emptyInputEmptyOutput(
+        new BufferingGABWViaOutputBufferDoFnFactory<>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoFixedWindows() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindows(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoSlidingWindows() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsIntoSlidingWindowsWithMinTimestamp(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsIntoOverlappingNonmergingWindows() throws Exception {
+    GroupAlsoByWindowsProperties.groupsIntoOverlappingNonmergingWindows(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsIntoSessions() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsInMergedSessions(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoFixedWindowsWithLatestTimestamp() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoSessionsWithEndOfWindowTimestamp() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+
+  @Test
+  public void testGroupsElementsIntoSessionsWithLatestTimestamp() throws Exception {
+    GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithLatestTimestamp(
+        new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
new file mode 100644
index 0000000..1cf05b6
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import java.util.Arrays;
+import org.apache.beam.runners.core.LateDataDroppingDoFnRunner.LateDataFilter;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+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;
+
+/**
+ * Unit tests for {@link LateDataDroppingDoFnRunner}.
+ */
+@RunWith(JUnit4.class)
+public class LateDataDroppingDoFnRunnerTest {
+  private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
+
+  @Mock private TimerInternals mockTimerInternals;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testLateDataFilter() throws Exception {
+    when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L));
+
+    InMemoryLongSumAggregator droppedDueToLateness =
+        new InMemoryLongSumAggregator("droppedDueToLateness");
+    LateDataFilter lateDataFilter = new LateDataFilter(
+        WindowingStrategy.of(WINDOW_FN), mockTimerInternals, droppedDueToLateness);
+
+    Iterable<WindowedValue<Integer>> actual = lateDataFilter.filter(
+        "a",
+        ImmutableList.of(
+            createDatum(13, 13L),
+            createDatum(5, 5L), // late element, earlier than 4L.
+            createDatum(16, 16L),
+            createDatum(18, 18L)));
+
+    Iterable<WindowedValue<Integer>> expected =  ImmutableList.of(
+        createDatum(13, 13L),
+        createDatum(16, 16L),
+        createDatum(18, 18L));
+    assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class)));
+    assertEquals(1, droppedDueToLateness.sum);
+  }
+
+  private <T> WindowedValue<T> createDatum(T element, long timestampMillis) {
+    Instant timestamp = new Instant(timestampMillis);
+    return WindowedValue.of(
+        element,
+        timestamp,
+        Arrays.asList(WINDOW_FN.assignWindow(timestamp)),
+        PaneInfo.NO_FIRING);
+  }
+
+  private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
+    private final String name;
+    private long sum = 0;
+
+    public InMemoryLongSumAggregator(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public void addValue(Long value) {
+      sum += value;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public CombineFn<Long, ?, Long> getCombineFn() {
+      return new Sum.SumLongFn();
+    }
+  }
+}


[31/50] [abbrv] incubator-beam git commit: Put classes in runners-core package into runners.core namespace

Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
deleted file mode 100644
index 7c3e4d7..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ /dev/null
@@ -1,983 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks;
-import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the
- * {@link PCollection} by key.
- *
- * <p>The {@link #onTrigger} relies on a {@link TriggerRunner} to manage the execution of
- * the triggering logic. The {@code ReduceFnRunner}s responsibilities are:
- *
- * <ul>
- * <li>Tracking the windows that are active (have buffered data) as elements arrive and
- * triggers are fired.
- * <li>Holding the watermark based on the timestamps of elements in a pane and releasing it
- * when the trigger fires.
- * <li>Calling the appropriate callbacks on {@link ReduceFn} based on trigger execution, timer
- * firings, etc, and providing appropriate contexts to the {@link ReduceFn} for actions
- * such as output.
- * <li>Scheduling garbage collection of state associated with a specific window, and making that
- * happen when the appropriate timer fires.
- * </ul>
- *
- * @param <K>       The type of key being processed.
- * @param <InputT>  The type of values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W>       The type of windows this operates on.
- */
-public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
-
-  /**
-   * The {@link ReduceFnRunner} depends on most aspects of the {@link WindowingStrategy}.
-   *
-   * <ul>
-   * <li>It runs the trigger from the {@link WindowingStrategy}.</li>
-   * <li>It merges windows according to the {@link WindowingStrategy}.</li>
-   * <li>It chooses how to track active windows and clear out expired windows
-   * according to the {@link WindowingStrategy}, based on the allowed lateness and
-   * whether windows can merge.</li>
-   * <li>It decides whether to emit empty final panes according to whether the
-   * {@link WindowingStrategy} requires it.<li>
-   * <li>It uses discarding or accumulation mode according to the {@link WindowingStrategy}.</li>
-   * </ul>
-   */
-  private final WindowingStrategy<Object, W> windowingStrategy;
-
-  private final OutputWindowedValue<KV<K, OutputT>> outputter;
-
-  private final StateInternals<K> stateInternals;
-
-  private final Aggregator<Long, Long> droppedDueToClosedWindow;
-
-  private final K key;
-
-  /**
-   * Track which windows are still active and the 'state address' windows which hold their state.
-   *
-   * <ul>
-   * <li>State: Global map for all active windows for this computation and key.
-   * <li>Lifetime: Cleared when no active windows need to be tracked. A window lives within
-   * the active window set until its trigger is closed or the window is garbage collected.
-   * </ul>
-   */
-  private final ActiveWindowSet<W> activeWindows;
-
-  /**
-   * Always a {@link SystemReduceFn}.
-   *
-   * <ul>
-   * <li>State: A bag of accumulated values, or the intermediate result of a combiner.
-   * <li>State style: RENAMED
-   * <li>Merging: Concatenate or otherwise combine the state from each merged window.
-   * <li>Lifetime: Cleared when a pane fires if DISCARDING_FIRED_PANES. Otherwise cleared
-   * when trigger is finished or when the window is garbage collected.
-   * </ul>
-   */
-  private final ReduceFn<K, InputT, OutputT, W> reduceFn;
-
-  /**
-   * Manage the setting and firing of timer events.
-   *
-   * <ul>
-   * <li>Merging: End-of-window and garbage collection timers are cancelled when windows are
-   * merged away. Timers created by triggers are never garbage collected and are left to
-   * fire and be ignored.
-   * <li>Lifetime: Timers automatically disappear after they fire.
-   * </ul>
-   */
-  private final TimerInternals timerInternals;
-
-  /**
-   * Manage the execution and state for triggers.
-   *
-   * <ul>
-   * <li>State: Tracks which sub-triggers have finished, and any additional state needed to
-   * determine when the trigger should fire.
-   * <li>State style: DIRECT
-   * <li>Merging: Finished bits are explicitly managed. Other state is eagerly merged as
-   * needed.
-   * <li>Lifetime: Most trigger state is cleared when the final pane is emitted. However
-   * the finished bits are left behind and must be cleared when the window is
-   * garbage collected.
-   * </ul>
-   */
-  private final TriggerRunner<W> triggerRunner;
-
-  /**
-   * Store the output watermark holds for each window.
-   *
-   * <ul>
-   * <li>State: Bag of hold timestamps.
-   * <li>State style: RENAMED
-   * <li>Merging: Depending on {@link OutputTimeFn}, may need to be recalculated on merging.
-   * When a pane fires it may be necessary to add (back) an end-of-window or garbage collection
-   * hold.
-   * <li>Lifetime: Cleared when a pane fires or when the window is garbage collected.
-   * </ul>
-   */
-  private final WatermarkHold<W> watermarkHold;
-
-  private final ReduceFnContextFactory<K, InputT, OutputT, W> contextFactory;
-
-  /**
-   * Store the previously emitted pane (if any) for each window.
-   *
-   * <ul>
-   * <li>State: The previous {@link PaneInfo} passed to the user's {@code DoFn.ProcessElement}
-   * method, if any.
-   * <li>Style style: DIRECT
-   * <li>Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}.
-   * Cleared when window is merged away.
-   * <li>Lifetime: Cleared when trigger is closed or window is garbage collected.
-   * </ul>
-   */
-  private final PaneInfoTracker paneInfoTracker;
-
-  /**
-   * Store whether we've seen any elements for a window since the last pane was emitted.
-   *
-   * <ul>
-   * <li>State: Unless DISCARDING_FIRED_PANES, a count of number of elements added so far.
-   * <li>State style: RENAMED.
-   * <li>Merging: Counts are summed when windows are merged.
-   * <li>Lifetime: Cleared when pane fires or window is garbage collected.
-   * </ul>
-   */
-  private final NonEmptyPanes<K, W> nonEmptyPanes;
-
-  public ReduceFnRunner(
-      K key,
-      WindowingStrategy<?, W> windowingStrategy,
-      StateInternals<K> stateInternals,
-      TimerInternals timerInternals,
-      WindowingInternals<?, KV<K, OutputT>> windowingInternals,
-      Aggregator<Long, Long> droppedDueToClosedWindow,
-      ReduceFn<K, InputT, OutputT, W> reduceFn,
-      PipelineOptions options) {
-    this.key = key;
-    this.timerInternals = timerInternals;
-    this.paneInfoTracker = new PaneInfoTracker(timerInternals);
-    this.stateInternals = stateInternals;
-    this.outputter = new OutputViaWindowingInternals<>(windowingInternals);
-    this.droppedDueToClosedWindow = droppedDueToClosedWindow;
-    this.reduceFn = reduceFn;
-
-    @SuppressWarnings("unchecked")
-    WindowingStrategy<Object, W> objectWindowingStrategy =
-        (WindowingStrategy<Object, W>) windowingStrategy;
-    this.windowingStrategy = objectWindowingStrategy;
-
-    this.nonEmptyPanes = NonEmptyPanes.create(this.windowingStrategy, this.reduceFn);
-
-    // Note this may incur I/O to load persisted window set data.
-    this.activeWindows = createActiveWindowSet();
-
-    this.contextFactory =
-        new ReduceFnContextFactory<K, InputT, OutputT, W>(key, reduceFn, this.windowingStrategy,
-            stateInternals, this.activeWindows, timerInternals, windowingInternals, options);
-
-    this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy);
-    this.triggerRunner =
-        new TriggerRunner<>(
-            windowingStrategy.getTrigger(),
-            new TriggerContextFactory<>(
-                windowingStrategy.getWindowFn(), stateInternals, activeWindows));
-  }
-
-  private ActiveWindowSet<W> createActiveWindowSet() {
-    return windowingStrategy.getWindowFn().isNonMerging()
-        ? new NonMergingActiveWindowSet<W>()
-        : new MergingActiveWindowSet<W>(windowingStrategy.getWindowFn(), stateInternals);
-  }
-
-  @VisibleForTesting
-  boolean isFinished(W window) {
-    return triggerRunner.isClosed(contextFactory.base(window, StateStyle.DIRECT).state());
-  }
-
-  @VisibleForTesting
-  boolean hasNoActiveWindows() {
-    return activeWindows.getActiveAndNewWindows().isEmpty();
-  }
-
-  /**
-   * Incorporate {@code values} into the underlying reduce function, and manage holds, timers,
-   * triggers, and window merging.
-   *
-   * <p>The general strategy is:
-   * <ol>
-   * <li>Use {@link WindowedValue#getWindows} (itself determined using
-   * {@link WindowFn#assignWindows}) to determine which windows each element belongs to. Some
-   * of those windows will already have state associated with them. The rest are considered
-   * NEW.
-   * <li>Use {@link WindowFn#mergeWindows} to attempt to merge currently ACTIVE and NEW windows.
-   * Each NEW window will become either ACTIVE or be discardedL.
-   * (See {@link ActiveWindowSet} for definitions of these terms.)
-   * <li>If at all possible, eagerly substitute NEW windows with their ACTIVE state address
-   * windows before any state is associated with the NEW window. In the common case that
-   * windows for new elements are merged into existing ACTIVE windows then no additional
-   * storage or merging overhead will be incurred.
-   * <li>Otherwise, keep track of the state address windows for ACTIVE windows so that their
-   * states can be merged on-demand when a pane fires.
-   * <li>Process the element for each of the windows it's windows have been merged into according
-   * to {@link ActiveWindowSet}. Processing may require running triggers, setting timers,
-   * setting holds, and invoking {@link ReduceFn#onTrigger}.
-   * </ol>
-   */
-  public void processElements(Iterable<WindowedValue<InputT>> values) throws Exception {
-    // If an incoming element introduces a new window, attempt to merge it into an existing
-    // window eagerly.
-    Map<W, W> windowToMergeResult = collectAndMergeWindows(values);
-
-    Set<W> windowsToConsider = new HashSet<>();
-
-    // Process each element, using the updated activeWindows determined by collectAndMergeWindows.
-    for (WindowedValue<InputT> value : values) {
-      windowsToConsider.addAll(processElement(windowToMergeResult, value));
-    }
-
-    // Trigger output from any window for which the trigger is ready
-    for (W mergedWindow : windowsToConsider) {
-      ReduceFn<K, InputT, OutputT, W>.Context directContext =
-          contextFactory.base(mergedWindow, StateStyle.DIRECT);
-      ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
-          contextFactory.base(mergedWindow, StateStyle.RENAMED);
-      triggerRunner.prefetchShouldFire(mergedWindow, directContext.state());
-      emitIfAppropriate(directContext, renamedContext);
-    }
-
-    // We're all done with merging and emitting elements so can compress the activeWindow state.
-    // Any windows which are still NEW must have come in on a new element which was then discarded
-    // due to the window's trigger being closed. We can thus delete them.
-    activeWindows.cleanupTemporaryWindows();
-  }
-
-  public void persist() {
-    activeWindows.persist();
-  }
-
-  /**
-   * Extract the windows associated with the values, and invoke merge. Return a map
-   * from windows to the merge result window. If a window is not in the domain of
-   * the result map then it did not get merged into a different window.
-   */
-  private Map<W, W> collectAndMergeWindows(Iterable<WindowedValue<InputT>> values)
-      throws Exception {
-    // No-op if no merging can take place
-    if (windowingStrategy.getWindowFn().isNonMerging()) {
-      return ImmutableMap.of();
-    }
-
-    // Collect the windows from all elements (except those which are too late) and
-    // make sure they are already in the active window set or are added as NEW windows.
-    for (WindowedValue<?> value : values) {
-      for (BoundedWindow untypedWindow : value.getWindows()) {
-        @SuppressWarnings("unchecked")
-        W window = (W) untypedWindow;
-
-        // For backwards compat with pre 1.4 only.
-        // We may still have ACTIVE windows with multiple state addresses, representing
-        // a window who's state has not yet been eagerly merged.
-        // We'll go ahead and merge that state now so that we don't have to worry about
-        // this legacy case anywhere else.
-        if (activeWindows.isActive(window)) {
-          Set<W> stateAddressWindows = activeWindows.readStateAddresses(window);
-          if (stateAddressWindows.size() > 1) {
-            // This is a legacy window who's state has not been eagerly merged.
-            // Do that now.
-            ReduceFn<K, InputT, OutputT, W>.OnMergeContext premergeContext =
-                contextFactory.forPremerge(window);
-            reduceFn.onMerge(premergeContext);
-            watermarkHold.onMerge(premergeContext);
-            activeWindows.merged(window);
-          }
-        }
-
-        // Add this window as NEW if it is not currently ACTIVE.
-        // If we had already seen this window and closed its trigger, then the
-        // window will not be currently ACTIVE. It will then be added as NEW here,
-        // and fall into the merging logic as usual.
-        activeWindows.ensureWindowExists(window);
-      }
-    }
-
-    // Merge all of the active windows and retain a mapping from source windows to result windows.
-    Map<W, W> windowToMergeResult = new HashMap<>();
-    activeWindows.merge(new OnMergeCallback(windowToMergeResult));
-    return windowToMergeResult;
-  }
-
-  private class OnMergeCallback implements ActiveWindowSet.MergeCallback<W> {
-    private final Map<W, W> windowToMergeResult;
-
-    OnMergeCallback(Map<W, W> windowToMergeResult) {
-      this.windowToMergeResult = windowToMergeResult;
-    }
-
-    /**
-     * Return the subset of {@code windows} which are currently ACTIVE. We only need to worry
-     * about merging state from ACTIVE windows. NEW windows by definition have no existing state.
-     */
-    private List<W> activeWindows(Iterable<W> windows) {
-      List<W> active = new ArrayList<>();
-      for (W window : windows) {
-        if (activeWindows.isActive(window)) {
-          active.add(window);
-        }
-      }
-      return active;
-    }
-
-    /**
-     * Called from the active window set to indicate {@code toBeMerged} (of which only
-     * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) will later
-     * be merged into {@code mergeResult}.
-     */
-    @Override
-    public void prefetchOnMerge(
-        Collection<W> toBeMerged, W mergeResult) throws Exception {
-      List<W> activeToBeMerged = activeWindows(toBeMerged);
-      ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
-          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
-      ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
-          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
-
-      // Prefetch various state.
-      triggerRunner.prefetchForMerge(mergeResult, activeToBeMerged, directMergeContext.state());
-      reduceFn.prefetchOnMerge(renamedMergeContext.state());
-      watermarkHold.prefetchOnMerge(renamedMergeContext.state());
-      nonEmptyPanes.prefetchOnMerge(renamedMergeContext.state());
-    }
-
-    /**
-     * Called from the active window set to indicate {@code toBeMerged} (of which only
-     * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) are about
-     * to be merged into {@code mergeResult}.
-     */
-    @Override
-    public void onMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {
-      // Remember we have merged these windows.
-      for (W window : toBeMerged) {
-        windowToMergeResult.put(window, mergeResult);
-      }
-
-      // At this point activeWindows has NOT incorporated the results of the merge.
-      List<W> activeToBeMerged = activeWindows(toBeMerged);
-      ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
-          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
-      ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
-          contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
-
-      // Run the reduceFn to perform any needed merging.
-      reduceFn.onMerge(renamedMergeContext);
-
-      // Merge the watermark holds.
-      watermarkHold.onMerge(renamedMergeContext);
-
-      // Merge non-empty pane state.
-      nonEmptyPanes.onMerge(renamedMergeContext.state());
-
-      // Have the trigger merge state as needed.
-      triggerRunner.onMerge(
-          directMergeContext.window(), directMergeContext.timers(), directMergeContext.state());
-
-      for (W active : activeToBeMerged) {
-        if (active.equals(mergeResult)) {
-          // Not merged away.
-          continue;
-        }
-        // Cleanup flavor A: Currently ACTIVE window is about to be merged away.
-        // Clear any state not already cleared by the onMerge calls above.
-        WindowTracing.debug("ReduceFnRunner.onMerge: Merging {} into {}", active, mergeResult);
-        ReduceFn<K, InputT, OutputT, W>.Context directClearContext =
-            contextFactory.base(active, StateStyle.DIRECT);
-        // No need for the end-of-window or garbage collection timers.
-        // We will establish a new end-of-window or garbage collection timer for the mergeResult
-        // window in processElement below. There must be at least one element for the mergeResult
-        // window since a new element with a new window must have triggered this onMerge.
-        cancelEndOfWindowAndGarbageCollectionTimers(directClearContext);
-        // We no longer care about any previous panes of merged away windows. The
-        // merge result window gets to start fresh if it is new.
-        paneInfoTracker.clear(directClearContext.state());
-      }
-    }
-  }
-
-  /**
-   * Process an element.
-   *
-   * @param value the value being processed
-   * @return the set of windows in which the element was actually processed
-   */
-  private Collection<W> processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT> value)
-      throws Exception {
-    // Redirect element windows to the ACTIVE windows they have been merged into.
-    // The compressed representation (value, {window1, window2, ...}) actually represents
-    // distinct elements (value, window1), (value, window2), ...
-    // so if window1 and window2 merge, the resulting window will contain both copies
-    // of the value.
-    Collection<W> windows = new ArrayList<>();
-    for (BoundedWindow untypedWindow : value.getWindows()) {
-      @SuppressWarnings("unchecked")
-      W window = (W) untypedWindow;
-      W mergeResult = windowToMergeResult.get(window);
-      if (mergeResult == null) {
-        mergeResult = window;
-      }
-      windows.add(mergeResult);
-    }
-
-    // Prefetch in each of the windows if we're going to need to process triggers
-    for (W window : windows) {
-      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
-          window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
-      triggerRunner.prefetchForValue(window, directContext.state());
-    }
-
-    // Process the element for each (mergeResultWindow, not closed) window it belongs to.
-    List<W> triggerableWindows = new ArrayList<>(windows.size());
-    for (W window : windows) {
-      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
-          window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
-      if (triggerRunner.isClosed(directContext.state())) {
-        // This window has already been closed.
-        droppedDueToClosedWindow.addValue(1L);
-        WindowTracing.debug(
-            "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
-            + "since window is no longer active at inputWatermark:{}; outputWatermark:{}",
-            value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
-            timerInternals.currentOutputWatermarkTime());
-        continue;
-      }
-
-      triggerableWindows.add(window);
-      activeWindows.ensureWindowIsActive(window);
-      ReduceFn<K, InputT, OutputT, W>.ProcessValueContext renamedContext = contextFactory.forValue(
-          window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED);
-
-      nonEmptyPanes.recordContent(renamedContext.state());
-
-      // Make sure we've scheduled the end-of-window or garbage collection timer for this window.
-      Instant timer = scheduleEndOfWindowOrGarbageCollectionTimer(directContext);
-
-      // Hold back progress of the output watermark until we have processed the pane this
-      // element will be included within. If the element is too late for that, place a hold at
-      // the end-of-window or garbage collection time to allow empty panes to contribute elements
-      // which won't be dropped due to lateness by a following computation (assuming the following
-      // computation uses the same allowed lateness value...)
-      @Nullable Instant hold = watermarkHold.addHolds(renamedContext);
-
-      if (hold != null) {
-        // Assert that holds have a proximate timer.
-        boolean holdInWindow = !hold.isAfter(window.maxTimestamp());
-        boolean timerInWindow = !timer.isAfter(window.maxTimestamp());
-        checkState(
-            holdInWindow == timerInWindow,
-            "set a hold at %s, a timer at %s, which disagree as to whether they are in window %s",
-            hold,
-            timer,
-            directContext.window());
-      }
-
-      // Execute the reduceFn, which will buffer the value as appropriate
-      reduceFn.processValue(renamedContext);
-
-      // Run the trigger to update its state
-      triggerRunner.processValue(
-          directContext.window(),
-          directContext.timestamp(),
-          directContext.timers(),
-          directContext.state());
-
-      // At this point, if triggerRunner.shouldFire before the processValue then
-      // triggerRunner.shouldFire after the processValue. In other words adding values
-      // cannot take a trigger state from firing to non-firing.
-      // (We don't actually assert this since it is too slow.)
-    }
-
-    return triggerableWindows;
-  }
-
-  /**
-   * Called when an end-of-window, garbage collection, or trigger-specific timer fires.
-   */
-  public void onTimer(TimerData timer) throws Exception {
-    // Which window is the timer for?
-    checkArgument(timer.getNamespace() instanceof WindowNamespace,
-        "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace());
-    @SuppressWarnings("unchecked")
-    WindowNamespace<W> windowNamespace = (WindowNamespace<W>) timer.getNamespace();
-    W window = windowNamespace.getWindow();
-    ReduceFn<K, InputT, OutputT, W>.Context directContext =
-        contextFactory.base(window, StateStyle.DIRECT);
-    ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
-        contextFactory.base(window, StateStyle.RENAMED);
-
-    // Has this window had its trigger finish?
-    // - The trigger may implement isClosed as constant false.
-    // - If the window function does not support windowing then all windows will be considered
-    // active.
-    // So we must take conjunction of activeWindows and triggerRunner state.
-    boolean windowIsActiveAndOpen =
-        activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state());
-
-    if (!windowIsActiveAndOpen) {
-      WindowTracing.debug(
-          "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window);
-    }
-
-    // If this is an end-of-window timer then we may need to set a garbage collection timer
-    // if allowed lateness is non-zero.
-    boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain()
-        && timer.getTimestamp().equals(window.maxTimestamp());
-
-    // If this is a garbage collection timer then we should trigger and garbage collect the window.
-    // We'll consider any timer at or after the end-of-window time to be a signal to garbage
-    // collect.
-    Instant cleanupTime = garbageCollectionTime(window);
-    boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain()
-        && !timer.getTimestamp().isBefore(cleanupTime);
-
-    if (isGarbageCollection) {
-      WindowTracing.debug(
-          "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with "
-          + "inputWatermark:{}; outputWatermark:{}",
-          key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
-          timerInternals.currentOutputWatermarkTime());
-
-      if (windowIsActiveAndOpen) {
-        // We need to call onTrigger to emit the final pane if required.
-        // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted,
-        // and the watermark has passed the end of the window.
-        @Nullable Instant newHold =
-            onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow);
-        checkState(newHold == null,
-            "Hold placed at %s despite isFinished being true.", newHold);
-      }
-
-      // Cleanup flavor B: Clear all the remaining state for this window since we'll never
-      // see elements for it again.
-      clearAllState(directContext, renamedContext, windowIsActiveAndOpen);
-    } else {
-      WindowTracing.debug(
-          "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with "
-          + "inputWatermark:{}; outputWatermark:{}",
-          key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
-          timerInternals.currentOutputWatermarkTime());
-      if (windowIsActiveAndOpen) {
-        emitIfAppropriate(directContext, renamedContext);
-      }
-
-      if (isEndOfWindow) {
-        // If the window strategy trigger includes a watermark trigger then at this point
-        // there should be no data holds, either because we'd already cleared them on an
-        // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate.
-        // We could assert this but it is very expensive.
-
-        // Since we are processing an on-time firing we should schedule the garbage collection
-        // timer. (If getAllowedLateness is zero then the timer event will be considered a
-        // cleanup event and handled by the above).
-        // Note we must do this even if the trigger is finished so that we are sure to cleanup
-        // any final trigger finished bits.
-        checkState(
-            windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO),
-            "Unexpected zero getAllowedLateness");
-        WindowTracing.debug(
-            "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with "
-            + "inputWatermark:{}; outputWatermark:{}",
-            key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(),
-            timerInternals.currentOutputWatermarkTime());
-        checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-                                 "Cleanup time %s is beyond end-of-time", cleanupTime);
-        directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME);
-      }
-    }
-  }
-
-  /**
-   * Clear all the state associated with {@code context}'s window.
-   * Should only be invoked if we know all future elements for this window will be considered
-   * beyond allowed lateness.
-   * This is a superset of the clearing done by {@link #emitIfAppropriate} below since:
-   * <ol>
-   * <li>We can clear the trigger finished bits since we'll never need to ask if the trigger is
-   * closed again.
-   * <li>We can clear any remaining garbage collection hold.
-   * </ol>
-   */
-  private void clearAllState(
-      ReduceFn<K, InputT, OutputT, W>.Context directContext,
-      ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
-      boolean windowIsActiveAndOpen)
-      throws Exception {
-    if (windowIsActiveAndOpen) {
-      // Since both the window is in the active window set AND the trigger was not yet closed,
-      // it is possible we still have state.
-      reduceFn.clearState(renamedContext);
-      watermarkHold.clearHolds(renamedContext);
-      nonEmptyPanes.clearPane(renamedContext.state());
-      // These calls work irrespective of whether the window is active or not, but
-      // are unnecessary if the window is not active.
-      triggerRunner.clearState(
-          directContext.window(), directContext.timers(), directContext.state());
-      paneInfoTracker.clear(directContext.state());
-    } else {
-      // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2).
-      // For (1), if !activeWindows.isActive then the window must be merging and has been
-      // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired
-      // and been closed, so this case reduces to (2).
-      // For (2), if triggerRunner.isClosed then the trigger was fired and entered the
-      // closed state. In that case emitIfAppropriate will have cleared all state in
-      // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows.
-      // We also know nonEmptyPanes must have been unconditionally cleared by the trigger.
-      // Since the trigger fired the existing watermark holds must have been cleared, and since
-      // the trigger closed no new end of window or garbage collection hold will have been
-      // placed by WatermarkHold.extractAndRelease.
-      // Thus all the state clearing above is unnecessary.
-      //
-      // But(!) for backwards compatibility we must allow a pipeline to be updated from
-      // an sdk version <= 1.3. In that case it is possible we have an end-of-window or
-      // garbage collection hold keyed by the current window (reached via directContext) rather
-      // than the state address window (reached via renamedContext).
-      // However this can only happen if:
-      // - We have merging windows.
-      // - We are DISCARDING_FIRED_PANES.
-      // - A pane has fired.
-      // - But the trigger is not (yet) closed.
-      if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES
-          && !windowingStrategy.getWindowFn().isNonMerging()) {
-        watermarkHold.clearHolds(directContext);
-      }
-    }
-
-    // Don't need to track address state windows anymore.
-    activeWindows.remove(directContext.window());
-    // We'll never need to test for the trigger being closed again.
-    triggerRunner.clearFinished(directContext.state());
-  }
-
-  /** Should the reduce function state be cleared? */
-  private boolean shouldDiscardAfterFiring(boolean isFinished) {
-    if (isFinished) {
-      // This is the last firing for trigger.
-      return true;
-    }
-    if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
-      // Nothing should be accumulated between panes.
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state.
-   */
-  private void emitIfAppropriate(ReduceFn<K, InputT, OutputT, W>.Context directContext,
-      ReduceFn<K, InputT, OutputT, W>.Context renamedContext)
-      throws Exception {
-    if (!triggerRunner.shouldFire(
-        directContext.window(), directContext.timers(), directContext.state())) {
-      // Ignore unless trigger is ready to fire
-      return;
-    }
-
-    // Inform the trigger of the transition to see if it is finished
-    triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state());
-    boolean isFinished = triggerRunner.isClosed(directContext.state());
-
-    // Will be able to clear all element state after triggering?
-    boolean shouldDiscard = shouldDiscardAfterFiring(isFinished);
-
-    // Run onTrigger to produce the actual pane contents.
-    // As a side effect it will clear all element holds, but not necessarily any
-    // end-of-window or garbage collection holds.
-    onTrigger(directContext, renamedContext, isFinished, false /*isEndOfWindow*/);
-
-    // Now that we've triggered, the pane is empty.
-    nonEmptyPanes.clearPane(renamedContext.state());
-
-    // Cleanup buffered data if appropriate
-    if (shouldDiscard) {
-      // Cleanup flavor C: The user does not want any buffered data to persist between panes.
-      reduceFn.clearState(renamedContext);
-    }
-
-    if (isFinished) {
-      // Cleanup flavor D: If trigger is closed we will ignore all new incoming elements.
-      // Clear state not otherwise cleared by onTrigger and clearPane above.
-      // Remember the trigger is, indeed, closed until the window is garbage collected.
-      triggerRunner.clearState(
-          directContext.window(), directContext.timers(), directContext.state());
-      paneInfoTracker.clear(directContext.state());
-      activeWindows.remove(directContext.window());
-    }
-  }
-
-  /**
-   * Do we need to emit a pane?
-   */
-  private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) {
-    if (!isEmpty) {
-      // The pane has elements.
-      return true;
-    }
-    if (timing == Timing.ON_TIME) {
-      // This is the unique ON_TIME pane.
-      return true;
-    }
-    if (isFinished && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) {
-      // This is known to be the final pane, and the user has requested it even when empty.
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Run the {@link ReduceFn#onTrigger} method and produce any necessary output.
-   *
-   * @return output watermark hold added, or {@literal null} if none.
-   */
-  @Nullable
-  private Instant onTrigger(
-      final ReduceFn<K, InputT, OutputT, W>.Context directContext,
-      ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
-      boolean isFinished, boolean isEndOfWindow)
-          throws Exception {
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-
-    // Prefetch necessary states
-    ReadableState<WatermarkHold.OldAndNewHolds> outputTimestampFuture =
-        watermarkHold.extractAndRelease(renamedContext, isFinished).readLater();
-    ReadableState<PaneInfo> paneFuture =
-        paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater();
-    ReadableState<Boolean> isEmptyFuture =
-        nonEmptyPanes.isEmpty(renamedContext.state()).readLater();
-
-    reduceFn.prefetchOnTrigger(directContext.state());
-    triggerRunner.prefetchOnFire(directContext.window(), directContext.state());
-
-    // Calculate the pane info.
-    final PaneInfo pane = paneFuture.read();
-    // Extract the window hold, and as a side effect clear it.
-
-    WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read();
-    final Instant outputTimestamp = pair.oldHold;
-    @Nullable Instant newHold = pair.newHold;
-
-    if (newHold != null) {
-      // We can't be finished yet.
-      checkState(
-        !isFinished, "new hold at %s but finished %s", newHold, directContext.window());
-      // The hold cannot be behind the input watermark.
-      checkState(
-        !newHold.isBefore(inputWM), "new hold %s is before input watermark %s", newHold, inputWM);
-      if (newHold.isAfter(directContext.window().maxTimestamp())) {
-        // The hold must be for garbage collection, which can't have happened yet.
-        checkState(
-          newHold.isEqual(garbageCollectionTime(directContext.window())),
-          "new hold %s should be at garbage collection for window %s plus %s",
-          newHold,
-          directContext.window(),
-          windowingStrategy.getAllowedLateness());
-      } else {
-        // The hold must be for the end-of-window, which can't have happened yet.
-        checkState(
-          newHold.isEqual(directContext.window().maxTimestamp()),
-          "new hold %s should be at end of window %s",
-          newHold,
-          directContext.window());
-        checkState(
-          !isEndOfWindow,
-          "new hold at %s for %s but this is the watermark trigger",
-          newHold,
-          directContext.window());
-      }
-    }
-
-    // Only emit a pane if it has data or empty panes are observable.
-    if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) {
-      // Run reduceFn.onTrigger method.
-      final List<W> windows = Collections.singletonList(directContext.window());
-      ReduceFn<K, InputT, OutputT, W>.OnTriggerContext renamedTriggerContext =
-          contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED,
-              new OnTriggerCallbacks<OutputT>() {
-                @Override
-                public void output(OutputT toOutput) {
-                  // We're going to output panes, so commit the (now used) PaneInfo.
-                  // TODO: This is unnecessary if the trigger isFinished since the saved
-                  // state will be immediately deleted.
-                  paneInfoTracker.storeCurrentPaneInfo(directContext, pane);
-
-                  // Output the actual value.
-                  outputter.outputWindowedValue(
-                      KV.of(key, toOutput), outputTimestamp, windows, pane);
-                }
-              });
-
-      reduceFn.onTrigger(renamedTriggerContext);
-    }
-
-    return newHold;
-  }
-
-  /**
-   * Make sure we'll eventually have a timer fire which will tell us to garbage collect
-   * the window state. For efficiency we may need to do this in two steps rather
-   * than one. Return the time at which the timer will fire.
-   *
-   * <ul>
-   * <li>If allowedLateness is zero then we'll garbage collect at the end of the window.
-   * For simplicity we'll set our own timer for this situation even though an
-   * {@link AfterWatermark} trigger may have also set an end-of-window timer.
-   * ({@code setTimer} is idempotent.)
-   * <li>If allowedLateness is non-zero then we could just always set a timer for the garbage
-   * collection time. However if the windows are large (eg hourly) and the allowedLateness is small
-   * (eg seconds) then we'll end up with nearly twice the number of timers in-flight. So we
-   * instead set an end-of-window timer and then roll that forward to a garbage collection timer
-   * when it fires. We use the input watermark to distinguish those cases.
-   * </ul>
-   */
-  private Instant scheduleEndOfWindowOrGarbageCollectionTimer(
-      ReduceFn<?, ?, ?, W>.Context directContext) {
-    Instant inputWM = timerInternals.currentInputWatermarkTime();
-    Instant endOfWindow = directContext.window().maxTimestamp();
-    String which;
-    Instant timer;
-    if (endOfWindow.isBefore(inputWM)) {
-      timer = garbageCollectionTime(directContext.window());
-      which = "garbage collection";
-    } else {
-      timer = endOfWindow;
-      which = "end-of-window";
-    }
-    WindowTracing.trace(
-        "ReduceFnRunner.scheduleEndOfWindowOrGarbageCollectionTimer: Scheduling {} timer at {} for "
-        + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
-        which,
-        timer,
-        key,
-        directContext.window(),
-        inputWM,
-        timerInternals.currentOutputWatermarkTime());
-    checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-                             "Timer %s is beyond end-of-time", timer);
-    directContext.timers().setTimer(timer, TimeDomain.EVENT_TIME);
-    return timer;
-  }
-
-  private void cancelEndOfWindowAndGarbageCollectionTimers(
-      ReduceFn<?, ?, ?, W>.Context directContext) {
-    WindowTracing.debug(
-        "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for "
-        + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
-        key, directContext.window(), timerInternals.currentInputWatermarkTime(),
-        timerInternals.currentOutputWatermarkTime());
-    Instant eow = directContext.window().maxTimestamp();
-    directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
-    Instant gc = garbageCollectionTime(directContext.window());
-    if (gc.isAfter(eow)) {
-      directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
-    }
-  }
-
-  /**
-   * 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) {
-
-    // 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 window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
-    }
-  }
-
-  /**
-   * An object that can output a value with all of its windowing information. This is a deliberately
-   * restricted subinterface of {@link WindowingInternals} to express how it is used here.
-   */
-  private interface OutputWindowedValue<OutputT> {
-    void outputWindowedValue(OutputT output, Instant timestamp,
-        Collection<? extends BoundedWindow> windows, PaneInfo pane);
-  }
-
-  private static class OutputViaWindowingInternals<OutputT>
-      implements OutputWindowedValue<OutputT> {
-
-    private final WindowingInternals<?, OutputT> windowingInternals;
-
-    public OutputViaWindowingInternals(WindowingInternals<?, OutputT> windowingInternals) {
-      this.windowingInternals = windowingInternals;
-    }
-
-    @Override
-    public void outputWindowedValue(
-        OutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      windowingInternals.outputWindowedValue(output, timestamp, windows, pane);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
deleted file mode 100644
index 1ebe5a8..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import java.util.List;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in.
- *
- * @param <InputT> the type of the {@link OldDoFn} (main) input elements
- * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
- */
-public class SimpleDoFnRunner<InputT, OutputT> extends DoFnRunnerBase<InputT, OutputT>{
-
-  protected SimpleDoFnRunner(PipelineOptions options, OldDoFn<InputT, OutputT> fn,
-      SideInputReader sideInputReader,
-      OutputManager outputManager,
-      TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
-      AggregatorFactory aggregatorFactory, WindowingStrategy<?, ?> windowingStrategy) {
-    super(options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext,
-        aggregatorFactory, windowingStrategy);
-  }
-
-  @Override
-  protected void invokeProcessElement(WindowedValue<InputT> elem) {
-    final OldDoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.processElement(processContext);
-    } catch (Exception ex) {
-      throw wrapUserCodeException(ex);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4bf3a3b3/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
deleted file mode 100644
index 28177a8..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
+++ /dev/null
@@ -1,138 +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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-
-/**
- * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}.
- *
- * @param <K> The type of key being processed.
- * @param <InputT> The type of values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W> The type of windows this operates on.
- */
-public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends BoundedWindow>
-    extends ReduceFn<K, InputT, OutputT, W> {
-  private static final String BUFFER_NAME = "buf";
-
-  /**
-   * Create a factory that produces {@link SystemReduceFn} instances that that buffer all of the
-   * input values in persistent state and produces an {@code Iterable<T>}.
-   */
-  public static <K, T, W extends BoundedWindow> SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>
-      buffering(final Coder<T> inputCoder) {
-    final StateTag<Object, BagState<T>> bufferTag =
-        StateTags.makeSystemTagInternal(StateTags.bag(BUFFER_NAME, inputCoder));
-    return new SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>(bufferTag) {
-      @Override
-      public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
-        StateMerging.prefetchBags(state, bufferTag);
-      }
-
-      @Override
-      public void onMerge(OnMergeContext c) throws Exception {
-        StateMerging.mergeBags(c.state(), bufferTag);
-      }
-    };
-  }
-
-  /**
-   * Create a factory that produces {@link SystemReduceFn} instances that combine all of the input
-   * values using a {@link CombineFn}.
-   */
-  public static <K, InputT, AccumT, OutputT, W extends BoundedWindow> SystemReduceFn<K, InputT,
-      AccumT, OutputT, W>
-      combining(
-          final Coder<K> keyCoder, final AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
-    final StateTag<K, AccumulatorCombiningState<InputT, AccumT, OutputT>> bufferTag;
-    if (combineFn.getFn() instanceof KeyedCombineFnWithContext) {
-      bufferTag = StateTags.makeSystemTagInternal(
-          StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValueWithContext(
-              BUFFER_NAME, combineFn.getAccumulatorCoder(),
-              (KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>) combineFn.getFn()));
-
-    } else {
-      bufferTag = StateTags.makeSystemTagInternal(
-            StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValue(
-                BUFFER_NAME, combineFn.getAccumulatorCoder(),
-                (KeyedCombineFn<K, InputT, AccumT, OutputT>) combineFn.getFn()));
-    }
-    return new SystemReduceFn<K, InputT, AccumT, OutputT, W>(bufferTag) {
-      @Override
-      public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
-        StateMerging.prefetchCombiningValues(state, bufferTag);
-      }
-
-      @Override
-      public void onMerge(OnMergeContext c) throws Exception {
-        StateMerging.mergeCombiningValues(c.state(), bufferTag);
-      }
-    };
-  }
-
-  private StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag;
-
-  public SystemReduceFn(
-      StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag) {
-    this.bufferTag = bufferTag;
-  }
-
-  @Override
-  public void processValue(ProcessValueContext c) throws Exception {
-    c.state().access(bufferTag).add(c.value());
-  }
-
-  @Override
-  @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT",
-    justification = "prefetch side effect")
-  public void prefetchOnTrigger(StateAccessor<K> state) {
-    state.access(bufferTag).readLater();
-  }
-
-  @Override
-  public void onTrigger(OnTriggerContext c) throws Exception {
-    c.output(c.state().access(bufferTag).read());
-  }
-
-  @Override
-  public void clearState(Context c) throws Exception {
-    c.state().access(bufferTag).clear();
-  }
-
-  @Override
-  public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
-    return state.access(bufferTag).isEmpty();
-  }
-}

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


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

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


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

Branch: refs/heads/gearpump-runner
Commit: ed7c4aaf96f766c8d82d3422a02a9c95446e7fb8
Parents: bf0a2ed 59ae94c
Author: Dan Halperin <dh...@google.com>
Authored: Mon Sep 12 17:40:15 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:15 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |    2 +-
 examples/java/pom.xml                           |  142 +-
 .../beam/examples/DebuggingWordCount.java       |   30 +-
 .../apache/beam/examples/MinimalWordCount.java  |    9 +-
 .../apache/beam/examples/WindowedWordCount.java |   44 +-
 .../org/apache/beam/examples/WordCount.java     |   57 +-
 .../common/ExampleBigQueryTableOptions.java     |    6 +-
 .../beam/examples/common/ExampleOptions.java    |   48 +-
 ...xamplePubsubTopicAndSubscriptionOptions.java |    2 +-
 .../common/ExamplePubsubTopicOptions.java       |    2 +-
 .../beam/examples/common/ExampleUtils.java      |   93 +-
 .../examples/common/PubsubFileInjector.java     |   16 +-
 .../beam/examples/complete/AutoComplete.java    |   76 +-
 .../org/apache/beam/examples/complete/README.md |    4 +-
 .../examples/complete/StreamingWordExtract.java |   20 +-
 .../apache/beam/examples/complete/TfIdf.java    |   42 +-
 .../examples/complete/TopWikipediaSessions.java |   26 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   29 +-
 .../beam/examples/complete/TrafficRoutes.java   |   39 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   30 +-
 .../cookbook/CombinePerKeyExamples.java         |   30 +-
 .../examples/cookbook/DatastoreWordCount.java   |   69 +-
 .../beam/examples/cookbook/DeDupExample.java    |   20 +-
 .../beam/examples/cookbook/FilterExamples.java  |   35 +-
 .../beam/examples/cookbook/JoinExamples.java    |   21 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   31 +-
 .../beam/examples/cookbook/TriggerExample.java  |   35 +-
 .../beam/examples/DebuggingWordCountTest.java   |    6 +-
 .../org/apache/beam/examples/WordCountIT.java   |  131 +-
 .../org/apache/beam/examples/WordCountTest.java |    6 +-
 .../examples/complete/AutoCompleteTest.java     |   12 +-
 .../beam/examples/complete/TfIdfTest.java       |    6 +-
 .../complete/TopWikipediaSessionsTest.java      |    7 +-
 .../examples/cookbook/BigQueryTornadoesIT.java  |    1 -
 .../cookbook/BigQueryTornadoesTest.java         |    7 +-
 .../cookbook/CombinePerKeyExamplesTest.java     |    7 +-
 .../examples/cookbook/DeDupExampleTest.java     |    6 +-
 .../examples/cookbook/FilterExamplesTest.java   |    9 +-
 .../examples/cookbook/JoinExamplesTest.java     |    9 +-
 .../cookbook/MaxPerKeyExamplesTest.java         |    9 +-
 .../examples/cookbook/TriggerExampleTest.java   |   17 +-
 examples/java8/pom.xml                          |   46 +-
 .../beam/examples/MinimalWordCountJava8.java    |    5 +-
 .../beam/examples/complete/game/GameStats.java  |    8 +-
 .../examples/complete/game/HourlyTeamScore.java |   10 +-
 .../examples/complete/game/LeaderBoard.java     |  121 +-
 .../beam/examples/complete/game/UserScore.java  |   10 +-
 .../complete/game/injector/Injector.java        |    8 +-
 .../complete/game/injector/InjectorUtils.java   |    1 -
 .../injector/RetryHttpInitializerWrapper.java   |    1 -
 .../complete/game/utils/WriteToBigQuery.java    |   18 +-
 .../game/utils/WriteWindowedToBigQuery.java     |    6 +-
 .../examples/MinimalWordCountJava8Test.java     |   23 +-
 .../examples/complete/game/GameStatsTest.java   |    8 +-
 .../complete/game/HourlyTeamScoreTest.java      |    8 +-
 .../examples/complete/game/LeaderBoardTest.java |  362 +++++
 .../examples/complete/game/UserScoreTest.java   |    8 +-
 pom.xml                                         |   12 +-
 .../apache/beam/runners/core/AssignWindows.java |   46 +
 .../beam/runners/core/AssignWindowsDoFn.java    |   79 +
 .../beam/runners/core/BatchTimerInternals.java  |  140 ++
 .../apache/beam/runners/core/DoFnRunner.java    |   65 +
 .../beam/runners/core/DoFnRunnerBase.java       |  559 +++++++
 .../apache/beam/runners/core/DoFnRunners.java   |  148 ++
 .../runners/core/ElementByteSizeObservable.java |   44 +
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |    5 +-
 .../runners/core/GroupAlsoByWindowsDoFn.java    |   66 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 ++
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  271 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  151 ++
 .../apache/beam/runners/core/NonEmptyPanes.java |  151 ++
 .../beam/runners/core/PaneInfoTracker.java      |  158 ++
 .../beam/runners/core/PeekingReiterator.java    |  100 ++
 .../core/PushbackSideInputDoFnRunner.java       |  116 ++
 .../org/apache/beam/runners/core/ReduceFn.java  |  130 ++
 .../runners/core/ReduceFnContextFactory.java    |  499 ++++++
 .../beam/runners/core/ReduceFnRunner.java       |  993 ++++++++++++
 .../beam/runners/core/SideInputHandler.java     |  243 +++
 .../beam/runners/core/SimpleDoFnRunner.java     |   58 +
 .../beam/runners/core/SystemReduceFn.java       |  139 ++
 .../apache/beam/runners/core/TriggerRunner.java |  247 +++
 .../core/UnboundedReadFromBoundedSource.java    |   43 +-
 .../apache/beam/runners/core/WatermarkHold.java |  539 +++++++
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 -
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   80 -
 .../beam/sdk/util/BatchTimerInternals.java      |  141 --
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   63 -
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  551 -------
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  222 ---
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   63 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 --
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  269 ----
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  147 --
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 --
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  158 --
 .../sdk/util/PushbackSideInputDoFnRunner.java   |  115 --
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  130 --
 .../beam/sdk/util/ReduceFnContextFactory.java   |  497 ------
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  988 ------------
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   55 -
 .../apache/beam/sdk/util/SystemReduceFn.java    |  139 --
 .../org/apache/beam/sdk/util/TriggerRunner.java |  245 ---
 .../org/apache/beam/sdk/util/WatermarkHold.java |  540 -------
 .../util/common/ElementByteSizeObservable.java  |   42 -
 .../beam/sdk/util/common/PeekingReiterator.java |   99 --
 .../beam/sdk/util/common/package-info.java      |   20 -
 .../org/apache/beam/sdk/util/package-info.java  |   20 -
 .../runners/core/BatchTimerInternalsTest.java   |  118 ++
 .../core/GroupAlsoByWindowsProperties.java      |  660 ++++++++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  110 ++
 .../core/LateDataDroppingDoFnRunnerTest.java    |  117 ++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  235 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   | 1446 +++++++++++++++++
 .../beam/runners/core/ReduceFnTester.java       |  796 ++++++++++
 .../beam/runners/core/SideInputHandlerTest.java |  220 +++
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   88 ++
 .../UnboundedReadFromBoundedSourceTest.java     |   27 +-
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  118 --
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  661 --------
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  110 --
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 --
 .../util/PushbackSideInputDoFnRunnerTest.java   |  234 ---
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1447 ------------------
 .../apache/beam/sdk/util/ReduceFnTester.java    |  789 ----------
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   86 --
 runners/direct-java/pom.xml                     |    6 +
 .../runners/direct/AggregatorContainer.java     |    8 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   18 +-
 .../beam/runners/direct/CloningThreadLocal.java |   43 -
 .../beam/runners/direct/CommittedResult.java    |    9 +-
 .../direct/ConsumerTrackingPipelineVisitor.java |   13 +-
 .../beam/runners/direct/DirectOptions.java      |   45 +-
 .../beam/runners/direct/DirectRegistrar.java    |    5 +-
 .../beam/runners/direct/DirectRunner.java       |   68 +-
 .../runners/direct/DirectTimerInternals.java    |    4 +-
 .../runners/direct/DoFnLifecycleManager.java    |  103 ++
 ...ecycleManagerRemovingTransformEvaluator.java |   78 +
 .../runners/direct/DoFnLifecycleManagers.java   |   45 +
 .../beam/runners/direct/EvaluationContext.java  |   36 +-
 .../direct/ExecutorServiceParallelExecutor.java |   45 +-
 .../FixedThreadPoolExecutorServiceFactory.java  |   45 -
 .../runners/direct/FlattenEvaluatorFactory.java |    3 +
 .../GroupAlsoByWindowEvaluatorFactory.java      |   20 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   21 +-
 .../ImmutabilityCheckingBundleFactory.java      |   10 +-
 .../direct/ImmutabilityEnforcementFactory.java  |    5 +-
 .../direct/ImmutableListBundleFactory.java      |    6 +-
 .../direct/KeyedPValueTrackingVisitor.java      |    5 +-
 .../beam/runners/direct/KeyedResourcePool.java  |   47 +
 .../runners/direct/LockedKeyedResourcePool.java |   95 ++
 .../beam/runners/direct/NanosOffsetClock.java   |   16 +-
 .../beam/runners/direct/ParDoEvaluator.java     |   22 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |   64 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   53 +-
 .../beam/runners/direct/PipelineExecutor.java   |    3 +-
 .../beam/runners/direct/SideInputContainer.java |   19 +-
 .../apache/beam/runners/direct/StepAndKey.java  |    4 +-
 .../runners/direct/StepTransformResult.java     |   16 +-
 .../direct/TestStreamEvaluatorFactory.java      |  218 +++
 ...readLocalInvalidatingTransformEvaluator.java |   63 -
 .../direct/TransformEvaluatorFactory.java       |   14 +-
 .../direct/TransformEvaluatorRegistry.java      |   60 +-
 .../beam/runners/direct/TransformExecutor.java  |    8 +-
 .../direct/TransformExecutorServices.java       |    1 -
 .../beam/runners/direct/TransformResult.java    |    7 +-
 .../direct/UnboundedReadDeduplicator.java       |   11 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   19 +-
 .../direct/UncommittedBundleOutputManager.java  |    5 +-
 .../runners/direct/ViewEvaluatorFactory.java    |    8 +-
 .../direct/WatermarkCallbackExecutor.java       |   11 +-
 .../beam/runners/direct/WatermarkManager.java   |   43 +-
 .../runners/direct/WindowEvaluatorFactory.java  |   13 +-
 .../direct/WriteWithShardingFactory.java        |   13 +-
 .../runners/direct/AggregatorContainerTest.java |    9 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |   13 +-
 .../runners/direct/CloningThreadLocalTest.java  |   92 --
 .../runners/direct/CommittedResultTest.java     |   13 +-
 .../ConsumerTrackingPipelineVisitorTest.java    |    6 +-
 .../runners/direct/DirectRegistrarTest.java     |    9 +-
 .../beam/runners/direct/DirectRunnerTest.java   |   16 +-
 .../direct/DirectTimerInternalsTest.java        |    1 -
 ...leManagerRemovingTransformEvaluatorTest.java |  142 ++
 .../direct/DoFnLifecycleManagerTest.java        |  169 ++
 .../direct/DoFnLifecycleManagersTest.java       |  139 ++
 .../EncodabilityEnforcementFactoryTest.java     |   10 +-
 .../runners/direct/EvaluationContextTest.java   |   18 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |    1 -
 .../direct/ForwardingPTransformTest.java        |    1 -
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |    8 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |    8 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |    1 -
 .../ImmutabilityEnforcementFactoryTest.java     |    6 +-
 .../direct/ImmutableListBundleFactoryTest.java  |   11 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |    9 +-
 .../direct/LockedKeyedResourcePoolTest.java     |  163 ++
 .../beam/runners/direct/ParDoEvaluatorTest.java |   14 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |    2 +-
 .../direct/ParDoSingleEvaluatorFactoryTest.java |    2 +-
 .../runners/direct/SideInputContainerTest.java  |   17 +-
 .../runners/direct/StepTransformResultTest.java |    1 -
 .../beam/runners/direct/StructuralKeyTest.java  |    1 -
 .../direct/TestStreamEvaluatorFactoryTest.java  |  206 +++
 ...LocalInvalidatingTransformEvaluatorTest.java |  135 --
 .../direct/TransformExecutorServicesTest.java   |    4 +-
 .../runners/direct/TransformExecutorTest.java   |   23 +-
 .../direct/UnboundedReadDeduplicatorTest.java   |   23 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   26 +-
 .../direct/ViewEvaluatorFactoryTest.java        |    4 +-
 .../direct/WatermarkCallbackExecutorTest.java   |    8 +-
 .../runners/direct/WatermarkManagerTest.java    |   21 +-
 .../direct/WindowEvaluatorFactoryTest.java      |   14 +-
 .../direct/WriteWithShardingFactoryTest.java    |   30 +-
 runners/flink/README.md                         |   17 +
 runners/flink/examples/pom.xml                  |    2 -
 .../beam/runners/flink/examples/TFIDF.java      |   20 +-
 .../beam/runners/flink/examples/WordCount.java  |   10 +-
 .../runners/flink/examples/package-info.java    |   22 +
 .../flink/examples/streaming/AutoComplete.java  |   11 +-
 .../flink/examples/streaming/JoinExamples.java  |    4 +-
 .../examples/streaming/KafkaIOExamples.java     |   46 +-
 .../KafkaWindowedWordCountExample.java          |   31 +-
 .../examples/streaming/WindowedWordCount.java   |   23 +-
 .../flink/examples/streaming/package-info.java  |   22 +
 runners/flink/runner/pom.xml                    |   13 +-
 .../flink/DefaultParallelismFactory.java        |    1 -
 .../FlinkPipelineExecutionEnvironment.java      |   21 +-
 .../runners/flink/FlinkPipelineOptions.java     |   65 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  475 +++++-
 .../runners/flink/FlinkRunnerRegistrar.java     |   11 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |   27 +-
 .../beam/runners/flink/TestFlinkRunner.java     |   33 +-
 .../apache/beam/runners/flink/package-info.java |   22 +
 .../FlinkBatchPipelineTranslator.java           |   16 +-
 .../FlinkBatchTransformTranslators.java         |   19 +-
 .../FlinkBatchTranslationContext.java           |   16 +-
 .../translation/FlinkPipelineTranslator.java    |    2 +-
 .../FlinkStreamingPipelineTranslator.java       |   60 +-
 .../FlinkStreamingTransformTranslators.java     |  972 +++++++++---
 .../FlinkStreamingTranslationContext.java       |    6 +-
 .../flink/translation/TranslationMode.java      |    8 +-
 .../functions/FlinkAssignContext.java           |    6 +-
 .../functions/FlinkAssignWindows.java           |    4 +-
 .../functions/FlinkDoFnFunction.java            |   20 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   17 +-
 .../FlinkMergingPartialReduceFunction.java      |   17 +-
 .../functions/FlinkMergingReduceFunction.java   |   21 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   42 +-
 .../FlinkMultiOutputProcessContext.java         |    6 +-
 .../FlinkMultiOutputPruningFunction.java        |    1 -
 .../functions/FlinkNoElementAssignContext.java  |    1 -
 .../functions/FlinkPartialReduceFunction.java   |   17 +-
 .../functions/FlinkProcessContext.java          |   15 +-
 .../functions/FlinkReduceFunction.java          |   21 +-
 .../functions/SideInputInitializer.java         |   10 +-
 .../translation/functions/package-info.java     |   22 +
 .../runners/flink/translation/package-info.java |   22 +
 .../translation/types/CoderTypeInformation.java |    5 +-
 .../translation/types/CoderTypeSerializer.java  |   10 +-
 .../types/EncodedValueComparator.java           |    6 +-
 .../types/EncodedValueSerializer.java           |  164 +-
 .../types/EncodedValueTypeInformation.java      |    4 -
 .../flink/translation/types/FlinkCoder.java     |   11 +-
 .../flink/translation/types/KvKeySelector.java  |    1 -
 .../flink/translation/types/package-info.java   |   22 +
 .../utils/SerializedPipelineOptions.java        |    6 +-
 .../flink/translation/utils/package-info.java   |   22 +
 .../wrappers/DataInputViewWrapper.java          |    3 +-
 .../wrappers/DataOutputViewWrapper.java         |    5 +-
 .../SerializableFnAggregatorWrapper.java        |    9 +-
 .../translation/wrappers/SourceInputFormat.java |    6 +-
 .../translation/wrappers/SourceInputSplit.java  |    1 -
 .../translation/wrappers/package-info.java      |   22 +
 .../wrappers/streaming/DoFnOperator.java        |  509 ++++++
 .../streaming/FlinkAbstractParDoWrapper.java    |  280 ----
 .../FlinkGroupAlsoByWindowWrapper.java          |  642 --------
 .../streaming/FlinkGroupByKeyWrapper.java       |   73 -
 .../streaming/FlinkParDoBoundMultiWrapper.java  |   79 -
 .../streaming/FlinkParDoBoundWrapper.java       |  104 --
 .../wrappers/streaming/FlinkStateInternals.java | 1035 +++++++++++++
 .../streaming/SingletonKeyedWorkItem.java       |   58 +
 .../streaming/SingletonKeyedWorkItemCoder.java  |  130 ++
 .../wrappers/streaming/WindowDoFnOperator.java  |  341 +++++
 .../wrappers/streaming/WorkItemKeySelector.java |   56 +
 .../streaming/io/BoundedSourceWrapper.java      |  216 +++
 .../io/FlinkStreamingCreateFunction.java        |   56 -
 .../streaming/io/UnboundedFlinkSink.java        |   27 +-
 .../streaming/io/UnboundedFlinkSource.java      |   36 +-
 .../streaming/io/UnboundedSocketSource.java     |   63 +-
 .../streaming/io/UnboundedSourceWrapper.java    |   15 +-
 .../wrappers/streaming/io/package-info.java     |   22 +
 .../wrappers/streaming/package-info.java        |   22 +
 .../state/AbstractFlinkTimerInternals.java      |  127 --
 .../streaming/state/FlinkStateInternals.java    |  733 ---------
 .../streaming/state/StateCheckpointReader.java  |   93 --
 .../streaming/state/StateCheckpointUtils.java   |  155 --
 .../streaming/state/StateCheckpointWriter.java  |  131 --
 .../wrappers/streaming/state/StateType.java     |   73 -
 .../flink/EncodedValueComparatorTest.java       |    1 -
 .../runners/flink/FlinkRunnerRegistrarTest.java |    1 -
 .../beam/runners/flink/PipelineOptionsTest.java |  106 +-
 .../beam/runners/flink/ReadSourceITCase.java    |    9 +-
 .../flink/ReadSourceStreamingITCase.java        |    4 +-
 .../beam/runners/flink/WriteSinkITCase.java     |   16 +-
 .../apache/beam/runners/flink/package-info.java |   22 +
 .../flink/streaming/DoFnOperatorTest.java       |  325 ++++
 .../streaming/FlinkStateInternalsTest.java      |  390 +++++
 .../flink/streaming/GroupAlsoByWindowTest.java  |  523 -------
 .../flink/streaming/GroupByNullKeyTest.java     |   17 +-
 .../flink/streaming/StateSerializationTest.java |  338 ----
 .../flink/streaming/TestCountingSource.java     |   12 +-
 .../streaming/TopWikipediaSessionsITCase.java   |   13 +-
 .../streaming/UnboundedSourceWrapperTest.java   |   43 +-
 .../runners/flink/streaming/package-info.java   |   22 +
 runners/gearpump/pom.xml                        |    7 +-
 .../gearpump/GearpumpPipelineOptions.java       |    8 +-
 .../gearpump/GearpumpPipelineResult.java        |    4 +-
 .../gearpump/GearpumpPipelineRunner.java        |   28 +-
 .../GearpumpPipelineRunnerRegistrar.java        |    9 +-
 .../gearpump/GearpumpPipelineTranslator.java    |    7 +-
 .../gearpump/examples/StreamingWordCount.java   |   15 +-
 .../gearpump/examples/UnboundedTextSource.java  |   15 +-
 .../translators/GroupByKeyTranslator.java       |   14 +-
 .../translators/ParDoBoundMultiTranslator.java  |   16 +-
 .../translators/TransformTranslator.java        |    3 +-
 .../translators/TranslationContext.java         |    6 +-
 .../translators/functions/DoFnFunction.java     |   14 +-
 .../translators/io/BoundedSourceWrapper.java    |    4 +-
 .../gearpump/translators/io/GearpumpSource.java |   12 +-
 .../translators/io/UnboundedSourceWrapper.java  |    4 +-
 .../gearpump/translators/io/ValuesSource.java   |   12 +-
 .../translators/utils/GearpumpDoFnRunner.java   |   28 +-
 .../translators/utils/NoOpSideInputReader.java  |    8 +-
 .../translators/utils/NoOpStepContext.java      |    6 +-
 runners/google-cloud-dataflow-java/pom.xml      |    8 +-
 .../dataflow/BlockingDataflowRunner.java        |    7 +-
 .../runners/dataflow/DataflowJobException.java  |    1 -
 .../runners/dataflow/DataflowPipelineJob.java   |  112 +-
 .../dataflow/DataflowPipelineRegistrar.java     |    5 +-
 .../dataflow/DataflowPipelineTranslator.java    |   59 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  192 +--
 .../runners/dataflow/DataflowRunnerHooks.java   |    3 +-
 .../dataflow/internal/CustomSources.java        |   18 +-
 .../internal/DataflowAggregatorTransforms.java  |   10 +-
 .../internal/DataflowMetricUpdateExtractor.java |   10 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   43 +-
 .../runners/dataflow/internal/IsmFormat.java    |   35 +-
 .../dataflow/internal/ReadTranslator.java       |    8 +-
 .../dataflow/options/CloudDebuggerOptions.java  |    6 +-
 .../options/DataflowPipelineDebugOptions.java   |   11 +-
 .../options/DataflowPipelineOptions.java        |   53 +-
 .../DataflowPipelineWorkerPoolOptions.java      |   19 +-
 .../options/DataflowProfilingOptions.java       |    3 +-
 .../options/DataflowWorkerLoggingOptions.java   |    8 +-
 .../dataflow/testing/TestDataflowRunner.java    |   27 +-
 .../dataflow/util/DataflowTransport.java        |    6 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    |   10 +-
 .../beam/runners/dataflow/util/GcsStager.java   |    6 +-
 .../runners/dataflow/util/MonitoringUtil.java   |   14 +-
 .../beam/runners/dataflow/util/PackageUtil.java |   47 +-
 .../runners/dataflow/util/RandomAccessData.java |   16 +-
 .../beam/runners/dataflow/util/Stager.java      |    1 -
 .../beam/runners/dataflow/util/TimeUtil.java    |    8 +-
 .../dataflow/BlockingDataflowRunnerTest.java    |    1 -
 .../dataflow/DataflowPipelineJobTest.java       |   65 +-
 .../dataflow/DataflowPipelineRegistrarTest.java |    9 +-
 .../DataflowPipelineTranslatorTest.java         |   38 +-
 .../runners/dataflow/DataflowRunnerTest.java    |  113 +-
 .../dataflow/RecordingPipelineVisitor.java      |    5 +-
 .../DataflowPipelineDebugOptionsTest.java       |    1 -
 .../options/DataflowPipelineOptionsTest.java    |   43 +-
 .../options/DataflowProfilingOptionsTest.java   |    4 +-
 .../DataflowWorkerLoggingOptionsTest.java       |    8 +-
 .../testing/TestDataflowRunnerTest.java         |   35 +-
 .../transforms/DataflowGroupByKeyTest.java      |    6 +-
 .../dataflow/transforms/DataflowViewTest.java   |    1 -
 .../dataflow/util/MonitoringUtilTest.java       |   17 +-
 .../runners/dataflow/util/PackageUtilTest.java  |   41 +-
 .../dataflow/util/RandomAccessDataTest.java     |   13 +-
 .../runners/dataflow/util/TimeUtilTest.java     |    1 -
 runners/spark/pom.xml                           |   12 +-
 .../runners/spark/SparkPipelineOptions.java     |   20 +-
 .../apache/beam/runners/spark/SparkRunner.java  |   21 +-
 .../runners/spark/SparkRunnerRegistrar.java     |    5 +-
 .../beam/runners/spark/TestSparkRunner.java     |    2 +-
 .../spark/aggregators/NamedAggregators.java     |   48 +-
 .../aggregators/metrics/AggregatorMetric.java   |   44 +
 .../metrics/AggregatorMetricSource.java         |   49 +
 .../metrics/WithNamedAggregatorsSupport.java    |  169 ++
 .../spark/aggregators/metrics/package-info.java |   22 +
 .../spark/aggregators/metrics/sink/CsvSink.java |   39 +
 .../aggregators/metrics/sink/GraphiteSink.java  |   39 +
 .../aggregators/metrics/sink/package-info.java  |   23 +
 .../beam/runners/spark/coders/CoderHelpers.java |   12 +-
 .../runners/spark/coders/NullWritableCoder.java |    6 +-
 .../runners/spark/coders/WritableCoder.java     |   14 +-
 .../beam/runners/spark/examples/WordCount.java  |    2 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |    2 +-
 .../beam/runners/spark/io/CreateStream.java     |    7 +-
 .../apache/beam/runners/spark/io/KafkaIO.java   |   14 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   16 +-
 .../spark/io/hadoop/ShardNameBuilder.java       |    3 +-
 .../io/hadoop/ShardNameTemplateHelper.java      |    9 +-
 .../io/hadoop/TemplatedAvroKeyOutputFormat.java |    5 +-
 .../TemplatedSequenceFileOutputFormat.java      |    3 +-
 .../io/hadoop/TemplatedTextOutputFormat.java    |    3 +-
 .../runners/spark/translation/DoFnFunction.java |   35 +-
 .../spark/translation/EvaluationContext.java    |   19 +-
 .../spark/translation/MultiDoFnFunction.java    |   17 +-
 .../translation/SparkPipelineTranslator.java    |    2 +-
 .../spark/translation/SparkProcessContext.java  |   32 +-
 .../spark/translation/SparkRuntimeContext.java  |   50 +-
 .../spark/translation/TransformEvaluator.java   |    3 +-
 .../spark/translation/TransformTranslator.java  |   44 +-
 .../streaming/StreamingEvaluationContext.java   |   14 +-
 .../streaming/StreamingTransformTranslator.java |   36 +-
 .../runners/spark/util/BroadcastHelper.java     |   16 +-
 .../beam/runners/spark/util/ByteArray.java      |    1 -
 .../apache/beam/runners/spark/DeDupTest.java    |   11 +-
 .../beam/runners/spark/EmptyInputTest.java      |    9 +-
 .../runners/spark/InMemoryMetricsSinkRule.java  |   32 +
 .../runners/spark/ProvidedSparkContextTest.java |  138 ++
 .../beam/runners/spark/SimpleWordCountTest.java |   27 +-
 .../runners/spark/SparkRunnerRegistrarTest.java |    9 +-
 .../apache/beam/runners/spark/TfIdfTest.java    |    6 +-
 .../metrics/sink/InMemoryMetrics.java           |   79 +
 .../runners/spark/coders/WritableCoderTest.java |    1 -
 .../beam/runners/spark/io/AvroPipelineTest.java |   25 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   21 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |    6 +-
 .../spark/io/hadoop/ShardNameBuilderTest.java   |    1 -
 .../spark/translation/CombineGloballyTest.java  |    9 +-
 .../spark/translation/CombinePerKeyTest.java    |   11 +-
 .../spark/translation/DoFnOutputTest.java       |    4 +-
 .../translation/MultiOutputWordCountTest.java   |   11 +-
 .../spark/translation/SerializationTest.java    |   27 +-
 .../spark/translation/SideEffectsTest.java      |    6 +-
 .../translation/SparkPipelineOptionsTest.java   |    1 -
 .../translation/TransformTranslatorTest.java    |   23 +-
 .../translation/WindowedWordCountTest.java      |    9 +-
 .../streaming/FlattenStreamingTest.java         |    8 +-
 .../streaming/KafkaStreamingTest.java           |   18 +-
 .../streaming/SimpleStreamingWordCountTest.java |   10 +-
 .../streaming/utils/EmbeddedKafkaCluster.java   |   12 +-
 .../streaming/utils/PAssertStreaming.java       |    4 +-
 .../spark/src/test/resources/metrics.properties |   29 +
 .../src/main/resources/beam/checkstyle.xml      |   13 +-
 .../src/main/resources/beam/findbugs-filter.xml |   14 +-
 sdks/java/core/pom.xml                          |    6 +
 .../beam/sdk/AggregatorPipelineExtractor.java   |   12 +-
 .../org/apache/beam/sdk/AggregatorValues.java   |    9 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |   23 +-
 .../org/apache/beam/sdk/PipelineResult.java     |    4 +-
 .../org/apache/beam/sdk/coders/AvroCoder.java   |   35 +-
 .../apache/beam/sdk/coders/BigDecimalCoder.java |    1 -
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |    1 -
 .../beam/sdk/coders/BigEndianLongCoder.java     |    1 -
 .../apache/beam/sdk/coders/BigIntegerCoder.java |    1 -
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |   11 +-
 .../org/apache/beam/sdk/coders/ByteCoder.java   |    1 -
 .../apache/beam/sdk/coders/ByteStringCoder.java |    7 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |   11 +-
 .../apache/beam/sdk/coders/CoderFactories.java  |    3 +-
 .../apache/beam/sdk/coders/CoderProviders.java  |    6 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |   24 +-
 .../apache/beam/sdk/coders/CollectionCoder.java |    4 +-
 .../org/apache/beam/sdk/coders/CustomCoder.java |   16 +-
 .../apache/beam/sdk/coders/DefaultCoder.java    |    3 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |    1 -
 .../org/apache/beam/sdk/coders/DoubleCoder.java |    1 -
 .../apache/beam/sdk/coders/DurationCoder.java   |    8 +-
 .../apache/beam/sdk/coders/InstantCoder.java    |   12 +-
 .../apache/beam/sdk/coders/IterableCoder.java   |    9 +-
 .../beam/sdk/coders/IterableLikeCoder.java      |    9 +-
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |   14 +-
 .../org/apache/beam/sdk/coders/KvCoder.java     |   13 +-
 .../org/apache/beam/sdk/coders/ListCoder.java   |    4 +-
 .../org/apache/beam/sdk/coders/MapCoder.java    |    9 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |   19 +-
 .../beam/sdk/coders/SerializableCoder.java      |    6 +-
 .../org/apache/beam/sdk/coders/SetCoder.java    |    4 +-
 .../apache/beam/sdk/coders/StandardCoder.java   |   11 +-
 .../beam/sdk/coders/StringDelegateCoder.java    |    5 +-
 .../apache/beam/sdk/coders/StringUtf8Coder.java |   11 +-
 .../beam/sdk/coders/TableRowJsonCoder.java      |    4 +-
 .../beam/sdk/coders/TextualIntegerCoder.java    |    1 -
 .../org/apache/beam/sdk/coders/VarIntCoder.java |    4 +-
 .../apache/beam/sdk/coders/VarLongCoder.java    |    4 +-
 .../org/apache/beam/sdk/coders/VoidCoder.java   |    1 -
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |   28 +-
 .../beam/sdk/coders/protobuf/ProtobufUtil.java  |    4 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   30 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |   45 +-
 .../apache/beam/sdk/io/BlockBasedSource.java    |    6 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |   31 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |   11 +-
 .../apache/beam/sdk/io/CompressedSource.java    |   96 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |    4 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |   11 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |   29 +-
 .../org/apache/beam/sdk/io/FileBasedSource.java |   17 +-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   10 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |   26 +-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   23 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |   47 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   14 +-
 .../main/java/org/apache/beam/sdk/io/Sink.java  |    3 +-
 .../java/org/apache/beam/sdk/io/Source.java     |    8 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  183 ++-
 .../org/apache/beam/sdk/io/UnboundedSource.java |   11 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   16 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   14 +-
 .../java/org/apache/beam/sdk/io/XmlSource.java  |   15 +-
 .../org/apache/beam/sdk/io/range/ByteKey.java   |    1 -
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |    6 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |    6 +-
 .../beam/sdk/io/range/OffsetRangeTracker.java   |    4 +-
 .../org/apache/beam/sdk/options/GcpOptions.java |   18 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |   15 +-
 .../beam/sdk/options/GoogleApiDebugOptions.java |    1 -
 .../beam/sdk/options/PipelineOptionSpec.java    |    4 +-
 .../beam/sdk/options/PipelineOptions.java       |   61 +-
 .../sdk/options/PipelineOptionsFactory.java     |   26 +-
 .../sdk/options/PipelineOptionsReflector.java   |    4 +-
 .../sdk/options/PipelineOptionsRegistrar.java   |    1 -
 .../sdk/options/PipelineOptionsValidator.java   |    6 +-
 .../sdk/options/ProxyInvocationHandler.java     |   38 +-
 .../apache/beam/sdk/runners/PipelineRunner.java |    1 +
 .../sdk/runners/PipelineRunnerRegistrar.java    |    1 -
 .../beam/sdk/runners/TransformHierarchy.java    |    9 +-
 .../beam/sdk/runners/TransformTreeNode.java     |   14 +-
 .../beam/sdk/testing/CoderProperties.java       |   23 +-
 .../beam/sdk/testing/FileChecksumMatcher.java   |  133 ++
 .../beam/sdk/testing/MatcherDeserializer.java   |    7 +-
 .../beam/sdk/testing/MatcherSerializer.java     |    7 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |   64 +-
 .../apache/beam/sdk/testing/PaneExtractors.java |   23 +-
 .../beam/sdk/testing/SerializableMatcher.java   |    3 +-
 .../beam/sdk/testing/SerializableMatchers.java  |   18 +-
 .../beam/sdk/testing/SourceTestUtils.java       |  147 +-
 .../apache/beam/sdk/testing/StaticWindows.java  |   10 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |   32 +-
 .../beam/sdk/testing/TestPipelineOptions.java   |    1 -
 .../org/apache/beam/sdk/testing/TestStream.java |  368 +++++
 .../beam/sdk/testing/WindowFnTestUtils.java     |   19 +-
 .../apache/beam/sdk/testing/WindowSupplier.java |   10 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |   20 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |    5 +-
 .../sdk/transforms/ApproximateQuantiles.java    |   29 +-
 .../beam/sdk/transforms/ApproximateUnique.java  |   23 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  108 +-
 .../beam/sdk/transforms/CombineFnBase.java      |   10 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |   57 +-
 .../org/apache/beam/sdk/transforms/Count.java   |   21 +-
 .../org/apache/beam/sdk/transforms/Create.java  |   44 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |   89 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |  284 ++++
 .../beam/sdk/transforms/DoFnReflector.java      | 1102 -------------
 .../apache/beam/sdk/transforms/DoFnTester.java  |  121 +-
 .../beam/sdk/transforms/FlatMapElements.java    |    7 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   12 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |    7 +-
 .../transforms/IntraBundleParallelization.java  |   11 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |    8 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |    9 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |  203 +++
 .../apache/beam/sdk/transforms/MapElements.java |   16 +-
 .../org/apache/beam/sdk/transforms/Max.java     |   32 +-
 .../org/apache/beam/sdk/transforms/Mean.java    |   12 +-
 .../org/apache/beam/sdk/transforms/Min.java     |   33 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   44 +-
 .../apache/beam/sdk/transforms/PTransform.java  |    9 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |  321 ++--
 .../apache/beam/sdk/transforms/Partition.java   |    3 +-
 .../beam/sdk/transforms/RemoveDuplicates.java   |    8 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |    7 +-
 .../beam/sdk/transforms/SimpleFunction.java     |    6 +-
 .../org/apache/beam/sdk/transforms/Sum.java     |   27 +-
 .../org/apache/beam/sdk/transforms/Top.java     |   20 +-
 .../org/apache/beam/sdk/transforms/Values.java  |    8 +-
 .../org/apache/beam/sdk/transforms/View.java    |    5 +-
 .../org/apache/beam/sdk/transforms/ViewFn.java  |    3 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |    9 +-
 .../beam/sdk/transforms/WithTimestamps.java     |    1 -
 .../sdk/transforms/display/DisplayData.java     |   21 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |   30 +-
 .../sdk/transforms/join/CoGbkResultSchema.java  |   10 +-
 .../beam/sdk/transforms/join/CoGroupByKey.java  |    5 +-
 .../transforms/join/KeyedPCollectionTuple.java  |    7 +-
 .../beam/sdk/transforms/join/RawUnionValue.java |   25 +
 .../beam/sdk/transforms/join/UnionCoder.java    |   14 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   61 +
 .../sdk/transforms/reflect/DoFnInvokers.java    |  503 ++++++
 .../sdk/transforms/reflect/DoFnSignature.java   |  110 ++
 .../sdk/transforms/reflect/DoFnSignatures.java  |  319 ++++
 .../sdk/transforms/reflect/package-info.java    |   23 +
 .../beam/sdk/transforms/windowing/AfterAll.java |    9 +-
 .../windowing/AfterDelayFromFirstElement.java   |   16 +-
 .../sdk/transforms/windowing/AfterEach.java     |    9 +-
 .../sdk/transforms/windowing/AfterFirst.java    |    9 +-
 .../sdk/transforms/windowing/AfterPane.java     |    9 +-
 .../windowing/AfterProcessingTime.java          |    9 +-
 .../AfterSynchronizedProcessingTime.java        |   10 +-
 .../transforms/windowing/AfterWatermark.java    |   12 +-
 .../sdk/transforms/windowing/BoundedWindow.java |    3 +-
 .../transforms/windowing/CalendarWindows.java   |    1 -
 .../transforms/windowing/DefaultTrigger.java    |    4 +-
 .../sdk/transforms/windowing/FixedWindows.java  |    4 +-
 .../sdk/transforms/windowing/GlobalWindow.java  |    6 +-
 .../sdk/transforms/windowing/GlobalWindows.java |    6 +-
 .../transforms/windowing/IntervalWindow.java    |   10 +-
 .../transforms/windowing/InvalidWindows.java    |    4 +-
 .../beam/sdk/transforms/windowing/Never.java    |    4 +-
 .../transforms/windowing/OrFinallyTrigger.java  |    7 +-
 .../sdk/transforms/windowing/OutputTimeFn.java  |    7 +-
 .../sdk/transforms/windowing/OutputTimeFns.java |    7 +-
 .../beam/sdk/transforms/windowing/PaneInfo.java |   22 +-
 .../windowing/PartitioningWindowFn.java         |    3 +-
 .../sdk/transforms/windowing/Repeatedly.java    |    6 +-
 .../beam/sdk/transforms/windowing/Sessions.java |    8 +-
 .../transforms/windowing/SlidingWindows.java    |   10 +-
 .../beam/sdk/transforms/windowing/Trigger.java  |   16 +-
 .../beam/sdk/transforms/windowing/Window.java   |   15 +-
 .../beam/sdk/transforms/windowing/WindowFn.java |    6 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   |    6 +-
 .../org/apache/beam/sdk/util/ApiSurface.java    |    6 +-
 .../apache/beam/sdk/util/AppliedCombineFn.java  |    6 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java |  174 ---
 .../util/AttemptBoundedExponentialBackOff.java  |   85 -
 .../org/apache/beam/sdk/util/AvroUtils.java     |    7 +-
 .../beam/sdk/util/BaseExecutionContext.java     |   13 +-
 .../org/apache/beam/sdk/util/BitSetCoder.java   |   11 +-
 .../apache/beam/sdk/util/BucketingFunction.java |    3 +-
 .../BufferedElementCountingOutputStream.java    |    4 +-
 .../apache/beam/sdk/util/CloudKnownType.java    |    1 -
 .../org/apache/beam/sdk/util/CloudObject.java   |    2 -
 .../org/apache/beam/sdk/util/CoderUtils.java    |   17 +-
 .../org/apache/beam/sdk/util/CombineFnUtil.java |    7 +-
 .../apache/beam/sdk/util/CounterAggregator.java |  128 --
 .../apache/beam/sdk/util/CredentialFactory.java |    1 -
 .../org/apache/beam/sdk/util/Credentials.java   |    9 +-
 .../beam/sdk/util/DirectSideInputReader.java    |    5 +-
 .../apache/beam/sdk/util/ExecutableTrigger.java |    7 +-
 .../apache/beam/sdk/util/ExecutionContext.java  |    5 +-
 .../sdk/util/ExposedByteArrayInputStream.java   |    3 +-
 .../sdk/util/ExposedByteArrayOutputStream.java  |    3 +-
 .../beam/sdk/util/FileIOChannelFactory.java     |    6 +-
 .../beam/sdk/util/FinishedTriggersSet.java      |    1 -
 .../org/apache/beam/sdk/util/FluentBackoff.java |  229 +++
 .../beam/sdk/util/GcpCredentialFactory.java     |    6 +-
 .../beam/sdk/util/GcsIOChannelFactory.java      |    5 +-
 .../apache/beam/sdk/util/GcsPathValidator.java  |    3 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  275 ++--
 .../apache/beam/sdk/util/IOChannelUtils.java    |    5 +-
 .../apache/beam/sdk/util/IdentityWindowFn.java  |    6 +-
 .../apache/beam/sdk/util/InstanceBuilder.java   |    5 +-
 .../util/IntervalBoundedExponentialBackOff.java |    1 +
 .../beam/sdk/util/KeyedWorkItemCoder.java       |   17 +-
 .../apache/beam/sdk/util/KeyedWorkItems.java    |    4 +-
 .../beam/sdk/util/MergingActiveWindowSet.java   |   21 +-
 .../apache/beam/sdk/util/MovingFunction.java    |    3 +-
 .../apache/beam/sdk/util/MutationDetectors.java |    5 +-
 .../sdk/util/NonMergingActiveWindowSet.java     |    6 +-
 .../beam/sdk/util/NoopCredentialFactory.java    |    4 +-
 .../beam/sdk/util/NullSideInputReader.java      |    6 +-
 .../beam/sdk/util/PCollectionViewWindow.java    |    3 +-
 .../apache/beam/sdk/util/PCollectionViews.java  |   25 +-
 .../java/org/apache/beam/sdk/util/PTuple.java   |    3 +-
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |    5 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   |    6 +-
 .../org/apache/beam/sdk/util/PubsubClient.java  |    5 +-
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |    8 +-
 .../apache/beam/sdk/util/PubsubJsonClient.java  |    5 +-
 .../apache/beam/sdk/util/PubsubTestClient.java  |    5 +-
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |    4 +-
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |    6 +-
 .../org/apache/beam/sdk/util/Reshuffle.java     |    1 -
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |    4 +-
 .../sdk/util/RetryHttpRequestInitializer.java   |    7 +-
 .../apache/beam/sdk/util/SerializableUtils.java |   15 +-
 .../org/apache/beam/sdk/util/Serializer.java    |    2 -
 .../apache/beam/sdk/util/SideInputReader.java   |    3 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |    4 +-
 .../java/org/apache/beam/sdk/util/Structs.java  |    2 -
 .../beam/sdk/util/SystemDoFnInternal.java       |    8 +-
 .../apache/beam/sdk/util/TestCredential.java    |    1 -
 .../apache/beam/sdk/util/TimerInternals.java    |   26 +-
 .../java/org/apache/beam/sdk/util/Timers.java   |    4 +-
 .../org/apache/beam/sdk/util/Transport.java     |    8 +-
 .../beam/sdk/util/TriggerContextFactory.java    |   18 +-
 .../beam/sdk/util/UnownedInputStream.java       |    1 -
 .../beam/sdk/util/UnownedOutputStream.java      |    1 -
 .../sdk/util/UploadIdResponseInterceptor.java   |    4 +-
 .../apache/beam/sdk/util/ValueWithRecordId.java |   13 +-
 .../java/org/apache/beam/sdk/util/Values.java   |    1 -
 .../org/apache/beam/sdk/util/WindowedValue.java |   32 +-
 .../beam/sdk/util/WindowingInternals.java       |    9 +-
 .../apache/beam/sdk/util/WindowingStrategy.java |   11 +-
 .../java/org/apache/beam/sdk/util/ZipFiles.java |    4 +-
 .../apache/beam/sdk/util/common/Counter.java    | 1287 ----------------
 .../beam/sdk/util/common/CounterName.java       |  153 --
 .../beam/sdk/util/common/CounterProvider.java   |   27 -
 .../apache/beam/sdk/util/common/CounterSet.java |  179 ---
 .../util/common/ElementByteSizeObserver.java    |   24 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |    3 -
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java |    2 -
 .../CopyOnAccessInMemoryStateInternals.java     |   16 +-
 .../sdk/util/state/InMemoryStateInternals.java  |   13 +-
 .../sdk/util/state/MergingStateAccessor.java    |    3 +-
 .../beam/sdk/util/state/StateContexts.java      |    3 +-
 .../sdk/util/state/StateInternalsFactory.java   |    3 +-
 .../beam/sdk/util/state/StateMerging.java       |    9 +-
 .../beam/sdk/util/state/StateNamespaces.java    |   10 +-
 .../apache/beam/sdk/util/state/StateTable.java  |    4 +-
 .../apache/beam/sdk/util/state/StateTag.java    |    5 +-
 .../apache/beam/sdk/util/state/StateTags.java   |   10 +-
 .../beam/sdk/util/state/WatermarkHoldState.java |    1 -
 .../java/org/apache/beam/sdk/values/KV.java     |   10 +-
 .../java/org/apache/beam/sdk/values/PBegin.java |    5 +-
 .../apache/beam/sdk/values/PCollectionList.java |   12 +-
 .../beam/sdk/values/PCollectionTuple.java       |   12 +-
 .../apache/beam/sdk/values/PCollectionView.java |    3 +-
 .../java/org/apache/beam/sdk/values/PDone.java  |    5 +-
 .../java/org/apache/beam/sdk/values/PInput.java |    3 +-
 .../org/apache/beam/sdk/values/POutput.java     |    3 +-
 .../org/apache/beam/sdk/values/PValueBase.java  |    5 +-
 .../beam/sdk/values/TimestampedValue.java       |   26 +-
 .../org/apache/beam/sdk/values/TupleTag.java    |   13 +-
 .../apache/beam/sdk/values/TupleTagList.java    |    4 +-
 .../apache/beam/sdk/values/TypeDescriptor.java  |    2 -
 .../apache/beam/sdk/values/TypeDescriptors.java |    1 -
 .../dataflow/util/GcsPathValidatorTest.java     |    1 -
 .../sdk/AggregatorPipelineExtractorTest.java    |   15 +-
 .../org/apache/beam/sdk/DataflowMatchers.java   |    4 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |   16 +-
 .../java/org/apache/beam/sdk/TestUtils.java     |   10 +-
 .../org/apache/beam/sdk/WindowMatchers.java     |    9 +-
 .../org/apache/beam/sdk/WindowMatchersTest.java |    4 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   60 +-
 .../beam/sdk/coders/BigDecimalCoderTest.java    |    6 +-
 .../sdk/coders/BigEndianIntegerCoderTest.java   |    6 +-
 .../beam/sdk/coders/BigEndianLongCoderTest.java |    6 +-
 .../beam/sdk/coders/BigIntegerCoderTest.java    |    6 +-
 .../beam/sdk/coders/ByteArrayCoderTest.java     |   10 +-
 .../apache/beam/sdk/coders/ByteCoderTest.java   |    6 +-
 .../beam/sdk/coders/ByteStringCoderTest.java    |   11 +-
 .../beam/sdk/coders/CoderFactoriesTest.java     |    5 +-
 .../beam/sdk/coders/CoderProvidersTest.java     |    4 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   31 +-
 .../org/apache/beam/sdk/coders/CoderTest.java   |    4 +-
 .../beam/sdk/coders/CollectionCoderTest.java    |   14 +-
 .../apache/beam/sdk/coders/CustomCoderTest.java |   12 +-
 .../beam/sdk/coders/DefaultCoderTest.java       |    7 +-
 .../beam/sdk/coders/DelegateCoderTest.java      |   11 +-
 .../apache/beam/sdk/coders/DoubleCoderTest.java |    6 +-
 .../beam/sdk/coders/DurationCoderTest.java      |    9 +-
 .../beam/sdk/coders/InstantCoderTest.java       |   15 +-
 .../beam/sdk/coders/IterableCoderTest.java      |   10 +-
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |   29 +-
 .../org/apache/beam/sdk/coders/KvCoderTest.java |   13 +-
 .../apache/beam/sdk/coders/ListCoderTest.java   |   10 +-
 .../apache/beam/sdk/coders/MapCoderTest.java    |   17 +-
 .../beam/sdk/coders/NullableCoderTest.java      |   13 +-
 .../beam/sdk/coders/PrintBase64Encodings.java   |    4 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   14 +-
 .../apache/beam/sdk/coders/SetCoderTest.java    |   12 +-
 .../beam/sdk/coders/StandardCoderTest.java      |   16 +-
 .../sdk/coders/StringDelegateCoderTest.java     |    8 +-
 .../beam/sdk/coders/StringUtf8CoderTest.java    |    6 +-
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |    9 +-
 .../sdk/coders/TextualIntegerCoderTest.java     |    6 +-
 .../apache/beam/sdk/coders/VarIntCoderTest.java |    6 +-
 .../beam/sdk/coders/VarLongCoderTest.java       |    6 +-
 .../sdk/coders/protobuf/ProtoCoderTest.java     |   14 +-
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |   10 +-
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |   26 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |   32 +-
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |   46 +-
 .../io/BoundedReadFromUnboundedSourceTest.java  |   11 +-
 .../beam/sdk/io/CompressedSourceTest.java       |   43 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |    2 -
 .../apache/beam/sdk/io/CountingSourceTest.java  |    6 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |   22 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |   32 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |   11 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |    7 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |   13 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |   19 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |   12 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  403 ++++-
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   68 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |   27 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |   29 +-
 .../range/ByteKeyRangeEstimateFractionTest.java |    1 -
 .../range/ByteKeyRangeInterpolateKeyTest.java   |    1 -
 .../beam/sdk/io/range/ByteKeyRangeTest.java     |    6 +-
 .../apache/beam/sdk/io/range/ByteKeyTest.java   |    3 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |   16 +-
 .../sdk/options/GoogleApiDebugOptionsTest.java  |    8 +-
 .../sdk/options/PipelineOptionsFactoryTest.java |   27 +-
 .../options/PipelineOptionsReflectorTest.java   |    6 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |   12 +-
 .../options/PipelineOptionsValidatorTest.java   |    1 -
 .../sdk/options/ProxyInvocationHandlerTest.java |   28 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |    1 -
 .../beam/sdk/runners/TransformTreeTest.java     |    8 +-
 .../runners/dataflow/TestCountingSource.java    |   13 +-
 .../dataflow/TestCountingSourceTest.java        |    4 +-
 .../beam/sdk/testing/CoderPropertiesTest.java   |   11 +-
 .../beam/sdk/testing/CrashingRunnerTest.java    |    1 -
 .../apache/beam/sdk/testing/ExpectedLogs.java   |   12 +-
 .../beam/sdk/testing/ExpectedLogsTest.java      |   13 +-
 .../sdk/testing/FastNanoClockAndSleeper.java    |    1 -
 .../testing/FastNanoClockAndSleeperTest.java    |    3 +-
 .../sdk/testing/FileChecksumMatcherTest.java    |  104 ++
 .../apache/beam/sdk/testing/PAssertTest.java    |   17 +-
 .../sdk/testing/PCollectionViewTesting.java     |   15 +-
 .../beam/sdk/testing/PaneExtractorsTest.java    |    4 +-
 .../sdk/testing/RestoreSystemProperties.java    |    5 +-
 .../sdk/testing/SerializableMatchersTest.java   |   12 +-
 .../beam/sdk/testing/SourceTestUtilsTest.java   |   63 +
 .../beam/sdk/testing/StaticWindowsTest.java     |    4 +-
 .../beam/sdk/testing/SystemNanoTimeSleeper.java |    1 -
 .../sdk/testing/SystemNanoTimeSleeperTest.java  |    1 -
 .../beam/sdk/testing/TestPipelineTest.java      |   12 +-
 .../apache/beam/sdk/testing/TestStreamTest.java |  355 +++++
 .../beam/sdk/testing/WindowSupplierTest.java    |   16 +-
 .../transforms/ApproximateQuantilesTest.java    |   13 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |   18 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   18 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   68 +-
 .../apache/beam/sdk/transforms/CountTest.java   |    7 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |   26 +-
 .../DoFnDelegatingAggregatorTest.java           |    3 +-
 .../beam/sdk/transforms/DoFnReflectorTest.java  |  757 ---------
 .../apache/beam/sdk/transforms/DoFnTest.java    |    4 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |   41 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |    5 +-
 .../sdk/transforms/FlatMapElementsTest.java     |   16 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |   21 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |   30 +-
 .../IntraBundleParallelizationTest.java         |    7 +-
 .../apache/beam/sdk/transforms/KeysTest.java    |    4 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |    4 +-
 .../beam/sdk/transforms/LatestFnTests.java      |  233 +++
 .../apache/beam/sdk/transforms/LatestTest.java  |  146 ++
 .../beam/sdk/transforms/MapElementsTest.java    |   15 +-
 .../org/apache/beam/sdk/transforms/MaxTest.java |    5 +-
 .../apache/beam/sdk/transforms/MeanTest.java    |   10 +-
 .../org/apache/beam/sdk/transforms/MinTest.java |    5 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |    3 +-
 .../apache/beam/sdk/transforms/OldDoFnTest.java |    9 +-
 .../beam/sdk/transforms/PTransformTest.java     |    1 -
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  447 ++++++
 .../apache/beam/sdk/transforms/ParDoTest.java   |   40 +-
 .../beam/sdk/transforms/PartitionTest.java      |    9 +-
 .../sdk/transforms/RemoveDuplicatesTest.java    |   10 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |   21 +-
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |    7 +-
 .../org/apache/beam/sdk/transforms/SumTest.java |    5 +-
 .../org/apache/beam/sdk/transforms/TopTest.java |   13 +-
 .../apache/beam/sdk/transforms/ValuesTest.java  |    4 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   31 +-
 .../beam/sdk/transforms/WithKeysTest.java       |    6 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |    4 +-
 .../display/DisplayDataEvaluator.java           |    8 +-
 .../display/DisplayDataEvaluatorTest.java       |    7 +-
 .../transforms/display/DisplayDataMatchers.java |    7 +-
 .../display/DisplayDataMatchersTest.java        |    2 -
 .../sdk/transforms/display/DisplayDataTest.java |   26 +-
 .../dofnreflector/DoFnReflectorTestHelper.java  |  116 --
 .../transforms/join/CoGbkResultCoderTest.java   |    4 +-
 .../sdk/transforms/join/CoGbkResultTest.java    |    6 +-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |   15 +-
 .../sdk/transforms/join/UnionCoderTest.java     |    4 +-
 .../transforms/reflect/DoFnInvokersTest.java    |  500 ++++++
 .../reflect/DoFnInvokersTestHelper.java         |  116 ++
 .../transforms/reflect/DoFnSignaturesTest.java  |  368 +++++
 .../sdk/transforms/windowing/AfterAllTest.java  |    1 -
 .../sdk/transforms/windowing/AfterEachTest.java |    1 -
 .../transforms/windowing/AfterFirstTest.java    |    1 -
 .../sdk/transforms/windowing/AfterPaneTest.java |    1 -
 .../windowing/AfterProcessingTimeTest.java      |    1 -
 .../AfterSynchronizedProcessingTimeTest.java    |    1 -
 .../windowing/AfterWatermarkTest.java           |    1 -
 .../windowing/CalendarWindowsTest.java          |   13 +-
 .../windowing/DefaultTriggerTest.java           |    1 -
 .../transforms/windowing/FixedWindowsTest.java  |   11 +-
 .../windowing/IntervalWindowTest.java           |    7 +-
 .../sdk/transforms/windowing/NeverTest.java     |    1 -
 .../windowing/OrFinallyTriggerTest.java         |    1 -
 .../sdk/transforms/windowing/PaneInfoTest.java  |    1 -
 .../transforms/windowing/RepeatedlyTest.java    |    1 -
 .../sdk/transforms/windowing/SessionsTest.java  |   16 +-
 .../windowing/SlidingWindowsTest.java           |   11 +-
 .../sdk/transforms/windowing/StubTrigger.java   |    4 +-
 .../sdk/transforms/windowing/TriggerTest.java   |    5 +-
 .../sdk/transforms/windowing/WindowTest.java    |    5 +-
 .../sdk/transforms/windowing/WindowingTest.java |   10 +-
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |   10 +-
 ...mptAndTimeBoundedExponentialBackOffTest.java |  214 ---
 .../AttemptBoundedExponentialBackOffTest.java   |   85 -
 .../org/apache/beam/sdk/util/AvroUtilsTest.java |   20 +-
 .../beam/sdk/util/BucketingFunctionTest.java    |    1 -
 ...BufferedElementCountingOutputStreamTest.java |   19 +-
 .../apache/beam/sdk/util/CoderUtilsTest.java    |    7 +-
 .../apache/beam/sdk/util/CombineFnUtilTest.java |   13 +-
 .../beam/sdk/util/CounterAggregatorTest.java    |  256 ----
 .../beam/sdk/util/ExecutableTriggerTest.java    |    6 +-
 .../util/ExposedByteArrayInputStreamTest.java   |    5 +-
 .../util/ExposedByteArrayOutputStreamTest.java  |    5 +-
 .../beam/sdk/util/FileIOChannelFactoryTest.java |   16 +-
 .../beam/sdk/util/FinishedTriggersSetTest.java  |    3 +-
 .../apache/beam/sdk/util/FluentBackoffTest.java |  226 +++
 .../beam/sdk/util/GatherAllPanesTest.java       |    7 +-
 .../beam/sdk/util/GcsIOChannelFactoryTest.java  |    1 -
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  102 +-
 .../beam/sdk/util/IOChannelUtilsTest.java       |    6 +-
 .../sdk/util/IdentitySideInputWindowFn.java     |    5 +-
 .../beam/sdk/util/InstanceBuilderTest.java      |    1 -
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |    4 +-
 .../sdk/util/MergingActiveWindowSetTest.java    |   13 +-
 .../beam/sdk/util/MovingFunctionTest.java       |    1 -
 .../beam/sdk/util/MutationDetectorsTest.java    |   17 +-
 .../org/apache/beam/sdk/util/PTupleTest.java    |    1 -
 .../apache/beam/sdk/util/PubsubClientTest.java  |   10 +-
 .../beam/sdk/util/PubsubGrpcClientTest.java     |   18 +-
 .../beam/sdk/util/PubsubJsonClientTest.java     |   18 +-
 .../beam/sdk/util/PubsubTestClientTest.java     |   18 +-
 .../org/apache/beam/sdk/util/ReshuffleTest.java |    7 +-
 .../beam/sdk/util/ReshuffleTriggerTest.java     |    1 -
 .../util/RetryHttpRequestInitializerTest.java   |   12 +-
 .../beam/sdk/util/SerializableUtilsTest.java    |   19 +-
 .../apache/beam/sdk/util/StreamUtilsTest.java   |    9 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |    1 -
 .../org/apache/beam/sdk/util/StructsTest.java   |   11 +-
 .../beam/sdk/util/TimerInternalsTest.java       |    1 -
 .../org/apache/beam/sdk/util/TriggerTester.java |   31 +-
 .../beam/sdk/util/UnownedInputStreamTest.java   |    3 +-
 .../beam/sdk/util/UnownedOutputStreamTest.java  |    3 +-
 .../util/UploadIdResponseInterceptorTest.java   |    7 +-
 .../beam/sdk/util/UserCodeExceptionTest.java    |    3 +-
 .../org/apache/beam/sdk/util/VarIntTest.java    |    9 +-
 .../apache/beam/sdk/util/WindowedValueTest.java |    9 +-
 .../org/apache/beam/sdk/util/ZipFilesTest.java  |   14 +-
 .../beam/sdk/util/common/CounterSetTest.java    |  227 ---
 .../beam/sdk/util/common/CounterTest.java       |  736 ---------
 .../sdk/util/common/ReflectHelpersTest.java     |    6 +-
 .../apache/beam/sdk/util/gcsfs/GcsPathTest.java |   11 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |    1 -
 .../util/state/InMemoryStateInternalsTest.java  |    4 +-
 .../sdk/util/state/StateNamespacesTest.java     |    1 -
 .../beam/sdk/util/state/StateTagTest.java       |    1 -
 .../java/org/apache/beam/sdk/values/KVTest.java |    4 +-
 .../beam/sdk/values/PCollectionListTest.java    |    3 +-
 .../beam/sdk/values/PCollectionTupleTest.java   |    8 +-
 .../org/apache/beam/sdk/values/PDoneTest.java   |    4 +-
 .../beam/sdk/values/TimestampedValueTest.java   |   83 +
 .../beam/sdk/values/TypeDescriptorTest.java     |   10 +-
 .../beam/sdk/values/TypeDescriptorsTest.java    |    6 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |    1 -
 .../extensions/joinlibrary/InnerJoinTest.java   |    6 +-
 .../joinlibrary/OuterLeftJoinTest.java          |    6 +-
 .../joinlibrary/OuterRightJoinTest.java         |    6 +-
 sdks/java/io/google-cloud-platform/pom.xml      |    4 +-
 .../sdk/io/gcp/bigquery/BigQueryAvroUtils.java  |    7 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  188 ++-
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |    5 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   |  173 ++-
 .../io/gcp/bigquery/BigQueryTableInserter.java  |   14 +-
 .../gcp/bigquery/BigQueryTableRowIterator.java  |   15 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |   60 +-
 .../sdk/io/gcp/bigtable/BigtableService.java    |   15 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   20 +-
 .../beam/sdk/io/gcp/datastore/DatastoreIO.java  |   12 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  | 1059 +++++++++++++
 .../beam/sdk/io/gcp/datastore/V1Beta3.java      |  969 ------------
 .../beam/sdk/io/gcp/datastore/package-info.java |    2 +-
 .../io/gcp/bigquery/BigQueryAvroUtilsTest.java  |   11 +-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  301 ++--
 .../gcp/bigquery/BigQueryServicesImplTest.java  |   44 +-
 .../gcp/bigquery/BigQueryTableInserterTest.java |   20 +-
 .../bigquery/BigQueryTableRowIteratorTest.java  |   10 +-
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |   19 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |   56 +-
 .../sdk/io/gcp/bigtable/BigtableReadIT.java     |    6 +-
 .../sdk/io/gcp/bigtable/BigtableWriteIT.java    |   29 +-
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   |  837 ++++++++++
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    |   97 ++
 .../sdk/io/gcp/datastore/V1Beta3ReadIT.java     |  114 --
 .../beam/sdk/io/gcp/datastore/V1Beta3Test.java  |  587 -------
 .../io/gcp/datastore/V1Beta3TestOptions.java    |   44 -
 .../sdk/io/gcp/datastore/V1Beta3TestUtil.java   |  382 -----
 .../sdk/io/gcp/datastore/V1Beta3WriteIT.java    |   85 -
 .../beam/sdk/io/gcp/datastore/V1ReadIT.java     |  111 ++
 .../sdk/io/gcp/datastore/V1TestOptions.java     |   43 +
 .../beam/sdk/io/gcp/datastore/V1TestUtil.java   |  382 +++++
 .../beam/sdk/io/gcp/datastore/V1WriteIT.java    |   83 +
 sdks/java/io/hdfs/pom.xml                       |    5 -
 .../beam/sdk/io/hdfs/AvroHDFSFileSource.java    |   23 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |   16 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSink.java   |   17 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |   30 +-
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |   12 +-
 .../SimpleAuthAvroHDFSFileSource.java           |   12 +-
 .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java |    3 +-
 .../simpleauth/SimpleAuthHDFSFileSource.java    |   12 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java  |    5 +-
 .../beam/sdk/io/hdfs/HDFSFileSourceTest.java    |   12 +-
 .../beam/sdk/io/hdfs/WritableCoderTest.java     |    1 -
 .../beam/sdk/io/jms/JmsCheckpointMark.java      |    9 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  |   47 +-
 .../org/apache/beam/sdk/io/jms/JmsRecord.java   |    1 -
 .../org/apache/beam/sdk/io/jms/JmsIOTest.java   |   25 +-
 .../beam/sdk/io/kafka/KafkaCheckpointMark.java  |    8 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |   99 +-
 .../apache/beam/sdk/io/kafka/KafkaRecord.java   |    3 +-
 .../beam/sdk/io/kafka/KafkaRecordCoder.java     |   14 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |   33 +-
 sdks/java/io/kinesis/pom.xml                    |  179 +++
 .../sdk/io/kinesis/CheckpointGenerator.java     |   30 +
 .../beam/sdk/io/kinesis/CustomOptional.java     |   85 +
 .../io/kinesis/DynamicCheckpointGenerator.java  |   56 +
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |   54 +
 .../sdk/io/kinesis/KinesisClientProvider.java   |   31 +
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  190 +++
 .../beam/sdk/io/kinesis/KinesisReader.java      |  145 ++
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |   96 ++
 .../beam/sdk/io/kinesis/KinesisRecord.java      |  121 ++
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |   74 +
 .../beam/sdk/io/kinesis/KinesisSource.java      |  112 ++
 .../beam/sdk/io/kinesis/RecordFilter.java       |   41 +
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |   53 +
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |  175 +++
 .../sdk/io/kinesis/ShardRecordsIterator.java    |   98 ++
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |  157 ++
 .../beam/sdk/io/kinesis/StartingPoint.java      |   85 +
 .../io/kinesis/StaticCheckpointGenerator.java   |   42 +
 .../io/kinesis/TransientKinesisException.java   |   29 +
 .../beam/sdk/io/kinesis/package-info.java       |   22 +
 .../beam/sdk/io/kinesis/AmazonKinesisMock.java  |  375 +++++
 .../beam/sdk/io/kinesis/CustomOptionalTest.java |   31 +
 .../kinesis/DynamicCheckpointGeneratorTest.java |   57 +
 .../sdk/io/kinesis/KinesisMockReadTest.java     |   91 ++
 .../io/kinesis/KinesisReaderCheckpointTest.java |   67 +
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |  119 ++
 .../beam/sdk/io/kinesis/KinesisReaderTest.java  |  120 ++
 .../sdk/io/kinesis/KinesisRecordCoderTest.java  |   45 +
 .../beam/sdk/io/kinesis/KinesisTestOptions.java |   47 +
 .../beam/sdk/io/kinesis/KinesisUploader.java    |   84 +
 .../beam/sdk/io/kinesis/RecordFilterTest.java   |   66 +
 .../beam/sdk/io/kinesis/RoundRobinTest.java     |   57 +
 .../sdk/io/kinesis/ShardCheckpointTest.java     |  149 ++
 .../io/kinesis/ShardRecordsIteratorTest.java    |  151 ++
 .../io/kinesis/SimplifiedKinesisClientTest.java |  224 +++
 .../beam/sdk/io/kinesis/package-info.java       |   22 +
 sdks/java/io/mongodb/pom.xml                    |  129 ++
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |  553 +++++++
 .../beam/sdk/io/mongodb/package-info.java       |   22 +
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  209 +++
 .../beam/sdk/io/mongodb/package-info.java       |   22 +
 sdks/java/io/pom.xml                            |    2 +
 .../beam/sdk/transforms/CombineJava8Test.java   |    7 +-
 .../beam/sdk/transforms/FilterJava8Test.java    |    4 +-
 .../transforms/FlatMapElementsJava8Test.java    |    9 +-
 .../sdk/transforms/MapElementsJava8Test.java    |    4 +-
 .../beam/sdk/transforms/PartitionJava8Test.java |    4 +-
 .../transforms/RemoveDuplicatesJava8Test.java   |   11 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |    1 -
 .../sdk/transforms/WithTimestampsJava8Test.java |    4 +-
 .../src/main/java/DebuggingWordCount.java       |    9 +-
 .../src/main/java/WindowedWordCount.java        |   19 +-
 .../main/java/common/DataflowExampleUtils.java  |   25 +-
 .../common/ExampleBigQueryTableOptions.java     |    3 +-
 .../main/java/common/PubsubFileInjector.java    |   14 +-
 .../src/test/java/DebuggingWordCountTest.java   |    6 +-
 .../src/test/java/WordCountTest.java            |    5 +-
 .../src/main/java/StarterPipeline.java          |    1 -
 .../src/main/java/it/pkg/StarterPipeline.java   |    1 -
 .../coders/AvroCoderBenchmark.java              |    6 +-
 .../coders/ByteArrayCoderBenchmark.java         |    6 +-
 .../coders/CoderBenchmarking.java               |    3 +-
 .../coders/StringUtf8CoderBenchmark.java        |    6 +-
 .../transforms/DoFnInvokersBenchmark.java       |  224 +++
 .../transforms/DoFnReflectorBenchmark.java      |  232 ---
 1084 files changed, 34837 insertions(+), 27954 deletions(-)
----------------------------------------------------------------------



[07/50] [abbrv] incubator-beam git commit: Organize imports in Kinesis

Posted by dh...@apache.org.
Organize imports in Kinesis


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

Branch: refs/heads/gearpump-runner
Commit: aee5fbf0f259dee90618319947eb2aad7024078b
Parents: bce9aef
Author: Dan Halperin <dh...@google.com>
Authored: Fri Aug 26 15:17:05 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:10 2016 -0700

----------------------------------------------------------------------
 .../io/kinesis/DynamicCheckpointGenerator.java  |  4 +--
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |  2 +-
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  2 +-
 .../beam/sdk/io/kinesis/KinesisReader.java      |  8 +++---
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |  4 +--
 .../beam/sdk/io/kinesis/KinesisRecord.java      |  8 +++---
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |  9 +++----
 .../beam/sdk/io/kinesis/KinesisSource.java      | 10 +++----
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |  1 -
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |  8 +++---
 .../sdk/io/kinesis/ShardRecordsIterator.java    |  2 +-
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |  5 ++--
 .../beam/sdk/io/kinesis/StartingPoint.java      |  2 +-
 .../beam/sdk/io/kinesis/AmazonKinesisMock.java  | 12 ++++-----
 .../beam/sdk/io/kinesis/CustomOptionalTest.java |  2 +-
 .../kinesis/DynamicCheckpointGeneratorTest.java |  7 ++---
 .../sdk/io/kinesis/KinesisMockReadTest.java     | 11 ++++----
 .../io/kinesis/KinesisReaderCheckpointTest.java | 10 +++----
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    | 28 ++++++++++----------
 .../beam/sdk/io/kinesis/KinesisReaderTest.java  |  7 ++---
 .../sdk/io/kinesis/KinesisRecordCoderTest.java  |  3 +--
 .../beam/sdk/io/kinesis/KinesisUploader.java    |  4 +--
 .../beam/sdk/io/kinesis/RecordFilterTest.java   |  8 +++---
 .../beam/sdk/io/kinesis/RoundRobinTest.java     |  4 +--
 .../sdk/io/kinesis/ShardCheckpointTest.java     |  7 ++---
 .../io/kinesis/ShardRecordsIteratorTest.java    | 11 ++++----
 .../io/kinesis/SimplifiedKinesisClientTest.java | 11 ++++----
 27 files changed, 92 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java
index d86960f..2ec293c 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java
@@ -20,10 +20,8 @@ package org.apache.beam.sdk.io.kinesis;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.collect.Lists.transform;
 
-
-import com.google.common.base.Function;
-
 import com.amazonaws.services.kinesis.model.Shard;
+import com.google.common.base.Function;
 
 /**
  * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java
index f48b9d5..c0f00de 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java
@@ -18,9 +18,9 @@
 package org.apache.beam.sdk.io.kinesis;
 
 import static com.google.common.collect.Lists.transform;
-import com.google.common.base.Function;
 
 import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import com.google.common.base.Function;
 import java.util.List;
 import javax.annotation.Nullable;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
index b3cb464..811051c 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.io.kinesis;
 
 
-import org.apache.beam.sdk.transforms.PTransform;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.amazonaws.auth.AWSCredentialsProvider;
@@ -28,6 +27,7 @@ import com.amazonaws.regions.Regions;
 import com.amazonaws.services.kinesis.AmazonKinesis;
 import com.amazonaws.services.kinesis.AmazonKinesisClient;
 import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import org.apache.beam.sdk.transforms.PTransform;
 import org.joda.time.Instant;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
index 38a0050..219a705 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
@@ -18,16 +18,16 @@
 package org.apache.beam.sdk.io.kinesis;
 
 
-import org.apache.beam.sdk.io.UnboundedSource;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.collect.Lists.newArrayList;
 
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.List;
 import java.util.NoSuchElementException;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /***

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java
index 6ceb742..663ba44 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java
@@ -17,20 +17,18 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import org.apache.beam.sdk.io.UnboundedSource;
 import static com.google.common.collect.Iterables.transform;
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Lists.partition;
 
-
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Iterator;
 import java.util.List;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.UnboundedSource;
 
 /***
  * Checkpoint representing a total progress in a set of shards in single stream.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
index cdb495c..fe2a33d 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
@@ -17,15 +17,15 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import com.google.common.base.Charsets;
+import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode;
 
 import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
 import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
-import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode;
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.joda.time.Instant;
+import com.google.common.base.Charsets;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.joda.time.Instant;
 
 /**
  * {@link UserRecord} enhanced with utility methods.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java
index c383a4f..5b13e31 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java
@@ -17,18 +17,17 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.InstantCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
-
 import org.joda.time.Instant;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
 
 /***
  * A {@link Coder} for {@link KinesisRecord}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
index 38c9fa4..62cba08 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
@@ -17,18 +17,16 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Lists.newArrayList;
+
+import java.util.List;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
-
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.collect.Lists.newArrayList;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.util.List;
 
 
 /***

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java
index 7257aa1..7adae4b 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.io.kinesis;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.collect.Queues.newArrayDeque;
 
-
 import java.util.Deque;
 import java.util.Iterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
index 1d8628b..9920aca 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
@@ -18,17 +18,17 @@
 package org.apache.beam.sdk.io.kinesis;
 
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
 import com.amazonaws.services.kinesis.model.Record;
 import com.amazonaws.services.kinesis.model.ShardIteratorType;
-import org.joda.time.Instant;
 import java.io.Serializable;
+import org.joda.time.Instant;
 
 
 /***

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java
index 7dfe158..d17996a 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java
@@ -21,9 +21,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.collect.Queues.newArrayDeque;
 
 import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import java.util.Deque;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.util.Deque;
 
 /***
  * Iterates over records in a single shard.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
index f9a1ea2..96267d1 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
@@ -18,8 +18,6 @@
 package org.apache.beam.sdk.io.kinesis;
 
 
-import com.google.common.collect.Lists;
-
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.kinesis.AmazonKinesis;
 import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
@@ -32,10 +30,11 @@ import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededExcepti
 import com.amazonaws.services.kinesis.model.Shard;
 import com.amazonaws.services.kinesis.model.ShardIteratorType;
 import com.amazonaws.services.kinesis.model.StreamDescription;
-import org.joda.time.Instant;
+import com.google.common.collect.Lists;
 import java.util.Date;
 import java.util.List;
 import java.util.concurrent.Callable;
+import org.joda.time.Instant;
 
 /***
  * Wraps {@link AmazonKinesis} class providing much simpler interface and

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
index 8140269..b7ee917 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
@@ -22,9 +22,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
 import com.amazonaws.services.kinesis.model.ShardIteratorType;
-import org.joda.time.Instant;
 import java.io.Serializable;
 import java.util.Objects;
+import org.joda.time.Instant;
 
 /***
  * Denotes a point at which the reader should start reading from a Kinesis stream.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
index 7ca8e0b..b007fa4 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
@@ -19,7 +19,9 @@ package org.apache.beam.sdk.io.kinesis;
 
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Lists.transform;
-import com.google.common.base.Function;
+import static java.lang.Integer.parseInt;
+import static java.lang.Math.min;
+import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode;
 
 import com.amazonaws.AmazonWebServiceRequest;
 import com.amazonaws.ResponseMetadata;
@@ -63,15 +65,13 @@ import com.amazonaws.services.kinesis.model.ShardIteratorType;
 import com.amazonaws.services.kinesis.model.SplitShardRequest;
 import com.amazonaws.services.kinesis.model.SplitShardResult;
 import com.amazonaws.services.kinesis.model.StreamDescription;
-import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode;
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.joda.time.Instant;
-import static java.lang.Integer.parseInt;
-import static java.lang.Math.min;
+import com.google.common.base.Function;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.List;
 import javax.annotation.Nullable;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.joda.time.Instant;
 
 /**
  * Created by p.pastuszka on 21.07.2016.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
index 152fd6d..cb0d0e2 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import org.junit.Test;
 import java.util.NoSuchElementException;
+import org.junit.Test;
 
 /**
  * Created by ppastuszka on 12.12.15.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
index a9e5a69..c92ac9a 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
@@ -17,15 +17,16 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
-import com.amazonaws.services.kinesis.model.Shard;
+import static java.util.Arrays.asList;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.BDDMockito.given;
+
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.model.Shard;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import static java.util.Arrays.asList;
 
 
 /***

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
index 61a858f..304220b 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
@@ -17,20 +17,19 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import static com.google.common.collect.Lists.newArrayList;
+
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.google.common.collect.Iterables;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
-import static com.google.common.collect.Lists.newArrayList;
-
-import com.google.common.collect.Iterables;
-
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
 import org.joda.time.DateTime;
 import org.junit.Test;
-import java.util.List;
 
 /**
  * Created by p.pastuszka on 22.07.2016.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
index 205f050..8c8da64 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
@@ -18,17 +18,17 @@
 package org.apache.beam.sdk.io.kinesis;
 
 
-import com.google.common.collect.Iterables;
-
+import static java.util.Arrays.asList;
 import static org.assertj.core.api.Assertions.assertThat;
+
+import com.google.common.collect.Iterables;
+import java.util.Iterator;
+import java.util.List;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import static java.util.Arrays.asList;
-import java.util.Iterator;
-import java.util.List;
 
 /***
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
index fbc7c66..73a2455 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
@@ -17,6 +17,20 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Lists.newArrayList;
+import static java.util.concurrent.Executors.newSingleThreadExecutor;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.amazonaws.regions.Regions;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -25,25 +39,11 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.collect.Lists.newArrayList;
-
-import com.amazonaws.regions.Regions;
-import static org.assertj.core.api.Assertions.assertThat;
 import org.apache.commons.lang.RandomStringUtils;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Ignore;
 import org.junit.Test;
-import static java.util.concurrent.Executors.newSingleThreadExecutor;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 /**
  * Integration test, that reads from the real Kinesis.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
index 793fb57..29a24821 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
@@ -17,16 +17,17 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import static java.util.Arrays.asList;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import static java.util.Arrays.asList;
-import java.io.IOException;
-import java.util.NoSuchElementException;
 
 /**
  * Created by ppastuszka on 12.12.15.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
index b09b7eb..d301f25 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java
@@ -17,11 +17,10 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import java.nio.ByteBuffer;
 import org.apache.beam.sdk.testing.CoderProperties;
-
 import org.joda.time.Instant;
 import org.junit.Test;
-import java.nio.ByteBuffer;
 
 /**
  * Created by p.pastuszka on 20.07.2016.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java
index 0dcede9..c98242b 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java
@@ -18,8 +18,6 @@
 package org.apache.beam.sdk.io.kinesis;
 
 import static com.google.common.collect.Lists.newArrayList;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Lists;
 
 import com.amazonaws.auth.BasicAWSCredentials;
 import com.amazonaws.internal.StaticCredentialsProvider;
@@ -30,6 +28,8 @@ import com.amazonaws.services.kinesis.model.PutRecordsRequest;
 import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry;
 import com.amazonaws.services.kinesis.model.PutRecordsResult;
 import com.amazonaws.services.kinesis.model.PutRecordsResultEntry;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
 import java.nio.ByteBuffer;
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
index 360106d..f979c01 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
@@ -17,16 +17,16 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import com.google.common.collect.Lists;
-
 import static org.mockito.BDDMockito.given;
+
+import com.google.common.collect.Lists;
+import java.util.Collections;
+import java.util.List;
 import org.assertj.core.api.Assertions;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import java.util.Collections;
-import java.util.List;
 
 
 /***

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
index a508ddf..aedc89e 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java
@@ -18,11 +18,11 @@
 package org.apache.beam.sdk.io.kinesis;
 
 import static com.google.common.collect.Lists.newArrayList;
-
 import static org.assertj.core.api.Assertions.assertThat;
-import org.junit.Test;
+
 import java.util.Collections;
 import java.util.List;
+import org.junit.Test;
 
 /**
  * Created by ppastuszka on 12.12.15.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
index 2227cef..39ab36f 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
@@ -22,8 +22,6 @@ import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPos
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
 import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
-import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
-import com.amazonaws.services.kinesis.model.ShardIteratorType;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.BDDMockito.given;
 import static org.mockito.Matchers.anyString;
@@ -31,6 +29,10 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import java.io.IOException;
 import org.joda.time.DateTime;
 import org.joda.time.Instant;
 import org.junit.Before;
@@ -38,7 +40,6 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import java.io.IOException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
index e2a3ccc..585b884 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
@@ -17,11 +17,16 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
-import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyListOf;
 import static org.mockito.Mockito.when;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import java.io.IOException;
+import java.util.Collections;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -29,10 +34,6 @@ import org.mockito.Mock;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.runners.MockitoJUnitRunner;
 import org.mockito.stubbing.Answer;
-import static java.util.Arrays.asList;
-import static java.util.Collections.singletonList;
-import java.io.IOException;
-import java.util.Collections;
 
 /**
  * Created by ppastuszka on 12.12.15.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aee5fbf0/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
index 44d29d6..96434fd 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
@@ -17,6 +17,11 @@
  */
 package org.apache.beam.sdk.io.kinesis;
 
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.reset;
+
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.AmazonServiceException.ErrorType;
 import com.amazonaws.services.kinesis.AmazonKinesis;
@@ -29,17 +34,13 @@ import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededExcepti
 import com.amazonaws.services.kinesis.model.Shard;
 import com.amazonaws.services.kinesis.model.ShardIteratorType;
 import com.amazonaws.services.kinesis.model.StreamDescription;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.reset;
+import java.util.List;
 import org.joda.time.Instant;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.InjectMocks;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
-import java.util.List;
 
 /***
  */



[15/50] [abbrv] incubator-beam git commit: Test that multiple instances of TestStream are supported

Posted by dh...@apache.org.
Test that multiple instances of TestStream are supported

Add KeyedResourcePool

This interface represents some shared pool of values that may be used by
at most one caller at a time.

Add LockedKeyedResourcePool which has at most one value per key and
at most one user per value at a time.

Use KeyedResourcePool in TestStream


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

Branch: refs/heads/gearpump-runner
Commit: 4251761de2eced90235696767cef941332e8427c
Parents: 798566c
Author: Thomas Groh <tg...@google.com>
Authored: Tue Aug 30 14:17:50 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/KeyedResourcePool.java  |  47 +++++
 .../runners/direct/LockedKeyedResourcePool.java |  95 +++++++++
 .../direct/TestStreamEvaluatorFactory.java      | 141 +++++++------
 .../direct/LockedKeyedResourcePoolTest.java     | 163 +++++++++++++++
 .../direct/TestStreamEvaluatorFactoryTest.java  | 206 +++++++++++++++++++
 .../apache/beam/sdk/testing/TestStreamTest.java |  29 +++
 6 files changed, 623 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4251761d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
new file mode 100644
index 0000000..b976b69
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import com.google.common.base.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * A pool of resources associated with specific keys. Implementations enforce specific use patterns,
+ * such as limiting the the number of outstanding elements available per key.
+ */
+interface KeyedResourcePool<K, V> {
+  /**
+   * Tries to acquire a value for the provided key, loading it via the provided loader if necessary.
+   *
+   * <p>If the returned {@link Optional} contains a value, the caller obtains ownership of that
+   * value. The value should be released back to this {@link KeyedResourcePool} after the
+   * caller no longer has use of it using {@link #release(Object, Object)}.
+   *
+   * <p>The provided {@link Callable} <b>must not</b> return null; it may either return a non-null
+   * value or throw an exception.
+   */
+  Optional<V> tryAcquire(K key, Callable<V> loader) throws ExecutionException;
+
+  /**
+   * Release the provided value, relinquishing ownership of it. Future calls to
+   * {@link #tryAcquire(Object, Callable)} may return the released value.
+   */
+  void release(K key, V value);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4251761d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
new file mode 100644
index 0000000..8b1e0b1
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ExecutionError;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * A {@link KeyedResourcePool} which is limited to at most one outstanding instance at a time for
+ * each key.
+ */
+class LockedKeyedResourcePool<K, V> implements KeyedResourcePool<K, V> {
+  /**
+   * A map from each key to an {@link Optional} of the associated value. At most one value is stored
+   * per key, and it is obtained by at most one thread at a time.
+   *
+   * <p>For each key in this map:
+   *
+   * <ul>
+   * <li>If there is no associated value, then no value has been stored yet.
+   * <li>If the value is {@code Optional.absent()} then the value is currently in use.
+   * <li>If the value is {@code Optional.present()} then the contained value is available for use.
+   * </ul>
+   */
+  public static <K, V> LockedKeyedResourcePool<K, V> create() {
+    return new LockedKeyedResourcePool<>();
+  }
+
+  private final ConcurrentMap<K, Optional<V>> cache;
+
+  private LockedKeyedResourcePool() {
+    cache = new ConcurrentHashMap<>();
+  }
+
+  @Override
+  public Optional<V> tryAcquire(K key, Callable<V> loader) throws ExecutionException {
+    Optional<V> value = cache.replace(key, Optional.<V>absent());
+    if (value == null) {
+      // No value already existed, so populate the cache with the value returned by the loader
+      cache.putIfAbsent(key, Optional.of(load(loader)));
+      // Some other thread may obtain the result after the putIfAbsent, so retry acquisition
+      value = cache.replace(key, Optional.<V>absent());
+    }
+    return value;
+  }
+
+  private V load(Callable<V> loader) throws ExecutionException {
+    try {
+      return loader.call();
+    } catch (Error t) {
+      throw new ExecutionError(t);
+    } catch (RuntimeException e) {
+      throw new UncheckedExecutionException(e);
+    } catch (Exception e) {
+      throw new ExecutionException(e);
+    }
+  }
+
+  @Override
+  public void release(K key, V value) {
+    Optional<V> replaced = cache.replace(key, Optional.of(value));
+    checkNotNull(replaced, "Tried to release before a value was acquired");
+    checkState(
+        !replaced.isPresent(),
+        "Released a value to a %s where there is already a value present for key %s (%s). "
+            + "At most one value may be present at a time.",
+        LockedKeyedResourcePool.class.getSimpleName(),
+        key,
+        replaced);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4251761d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
index e9f37ba..3dbd886 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
@@ -22,12 +22,12 @@ import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.base.Supplier;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.testing.TestStream;
 import org.apache.beam.sdk.testing.TestStream.ElementEvent;
@@ -49,43 +49,52 @@ import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
-/**
- * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive.
- */
+/** The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */
 class TestStreamEvaluatorFactory implements TransformEvaluatorFactory {
-  private final AtomicBoolean inUse = new AtomicBoolean(false);
-  private final AtomicReference<Evaluator<?>> evaluator = new AtomicReference<>();
+  private final KeyedResourcePool<AppliedPTransform<?, ?, ?>, Evaluator<?>> evaluators =
+      LockedKeyedResourcePool.create();
 
   @Nullable
   @Override
   public <InputT> TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application,
       @Nullable CommittedBundle<?> inputBundle,
-      EvaluationContext evaluationContext) throws Exception {
+      EvaluationContext evaluationContext)
+      throws Exception {
     return createEvaluator((AppliedPTransform) application, evaluationContext);
   }
 
   @Override
   public void cleanup() throws Exception {}
 
+  /**
+   * Returns the evaluator for the provided application of {@link TestStream}, or null if it is
+   * already in use.
+   *
+   * <p>The documented behavior of {@link TestStream} requires the output of one event to travel
+   * completely through the pipeline before any additional event, so additional instances that have
+   * a separate collection of events cannot be created.
+   */
   private <InputT, OutputT> TransformEvaluator<? super InputT> createEvaluator(
       AppliedPTransform<PBegin, PCollection<OutputT>, TestStream<OutputT>> application,
-      EvaluationContext evaluationContext) {
-    if (evaluator.get() == null) {
-      Evaluator<OutputT> createdEvaluator = new Evaluator<>(application, evaluationContext, inUse);
-      evaluator.compareAndSet(null, createdEvaluator);
-    }
-    if (inUse.compareAndSet(false, true)) {
-      return evaluator.get();
-    } else {
-      return null;
-    }
+      EvaluationContext evaluationContext)
+      throws ExecutionException {
+    return evaluators
+        .tryAcquire(application, new CreateEvaluator<>(application, evaluationContext, evaluators))
+        .orNull();
   }
 
+  /**
+   * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link
+   * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will
+   * return this evaluator.
+   */
+  private void completeEvaluation(Evaluator<?> evaluator) {}
+
   private static class Evaluator<T> implements TransformEvaluator<Object> {
     private final AppliedPTransform<PBegin, PCollection<T>, TestStream<T>> application;
     private final EvaluationContext context;
-    private final AtomicBoolean inUse;
+    private final KeyedResourcePool<AppliedPTransform<?, ?, ?>, Evaluator<?>> cache;
     private final List<Event<T>> events;
     private int index;
     private Instant currentWatermark;
@@ -93,49 +102,48 @@ class TestStreamEvaluatorFactory implements TransformEvaluatorFactory {
     private Evaluator(
         AppliedPTransform<PBegin, PCollection<T>, TestStream<T>> application,
         EvaluationContext context,
-        AtomicBoolean inUse) {
+        KeyedResourcePool<AppliedPTransform<?, ?, ?>, Evaluator<?>> cache) {
       this.application = application;
       this.context = context;
-      this.inUse = inUse;
+      this.cache = cache;
       this.events = application.getTransform().getEvents();
       index = 0;
       currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
     }
 
     @Override
-    public void processElement(WindowedValue<Object> element) throws Exception {
-    }
+    public void processElement(WindowedValue<Object> element) throws Exception {}
 
     @Override
     public TransformResult finishBundle() throws Exception {
-      if (index >= events.size()) {
-        return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE).build();
-      }
-      Event<T> event = events.get(index);
-      if (event.getType().equals(EventType.WATERMARK)) {
-        currentWatermark = ((WatermarkEvent<T>) event).getWatermark();
-      }
-      StepTransformResult.Builder result =
-          StepTransformResult.withHold(application, currentWatermark);
-      if (event.getType().equals(EventType.ELEMENT)) {
-        UncommittedBundle<T> bundle = context.createRootBundle(application.getOutput());
-        for (TimestampedValue<T> elem : ((ElementEvent<T>) event).getElements()) {
-          bundle.add(WindowedValue.timestampedValueInGlobalWindow(elem.getValue(),
-              elem.getTimestamp()));
+      try {
+        if (index >= events.size()) {
+          return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE)
+              .build();
         }
-        result.addOutput(bundle);
-      }
-      if (event.getType().equals(EventType.PROCESSING_TIME)) {
-        ((TestClock) context.getClock())
-            .advance(((ProcessingTimeEvent<T>) event).getProcessingTimeAdvance());
+        Event<T> event = events.get(index);
+        if (event.getType().equals(EventType.WATERMARK)) {
+          currentWatermark = ((WatermarkEvent<T>) event).getWatermark();
+        }
+        StepTransformResult.Builder result =
+            StepTransformResult.withHold(application, currentWatermark);
+        if (event.getType().equals(EventType.ELEMENT)) {
+          UncommittedBundle<T> bundle = context.createRootBundle(application.getOutput());
+          for (TimestampedValue<T> elem : ((ElementEvent<T>) event).getElements()) {
+            bundle.add(
+                WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()));
+          }
+          result.addOutput(bundle);
+        }
+        if (event.getType().equals(EventType.PROCESSING_TIME)) {
+          ((TestClock) context.getClock())
+              .advance(((ProcessingTimeEvent<T>) event).getProcessingTimeAdvance());
+        }
+        index++;
+        return result.build();
+      } finally {
+        cache.release(application, this);
       }
-      index++;
-      checkState(inUse.compareAndSet(true, false),
-          "The InUse flag of a %s was changed while the source evaluator was executing. "
-              + "%s cannot be split or evaluated in parallel.",
-          TestStream.class.getSimpleName(),
-          TestStream.class.getSimpleName());
-      return result.build();
     }
   }
 
@@ -181,20 +189,37 @@ class TestStreamEvaluatorFactory implements TransformEvaluatorFactory {
 
       @Override
       public PCollection<T> apply(PBegin input) {
-        setup(input.getPipeline());
-        return PCollection.<T>createPrimitiveOutputInternal(
-                input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED)
-            .setCoder(original.getValueCoder());
-      }
-
-      private void setup(Pipeline p) {
-        PipelineRunner runner = p.getRunner();
-        checkState(runner instanceof DirectRunner,
+        PipelineRunner runner = input.getPipeline().getRunner();
+        checkState(
+            runner instanceof DirectRunner,
             "%s can only be used when running with the %s",
             getClass().getSimpleName(),
             DirectRunner.class.getSimpleName());
         ((DirectRunner) runner).setClockSupplier(new TestClockSupplier());
+        return PCollection.<T>createPrimitiveOutputInternal(
+                input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED)
+            .setCoder(original.getValueCoder());
       }
     }
   }
+
+  private static class CreateEvaluator<OutputT> implements Callable<Evaluator<?>> {
+    private final AppliedPTransform<PBegin, PCollection<OutputT>, TestStream<OutputT>> application;
+    private final EvaluationContext evaluationContext;
+    private final KeyedResourcePool<AppliedPTransform<?, ?, ?>, Evaluator<?>> evaluators;
+
+    public CreateEvaluator(
+        AppliedPTransform<PBegin, PCollection<OutputT>, TestStream<OutputT>> application,
+        EvaluationContext evaluationContext,
+        KeyedResourcePool<AppliedPTransform<?, ?, ?>, Evaluator<?>> evaluators) {
+      this.application = application;
+      this.evaluationContext = evaluationContext;
+      this.evaluators = evaluators;
+    }
+
+    @Override
+    public Evaluator<?> call() throws Exception {
+      return new Evaluator<>(application, evaluationContext, evaluators);
+    }
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4251761d/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
new file mode 100644
index 0000000..7703881
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.Iterables;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link TestStreamEvaluatorFactory}. */
+@RunWith(JUnit4.class)
+public class TestStreamEvaluatorFactoryTest {
+  private TestStreamEvaluatorFactory factory = new TestStreamEvaluatorFactory();
+  private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
+
+  /** Demonstrates that returned evaluators produce elements in sequence. */
+  @Test
+  public void producesElementsInSequence() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> streamVals =
+        p.apply(
+            TestStream.create(VarIntCoder.of())
+                .addElements(1, 2, 3)
+                .addElements(4, 5, 6)
+                .advanceWatermarkToInfinity());
+
+    EvaluationContext context = mock(EvaluationContext.class);
+    when(context.createRootBundle(streamVals))
+        .thenReturn(
+            bundleFactory.createRootBundle(streamVals), bundleFactory.createRootBundle(streamVals));
+
+    TransformEvaluator<Object> firstEvaluator =
+        factory.forApplication(streamVals.getProducingTransformInternal(), null, context);
+    TransformResult firstResult = firstEvaluator.finishBundle();
+
+    TransformEvaluator<Object> secondEvaluator =
+        factory.forApplication(streamVals.getProducingTransformInternal(), null, context);
+    TransformResult secondResult = secondEvaluator.finishBundle();
+
+    TransformEvaluator<Object> thirdEvaluator =
+        factory.forApplication(streamVals.getProducingTransformInternal(), null, context);
+    TransformResult thirdResult = thirdEvaluator.finishBundle();
+
+    assertThat(
+        Iterables.getOnlyElement(firstResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(1),
+            WindowedValue.valueInGlobalWindow(2),
+            WindowedValue.valueInGlobalWindow(3)));
+    assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+    assertThat(
+        Iterables.getOnlyElement(secondResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(
+            WindowedValue.valueInGlobalWindow(4),
+            WindowedValue.valueInGlobalWindow(5),
+            WindowedValue.valueInGlobalWindow(6)));
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+    assertThat(Iterables.isEmpty(thirdResult.getOutputBundles()), is(true));
+    assertThat(thirdResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE));
+  }
+
+  /** Demonstrates that at most one evaluator for an application is available at a time. */
+  @Test
+  public void onlyOneEvaluatorAtATime() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> streamVals =
+        p.apply(
+            TestStream.create(VarIntCoder.of()).addElements(4, 5, 6).advanceWatermarkToInfinity());
+
+    EvaluationContext context = mock(EvaluationContext.class);
+    TransformEvaluator<Object> firstEvaluator =
+        factory.forApplication(streamVals.getProducingTransformInternal(), null, context);
+
+    // create a second evaluator before the first is finished. The evaluator should not be available
+    TransformEvaluator<Object> secondEvaluator =
+        factory.forApplication(streamVals.getProducingTransformInternal(), null, context);
+    assertThat(secondEvaluator, is(nullValue()));
+  }
+
+  /**
+   * Demonstrates that multiple applications of the same {@link TestStream} produce separate
+   * evaluators.
+   */
+  @Test
+  public void multipleApplicationsMultipleEvaluators() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    TestStream<Integer> stream =
+        TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity();
+    PCollection<Integer> firstVals = p.apply("Stream One", stream);
+    PCollection<Integer> secondVals = p.apply("Stream A", stream);
+
+    EvaluationContext context = mock(EvaluationContext.class);
+    when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals));
+    when(context.createRootBundle(secondVals))
+        .thenReturn(bundleFactory.createRootBundle(secondVals));
+
+    TransformEvaluator<Object> firstEvaluator =
+        factory.forApplication(firstVals.getProducingTransformInternal(), null, context);
+    // The two evaluators can exist independently
+    TransformEvaluator<Object> secondEvaluator =
+        factory.forApplication(secondVals.getProducingTransformInternal(), null, context);
+
+    TransformResult firstResult = firstEvaluator.finishBundle();
+    TransformResult secondResult = secondEvaluator.finishBundle();
+
+    assertThat(
+        Iterables.getOnlyElement(firstResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2)));
+    assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+    // They both produce equal results, and don't interfere with each other
+    assertThat(
+        Iterables.getOnlyElement(secondResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2)));
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+  }
+
+  /**
+   * Demonstrates that multiple applications of different {@link TestStream} produce independent
+   * evaluators.
+   */
+  @Test
+  public void multipleStreamsMultipleEvaluators() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> firstVals =
+        p.apply(
+            "Stream One",
+            TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity());
+    PCollection<String> secondVals =
+        p.apply(
+            "Stream A",
+            TestStream.create(StringUtf8Coder.of())
+                .addElements("Two")
+                .advanceWatermarkToInfinity());
+
+    EvaluationContext context = mock(EvaluationContext.class);
+    when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals));
+    when(context.createRootBundle(secondVals))
+        .thenReturn(bundleFactory.createRootBundle(secondVals));
+
+    TransformEvaluator<Object> firstEvaluator =
+        factory.forApplication(firstVals.getProducingTransformInternal(), null, context);
+    // The two evaluators can exist independently
+    TransformEvaluator<Object> secondEvaluator =
+        factory.forApplication(secondVals.getProducingTransformInternal(), null, context);
+
+    TransformResult firstResult = firstEvaluator.finishBundle();
+    TransformResult secondResult = secondEvaluator.finishBundle();
+
+    assertThat(
+        Iterables.getOnlyElement(firstResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2)));
+    assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+
+    assertThat(
+        Iterables.getOnlyElement(secondResult.getOutputBundles())
+            .commit(Instant.now())
+            .getElements(),
+        Matchers.<WindowedValue<?>>containsInAnyOrder(WindowedValue.valueInGlobalWindow("Two")));
+    assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4251761d/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index 6457f91..a1b4e4a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -265,6 +265,35 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
+  @Category(NeedsRunner.class)
+  public void testMultipleStreams() {
+    TestStream<String> stream = TestStream.create(StringUtf8Coder.of())
+        .addElements("foo", "bar")
+        .advanceWatermarkToInfinity();
+
+    TestStream<Integer> other =
+        TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity();
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<String> createStrings =
+        p.apply("CreateStrings", stream)
+            .apply("WindowStrings",
+                Window.<String>triggering(AfterPane.elementCountAtLeast(2))
+                    .withAllowedLateness(Duration.ZERO)
+                    .accumulatingFiredPanes());
+    PAssert.that(createStrings).containsInAnyOrder("foo", "bar");
+    PCollection<Integer> createInts =
+        p.apply("CreateInts", other)
+            .apply("WindowInts",
+                Window.<Integer>triggering(AfterPane.elementCountAtLeast(4))
+                    .withAllowedLateness(Duration.ZERO)
+                    .accumulatingFiredPanes());
+    PAssert.that(createInts).containsInAnyOrder(1, 2, 3, 4);
+
+    p.run();
+  }
+
+  @Test
   public void testElementAtPositiveInfinityThrows() {
     Builder<Integer> stream =
         TestStream.create(VarIntCoder.of())


[23/50] [abbrv] incubator-beam git commit: FluentBackoff: a replacement for a variety of custom backoff implementations

Posted by dh...@apache.org.
FluentBackoff: a replacement for a variety of custom backoff implementations

We have 3 different backoff classes, which don't really have that much
different functionality. Add a single, flexible backoff implementation
that can be used to replace all three classes. Additionally, this new
backoff actually supports more functionality than any of the other three
did -- you can limit retries, cap the exponential growth of an
individual backoff, and cap the cumulative time spent in backoff; prior
implementations did not allow all 3.

This also makes the parameters self-obvious (Duration, not
number-that-is-also-millis) where appropriate.

This initial PR should have no functional changes.

* Implement FluentBackoff
* Replace other custom BackOff implementations with FluentBackoff


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

Branch: refs/heads/gearpump-runner
Commit: 3f48566618552c4b0fa026aa3a75ef6f1875da63
Parents: c92e45d
Author: Dan Halperin <dh...@google.com>
Authored: Wed Aug 24 22:35:26 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:12 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/common/ExampleUtils.java      |   7 +-
 .../runners/dataflow/DataflowPipelineJob.java   |  72 +++---
 .../beam/runners/dataflow/util/PackageUtil.java |  31 +--
 .../dataflow/DataflowPipelineJobTest.java       |  32 ++-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |   8 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java | 172 --------------
 .../util/AttemptBoundedExponentialBackOff.java  |  85 -------
 .../org/apache/beam/sdk/util/FluentBackoff.java | 229 +++++++++++++++++++
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |  18 +-
 .../util/IntervalBoundedExponentialBackOff.java |   1 +
 ...mptAndTimeBoundedExponentialBackOffTest.java | 212 -----------------
 .../AttemptBoundedExponentialBackOffTest.java   |  84 -------
 .../apache/beam/sdk/util/FluentBackoffTest.java | 226 ++++++++++++++++++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  10 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  21 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 152 ++++++------
 .../gcp/bigquery/BigQueryTableRowIterator.java  |   5 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  26 +--
 .../gcp/bigquery/BigQueryServicesImplTest.java  |  17 +-
 .../beam/sdk/io/gcp/datastore/V1TestUtil.java   |   9 +-
 20 files changed, 675 insertions(+), 742 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
index eadb580..2e8dcf6 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
@@ -45,8 +45,9 @@ import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.BigQueryOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.Transport;
+import org.joda.time.Duration;
 
 /**
  * The utility class that sets up and tears down external resources,
@@ -79,7 +80,9 @@ public class ExampleUtils {
    */
   public void setup() throws IOException {
     Sleeper sleeper = Sleeper.DEFAULT;
-    BackOff backOff = new AttemptBoundedExponentialBackOff(3, 200);
+    BackOff backOff =
+        FluentBackoff.DEFAULT
+            .withMaxRetries(3).withInitialBackoff(Duration.millis(200)).backoff();
     Throwable lastException = null;
     try {
       do {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/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 9a515fa..dad59f2 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
@@ -34,7 +34,6 @@ import java.io.IOException;
 import java.net.SocketTimeoutException;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
 import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor;
@@ -44,8 +43,7 @@ import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.AggregatorValues;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -95,14 +93,27 @@ public class DataflowPipelineJob implements PipelineResult {
   /**
    * The polling interval for job status and messages information.
    */
-  static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
-  static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
+  static final Duration MESSAGES_POLLING_INTERVAL = Duration.standardSeconds(2);
+  static final Duration STATUS_POLLING_INTERVAL = Duration.standardSeconds(2);
+
+  static final double DEFAULT_BACKOFF_EXPONENT = 1.5;
 
   /**
-   * The amount of polling attempts for job status and messages information.
+   * The amount of polling retries for job status and messages information.
    */
-  static final int MESSAGES_POLLING_ATTEMPTS = 12;
-  static final int STATUS_POLLING_ATTEMPTS = 5;
+  static final int MESSAGES_POLLING_RETRIES = 11;
+  static final int STATUS_POLLING_RETRIES = 4;
+
+  private static final FluentBackoff MESSAGES_BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT
+          .withInitialBackoff(MESSAGES_POLLING_INTERVAL)
+          .withMaxRetries(MESSAGES_POLLING_RETRIES)
+          .withExponent(DEFAULT_BACKOFF_EXPONENT);
+  protected static final FluentBackoff STATUS_BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT
+          .withInitialBackoff(STATUS_POLLING_INTERVAL)
+          .withMaxRetries(STATUS_POLLING_RETRIES)
+          .withExponent(DEFAULT_BACKOFF_EXPONENT);
 
   /**
    * Constructs the job.
@@ -214,21 +225,23 @@ public class DataflowPipelineJob implements PipelineResult {
     MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowOptions.getDataflowClient());
 
     long lastTimestamp = 0;
-    BackOff backoff =
-        duration.getMillis() > 0
-            ? new AttemptAndTimeBoundedExponentialBackOff(
-                MESSAGES_POLLING_ATTEMPTS,
-                MESSAGES_POLLING_INTERVAL,
-                duration.getMillis(),
-                AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
-                nanoClock)
-            : new AttemptBoundedExponentialBackOff(
-                MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL);
+    BackOff backoff;
+    if (!duration.isLongerThan(Duration.ZERO)) {
+      backoff = MESSAGES_BACKOFF_FACTORY.backoff();
+    } else {
+      backoff = MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration).backoff();
+    }
+
+    // This function tracks the cumulative time from the *first request* to enforce the wall-clock
+    // limit. Any backoff instance could, at best, track the the time since the first attempt at a
+    // given request. Thus, we need to track the cumulative time ourselves.
+    long startNanos = nanoClock.nanoTime();
+
     State state;
     do {
       // Get the state of the job before listing messages. This ensures we always fetch job
       // messages after the job finishes to ensure we have all them.
-      state = getStateWithRetries(1, sleeper);
+      state = getStateWithRetries(STATUS_BACKOFF_FACTORY.withMaxRetries(0).backoff(), sleeper);
       boolean hasError = state == State.UNKNOWN;
 
       if (messageHandler != null && !hasError) {
@@ -250,7 +263,16 @@ public class DataflowPipelineJob implements PipelineResult {
       }
 
       if (!hasError) {
+        // Reset the backoff.
         backoff.reset();
+        // If duration is set, update the new cumulative sleep time to be the remaining
+        // part of the total input sleep duration.
+        if (duration.isLongerThan(Duration.ZERO)) {
+          long nanosConsumed = nanoClock.nanoTime() - startNanos;
+          Duration consumed = Duration.millis((nanosConsumed + 999999) / 1000000);
+          backoff =
+              MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration.minus(consumed)).backoff();
+        }
         // Check if the job is done.
         if (state.isTerminal()) {
           return state;
@@ -287,7 +309,7 @@ public class DataflowPipelineJob implements PipelineResult {
       return terminalState;
     }
 
-    return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT);
+    return getStateWithRetries(STATUS_BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT);
   }
 
   /**
@@ -299,7 +321,7 @@ public class DataflowPipelineJob implements PipelineResult {
    * @return The state of the job or State.UNKNOWN in case of failure.
    */
   @VisibleForTesting
-  State getStateWithRetries(int attempts, Sleeper sleeper) {
+  State getStateWithRetries(BackOff attempts, Sleeper sleeper) {
     if (terminalState != null) {
       return terminalState;
     }
@@ -318,17 +340,13 @@ public class DataflowPipelineJob implements PipelineResult {
    * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the
    * maximum number of passed in attempts.
    *
-   * @param attempts The amount of attempts to make.
+   * @param backoff the {@link BackOff} used to control retries.
    * @param sleeper Object used to do the sleeps between attempts.
    * @return The underlying {@link Job} object.
    * @throws IOException When the maximum number of retries is exhausted, the last exception is
    * thrown.
    */
-  @VisibleForTesting
-  Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException {
-    AttemptBoundedExponentialBackOff backoff =
-        new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL);
-
+  private Job getJobWithRetries(BackOff backoff, Sleeper sleeper) throws IOException {
     // Retry loop ends in return or throw
     while (true) {
       try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
index bf1f666..6d910ba 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.dataflow.util;
 
 import com.fasterxml.jackson.core.Base64Variants;
-import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.BackOff;
 import com.google.api.client.util.Sleeper;
 import com.google.api.services.dataflow.model.DataflowPackage;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
@@ -37,10 +37,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
 import org.apache.beam.sdk.util.ZipFiles;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,11 +55,15 @@ public class PackageUtil {
   /**
    * The initial interval to use between package staging attempts.
    */
-  private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L;
+  private static final Duration INITIAL_BACKOFF_INTERVAL = Duration.standardSeconds(5);
   /**
-   * The maximum number of attempts when staging a file.
+   * The maximum number of retries when staging a file.
    */
-  private static final int MAX_ATTEMPTS = 5;
+  private static final int MAX_RETRIES = 4;
+
+  private static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT
+          .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_INTERVAL);
 
   /**
    * Translates exceptions from API calls.
@@ -199,9 +204,7 @@ public class PackageUtil {
         }
 
         // Upload file, retrying on failure.
-        AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
-            MAX_ATTEMPTS,
-            INITIAL_BACKOFF_INTERVAL_MS);
+        BackOff backoff = BACKOFF_FACTORY.backoff();
         while (true) {
           try {
             LOG.debug("Uploading classpath element {} to {}", classpathElement, target);
@@ -219,15 +222,17 @@ public class PackageUtil {
                   + "'gcloud auth login'.", classpathElement, target);
               LOG.error(errorMessage);
               throw new IOException(errorMessage, e);
-            } else if (!backoff.atMaxAttempts()) {
-              LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
-                  classpathElement, e);
-              BackOffUtils.next(retrySleeper, backoff);
-            } else {
+            }
+            long sleep = backoff.nextBackOffMillis();
+            if (sleep == BackOff.STOP) {
               // Rethrow last error, to be included as a cause in the catch below.
               LOG.error("Upload failed, will NOT retry staging of classpath: {}",
                   classpathElement, e);
               throw e;
+            } else {
+              LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
+                  classpathElement, e);
+              retrySleeper.sleep(sleep);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
index 22b5400..226140a 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
@@ -60,7 +60,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.joda.time.Duration;
@@ -111,22 +110,21 @@ public class DataflowPipelineJobTest {
    * AttemptBoundedExponentialBackOff given the number of retries and
    * an initial polling interval.
    *
-   * @param pollingIntervalMillis The initial polling interval given.
-   * @param attempts The number of attempts made
+   * @param pollingInterval The initial polling interval given.
+   * @param retries The number of retries made
    * @param timeSleptMillis The amount of time slept by the clock. This is checked
    * against the valid interval.
    */
-  void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) {
+  private void checkValidInterval(Duration pollingInterval, int retries, long timeSleptMillis) {
     long highSum = 0;
     long lowSum = 0;
-    for (int i = 1; i < attempts; i++) {
+    for (int i = 0; i < retries; i++) {
       double currentInterval =
-          pollingIntervalMillis
-          * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1);
-      double offset =
-          AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval;
-      highSum += Math.round(currentInterval + offset);
-      lowSum += Math.round(currentInterval - offset);
+          pollingInterval.getMillis()
+          * Math.pow(DataflowPipelineJob.DEFAULT_BACKOFF_EXPONENT, i);
+      double randomOffset = 0.5 * currentInterval;
+      highSum += Math.round(currentInterval + randomOffset);
+      lowSum += Math.round(currentInterval - randomOffset);
     }
     assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum)));
   }
@@ -228,7 +226,7 @@ public class DataflowPipelineJobTest {
     assertEquals(null, state);
     long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
     checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL,
-        DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff);
+        DataflowPipelineJob.MESSAGES_POLLING_RETRIES, timeDiff);
   }
 
   @Test
@@ -246,8 +244,8 @@ public class DataflowPipelineJobTest {
     State state = job.waitUntilFinish(Duration.millis(4), null, fastClock, fastClock);
     assertEquals(null, state);
     long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
-    // Should only sleep for the 4 ms remaining.
-    assertEquals(timeDiff, 4L);
+    // Should only have slept for the 4 ms allowed.
+    assertEquals(4L, timeDiff);
   }
 
   @Test
@@ -268,7 +266,7 @@ public class DataflowPipelineJobTest {
 
     assertEquals(
         State.RUNNING,
-        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+        job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock));
   }
 
   @Test
@@ -286,10 +284,10 @@ public class DataflowPipelineJobTest {
     long startTime = fastClock.nanoTime();
     assertEquals(
         State.UNKNOWN,
-        job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+        job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock));
     long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
     checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL,
-        DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff);
+        DataflowPipelineJob.STATUS_POLLING_RETRIES, timeDiff);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
index 28d7746..b41c655 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
@@ -34,7 +34,7 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.RemoveDuplicates;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.ValueWithRecordId;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
@@ -52,6 +52,10 @@ class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PCollection<T
   private final UnboundedSource<T, ?> source;
   private final long maxNumRecords;
   private final Duration maxReadTime;
+  private static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT
+          .withInitialBackoff(Duration.millis(10))
+          .withMaxBackoff(Duration.standardSeconds(10));
 
   /**
    * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount
@@ -241,7 +245,7 @@ class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PCollection<T
 
       private boolean advanceWithBackoff() throws IOException {
         // Try reading from the source with exponential backoff
-        BackOff backoff = new IntervalBoundedExponentialBackOff(10000L, 10L);
+        BackOff backoff = BACKOFF_FACTORY.backoff();
         long nextSleep = backoff.nextBackOffMillis();
         while (nextSleep != BackOff.STOP) {
           if (endTime != null && Instant.now().isAfter(endTime)) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
deleted file mode 100644
index d8050e0..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.NanoClock;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff
- * is happening as well as the amount of retries. Acts exactly as a AttemptBoundedExponentialBackOff
- * unless the time interval has expired since the object was created. At this point, it will always
- * return BackOff.STOP. Calling reset() resets both the timer and the number of retry attempts,
- * unless a custom ResetPolicy (ResetPolicy.ATTEMPTS or ResetPolicy.TIMER) is passed to the
- * constructor.
- *
- * <p>Implementation is not thread-safe.
- */
-public class AttemptAndTimeBoundedExponentialBackOff extends AttemptBoundedExponentialBackOff {
-  private long endTimeMillis;
-  private long maximumTotalWaitTimeMillis;
-  private ResetPolicy resetPolicy;
-  private final NanoClock nanoClock;
-  // NanoClock.SYSTEM has a max elapsed time of 292 years or 2^63 ns.  Here, we choose 2^53 ns as
-  // a smaller but still huge limit.
-  private static final long MAX_ELAPSED_TIME_MILLIS = 1L << 53;
-
-  /**
-   * A ResetPolicy controls the behavior of this BackOff when reset() is called.  By default, both
-   * the number of attempts and the time bound for the BackOff are reset, but an alternative
-   * ResetPolicy may be set to only reset one of these two.
-   */
-  public static enum ResetPolicy {
-    ALL,
-    ATTEMPTS,
-    TIMER
-  }
-
-  /**
-   * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff.
-   *
-   * @param maximumNumberOfAttempts The maximum number of attempts it will make.
-   * @param initialIntervalMillis The original interval to wait between attempts in milliseconds.
-   * @param maximumTotalWaitTimeMillis The maximum total time that this object will
-   *    allow more attempts in milliseconds.
-   */
-  public AttemptAndTimeBoundedExponentialBackOff(
-      int maximumNumberOfAttempts, long initialIntervalMillis, long maximumTotalWaitTimeMillis) {
-    this(
-        maximumNumberOfAttempts,
-        initialIntervalMillis,
-        maximumTotalWaitTimeMillis,
-        ResetPolicy.ALL,
-        NanoClock.SYSTEM);
-  }
-
-  /**
-   * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff.
-   *
-   * @param maximumNumberOfAttempts The maximum number of attempts it will make.
-   * @param initialIntervalMillis The original interval to wait between attempts in milliseconds.
-   * @param maximumTotalWaitTimeMillis The maximum total time that this object will
-   *    allow more attempts in milliseconds.
-   * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject
-   *    to being reset.
-   */
-  public AttemptAndTimeBoundedExponentialBackOff(
-      int maximumNumberOfAttempts,
-      long initialIntervalMillis,
-      long maximumTotalWaitTimeMillis,
-      ResetPolicy resetPolicy) {
-    this(
-        maximumNumberOfAttempts,
-        initialIntervalMillis,
-        maximumTotalWaitTimeMillis,
-        resetPolicy,
-        NanoClock.SYSTEM);
-  }
-
-  /**
-   * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff.
-   *
-   * @param maximumNumberOfAttempts The maximum number of attempts it will make.
-   * @param initialIntervalMillis The original interval to wait between attempts in milliseconds.
-   * @param maximumTotalWaitTimeMillis The maximum total time that this object will
-   *    allow more attempts in milliseconds.
-   * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject
-   *    to being reset.
-   * @param nanoClock clock used to measure the time that has passed.
-   */
-  public AttemptAndTimeBoundedExponentialBackOff(
-      int maximumNumberOfAttempts,
-      long initialIntervalMillis,
-      long maximumTotalWaitTimeMillis,
-      ResetPolicy resetPolicy,
-      NanoClock nanoClock) {
-    super(maximumNumberOfAttempts, initialIntervalMillis);
-    checkArgument(
-        maximumTotalWaitTimeMillis > 0, "Maximum total wait time must be greater than zero.");
-    checkArgument(
-        maximumTotalWaitTimeMillis < MAX_ELAPSED_TIME_MILLIS,
-        "Maximum total wait time must be less than " + MAX_ELAPSED_TIME_MILLIS + " milliseconds");
-    checkArgument(resetPolicy != null, "resetPolicy may not be null");
-    checkArgument(nanoClock != null, "nanoClock may not be null");
-    this.maximumTotalWaitTimeMillis = maximumTotalWaitTimeMillis;
-    this.resetPolicy = resetPolicy;
-    this.nanoClock = nanoClock;
-    // Set the end time for this BackOff.  Note that we cannot simply call reset() here since the
-    // resetPolicy may not be set to reset the time bound.
-    endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis;
-  }
-
-  @Override
-  @SuppressFBWarnings(value = "UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR",
-      justification = "Explicitly handled in implementation.")
-  public void reset() {
-    // reset() is called in the constructor of the parent class before resetPolicy and nanoClock are
-    // set.  In this case, we call the parent class's reset() method and return.
-    if (resetPolicy == null) {
-      super.reset();
-      return;
-    }
-    // Reset the number of attempts.
-    if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.ATTEMPTS) {
-      super.reset();
-    }
-    // Reset the time bound.
-    if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.TIMER) {
-      endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis;
-    }
-  }
-
-  public void setEndtimeMillis(long endTimeMillis) {
-    this.endTimeMillis = endTimeMillis;
-  }
-
-  @Override
-  public long nextBackOffMillis() {
-    if (atMaxAttempts()) {
-      return BackOff.STOP;
-    }
-    long backoff = Math.min(super.nextBackOffMillis(), endTimeMillis - getTimeMillis());
-    return (backoff > 0 ? backoff : BackOff.STOP);
-  }
-
-  private long getTimeMillis() {
-    return TimeUnit.NANOSECONDS.toMillis(nanoClock.nanoTime());
-  }
-
-  @Override
-  public boolean atMaxAttempts() {
-    return super.atMaxAttempts() || getTimeMillis() >= endTimeMillis;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
deleted file mode 100644
index 5707293..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.api.client.util.BackOff;
-
-
-/**
- * Implementation of {@link BackOff} that increases the back off period for each retry attempt
- * using a randomization function that grows exponentially.
- *
- * <p>Example: The initial interval is .5 seconds and the maximum number of retries is 10.
- * For 10 tries the sequence will be (values in seconds):
- *
- * <pre>
- * retry#      retry_interval     randomized_interval
- * 1             0.5                [0.25,   0.75]
- * 2             0.75               [0.375,  1.125]
- * 3             1.125              [0.562,  1.687]
- * 4             1.687              [0.8435, 2.53]
- * 5             2.53               [1.265,  3.795]
- * 6             3.795              [1.897,  5.692]
- * 7             5.692              [2.846,  8.538]
- * 8             8.538              [4.269, 12.807]
- * 9            12.807              [6.403, 19.210]
- * 10           {@link BackOff#STOP}
- * </pre>
- *
- * <p>Implementation is not thread-safe.
- */
-public class AttemptBoundedExponentialBackOff implements BackOff {
-  public static final double DEFAULT_MULTIPLIER = 1.5;
-  public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5;
-  private final int maximumNumberOfAttempts;
-  private final long initialIntervalMillis;
-  private int currentAttempt;
-
-  public AttemptBoundedExponentialBackOff(int maximumNumberOfAttempts, long initialIntervalMillis) {
-    checkArgument(maximumNumberOfAttempts > 0,
-        "Maximum number of attempts must be greater than zero.");
-    checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero.");
-    this.maximumNumberOfAttempts = maximumNumberOfAttempts;
-    this.initialIntervalMillis = initialIntervalMillis;
-    reset();
-  }
-
-  @Override
-  public void reset() {
-    currentAttempt = 1;
-  }
-
-  @Override
-  public long nextBackOffMillis() {
-    if (currentAttempt >= maximumNumberOfAttempts) {
-      return BackOff.STOP;
-    }
-    double currentIntervalMillis = initialIntervalMillis
-        * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1);
-    double randomOffset = (Math.random() * 2 - 1)
-        * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis;
-    currentAttempt += 1;
-    return Math.round(currentIntervalMillis + randomOffset);
-  }
-
-  public boolean atMaxAttempts() {
-    return currentAttempt >= maximumNumberOfAttempts;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
new file mode 100644
index 0000000..479d7a8
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.client.util.BackOff;
+import com.google.common.base.MoreObjects;
+import org.joda.time.Duration;
+
+/**
+ * A fluent builder for {@link BackOff} objects that allows customization of the retry algorithm.
+ *
+ * @see #DEFAULT for the default configuration parameters.
+ */
+public final class FluentBackoff {
+
+  private static final double DEFAULT_EXPONENT = 1.5;
+  private static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5;
+  private static final Duration DEFAULT_MIN_BACKOFF = Duration.standardSeconds(1);
+  private static final Duration DEFAULT_MAX_BACKOFF = Duration.standardDays(1000);
+  private static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE;
+  private static final Duration DEFAULT_MAX_CUM_BACKOFF = Duration.standardDays(1000);
+
+  private final double exponent;
+  private final Duration initialBackoff;
+  private final Duration maxBackoff;
+  private final Duration maxCumulativeBackoff;
+  private final int maxRetries;
+
+  /**
+   * By default the {@link BackOff} created by this builder will use exponential backoff (base
+   * exponent 1.5) with an initial backoff of 1 second. These parameters can be overridden with
+   * {@link #withExponent(double)} and {@link #withInitialBackoff(Duration)},
+   * respectively, and the maximum backoff after exponential increase can be capped using {@link
+   * FluentBackoff#withMaxBackoff(Duration)}.
+   *
+   * <p>The default {@link BackOff} does not limit the number of retries. To limit the backoff, the
+   * maximum total number of retries can be set using {@link #withMaxRetries(int)}. The
+   * total time spent in backoff can be time-bounded as well by configuring {@link
+   * #withMaxCumulativeBackoff(Duration)}. If either of these limits are reached, calls
+   * to {@link BackOff#nextBackOffMillis()} will return {@link BackOff#STOP} to signal that no more
+   * retries should continue.
+   */
+  public static final FluentBackoff DEFAULT = new FluentBackoff(
+      DEFAULT_EXPONENT,
+      DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_MAX_CUM_BACKOFF,
+      DEFAULT_MAX_RETRIES);
+
+  /**
+   * Instantiates a {@link BackOff} that will obey the current configuration.
+   *
+   * @see FluentBackoff
+   */
+  public BackOff backoff() {
+    return new BackoffImpl(this);
+  }
+
+  /**
+   * Returns a copy of this {@link FluentBackoff} that instead uses the specified exponent to
+   * control the exponential growth of delay.
+   *
+   * <p>Does not modify this object.
+   *
+   * @see FluentBackoff
+   */
+  public FluentBackoff withExponent(double exponent) {
+    checkArgument(exponent > 0, "exponent %s must be greater than 0", exponent);
+    return new FluentBackoff(
+        exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries);
+  }
+
+  /**
+   * Returns a copy of this {@link FluentBackoff} that instead uses the specified initial backoff
+   * duration.
+   *
+   * <p>Does not modify this object.
+   *
+   * @see FluentBackoff
+   */
+  public FluentBackoff withInitialBackoff(Duration initialBackoff) {
+    checkArgument(
+        initialBackoff.isLongerThan(Duration.ZERO),
+        "initialBackoff %s must be at least 1 millisecond",
+        initialBackoff);
+    return new FluentBackoff(
+        exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries);
+  }
+
+  /**
+   * Returns a copy of this {@link FluentBackoff} that limits the maximum backoff of an individual
+   * attempt to the specified duration.
+   *
+   * <p>Does not modify this object.
+   *
+   * @see FluentBackoff
+   */
+  public FluentBackoff withMaxBackoff(Duration maxBackoff) {
+    checkArgument(
+        maxBackoff.getMillis() > 0,
+        "maxBackoff %s must be at least 1 millisecond",
+        maxBackoff);
+    return new FluentBackoff(
+        exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries);
+  }
+
+  /**
+   * Returns a copy of this {@link FluentBackoff} that limits the total time spent in backoff
+   * returned across all calls to {@link BackOff#nextBackOffMillis()}.
+   *
+   * <p>Does not modify this object.
+   *
+   * @see FluentBackoff
+   */
+  public FluentBackoff withMaxCumulativeBackoff(Duration maxCumulativeBackoff) {
+    checkArgument(maxCumulativeBackoff.isLongerThan(Duration.ZERO),
+        "maxCumulativeBackoff %s must be at least 1 millisecond", maxCumulativeBackoff);
+    return new FluentBackoff(
+        exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries);
+  }
+
+  /**
+   * Returns a copy of this {@link FluentBackoff} that limits the total number of retries, aka
+   * the total number of calls to {@link BackOff#nextBackOffMillis()} before returning
+   * {@link BackOff#STOP}.
+   *
+   * <p>Does not modify this object.
+   *
+   * @see FluentBackoff
+   */
+  public FluentBackoff withMaxRetries(int maxRetries) {
+    checkArgument(maxRetries >= 0, "maxRetries %s cannot be negative", maxRetries);
+    return new FluentBackoff(
+        exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries);
+  }
+
+  public String toString() {
+    return MoreObjects.toStringHelper(FluentBackoff.class)
+        .add("exponent", exponent)
+        .add("initialBackoff", initialBackoff)
+        .add("maxBackoff", maxBackoff)
+        .add("maxRetries", maxRetries)
+        .add("maxCumulativeBackoff", maxCumulativeBackoff)
+        .toString();
+  }
+
+  private static class BackoffImpl implements BackOff {
+
+    // Customization of this backoff.
+    private final FluentBackoff backoffConfig;
+    // Current state
+    private Duration currentCumulativeBackoff;
+    private int currentRetry;
+
+    @Override
+    public void reset() {
+      currentRetry = 0;
+      currentCumulativeBackoff = Duration.ZERO;
+    }
+
+    @Override
+    public long nextBackOffMillis() {
+      // Maximum number of retries reached.
+      if (currentRetry >= backoffConfig.maxRetries) {
+        return BackOff.STOP;
+      }
+      // Maximum cumulative backoff reached.
+      if (currentCumulativeBackoff.compareTo(backoffConfig.maxCumulativeBackoff) >= 0) {
+        return BackOff.STOP;
+      }
+
+      double currentIntervalMillis =
+          Math.min(
+              backoffConfig.initialBackoff.getMillis()
+                  * Math.pow(backoffConfig.exponent, currentRetry),
+              backoffConfig.maxBackoff.getMillis());
+      double randomOffset =
+          (Math.random() * 2 - 1) * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis;
+      long nextBackoffMillis = Math.round(currentIntervalMillis + randomOffset);
+      // Cap to limit on cumulative backoff
+      Duration remainingCumulative =
+          backoffConfig.maxCumulativeBackoff.minus(currentCumulativeBackoff);
+      nextBackoffMillis = Math.min(nextBackoffMillis, remainingCumulative.getMillis());
+
+      // Update state and return backoff.
+      currentCumulativeBackoff = currentCumulativeBackoff.plus(nextBackoffMillis);
+      currentRetry += 1;
+      return nextBackoffMillis;
+    }
+
+    private BackoffImpl(FluentBackoff backoffConfig) {
+      this.backoffConfig = backoffConfig;
+      this.reset();
+    }
+
+    public String toString() {
+      return MoreObjects.toStringHelper(BackoffImpl.class)
+          .add("backoffConfig", backoffConfig)
+          .add("currentRetry", currentRetry)
+          .add("currentCumulativeBackoff", currentCumulativeBackoff)
+          .toString();
+    }
+  }
+
+  private FluentBackoff(
+      double exponent, Duration initialBackoff, Duration maxBackoff, Duration maxCumulativeBackoff,
+      int maxRetries) {
+    this.exponent = exponent;
+    this.initialBackoff = initialBackoff;
+    this.maxBackoff = maxBackoff;
+    this.maxRetries = maxRetries;
+    this.maxCumulativeBackoff = maxCumulativeBackoff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
index 44a182e..41c372e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
@@ -66,6 +66,7 @@ import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -119,6 +120,9 @@ public class GcsUtil {
    */
   private static final int MAX_CONCURRENT_BATCHES = 256;
 
+  private static final FluentBackoff BACKOFF_FACTORY =
+      FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200));
+
   /////////////////////////////////////////////////////////////////////////////
 
   /** Client for the GCS API. */
@@ -177,7 +181,7 @@ public class GcsUtil {
         // the request has strong global consistency.
         ResilientOperation.retry(
             ResilientOperation.getGoogleRequestCallable(getObject),
-            new AttemptBoundedExponentialBackOff(3, 200),
+            BACKOFF_FACTORY.backoff(),
             RetryDeterminer.SOCKET_ERRORS,
             IOException.class);
         return ImmutableList.of(gcsPattern);
@@ -216,7 +220,7 @@ public class GcsUtil {
       try {
         objects = ResilientOperation.retry(
             ResilientOperation.getGoogleRequestCallable(listObject),
-            new AttemptBoundedExponentialBackOff(3, 200),
+            BACKOFF_FACTORY.backoff(),
             RetryDeterminer.SOCKET_ERRORS,
             IOException.class);
       } catch (Exception e) {
@@ -257,7 +261,10 @@ public class GcsUtil {
    * if the resource does not exist.
    */
   public long fileSize(GcsPath path) throws IOException {
-    return fileSize(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT);
+    return fileSize(
+        path,
+        BACKOFF_FACTORY.backoff(),
+        Sleeper.DEFAULT);
   }
 
   /**
@@ -335,7 +342,10 @@ public class GcsUtil {
    * be accessible otherwise the permissions exception will be propagated.
    */
   public boolean bucketExists(GcsPath path) throws IOException {
-    return bucketExists(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT);
+    return bucketExists(
+        path,
+        BACKOFF_FACTORY.backoff(),
+        Sleeper.DEFAULT);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
index 519776a..6fac6dc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
@@ -49,6 +49,7 @@ import com.google.api.client.util.BackOff;
  *
  * <p>Implementation is not thread-safe.
  */
+@Deprecated
 public class IntervalBoundedExponentialBackOff implements BackOff {
   public static final double DEFAULT_MULTIPLIER = 1.5;
   public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
deleted file mode 100644
index 59e0fb7..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.lessThan;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import com.google.api.client.util.BackOff;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Unit tests for {@link AttemptAndTimeBoundedExponentialBackOff}. */
-@RunWith(JUnit4.class)
-public class AttemptAndTimeBoundedExponentialBackOffTest {
-  @Rule public ExpectedException exception = ExpectedException.none();
-  @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
-
-  @Test
-  public void testUsingInvalidInitialInterval() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Initial interval must be greater than zero.");
-    new AttemptAndTimeBoundedExponentialBackOff(10, 0L, 1000L);
-  }
-
-  @Test
-  public void testUsingInvalidTimeInterval() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Maximum total wait time must be greater than zero.");
-    new AttemptAndTimeBoundedExponentialBackOff(10, 2L, 0L);
-  }
-
-  @Test
-  public void testUsingInvalidMaximumNumberOfRetries() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Maximum number of attempts must be greater than zero.");
-    new AttemptAndTimeBoundedExponentialBackOff(-1, 10L, 1000L);
-  }
-
-  @Test
-  public void testThatFixedNumberOfAttemptsExits() throws Exception {
-    BackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3,
-            500L,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testThatResettingAllowsReuse() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3,
-            500,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-
-    backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            30,
-            500,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    fastClock.sleep(2000L);
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-  }
-
-  @Test
-  public void testThatResettingAttemptsAllowsReuse() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3,
-            500,
-            1000,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testThatResettingAttemptsDoesNotAllowsReuse() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            30,
-            500,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    fastClock.sleep(2000L);
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testThatResettingTimerAllowsReuse() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            30,
-            500,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    fastClock.sleep(2000L);
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(561L), lessThan(1688L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(843L), lessThan(2531L)));
-  }
-
-  @Test
-  public void testThatResettingTimerDoesNotAllowReuse() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3,
-            500,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER,
-            fastClock);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testTimeBound() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3, 500L, 5L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock);
-    assertEquals(backOff.nextBackOffMillis(), 5L);
-  }
-
-  @Test
-  public void testAtMaxAttempts() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3,
-            500L,
-            1000L,
-            AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL,
-            fastClock);
-    assertFalse(backOff.atMaxAttempts());
-    backOff.nextBackOffMillis();
-    assertFalse(backOff.atMaxAttempts());
-    backOff.nextBackOffMillis();
-    assertTrue(backOff.atMaxAttempts());
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testAtMaxTime() throws Exception {
-    AttemptBoundedExponentialBackOff backOff =
-        new AttemptAndTimeBoundedExponentialBackOff(
-            3, 500L, 1L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock);
-    fastClock.sleep(2);
-    assertTrue(backOff.atMaxAttempts());
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
deleted file mode 100644
index 3cfa961..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.lessThan;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import com.google.api.client.util.BackOff;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Unit tests for {@link AttemptBoundedExponentialBackOff}. */
-@RunWith(JUnit4.class)
-public class AttemptBoundedExponentialBackOffTest {
-  @Rule public ExpectedException exception = ExpectedException.none();
-
-  @Test
-  public void testUsingInvalidInitialInterval() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Initial interval must be greater than zero.");
-    new AttemptBoundedExponentialBackOff(10, 0L);
-  }
-
-  @Test
-  public void testUsingInvalidMaximumNumberOfRetries() throws Exception {
-    exception.expect(IllegalArgumentException.class);
-    exception.expectMessage("Maximum number of attempts must be greater than zero.");
-    new AttemptBoundedExponentialBackOff(-1, 10L);
-  }
-
-  @Test
-  public void testThatFixedNumberOfAttemptsExits() throws Exception {
-    BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testThatResettingAllowsReuse() throws Exception {
-    BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500);
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-    backOff.reset();
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L)));
-    assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L)));
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-
-  @Test
-  public void testAtMaxAttempts() throws Exception {
-    AttemptBoundedExponentialBackOff backOff = new AttemptBoundedExponentialBackOff(3, 500);
-    assertFalse(backOff.atMaxAttempts());
-    backOff.nextBackOffMillis();
-    assertFalse(backOff.atMaxAttempts());
-    backOff.nextBackOffMillis();
-    assertTrue(backOff.atMaxAttempts());
-    assertEquals(BackOff.STOP, backOff.nextBackOffMillis());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java
new file mode 100644
index 0000000..20b03cf
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.lessThan;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import com.google.api.client.util.BackOff;
+import java.io.IOException;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link FluentBackoff}.
+ */
+@RunWith(JUnit4.class)
+public class FluentBackoffTest {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  private final FluentBackoff defaultBackoff = FluentBackoff.DEFAULT;
+
+  @Test
+  public void testInvalidExponent() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("exponent -2.0 must be greater than 0");
+    defaultBackoff.withExponent(-2.0);
+  }
+
+  @Test
+  public void testInvalidInitialBackoff() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("initialBackoff PT0S must be at least 1 millisecond");
+    defaultBackoff.withInitialBackoff(Duration.ZERO);
+  }
+
+  @Test
+  public void testInvalidMaxBackoff() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("maxBackoff PT0S must be at least 1 millisecond");
+    defaultBackoff.withMaxBackoff(Duration.ZERO);
+  }
+
+  @Test
+  public void testInvalidMaxRetries() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("maxRetries -1 cannot be negative");
+    defaultBackoff.withMaxRetries(-1);
+  }
+
+  @Test
+  public void testInvalidCumulativeBackoff() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("maxCumulativeBackoff PT-0.002S must be at least 1 millisecond");
+    defaultBackoff.withMaxCumulativeBackoff(Duration.millis(-2));
+  }
+
+  /**
+   * Tests with bounded interval, custom exponent, and unlimited retries.
+   */
+  @Test
+  public void testBoundedIntervalWithReset() throws Exception {
+    BackOff backOff =
+        FluentBackoff.DEFAULT
+            .withInitialBackoff(Duration.millis(500))
+            .withMaxBackoff(Duration.standardSeconds(1)).backoff();
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+
+    // Reset, should go back to short times.
+    backOff.reset();
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L),
+        lessThanOrEqualTo(1500L)));
+
+  }
+
+  /**
+   * Tests with bounded interval, custom exponent, limited retries, and a reset.
+   */
+  @Test
+  public void testMaxRetriesWithReset() throws Exception {
+    BackOff backOff =
+        FluentBackoff.DEFAULT
+            .withInitialBackoff(Duration.millis(500))
+            .withMaxRetries(1)
+            .backoff();
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP));
+    assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP));
+    assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP));
+    assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP));
+
+    backOff.reset();
+    assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L)));
+    assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP));
+  }
+
+  private static long countMaximumBackoff(BackOff backOff) throws IOException {
+    long cumulativeBackoffMillis = 0;
+    long currentBackoffMillis = backOff.nextBackOffMillis();
+    while (currentBackoffMillis != BackOff.STOP) {
+      cumulativeBackoffMillis += currentBackoffMillis;
+      currentBackoffMillis = backOff.nextBackOffMillis();
+    }
+    return cumulativeBackoffMillis;
+  }
+
+  /**
+   * Tests with bounded interval, custom exponent, limited cumulative time, and a reset.
+   */
+  @Test
+  public void testBoundedIntervalAndCumTimeWithReset() throws Exception {
+    BackOff backOff =
+        FluentBackoff.DEFAULT
+            .withInitialBackoff(Duration.millis(500))
+            .withMaxBackoff(Duration.standardSeconds(1))
+            .withMaxCumulativeBackoff(Duration.standardMinutes(1)).backoff();
+
+    assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis()));
+
+    backOff.reset();
+    assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis()));
+    // sanity check: should get 0 if we don't reset
+    assertThat(countMaximumBackoff(backOff), equalTo(0L));
+
+    backOff.reset();
+    assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis()));
+  }
+
+  /**
+   * Tests with bounded interval, custom exponent, limited cumulative time and retries.
+   */
+  @Test
+  public void testBoundedIntervalAndCumTimeAndRetriesWithReset() throws Exception {
+    BackOff backOff =
+        FluentBackoff.DEFAULT
+            .withInitialBackoff(Duration.millis(500))
+            .withMaxBackoff(Duration.standardSeconds(1))
+            .withMaxCumulativeBackoff(Duration.standardMinutes(1))
+            .backoff();
+
+    long cumulativeBackoffMillis = 0;
+    long currentBackoffMillis = backOff.nextBackOffMillis();
+    while (currentBackoffMillis != BackOff.STOP) {
+      cumulativeBackoffMillis += currentBackoffMillis;
+      currentBackoffMillis = backOff.nextBackOffMillis();
+    }
+    assertThat(cumulativeBackoffMillis, equalTo(Duration.standardMinutes(1).getMillis()));
+  }
+
+  @Test
+  public void testFluentBackoffToString() throws IOException {
+    FluentBackoff config = FluentBackoff.DEFAULT
+        .withExponent(3.4)
+        .withMaxRetries(4)
+        .withInitialBackoff(Duration.standardSeconds(3))
+        .withMaxBackoff(Duration.standardHours(1))
+        .withMaxCumulativeBackoff(Duration.standardDays(1));
+
+    assertEquals(
+        "FluentBackoff{exponent=3.4, initialBackoff=PT3S, maxBackoff=PT3600S,"
+            + " maxRetries=4, maxCumulativeBackoff=PT86400S}",
+        config.toString());
+  }
+  @Test
+  public void testBackoffImplToString() throws IOException {
+    FluentBackoff config = FluentBackoff.DEFAULT
+        .withExponent(3.4)
+        .withMaxRetries(4)
+        .withInitialBackoff(Duration.standardSeconds(3))
+        .withMaxBackoff(Duration.standardHours(1))
+        .withMaxCumulativeBackoff(Duration.standardDays(1));
+    BackOff backOff = config.backoff();
+
+    assertEquals(
+        "BackoffImpl{backoffConfig=" + config.toString() + ","
+            + " currentRetry=0, currentCumulativeBackoff=PT0S}",
+        backOff.toString());
+
+    // backoff once, ignoring result
+    backOff.nextBackOffMillis();
+
+    // currentRetry is exact, we can test it.
+    assertThat(backOff.toString(), containsString("currentRetry=1"));
+    // currentCumulativeBackoff is not exact; we cannot even check that it's non-zero (randomness).
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
index 681b0aa..9504b4c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java
@@ -365,7 +365,7 @@ public class GcsUtilTest {
     Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class);
     Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class);
 
-    BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200);
+    BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff();
 
     when(mockStorage.objects()).thenReturn(mockStorageObjects);
     when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet);
@@ -376,7 +376,7 @@ public class GcsUtilTest {
 
     assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"),
         mockBackOff, new FastNanoClockAndSleeper()));
-    assertEquals(mockBackOff.nextBackOffMillis(), BackOff.STOP);
+    assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis());
   }
 
   @Test
@@ -390,7 +390,7 @@ public class GcsUtilTest {
     Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
     Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
 
-    BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200);
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
 
     when(mockStorage.buckets()).thenReturn(mockStorageObjects);
     when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);
@@ -413,7 +413,7 @@ public class GcsUtilTest {
     Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
     Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
 
-    BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200);
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
     GoogleJsonResponseException expectedException =
         googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN,
             "Waves hand mysteriously", "These aren't the buckets your looking for");
@@ -438,7 +438,7 @@ public class GcsUtilTest {
     Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class);
     Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class);
 
-    BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200);
+    BackOff mockBackOff = FluentBackoff.DEFAULT.backoff();
 
     when(mockStorage.buckets()).thenReturn(mockStorageObjects);
     when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3f485666/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 304dc82..6dde581 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -24,9 +24,6 @@ import static com.google.common.base.Preconditions.checkState;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.api.client.json.JsonFactory;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
 import com.google.api.services.bigquery.Bigquery;
 import com.google.api.services.bigquery.model.Job;
 import com.google.api.services.bigquery.model.JobConfigurationExtract;
@@ -69,7 +66,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -108,7 +104,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
 import org.apache.beam.sdk.util.FileIOChannelFactory;
 import org.apache.beam.sdk.util.GcsIOChannelFactory;
 import org.apache.beam.sdk.util.GcsUtil;
@@ -129,6 +124,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -294,7 +290,7 @@ public class BigQueryIO {
    *
    * <p>If the project id is omitted, the default project id is used.
    */
-  public static TableReference parseTableSpec(String tableSpec) {
+  static TableReference parseTableSpec(String tableSpec) {
     Matcher match = TABLE_SPEC.matcher(tableSpec);
     if (!match.matches()) {
       throw new IllegalArgumentException(
@@ -953,14 +949,14 @@ public class BigQueryIO {
    * ...
    */
   private abstract static class BigQuerySourceBase extends BoundedSource<TableRow> {
-    // The maximum number of attempts to verify temp files.
-    private static final int MAX_FILES_VERIFY_ATTEMPTS = 10;
+    // The maximum number of retries to verify temp files.
+    private static final int MAX_FILES_VERIFY_RETRIES = 9;
 
     // The maximum number of retries to poll a BigQuery job.
     protected static final int JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
 
     // The initial backoff for verifying temp files.
-    private static final long INITIAL_FILES_VERIFY_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1);
+    private static final Duration INITIAL_FILES_VERIFY_BACKOFF = Duration.standardSeconds(1);
 
     protected final String jobIdToken;
     protected final String extractDestinationDir;
@@ -1055,14 +1051,7 @@ public class BigQueryIO {
             }};
 
       List<BoundedSource<TableRow>> avroSources = Lists.newArrayList();
-      BackOff backoff = new AttemptBoundedExponentialBackOff(
-          MAX_FILES_VERIFY_ATTEMPTS, INITIAL_FILES_VERIFY_BACKOFF_MILLIS);
       for (String fileName : files) {
-        while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) {
-          if (IOChannelUtils.getFactory(fileName).getSizeBytes(fileName) != -1) {
-            break;
-          }
-        }
         avroSources.add(new TransformingSource<>(
             AvroSource.from(fileName), function, getDefaultOutputCoder()));
       }


[14/50] [abbrv] incubator-beam git commit: Remove empty unused method in TestStreamEvaluatorFactory

Posted by dh...@apache.org.
Remove empty unused method in TestStreamEvaluatorFactory


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

Branch: refs/heads/gearpump-runner
Commit: 28ad44d2ad5c995044966f8c70f6dcec91343eb6
Parents: 4251761
Author: Thomas Groh <tg...@google.com>
Authored: Wed Aug 31 15:34:21 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/TestStreamEvaluatorFactory.java       | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28ad44d2/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
index 3dbd886..5fe771c 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
@@ -84,13 +84,6 @@ class TestStreamEvaluatorFactory implements TransformEvaluatorFactory {
         .orNull();
   }
 
-  /**
-   * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link
-   * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will
-   * return this evaluator.
-   */
-  private void completeEvaluation(Evaluator<?> evaluator) {}
-
   private static class Evaluator<T> implements TransformEvaluator<Object> {
     private final AppliedPTransform<PBegin, PCollection<T>, TestStream<T>> application;
     private final EvaluationContext context;


[41/50] [abbrv] incubator-beam git commit: Add header/footer support to TextIO.Write

Posted by dh...@apache.org.
Add header/footer support to TextIO.Write


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

Branch: refs/heads/gearpump-runner
Commit: b23559578f2a7acda477b4ebccccb6a6f7c9b03e
Parents: 50c1c88
Author: Stas Levin <st...@gmail.com>
Authored: Mon Sep 5 20:26:12 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 114 ++++++++++++++++---
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  60 +++++++++-
 2 files changed, 152 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b2355957/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 242470b..c754a0b 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
@@ -448,7 +448,15 @@ public class TextIO {
       return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation();
     }
 
-    // TODO: appendingNewlines, header, footer, etc.
+    public static Bound<String> withHeader(String header) {
+      return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header);
+    }
+
+    public static Bound<String> withFooter(String footer) {
+      return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer);
+    }
+
+    // TODO: appendingNewlines, etc.
 
     /**
      * A PTransform that writes a bounded PCollection to a text file (or
@@ -465,6 +473,12 @@ public class TextIO {
       /** The suffix of each file written, combined with prefix and shardTemplate. */
       private final String filenameSuffix;
 
+      /** An optional header to add to each file. */
+      private final String header;
+
+      /** An optional footer to add to each file. */
+      private final String footer;
+
       /** The Coder to use to decode each line. */
       private final Coder<T> coder;
 
@@ -478,12 +492,15 @@ public class TextIO {
       private final boolean validate;
 
       Bound(Coder<T> coder) {
-        this(null, null, "", coder, 0, DEFAULT_SHARD_TEMPLATE, true);
+        this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true);
       }
 
-      private Bound(String name, String filenamePrefix, String filenameSuffix, Coder<T> coder,
-          int numShards, String shardTemplate, boolean validate) {
+      private Bound(String name, String filenamePrefix, String filenameSuffix, String header,
+                    String footer, Coder<T> coder, int numShards, String shardTemplate,
+                    boolean validate) {
         super(name);
+        this.header = header;
+        this.footer = footer;
         this.coder = coder;
         this.filenamePrefix = filenamePrefix;
         this.filenameSuffix = filenameSuffix;
@@ -502,7 +519,7 @@ public class TextIO {
        */
       public Bound<T> to(String filenamePrefix) {
         validateOutputComponent(filenamePrefix);
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
 
@@ -516,7 +533,7 @@ public class TextIO {
        */
       public Bound<T> withSuffix(String nameExtension) {
         validateOutputComponent(nameExtension);
-        return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards,
+        return new Bound<>(name, filenamePrefix, nameExtension, header, footer, coder, numShards,
             shardTemplate, validate);
       }
 
@@ -536,7 +553,7 @@ public class TextIO {
        */
       public Bound<T> withNumShards(int numShards) {
         checkArgument(numShards >= 0);
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
 
@@ -549,7 +566,7 @@ public class TextIO {
        * @see ShardNameTemplate
        */
       public Bound<T> withShardNameTemplate(String shardTemplate) {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
 
@@ -567,7 +584,8 @@ public class TextIO {
        * <p>Does not modify this object.
        */
       public Bound<T> withoutSharding() {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, "", validate);
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, 1, "",
+            validate);
       }
 
       /**
@@ -579,7 +597,7 @@ public class TextIO {
        * @param <X> the type of the elements of the input {@link PCollection}
        */
       public <X> Bound<X> withCoder(Coder<X> coder) {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
 
@@ -594,7 +612,17 @@ public class TextIO {
        * <p>Does not modify this object.
        */
       public Bound<T> withoutValidation() {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
+            shardTemplate, false);
+      }
+
+      public Bound<T> withHeader(String header) {
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
+            shardTemplate, false);
+      }
+
+      public Bound<T> withFooter(String footer) {
+        return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
 
@@ -607,7 +635,8 @@ public class TextIO {
 
         org.apache.beam.sdk.io.Write.Bound<T> write =
             org.apache.beam.sdk.io.Write.to(
-                new TextSink<>(filenamePrefix, filenameSuffix, shardTemplate, coder));
+                new TextSink<>(filenamePrefix, filenameSuffix, header, footer, shardTemplate,
+                    coder));
         if (getNumShards() > 0) {
           write = write.withNumShards(getNumShards());
         }
@@ -629,7 +658,11 @@ public class TextIO {
             .addIfNotDefault(DisplayData.item("validation", validate)
               .withLabel("Validation Enabled"), true)
             .addIfNotDefault(DisplayData.item("numShards", numShards)
-              .withLabel("Maximum Output Shards"), 0);
+              .withLabel("Maximum Output Shards"), 0)
+            .addIfNotNull(DisplayData.item("fileHeader", header)
+              .withLabel("Output file header"))
+            .addIfNotNull(DisplayData.item("fileFooter", footer)
+                .withLabel("Output file footer"));
       }
 
       /**
@@ -664,6 +697,14 @@ public class TextIO {
         return coder;
       }
 
+      public String getHeader() {
+        return header;
+      }
+
+      public String getFooter() {
+        return footer;
+      }
+
       public boolean needsValidation() {
         return validate;
       }
@@ -946,17 +987,22 @@ public class TextIO {
   @VisibleForTesting
   static class TextSink<T> extends FileBasedSink<T> {
     private final Coder<T> coder;
+    private final String header;
+    private final String footer;
 
     @VisibleForTesting
     TextSink(
-        String baseOutputFilename, String extension, String fileNameTemplate, Coder<T> coder) {
+        String baseOutputFilename, String extension, String header, String footer,
+        String fileNameTemplate, Coder<T> coder) {
       super(baseOutputFilename, extension, fileNameTemplate);
       this.coder = coder;
+      this.header = header;
+      this.footer = footer;
     }
 
     @Override
     public FileBasedSink.FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options) {
-      return new TextWriteOperation<>(this, coder);
+      return new TextWriteOperation<>(this, coder, header, footer);
     }
 
     /**
@@ -965,15 +1011,19 @@ public class TextIO {
      */
     private static class TextWriteOperation<T> extends FileBasedWriteOperation<T> {
       private final Coder<T> coder;
+      private final String header;
+      private final String footer;
 
-      private TextWriteOperation(TextSink<T> sink, Coder<T> coder) {
+      private TextWriteOperation(TextSink<T> sink, Coder<T> coder, String header, String footer) {
         super(sink);
         this.coder = coder;
+        this.header = header;
+        this.footer = footer;
       }
 
       @Override
       public FileBasedWriter<T> createWriter(PipelineOptions options) throws Exception {
-        return new TextWriter<>(this, coder);
+        return new TextWriter<>(this, coder, header, footer);
       }
     }
 
@@ -984,20 +1034,50 @@ public class TextIO {
     private static class TextWriter<T> extends FileBasedWriter<T> {
       private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
       private final Coder<T> coder;
+      private final String header;
+      private final String footer;
       private OutputStream out;
 
       public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder) {
+        this(writeOperation, coder, null, null);
+      }
+
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header) {
+        this(writeOperation, coder, header, null);
+      }
+
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header,
+                        String footer) {
         super(writeOperation);
+        this.header = header;
+        this.footer = footer;
         this.mimeType = MimeTypes.TEXT;
         this.coder = coder;
       }
 
+      private void writeLine(String line) throws IOException {
+        if (line != null) {
+          out.write(line.getBytes(StandardCharsets.UTF_8));
+          out.write(NEWLINE);
+        }
+      }
+
       @Override
       protected void prepareWrite(WritableByteChannel channel) throws Exception {
         out = Channels.newOutputStream(channel);
       }
 
       @Override
+      protected void writeHeader() throws Exception {
+        writeLine(header);
+      }
+
+      @Override
+      protected void writeFooter() throws Exception {
+        writeLine(footer);
+      }
+
+      @Override
       public void write(T value) throws Exception {
         coder.encode(value, out, Context.OUTER);
         out.write(NEWLINE);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b2355957/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 8f94766..2ab2683 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -34,6 +34,8 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -48,6 +50,7 @@ import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 import java.util.zip.GZIPOutputStream;
@@ -101,6 +104,9 @@ import org.mockito.stubbing.Answer;
 @SuppressWarnings("unchecked")
 public class TextIOTest {
 
+  private static final String MY_HEADER = "myHeader";
+  private static final String MY_FOOTER = "myFooter";
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule
@@ -216,10 +222,19 @@ public class TextIOTest {
   }
 
   <T> void runTestWrite(T[] elems, Coder<T> coder) throws Exception {
-    runTestWrite(elems, coder, 1);
+    runTestWrite(elems, null, null, coder, 1);
   }
 
   <T> void runTestWrite(T[] elems, Coder<T> coder, int numShards) throws Exception {
+    runTestWrite(elems, null, null, coder, numShards);
+  }
+
+  <T> void runTestWrite(T[] elems, Coder<T> coder, String header, String footer) throws Exception {
+    runTestWrite(elems, header, footer, coder, 1);
+  }
+
+  <T> void runTestWrite(T[] elems, String header, String footer, Coder<T> coder, int numShards)
+      throws Exception {
     String outputName = "file.txt";
     String baseFilename = tmpFolder.newFile(outputName).getPath();
 
@@ -235,6 +250,8 @@ public class TextIOTest {
     } else {
       write = TextIO.Write.to(baseFilename).withCoder(coder);
     }
+    write = write.withHeader(header).withFooter(footer);
+
     if (numShards == 1) {
       write = write.withoutSharding();
     } else if (numShards > 0) {
@@ -244,11 +261,14 @@ public class TextIOTest {
 
     p.run();
 
-    assertOutputFiles(elems, coder, numShards, tmpFolder, outputName, write.getShardNameTemplate());
+    assertOutputFiles(elems, header, footer, coder, numShards, tmpFolder, outputName,
+        write.getShardNameTemplate());
   }
 
   public static <T> void assertOutputFiles(
       T[] elems,
+      String header,
+      String footer,
       Coder<T> coder,
       int numShards,
       TemporaryFolder rootLocation,
@@ -284,15 +304,23 @@ public class TextIOTest {
       }
     }
 
-    String[] expected = new String[elems.length];
+    LinkedList<String> expected = Lists.newLinkedList();
+
     for (int i = 0; i < elems.length; i++) {
       T elem = elems[i];
       byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem);
       String line = new String(encodedElem);
-      expected[i] = line;
+      expected.add(line);
+    }
+
+    if (header != null) {
+      expected.addFirst(header);
+    }
+    if (footer != null) {
+      expected.addLast(footer);
     }
 
-    assertThat(actual, containsInAnyOrder(expected));
+    assertThat(actual, containsInAnyOrder(expected.toArray()));
   }
 
   @Test
@@ -332,18 +360,40 @@ public class TextIOTest {
   }
 
   @Test
+  @Category(NeedsRunner.class)
+  public void testWriteWithHeader() throws Exception {
+    runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, null);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testWriteWithFooter() throws Exception {
+    runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), null, MY_FOOTER);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testWriteWithHeaderAndFooter() throws Exception {
+    runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, MY_FOOTER);
+  }
+
+  @Test
   public void testWriteDisplayData() {
     TextIO.Write.Bound<?> write = TextIO.Write
         .to("foo")
         .withSuffix("bar")
         .withShardNameTemplate("-SS-of-NN-")
         .withNumShards(100)
+        .withFooter("myFooter")
+        .withHeader("myHeader")
         .withoutValidation();
 
     DisplayData displayData = DisplayData.from(write);
 
     assertThat(displayData, hasDisplayItem("filePrefix", "foo"));
     assertThat(displayData, hasDisplayItem("fileSuffix", "bar"));
+    assertThat(displayData, hasDisplayItem("fileHeader", "myHeader"));
+    assertThat(displayData, hasDisplayItem("fileFooter", "myFooter"));
     assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-"));
     assertThat(displayData, hasDisplayItem("numShards", 100));
     assertThat(displayData, hasDisplayItem("validation", false));


[08/50] [abbrv] incubator-beam git commit: Address comments of Flink Side-Input PR

Posted by dh...@apache.org.
Address comments of Flink Side-Input PR


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

Branch: refs/heads/gearpump-runner
Commit: 6ae4b6a3df5cf3b834505fcb3f21df0e90473a0f
Parents: 8007bdf
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Aug 25 11:00:39 2016 +0200
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:11 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/SideInputHandler.java     |  6 +-
 .../apache/beam/runners/flink/FlinkRunner.java  | 86 ++++++++++++++++++--
 .../wrappers/streaming/DoFnOperator.java        | 13 ++-
 .../wrappers/streaming/WindowDoFnOperator.java  |  2 -
 4 files changed, 89 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ae4b6a3/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
index a97d3f3..851ed37 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
@@ -60,7 +60,11 @@ public class SideInputHandler implements ReadyCheckingSideInputReader {
   /** The list of side inputs that we're handling. */
   protected final Collection<PCollectionView<?>> sideInputs;
 
-  /** State internals that are scoped not to the key of a value but instead to one key group. */
+  /**
+   * State internals that are scoped not to the key of a value but are global. The state can still
+   * be keep locally but if side inputs are broadcast to all parallel operators then all will
+   * have the same view of the state.
+   */
   private final StateInternals<Void> stateInternals;
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ae4b6a3/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
index 8b1f42e..d3c65c0 100644
--- 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
@@ -25,8 +25,13 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -35,6 +40,7 @@ import org.apache.beam.sdk.coders.ListCoder;
 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.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -47,6 +53,8 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+
 import org.apache.flink.api.common.JobExecutionResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -108,6 +116,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
 
   private FlinkRunner(FlinkPipelineOptions options) {
     this.options = options;
+    this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
 
     ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
     if (options.isStreaming()) {
@@ -124,6 +133,8 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
 
   @Override
   public FlinkRunnerResult run(Pipeline pipeline) {
+    logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
+
     LOG.info("Executing pipeline using FlinkRunner.");
 
     FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options);
@@ -176,6 +187,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
 
       PTransform<InputT, OutputT> customTransform =
           InstanceBuilder.ofType(customTransformClass)
+              .withArg(FlinkRunner.class, this)
               .withArg(transformClass, transform)
               .build();
 
@@ -223,6 +235,59 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
     return files;
   }
 
+  /** 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);
+  }
+
+  /** 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 Pipeline.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 "
+          + "the Flink runner will require deterministic key coders.",
+          ptransformViewNamesWithNonDeterministicKeyCoders);
+    }
+  }
+
+
+  /////////////////////////////////////////////////////////////////////////////
+
   /**
    * Specialized implementation for
    * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap}
@@ -231,8 +296,11 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
   private static class StreamingViewAsMap<K, V>
       extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
 
+    private final FlinkRunner runner;
+
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsMap(View.AsMap<K, V> transform) {
+    public StreamingViewAsMap(FlinkRunner runner, View.AsMap<K, V> transform) {
+      this.runner = runner;
     }
 
     @Override
@@ -248,7 +316,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
       try {
         inputCoder.getKeyCoder().verifyDeterministic();
       } catch (Coder.NonDeterministicException e) {
-//        runner.recordViewUsesNonDeterministicKeyCoder(this);
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
       }
 
       return input
@@ -270,11 +338,14 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
   private static class StreamingViewAsMultimap<K, V>
       extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
 
+    private final FlinkRunner runner;
+
     /**
      * Builds an instance of this class from the overridden transform.
      */
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsMultimap(View.AsMultimap<K, V> transform) {
+    public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap<K, V> transform) {
+      this.runner = runner;
     }
 
     @Override
@@ -290,7 +361,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
       try {
         inputCoder.getKeyCoder().verifyDeterministic();
       } catch (Coder.NonDeterministicException e) {
-//        runner.recordViewUsesNonDeterministicKeyCoder(this);
+        runner.recordViewUsesNonDeterministicKeyCoder(this);
       }
 
       return input
@@ -315,7 +386,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
      * Builds an instance of this class from the overridden transform.
      */
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsList(View.AsList<T> transform) {}
+    public StreamingViewAsList(FlinkRunner runner, View.AsList<T> transform) {}
 
     @Override
     public PCollectionView<List<T>> apply(PCollection<T> input) {
@@ -346,7 +417,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
      * Builds an instance of this class from the overridden transform.
      */
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsIterable(View.AsIterable<T> transform) { }
+    public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable<T> transform) { }
 
     @Override
     public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
@@ -386,7 +457,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
      * Builds an instance of this class from the overridden transform.
      */
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
-    public StreamingViewAsSingleton(View.AsSingleton<T> transform) {
+    public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton<T> transform) {
       this.transform = transform;
     }
 
@@ -443,6 +514,7 @@ public class FlinkRunner extends PipelineRunner<FlinkRunnerResult> {
      */
     @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply()
     public StreamingCombineGloballyAsSingletonView(
+        FlinkRunner runner,
         Combine.GloballyAsSingletonView<InputT, OutputT> transform) {
       this.transform = transform;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ae4b6a3/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index 000d69f..2c7ebc6 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -75,11 +75,12 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
 
 /**
- * Flink operator for executing {@link DoFn DoFns}.
+ * Flink operator for executing {@link OldDoFn DoFns}.
  *
- * @param <InputT>
- * @param <FnOutputT>
- * @param <OutputT>
+ * @param <InputT> the input type of the {@link OldDoFn}
+ * @param <FnOutputT> the output type of the {@link OldDoFn}
+ * @param <OutputT> the output type of the operator, this can be different from the fn output type when we have
+ *                 side outputs
  */
 public class DoFnOperator<InputT, FnOutputT, OutputT>
     extends AbstractStreamOperator<OutputT>
@@ -95,8 +96,6 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
   protected final Collection<PCollectionView<?>> sideInputs;
   protected final Map<Integer, PCollectionView<?>> sideInputTagMapping;
 
-  protected final boolean hasSideInputs;
-
   protected final WindowingStrategy<?, ?> windowingStrategy;
 
   protected final OutputManagerFactory<OutputT> outputManagerFactory;
@@ -136,8 +135,6 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
     this.windowingStrategy = windowingStrategy;
     this.outputManagerFactory = outputManagerFactory;
 
-    this.hasSideInputs = !sideInputs.isEmpty();
-
     this.pushedBackWatermarkDescriptor =
         new ReducingStateDescriptor<>(
             "pushed-back-elements-watermark-hold",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6ae4b6a3/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index c6dde51..01cfa5b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -201,8 +201,6 @@ public class WindowDoFnOperator<K, InputT, OutputT>
       if (timer != null && timer.f1.getTimestamp().getMillis() < actualInputWatermark) {
         fire = true;
 
-        System.out.println("FIRING: " + timer);
-
         watermarkTimersQueue.remove();
         watermarkTimers.remove(timer);
 


[39/50] [abbrv] incubator-beam git commit: Revised according to comments following a code review.

Posted by dh...@apache.org.
Revised according to comments following a code review.


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

Branch: refs/heads/gearpump-runner
Commit: 1b420dbdc08c3fde53dd5ce0a56260576cee3076
Parents: b235595
Author: Stas Levin <st...@gmail.com>
Authored: Tue Sep 6 23:22:11 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 12 17:40:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 72 +++++++++++--------
 .../java/org/apache/beam/sdk/io/TextIOTest.java | 73 +++++++++++++++-----
 2 files changed, 98 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1b420dbd/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 c754a0b..eefa867 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
@@ -22,6 +22,7 @@ import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -35,12 +36,14 @@ import java.util.regex.Pattern;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.io.Read.Bounded;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
 import org.apache.beam.sdk.values.PBegin;
@@ -474,10 +477,10 @@ public class TextIO {
       private final String filenameSuffix;
 
       /** An optional header to add to each file. */
-      private final String header;
+      private final T header;
 
       /** An optional footer to add to each file. */
-      private final String footer;
+      private final T footer;
 
       /** The Coder to use to decode each line. */
       private final Coder<T> coder;
@@ -495,8 +498,8 @@ public class TextIO {
         this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true);
       }
 
-      private Bound(String name, String filenamePrefix, String filenameSuffix, String header,
-                    String footer, Coder<T> coder, int numShards, String shardTemplate,
+      private Bound(String name, String filenamePrefix, String filenameSuffix, T header,
+                    T footer, Coder<T> coder, int numShards, String shardTemplate,
                     boolean validate) {
         super(name);
         this.header = header;
@@ -509,6 +512,14 @@ public class TextIO {
         this.validate = validate;
       }
 
+      private String asString(T obj, Coder<T> coder) {
+        try {
+          return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj));
+        } catch (CoderException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
       /**
        * Returns a transform for writing to text files that's like this one but
        * that writes to the file(s) with the given filename prefix.
@@ -594,9 +605,8 @@ public class TextIO {
        * the elements of the input {@link PCollection PCollection<X>} into an
        * output text line. Does not modify this object.
        *
-       * @param <X> the type of the elements of the input {@link PCollection}
        */
-      public <X> Bound<X> withCoder(Coder<X> coder) {
+      public Bound<?> withCoder(Coder<? super T> coder) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate);
       }
@@ -616,12 +626,12 @@ public class TextIO {
             shardTemplate, false);
       }
 
-      public Bound<T> withHeader(String header) {
+      public Bound<T> withHeader(T header) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
 
-      public Bound<T> withFooter(String footer) {
+      public Bound<T> withFooter(T footer) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, false);
       }
@@ -659,10 +669,10 @@ public class TextIO {
               .withLabel("Validation Enabled"), true)
             .addIfNotDefault(DisplayData.item("numShards", numShards)
               .withLabel("Maximum Output Shards"), 0)
-            .addIfNotNull(DisplayData.item("fileHeader", header)
-              .withLabel("Output file header"))
-            .addIfNotNull(DisplayData.item("fileFooter", footer)
-                .withLabel("Output file footer"));
+            .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder))
+              .withLabel("File Header"))
+            .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder))
+                .withLabel("File Footer"));
       }
 
       /**
@@ -697,11 +707,11 @@ public class TextIO {
         return coder;
       }
 
-      public String getHeader() {
+      public T getHeader() {
         return header;
       }
 
-      public String getFooter() {
+      public T getFooter() {
         return footer;
       }
 
@@ -987,17 +997,21 @@ public class TextIO {
   @VisibleForTesting
   static class TextSink<T> extends FileBasedSink<T> {
     private final Coder<T> coder;
-    private final String header;
-    private final String footer;
+    private final byte[] header;
+    private final byte[] footer;
 
     @VisibleForTesting
     TextSink(
-        String baseOutputFilename, String extension, String header, String footer,
+        String baseOutputFilename, String extension, T header, T footer,
         String fileNameTemplate, Coder<T> coder) {
       super(baseOutputFilename, extension, fileNameTemplate);
       this.coder = coder;
-      this.header = header;
-      this.footer = footer;
+      try {
+        this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header);
+        this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer);
+      } catch (CoderException e) {
+        throw new RuntimeException(e);
+      }
     }
 
     @Override
@@ -1011,10 +1025,10 @@ public class TextIO {
      */
     private static class TextWriteOperation<T> extends FileBasedWriteOperation<T> {
       private final Coder<T> coder;
-      private final String header;
-      private final String footer;
+      private final byte[] header;
+      private final byte[] footer;
 
-      private TextWriteOperation(TextSink<T> sink, Coder<T> coder, String header, String footer) {
+      private TextWriteOperation(TextSink<T> sink, Coder<T> coder, byte[] header, byte[] footer) {
         super(sink);
         this.coder = coder;
         this.header = header;
@@ -1034,20 +1048,20 @@ public class TextIO {
     private static class TextWriter<T> extends FileBasedWriter<T> {
       private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8);
       private final Coder<T> coder;
-      private final String header;
-      private final String footer;
+      private final byte[] header;
+      private final byte[] footer;
       private OutputStream out;
 
       public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder) {
         this(writeOperation, coder, null, null);
       }
 
-      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header) {
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, byte[] header) {
         this(writeOperation, coder, header, null);
       }
 
-      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, String header,
-                        String footer) {
+      public TextWriter(FileBasedWriteOperation<T> writeOperation, Coder<T> coder, byte[] header,
+                        byte[] footer) {
         super(writeOperation);
         this.header = header;
         this.footer = footer;
@@ -1055,9 +1069,9 @@ public class TextIO {
         this.coder = coder;
       }
 
-      private void writeLine(String line) throws IOException {
+      private void writeLine(byte[] line) throws IOException {
         if (line != null) {
-          out.write(line.getBytes(StandardCharsets.UTF_8));
+          out.write(line);
           out.write(NEWLINE);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1b420dbd/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 2ab2683..7028761 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -33,7 +33,11 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import java.io.BufferedReader;
@@ -229,11 +233,11 @@ public class TextIOTest {
     runTestWrite(elems, null, null, coder, numShards);
   }
 
-  <T> void runTestWrite(T[] elems, Coder<T> coder, String header, String footer) throws Exception {
+  <T> void runTestWrite(T[] elems, Coder<T> coder, T header, T footer) throws Exception {
     runTestWrite(elems, header, footer, coder, 1);
   }
 
-  <T> void runTestWrite(T[] elems, String header, String footer, Coder<T> coder, int numShards)
+  <T> void runTestWrite(T[] elems, T header, T footer, Coder<T> coder, int numShards)
       throws Exception {
     String outputName = "file.txt";
     String baseFilename = tmpFolder.newFile(outputName).getPath();
@@ -248,7 +252,7 @@ public class TextIOTest {
       // T==String
       write = (TextIO.Write.Bound<T>) writeStrings;
     } else {
-      write = TextIO.Write.to(baseFilename).withCoder(coder);
+      write = TextIO.Write.withCoder(coder).to(baseFilename);
     }
     write = write.withHeader(header).withFooter(footer);
 
@@ -267,9 +271,9 @@ public class TextIOTest {
 
   public static <T> void assertOutputFiles(
       T[] elems,
-      String header,
-      String footer,
-      Coder<T> coder,
+      final T header,
+      final T footer,
+      final Coder<T> coder,
       int numShards,
       TemporaryFolder rootLocation,
       String outputName,
@@ -291,36 +295,69 @@ public class TextIOTest {
       }
     }
 
-    List<String> actual = new ArrayList<>();
+    List<List<String>> actual = new ArrayList<>();
+
     for (File tmpFile : expectedFiles) {
       try (BufferedReader reader = new BufferedReader(new FileReader(tmpFile))) {
+        List<String> currentFile = Lists.newArrayList();
         for (;;) {
           String line = reader.readLine();
           if (line == null) {
             break;
           }
-          actual.add(line);
+          currentFile.add(line);
         }
+        actual.add(currentFile);
       }
     }
 
-    LinkedList<String> expected = Lists.newLinkedList();
+    LinkedList<String> expectedElements = Lists.newLinkedList();
 
     for (int i = 0; i < elems.length; i++) {
       T elem = elems[i];
       byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem);
       String line = new String(encodedElem);
-      expected.add(line);
+      expectedElements.add(line);
     }
 
-    if (header != null) {
-      expected.addFirst(header);
-    }
-    if (footer != null) {
-      expected.addLast(footer);
-    }
-
-    assertThat(actual, containsInAnyOrder(expected.toArray()));
+    final String headerString =
+        header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header));
+
+    final String footerString =
+        footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer));
+
+    ArrayList<String> actualElements =
+        Lists.newArrayList(
+            Iterables.concat(
+                FluentIterable
+                    .from(actual)
+                    .transform(new Function<List<String>, List<String>>() {
+                      @Nullable
+                      @Override
+                      public List<String> apply(List<String> lines) {
+                        ArrayList<String> newLines = Lists.newArrayList(lines);
+                        if (headerString != null) {
+                          newLines.remove(0);
+                        }
+                        if (footerString != null) {
+                          int last = newLines.size() - 1;
+                          newLines.remove(last);
+                        }
+                        return newLines;
+                      }
+                    })
+                    .toList()));
+
+    assertThat(actualElements, containsInAnyOrder(expectedElements.toArray()));
+
+    assertTrue(Iterables.all(actual, new Predicate<List<String>>() {
+      @Override
+      public boolean apply(@Nullable List<String> fileLines) {
+        int last = fileLines.size() - 1;
+        return (headerString == null || fileLines.get(0).equals(headerString))
+              && (footerString == null || fileLines.get(last).equals(footerString));
+      }
+    }));
   }
 
   @Test