You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2016/12/21 20:23:09 UTC

[01/50] incubator-beam git commit: Closes #1622

Repository: incubator-beam
Updated Branches:
  refs/heads/gearpump-runner b6e7bb659 -> 4c445dd0b


Closes #1622


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

Branch: refs/heads/gearpump-runner
Commit: 1ee191fb80ed1644b275a87fc02c11eae47904c2
Parents: cc28f0c cfd1633
Author: bchambers <bc...@google.com>
Authored: Fri Dec 16 14:26:35 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Fri Dec 16 14:26:35 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/metrics/CounterCell.java     | 10 +++++++++-
 .../apache/beam/sdk/metrics/DistributionCell.java    | 15 +++++++++++++--
 .../java/org/apache/beam/sdk/metrics/MetricCell.java |  2 +-
 3 files changed, 23 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[49/50] incubator-beam git commit: [BEAM-79] Upgrade to beam-0.5.0-incubating-SNAPSHOT

Posted by ke...@apache.org.
[BEAM-79] Upgrade to beam-0.5.0-incubating-SNAPSHOT


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

Branch: refs/heads/gearpump-runner
Commit: 647034cfc6ee419548b6da222e6d134792366a26
Parents: c2fb7c0
Author: manuzhang <ow...@gmail.com>
Authored: Wed Dec 21 09:32:35 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Wed Dec 21 09:32:35 2016 +0800

----------------------------------------------------------------------
 runners/gearpump/pom.xml                              |  2 +-
 .../runners/gearpump/examples/StreamingWordCount.java | 14 +++++++-------
 .../translators/ParDoBoundMultiTranslator.java        |  2 +-
 .../gearpump/translators/ParDoBoundTranslator.java    |  2 +-
 .../gearpump/translators/TranslationContext.java      |  3 +--
 .../gearpump/translators/utils/DoFnRunnerFactory.java |  2 +-
 .../translators/utils/NoOpAggregatorFactory.java      |  2 +-
 7 files changed, 13 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/runners/gearpump/pom.xml
----------------------------------------------------------------------
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index 9320561..bb35ad7 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.4.0-incubating-SNAPSHOT</version>
+    <version>0.5.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/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 1d85c25..b2d762a 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
@@ -24,7 +24,7 @@ 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.Count;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
@@ -42,10 +42,10 @@ import org.slf4j.LoggerFactory;
  */
 public class StreamingWordCount {
 
-  static class ExtractWordsFn extends OldDoFn<String, String> {
+  static class ExtractWordsFn extends DoFn<String, String> {
 
-    @Override
-    public void processElement(ProcessContext c) {
+    @ProcessElement
+    public void process(ProcessContext c) {
       // Split the line into words.
       String[] words = c.element().split("[^a-zA-Z']+");
 
@@ -58,11 +58,11 @@ public class StreamingWordCount {
     }
   }
 
-  static class FormatAsStringFn extends OldDoFn<KV<String, Long>, String> {
+  static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
     private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class);
 
-    @Override
-    public void processElement(ProcessContext c) {
+    @ProcessElement
+    public void process(ProcessContext c) {
       String row = c.element().getKey()
           + " - " + c.element().getValue()
           + " @ " + c.timestamp().toString();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/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 54f1c3f..24f9734 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
@@ -64,7 +64,7 @@ public class ParDoBoundMultiTranslator<InputT, OutputT> implements
     JavaStream<WindowedValue<KV<TupleTag<OutputT>, OutputT>>> outputStream = inputStream.flatMap(
         new DoFnMultiFunction<>(
             context.getPipelineOptions(),
-            transform.getNewFn(),
+            transform.getFn(),
             transform.getMainOutputTag(),
             transform.getSideOutputTags(),
             inputT.getWindowingStrategy(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java
index a796c83..689bc08 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java
@@ -38,7 +38,7 @@ public class ParDoBoundTranslator<InputT, OutputT> implements
 
   @Override
   public void translate(ParDo.Bound<InputT, OutputT> transform, TranslationContext context) {
-    DoFn<InputT, OutputT> doFn = transform.getNewFn();
+    DoFn<InputT, OutputT> doFn = transform.getFn();
     PCollection<OutputT> output = context.getOutput(transform);
     WindowingStrategy<?, ?> windowingStrategy = output.getWindowingStrategy();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/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 d9d6a8e..63fb619 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
@@ -54,8 +54,7 @@ public class TranslationContext {
   }
 
   public void setCurrentTransform(TransformHierarchy.Node treeNode) {
-    this.currentTransform = AppliedPTransform.of(treeNode.getFullName(),
-        treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform());
+    this.currentTransform = treeNode.toAppliedPTransform();
   }
 
   public GearpumpPipelineOptions getPipelineOptions() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java
index 7119a87..7e1402f 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java
@@ -21,12 +21,12 @@ package org.apache.beam.runners.gearpump.translators.utils;
 import java.io.Serializable;
 import java.util.List;
 
+import org.apache.beam.runners.core.AggregatorFactory;
 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.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.SideInputReader;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/647034cf/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java
index bfc73bf..22ffc4d 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java
@@ -20,8 +20,8 @@ package org.apache.beam.runners.gearpump.translators.utils;
 
 import java.io.Serializable;
 
+import org.apache.beam.runners.core.AggregatorFactory;
 import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.util.ExecutionContext;
 


[36/50] incubator-beam git commit: This closes #1657: Fix seed job fetch spec

Posted by ke...@apache.org.
This closes #1657: Fix seed job fetch spec


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

Branch: refs/heads/gearpump-runner
Commit: 66318d829d4e2a02268b295b1c11297f181ee13e
Parents: d738c68 2148adb
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 11:25:48 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:25:48 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[38/50] incubator-beam git commit: Closes #1638

Posted by ke...@apache.org.
Closes #1638


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

Branch: refs/heads/gearpump-runner
Commit: a972b23302146e686e0f0e141093f899e5e70555
Parents: 66318d8 998cabc
Author: bchambers <bc...@google.com>
Authored: Mon Dec 19 11:30:20 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Mon Dec 19 11:30:20 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                            |  3 +-
 .../beam/runners/direct/DirectRunnerTest.java   | 39 ------------
 runners/flink/runner/pom.xml                    |  6 +-
 runners/google-cloud-dataflow-java/pom.xml      |  3 +-
 runners/spark/pom.xml                           |  3 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |  3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |  5 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    | 24 ++++++++
 .../apache/beam/sdk/metrics/MetricMatchers.java |  4 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    | 63 +++++++++++++++++++-
 10 files changed, 103 insertions(+), 50 deletions(-)
----------------------------------------------------------------------



[20/50] incubator-beam git commit: Change counter name in TestDataflowRunner

Posted by ke...@apache.org.
Change counter name in TestDataflowRunner


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

Branch: refs/heads/gearpump-runner
Commit: 6b055d2debe879816808b4c1ee847e34cc1df5c0
Parents: 1ee191f
Author: Joshua Litt <jo...@google.com>
Authored: Sat Dec 17 11:12:12 2016 -0800
Committer: Joshua Litt <jo...@google.com>
Committed: Sat Dec 17 11:12:12 2016 -0800

----------------------------------------------------------------------
 .../dataflow/testing/TestDataflowRunner.java    | 29 ++++++++++++++++----
 .../testing/TestDataflowRunnerTest.java         | 16 ++++++++++-
 2 files changed, 39 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6b055d2d/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
index 4b0fcf2..0564448 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
@@ -61,7 +61,12 @@ import org.slf4j.LoggerFactory;
  */
 public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   private static final String TENTATIVE_COUNTER = "tentative";
-  private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
+  // See https://issues.apache.org/jira/browse/BEAM-1170
+  // we need to either fix the API or pipe the DRAINED signal through
+  @VisibleForTesting
+  static final String LEGACY_WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
+  @VisibleForTesting
+  static final String WATERMARK_METRIC_SUFFIX = "DataWatermark";
   private static final long MAX_WATERMARK_VALUE = -2L;
   private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class);
 
@@ -248,6 +253,23 @@ public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   }
 
   /**
+   * Checks wether a metric is a streaming watermark.
+   *
+   * @return true if the metric is a watermark.
+   */
+  boolean isWatermark(MetricUpdate metric) {
+    if (metric.getName() == null || metric.getName().getName() == null) {
+      return false; // no name -> shouldn't happen, not the watermark
+    }
+    if (metric.getScalar() == null) {
+      return false; // no scalar value -> not the watermark
+    }
+    String name = metric.getName().getName();
+    return name.endsWith(LEGACY_WATERMARK_METRIC_SUFFIX)
+        || name.endsWith(WATERMARK_METRIC_SUFFIX);
+  }
+
+  /**
    * Check watermarks of the streaming job. At least one watermark metric must exist.
    *
    * @return true if all watermarks are at max, false otherwise.
@@ -256,10 +278,7 @@ public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   boolean atMaxWatermark(DataflowPipelineJob job, JobMetrics metrics) {
     boolean hasMaxWatermark = false;
     for (MetricUpdate metric : metrics.getMetrics()) {
-      if (metric.getName() == null
-          || metric.getName().getName() == null
-          || !metric.getName().getName().endsWith(WATERMARK_METRIC_SUFFIX)
-          || metric.getScalar() == null) {
+      if (!isWatermark(metric)) {
         continue;
       }
       BigDecimal watermark = (BigDecimal) metric.getScalar();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6b055d2d/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
index 366c6a1..da5630b 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.dataflow.testing;
 
+import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.LEGACY_WATERMARK_METRIC_SUFFIX;
+import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.WATERMARK_METRIC_SUFFIX;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -95,7 +97,6 @@ public class TestDataflowRunnerTest {
   @Mock private MockLowLevelHttpRequest request;
   @Mock private GcsUtil mockGcsUtil;
 
-  private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
   private static final BigDecimal DEFAULT_MAX_WATERMARK = new BigDecimal(-2);
 
   private TestDataflowPipelineOptions options;
@@ -411,6 +412,19 @@ public class TestDataflowRunnerTest {
   }
 
   @Test
+  public void testCheckMaxWatermarkWithLegacyWatermarkAtMax() throws IOException {
+    DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null));
+    Pipeline p = TestPipeline.create(options);
+    p.apply(Create.of(1, 2, 3));
+
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
+    JobMetrics metrics = buildJobMetrics(generateMockStreamingMetrics(
+        ImmutableMap.of(LEGACY_WATERMARK_METRIC_SUFFIX, DEFAULT_MAX_WATERMARK)));
+    doReturn(State.RUNNING).when(job).getState();
+    assertTrue(runner.atMaxWatermark(job, metrics));
+  }
+
+  @Test
   public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOException {
     DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null));
     Pipeline p = TestPipeline.create(options);


[28/50] incubator-beam git commit: [BEAM-716] This closes #1577

Posted by ke...@apache.org.
[BEAM-716] This closes #1577


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

Branch: refs/heads/gearpump-runner
Commit: 1e148cd7d5f12e6742ac57440bf0731460d11b80
Parents: 1c9bf8d 30e14cf
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Mon Dec 19 07:40:39 2016 +0100
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Mon Dec 19 07:40:39 2016 +0100

----------------------------------------------------------------------
 sdks/java/io/jms/pom.xml                        |   7 +
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 338 +++++++++++++------
 2 files changed, 244 insertions(+), 101 deletions(-)
----------------------------------------------------------------------



[46/50] incubator-beam git commit: Closes #1656

Posted by ke...@apache.org.
Closes #1656


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

Branch: refs/heads/gearpump-runner
Commit: b3de17b3d1a394563d680af9ac34ecfe801c25c2
Parents: 28d7913 85422f9
Author: Dan Halperin <dh...@google.com>
Authored: Mon Dec 19 16:24:09 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Dec 19 16:24:09 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------



[32/50] incubator-beam git commit: This closes #1655: [BEAM-1178] Make naming of logger objects consistent

Posted by ke...@apache.org.
This closes #1655: [BEAM-1178] Make naming of logger objects consistent

  [BEAM-1178] Make naming of logger objects consistent
  Fix grammar error (repeated for)


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

Branch: refs/heads/gearpump-runner
Commit: e68a70e08c9fe00df9ec163d1532da130f69588a
Parents: 203b7b1 4cf5917
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 09:44:13 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 09:44:13 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |  4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |  8 +--
 .../beam/sdk/metrics/MetricsEnvironment.java    |  6 +--
 .../sdk/transforms/reflect/DoFnSignatures.java  |  2 +-
 .../org/apache/beam/sdk/util/ApiSurface.java    | 52 ++++++++++----------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    | 24 ++++-----
 .../io/gcp/bigtable/BigtableServiceImpl.java    |  6 +--
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   | 12 ++---
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |  8 +--
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  8 +--
 11 files changed, 67 insertions(+), 67 deletions(-)
----------------------------------------------------------------------



[02/50] incubator-beam git commit: Make {Metric, Counter, Distribution}Cell public

Posted by ke...@apache.org.
Make {Metric,Counter,Distribution}Cell public

These classes may be useful to hold on to within a runner where the step
context is known to be unchanging and there may be a desire to avoid the
extra indirection of using {Counter,Distribution} directly.


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

Branch: refs/heads/gearpump-runner
Commit: cfd1633a441f98d100bd995bd811b6b8fa2cd62d
Parents: cc28f0c
Author: bchambers <bc...@google.com>
Authored: Wed Dec 14 15:46:50 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Fri Dec 16 14:26:35 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/metrics/CounterCell.java     | 10 +++++++++-
 .../apache/beam/sdk/metrics/DistributionCell.java    | 15 +++++++++++++--
 .../java/org/apache/beam/sdk/metrics/MetricCell.java |  2 +-
 3 files changed, 23 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cfd1633a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java
index bb65833..93700e6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java
@@ -26,13 +26,21 @@ import org.apache.beam.sdk.annotations.Experimental.Kind;
  *
  * <p>This class generally shouldn't be used directly. The only exception is within a runner where
  * a counter is being reported for a specific step (rather than the counter in the current context).
+ * In that case retrieving the underlying cell and reporting directly to it avoids a step of
+ * indirection.
  */
 @Experimental(Kind.METRICS)
-class CounterCell implements MetricCell<Counter, Long>, Counter {
+public class CounterCell implements MetricCell<Counter, Long>, Counter {
 
   private final DirtyState dirty = new DirtyState();
   private final AtomicLong value = new AtomicLong();
 
+  /**
+   * Package-visibility because all {@link CounterCell CounterCells} should be created by
+   * {@link MetricsContainer#getCounter(MetricName)}.
+   */
+  CounterCell() {}
+
   /** Increment the counter by the given amount. */
   private void add(long n) {
     value.addAndGet(n);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cfd1633a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
index f0074a9..e095158 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
@@ -23,14 +23,25 @@ import org.apache.beam.sdk.annotations.Experimental.Kind;
 
 /**
  * Tracks the current value (and delta) for a Distribution metric.
+ *
+ * <p>This class generally shouldn't be used directly. The only exception is within a runner where
+ * a distribution is being reported for a specific step (rather than the distribution in the current
+ * context). In that case retrieving the underlying cell and reporting directly to it avoids a step
+ * of indirection.
  */
 @Experimental(Kind.METRICS)
-class DistributionCell implements MetricCell<Distribution, DistributionData>, Distribution {
+public class DistributionCell implements MetricCell<Distribution, DistributionData>, Distribution {
 
   private final DirtyState dirty = new DirtyState();
   private final AtomicReference<DistributionData> value =
       new AtomicReference<DistributionData>(DistributionData.EMPTY);
 
+  /**
+   * Package-visibility because all {@link DistributionCell DistributionCells} should be created by
+   * {@link MetricsContainer#getDistribution(MetricName)}.
+   */
+  DistributionCell() {}
+
   /** Increment the counter by the given amount. */
   @Override
   public void update(long n) {
@@ -55,4 +66,4 @@ class DistributionCell implements MetricCell<Distribution, DistributionData>, Di
   public Distribution getInterface() {
     return this;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cfd1633a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java
index 211b2dd..7cf9710 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java
@@ -28,7 +28,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind;
  * @param <DataT> The type of metric data stored (and extracted) from this cell.
  */
 @Experimental(Kind.METRICS)
-interface MetricCell<UserT extends Metric, DataT> {
+public interface MetricCell<UserT extends Metric, DataT> {
 
   /**
    * Return the {@link DirtyState} tracking whether this metric cell contains uncommitted changes.


[05/50] incubator-beam git commit: Revert "Move InMemoryTimerInternals to runners-core"

Posted by ke...@apache.org.
Revert "Move InMemoryTimerInternals to runners-core"

This reverts commit ec0bf7b4023ff75f4ec6723d2e77ed507eb57c51.


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

Branch: refs/heads/gearpump-runner
Commit: 45ed5c70c18a806d0fc2e7385886285206fd18e4
Parents: 954e57d
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 16 16:33:51 2016 -0800
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Fri Dec 16 16:39:20 2016 -0800

----------------------------------------------------------------------
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   1 +
 .../runners/core/InMemoryTimerInternals.java    | 276 -------------------
 .../core/InMemoryTimerInternalsTest.java        | 155 -----------
 .../beam/runners/core/ReduceFnTester.java       |   1 +
 .../beam/runners/core/SplittableParDoTest.java  |  16 +-
 .../triggers/TriggerStateMachineTester.java     |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |   2 +-
 .../apache/beam/sdk/transforms/DoFnTester.java  |  36 +++
 .../sdk/util/state/InMemoryTimerInternals.java  | 275 ++++++++++++++++++
 .../util/state/InMemoryTimerInternalsTest.java  | 153 ++++++++++
 10 files changed, 471 insertions(+), 446 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/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
index efcd771..9189191 100644
--- 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
@@ -27,6 +27,7 @@ import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
deleted file mode 100644
index b22fcb3..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.apache.beam.sdk.util.WindowTracing;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.joda.time.Instant;
-
-/**
- * Simulates the firing of timers and progression of input and output watermarks for a single
- * computation and key in a Windmill-like streaming environment.
- */
-public class InMemoryTimerInternals 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);
-
-  /** Pending synchronized processing time timers, in timestamp order. */
-  private PriorityQueue<TimerData> synchronizedProcessingTimers = new PriorityQueue<>(11);
-
-  /** Current input watermark. */
-  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. */
-  private Instant synchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
-  @Override
-  @Nullable
-  public Instant currentOutputWatermarkTime() {
-    return outputWatermarkTime;
-  }
-
-  /**
-   * Returns when the next timer in the given time domain will fire, or {@code null}
-   * if there are no timers scheduled in that time domain.
-   */
-  @Nullable
-  public Instant getNextTimer(TimeDomain domain) {
-    final TimerData data;
-    switch (domain) {
-      case EVENT_TIME:
-        data = watermarkTimers.peek();
-        break;
-      case PROCESSING_TIME:
-        data = processingTimers.peek();
-        break;
-      case SYNCHRONIZED_PROCESSING_TIME:
-        data = synchronizedProcessingTimers.peek();
-        break;
-      default:
-        throw new IllegalArgumentException("Unexpected time domain: " + domain);
-    }
-    return (data == null) ? null : data.getTimestamp();
-  }
-
-  private PriorityQueue<TimerData> queue(TimeDomain domain) {
-    switch (domain) {
-      case EVENT_TIME:
-        return watermarkTimers;
-      case PROCESSING_TIME:
-        return processingTimers;
-      case SYNCHRONIZED_PROCESSING_TIME:
-        return synchronizedProcessingTimers;
-      default:
-        throw new IllegalArgumentException("Unexpected time domain: " + domain);
-    }
-  }
-
-  @Override
-  public void setTimer(StateNamespace namespace, String timerId, Instant target,
-      TimeDomain timeDomain) {
-    throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
-  }
-
-  @Override
-  public void setTimer(TimerData timerData) {
-    WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData);
-    if (existingTimers.add(timerData)) {
-      queue(timerData.getDomain()).add(timerData);
-    }
-  }
-
-  @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
-    throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported.");
-  }
-
-  @Override
-  public void deleteTimer(TimerData timer) {
-    WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), 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 inputWatermarkTime;
-  }
-
-  @Override
-  public String toString() {
-    return MoreObjects.toStringHelper(getClass())
-        .add("watermarkTimers", watermarkTimers)
-        .add("processingTimers", processingTimers)
-        .add("synchronizedProcessingTimers", synchronizedProcessingTimers)
-        .add("inputWatermarkTime", inputWatermarkTime)
-        .add("outputWatermarkTime", outputWatermarkTime)
-        .add("processingTime", processingTime)
-        .toString();
-  }
-
-  /** Advances input watermark to the given value. */
-  public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
-    checkNotNull(newInputWatermark);
-    checkState(
-        !newInputWatermark.isBefore(inputWatermarkTime),
-        "Cannot move input watermark time backwards from %s to %s",
-        inputWatermarkTime,
-        newInputWatermark);
-    WindowTracing.trace(
-        "{}.advanceInputWatermark: from {} to {}",
-        getClass().getSimpleName(), inputWatermarkTime, newInputWatermark);
-    inputWatermarkTime = newInputWatermark;
-  }
-
-  /** Advances output watermark to the given value. */
-  public void advanceOutputWatermark(Instant newOutputWatermark) {
-    checkNotNull(newOutputWatermark);
-    final Instant adjustedOutputWatermark;
-    if (newOutputWatermark.isAfter(inputWatermarkTime)) {
-      WindowTracing.trace(
-          "{}.advanceOutputWatermark: clipping output watermark from {} to {}",
-          getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime);
-      adjustedOutputWatermark = inputWatermarkTime;
-    } else {
-      adjustedOutputWatermark = newOutputWatermark;
-    }
-
-    checkState(
-        outputWatermarkTime == null || !adjustedOutputWatermark.isBefore(outputWatermarkTime),
-        "Cannot move output watermark time backwards from %s to %s",
-        outputWatermarkTime,
-        adjustedOutputWatermark);
-    WindowTracing.trace(
-        "{}.advanceOutputWatermark: from {} to {}",
-        getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark);
-    outputWatermarkTime = adjustedOutputWatermark;
-  }
-
-  /** Advances processing time to the given value. */
-  public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
-    checkNotNull(newProcessingTime);
-    checkState(
-        !newProcessingTime.isBefore(processingTime),
-        "Cannot move processing time backwards from %s to %s",
-        processingTime,
-        newProcessingTime);
-    WindowTracing.trace(
-        "{}.advanceProcessingTime: from {} to {}",
-        getClass().getSimpleName(), processingTime, newProcessingTime);
-    processingTime = newProcessingTime;
-  }
-
-  /** Advances synchronized processing time to the given value. */
-  public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime)
-      throws Exception {
-    checkNotNull(newSynchronizedProcessingTime);
-    checkState(
-        !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
-        "Cannot move processing time backwards from %s to %s",
-        synchronizedProcessingTime,
-        newSynchronizedProcessingTime);
-    WindowTracing.trace(
-        "{}.advanceProcessingTime: from {} to {}",
-        getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime);
-    synchronizedProcessingTime = newSynchronizedProcessingTime;
-  }
-
-  /** Returns the next eligible event time timer, if none returns null. */
-  @Nullable
-  public TimerData removeNextEventTimer() {
-    TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME);
-    if (timer != null) {
-      WindowTracing.trace(
-          "{}.removeNextEventTimer: firing {} at {}",
-          getClass().getSimpleName(), timer, inputWatermarkTime);
-    }
-    return timer;
-  }
-
-  /** Returns the next eligible processing time timer, if none returns null. */
-  @Nullable
-  public TimerData removeNextProcessingTimer() {
-    TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME);
-    if (timer != null) {
-      WindowTracing.trace(
-          "{}.removeNextProcessingTimer: firing {} at {}",
-          getClass().getSimpleName(), timer, processingTime);
-    }
-    return timer;
-  }
-
-  /** Returns the next eligible synchronized processing time timer, if none returns null. */
-  @Nullable
-  public TimerData removeNextSynchronizedProcessingTimer() {
-    TimerData timer = removeNextTimer(
-        synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    if (timer != null) {
-      WindowTracing.trace(
-          "{}.removeNextSynchronizedProcessingTimer: firing {} at {}",
-          getClass().getSimpleName(), timer, synchronizedProcessingTime);
-    }
-    return timer;
-  }
-
-  @Nullable
-  private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) {
-    PriorityQueue<TimerData> queue = queue(domain);
-    if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) {
-      TimerData timer = queue.remove();
-      existingTimers.remove(timer);
-      return timer;
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
deleted file mode 100644
index 2caa874..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.core;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-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.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link InMemoryTimerInternals}.
- */
-@RunWith(JUnit4.class)
-public class InMemoryTimerInternalsTest {
-
-  private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
-
-  @Test
-  public void testFiringTimers() throws Exception {
-    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
-    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(new Instant(20));
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-
-    // Advancing just a little shouldn't refire
-    underTest.advanceProcessingTime(new Instant(21));
-    assertNull(underTest.removeNextProcessingTimer());
-
-    // Adding the timer and advancing a little should refire
-    underTest.setTimer(processingTime1);
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-
-    // And advancing the rest of the way should still have the other timer
-    underTest.advanceProcessingTime(new Instant(30));
-    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-  }
-
-  @Test
-  public void testFiringTimersWithCallback() throws Exception {
-    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
-    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(new Instant(20));
-    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1));
-    assertThat(underTest.removeNextProcessingTimer(), nullValue());
-
-    // Advancing just a little shouldn't refire
-    underTest.advanceProcessingTime(new Instant(21));
-    assertThat(underTest.removeNextProcessingTimer(), nullValue());
-
-    // Adding the timer and advancing a little should fire again
-    underTest.setTimer(processingTime1);
-    underTest.advanceProcessingTime(new Instant(21));
-    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1));
-    assertThat(underTest.removeNextProcessingTimer(), nullValue());
-
-    // And advancing the rest of the way should still have the other timer
-    underTest.advanceProcessingTime(new Instant(30));
-    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2));
-    assertThat(underTest.removeNextProcessingTimer(), nullValue());
-  }
-
-  @Test
-  public void testTimerOrdering() throws Exception {
-    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
-    TimerData eventTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
-    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTime1 = TimerData.of(
-        NS1, new Instant(19), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-    TimerData eventTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
-    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
-    TimerData synchronizedProcessingTime2 = TimerData.of(
-        NS1, new Instant(29), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
-
-    underTest.setTimer(processingTime1);
-    underTest.setTimer(eventTime1);
-    underTest.setTimer(synchronizedProcessingTime1);
-    underTest.setTimer(processingTime2);
-    underTest.setTimer(eventTime2);
-    underTest.setTimer(synchronizedProcessingTime2);
-
-    assertNull(underTest.removeNextEventTimer());
-    underTest.advanceInputWatermark(new Instant(30));
-    assertEquals(eventTime1, underTest.removeNextEventTimer());
-    assertEquals(eventTime2, underTest.removeNextEventTimer());
-    assertNull(underTest.removeNextEventTimer());
-
-    assertNull(underTest.removeNextProcessingTimer());
-    underTest.advanceProcessingTime(new Instant(30));
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-
-    assertNull(underTest.removeNextSynchronizedProcessingTimer());
-    underTest.advanceSynchronizedProcessingTime(new Instant(30));
-    assertEquals(synchronizedProcessingTime1, underTest.removeNextSynchronizedProcessingTimer());
-    assertEquals(synchronizedProcessingTime2, underTest.removeNextSynchronizedProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-  }
-
-  @Test
-  public void testDeduplicate() throws Exception {
-    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
-    TimerData eventTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
-    TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    underTest.setTimer(eventTime);
-    underTest.setTimer(eventTime);
-    underTest.setTimer(processingTime);
-    underTest.setTimer(processingTime);
-    underTest.advanceProcessingTime(new Instant(20));
-    underTest.advanceInputWatermark(new Instant(20));
-
-    assertEquals(processingTime, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-    assertEquals(eventTime, underTest.removeNextEventTimer());
-    assertNull(underTest.removeNextEventTimer());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/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
index 890195a..db0cf91 100644
--- 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
@@ -73,6 +73,7 @@ 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.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
 import org.apache.beam.sdk.util.state.StateTag;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
index 41d419b..cf96b66 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
@@ -28,7 +28,6 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -191,8 +190,6 @@ public class SplittableParDoTest {
         tester;
     private Instant currentProcessingTime;
 
-    private InMemoryTimerInternals timerInternals;
-
     ProcessFnTester(
         Instant currentProcessingTime,
         DoFn<InputT, OutputT> fn,
@@ -203,7 +200,6 @@ public class SplittableParDoTest {
           new SplittableParDo.ProcessFn<>(
               fn, inputCoder, restrictionCoder, IntervalWindow.getCoder());
       this.tester = DoFnTester.of(processFn);
-      this.timerInternals = new InMemoryTimerInternals();
       processFn.setStateInternalsFactory(
           new StateInternalsFactory<String>() {
             @Override
@@ -215,7 +211,7 @@ public class SplittableParDoTest {
           new TimerInternalsFactory<String>() {
             @Override
             public TimerInternals timerInternalsForKey(String key) {
-              return timerInternals;
+              return tester.getTimerInternals();
             }
           });
       processFn.setOutputWindowedValue(
@@ -251,7 +247,7 @@ public class SplittableParDoTest {
       // through the state/timer/output callbacks.
       this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE);
       this.tester.startBundle();
-      timerInternals.advanceProcessingTime(currentProcessingTime);
+      this.tester.advanceProcessingTime(currentProcessingTime);
 
       this.currentProcessingTime = currentProcessingTime;
     }
@@ -289,13 +285,7 @@ public class SplittableParDoTest {
      */
     boolean advanceProcessingTimeBy(Duration duration) throws Exception {
       currentProcessingTime = currentProcessingTime.plus(duration);
-      timerInternals.advanceProcessingTime(currentProcessingTime);
-
-      List<TimerInternals.TimerData> timers = new ArrayList<>();
-      TimerInternals.TimerData nextTimer;
-      while ((nextTimer = timerInternals.removeNextProcessingTimer()) != null) {
-        timers.add(nextTimer);
-      }
+      List<TimerInternals.TimerData> timers = tester.advanceProcessingTime(currentProcessingTime);
       if (timers.isEmpty()) {
         return false;
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
index 2a626d4..be63c06 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
@@ -34,7 +34,6 @@ import java.util.Set;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.core.ActiveWindowSet;
 import org.apache.beam.runners.core.ActiveWindowSet.MergeCallback;
-import org.apache.beam.runners.core.InMemoryTimerInternals;
 import org.apache.beam.runners.core.MergingActiveWindowSet;
 import org.apache.beam.runners.core.NonMergingActiveWindowSet;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -47,6 +46,7 @@ import org.apache.beam.sdk.util.Timers;
 import org.apache.beam.sdk.util.WindowTracing;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java
index 5432d58..87d3f50 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import org.apache.beam.runners.core.GroupAlsoByWindowsDoFn;
-import org.apache.beam.runners.core.InMemoryTimerInternals;
 import org.apache.beam.runners.core.OutputWindowedValue;
 import org.apache.beam.runners.core.ReduceFnRunner;
 import org.apache.beam.runners.core.SystemReduceFn;
@@ -38,6 +37,7 @@ import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/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 2d8684a..93b3f59 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
@@ -46,10 +46,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.Timer;
+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.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -141,6 +143,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return (StateInternals<K>) stateInternals;
   }
 
+  public TimerInternals getTimerInternals() {
+    return timerInternals;
+  }
+
   /**
    * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage
    * the lifecycle of the {@link DoFn}.
@@ -227,6 +233,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     context.setupDelegateAggregators();
     // State and timer internals are per-bundle.
     stateInternals = InMemoryStateInternals.forKey(new Object());
+    timerInternals = new InMemoryTimerInternals();
     try {
       fnInvoker.invokeStartBundle(context);
     } catch (UserCodeException e) {
@@ -535,6 +542,34 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return extractAggregatorValue(agg.getName(), agg.getCombineFn());
   }
 
+  public List<TimerInternals.TimerData> advanceInputWatermark(Instant newWatermark) {
+    try {
+      timerInternals.advanceInputWatermark(newWatermark);
+      final List<TimerInternals.TimerData> firedTimers = new ArrayList<>();
+      TimerInternals.TimerData timer;
+      while ((timer = timerInternals.removeNextEventTimer()) != null) {
+        firedTimers.add(timer);
+      }
+      return firedTimers;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public List<TimerInternals.TimerData> advanceProcessingTime(Instant newProcessingTime) {
+    try {
+      timerInternals.advanceProcessingTime(newProcessingTime);
+      final List<TimerInternals.TimerData> firedTimers = new ArrayList<>();
+      TimerInternals.TimerData timer;
+      while ((timer = timerInternals.removeNextProcessingTimer()) != null) {
+        firedTimers.add(timer);
+      }
+      return firedTimers;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
   private <AccumT, AggregateT> AggregateT extractAggregatorValue(
       String name, CombineFn<?, AccumT, AggregateT> combiner) {
     @SuppressWarnings("unchecked")
@@ -779,6 +814,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   private Map<TupleTag<?>, List<ValueInSingleWindow<?>>> outputs;
 
   private InMemoryStateInternals<?> stateInternals;
+  private InMemoryTimerInternals timerInternals;
 
   /** The state of processing of the {@link DoFn} under test. */
   private State state = State.UNINITIALIZED;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
new file mode 100644
index 0000000..44b44f0
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.state;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+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.apache.beam.sdk.util.WindowTracing;
+import org.joda.time.Instant;
+
+/**
+ * Simulates the firing of timers and progression of input and output watermarks for a single
+ * computation and key in a Windmill-like streaming environment.
+ */
+public class InMemoryTimerInternals 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);
+
+  /** Pending synchronized processing time timers, in timestamp order. */
+  private PriorityQueue<TimerData> synchronizedProcessingTimers = new PriorityQueue<>(11);
+
+  /** Current input watermark. */
+  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. */
+  private Instant synchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+  @Override
+  @Nullable
+  public Instant currentOutputWatermarkTime() {
+    return outputWatermarkTime;
+  }
+
+  /**
+   * Returns when the next timer in the given time domain will fire, or {@code null}
+   * if there are no timers scheduled in that time domain.
+   */
+  @Nullable
+  public Instant getNextTimer(TimeDomain domain) {
+    final TimerData data;
+    switch (domain) {
+      case EVENT_TIME:
+        data = watermarkTimers.peek();
+        break;
+      case PROCESSING_TIME:
+        data = processingTimers.peek();
+        break;
+      case SYNCHRONIZED_PROCESSING_TIME:
+        data = synchronizedProcessingTimers.peek();
+        break;
+      default:
+        throw new IllegalArgumentException("Unexpected time domain: " + domain);
+    }
+    return (data == null) ? null : data.getTimestamp();
+  }
+
+  private PriorityQueue<TimerData> queue(TimeDomain domain) {
+    switch (domain) {
+      case EVENT_TIME:
+        return watermarkTimers;
+      case PROCESSING_TIME:
+        return processingTimers;
+      case SYNCHRONIZED_PROCESSING_TIME:
+        return synchronizedProcessingTimers;
+      default:
+        throw new IllegalArgumentException("Unexpected time domain: " + domain);
+    }
+  }
+
+  @Override
+  public void setTimer(StateNamespace namespace, String timerId, Instant target,
+      TimeDomain timeDomain) {
+    throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
+  }
+
+  @Override
+  public void setTimer(TimerData timerData) {
+    WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData);
+    if (existingTimers.add(timerData)) {
+      queue(timerData.getDomain()).add(timerData);
+    }
+  }
+
+  @Override
+  public void deleteTimer(StateNamespace namespace, String timerId) {
+    throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported.");
+  }
+
+  @Override
+  public void deleteTimer(TimerData timer) {
+    WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), 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 inputWatermarkTime;
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(getClass())
+        .add("watermarkTimers", watermarkTimers)
+        .add("processingTimers", processingTimers)
+        .add("synchronizedProcessingTimers", synchronizedProcessingTimers)
+        .add("inputWatermarkTime", inputWatermarkTime)
+        .add("outputWatermarkTime", outputWatermarkTime)
+        .add("processingTime", processingTime)
+        .toString();
+  }
+
+  /** Advances input watermark to the given value. */
+  public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
+    checkNotNull(newInputWatermark);
+    checkState(
+        !newInputWatermark.isBefore(inputWatermarkTime),
+        "Cannot move input watermark time backwards from %s to %s",
+        inputWatermarkTime,
+        newInputWatermark);
+    WindowTracing.trace(
+        "{}.advanceInputWatermark: from {} to {}",
+        getClass().getSimpleName(), inputWatermarkTime, newInputWatermark);
+    inputWatermarkTime = newInputWatermark;
+  }
+
+  /** Advances output watermark to the given value. */
+  public void advanceOutputWatermark(Instant newOutputWatermark) {
+    checkNotNull(newOutputWatermark);
+    final Instant adjustedOutputWatermark;
+    if (newOutputWatermark.isAfter(inputWatermarkTime)) {
+      WindowTracing.trace(
+          "{}.advanceOutputWatermark: clipping output watermark from {} to {}",
+          getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime);
+      adjustedOutputWatermark = inputWatermarkTime;
+    } else {
+      adjustedOutputWatermark = newOutputWatermark;
+    }
+
+    checkState(
+        outputWatermarkTime == null || !adjustedOutputWatermark.isBefore(outputWatermarkTime),
+        "Cannot move output watermark time backwards from %s to %s",
+        outputWatermarkTime,
+        adjustedOutputWatermark);
+    WindowTracing.trace(
+        "{}.advanceOutputWatermark: from {} to {}",
+        getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark);
+    outputWatermarkTime = adjustedOutputWatermark;
+  }
+
+  /** Advances processing time to the given value. */
+  public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
+    checkNotNull(newProcessingTime);
+    checkState(
+        !newProcessingTime.isBefore(processingTime),
+        "Cannot move processing time backwards from %s to %s",
+        processingTime,
+        newProcessingTime);
+    WindowTracing.trace(
+        "{}.advanceProcessingTime: from {} to {}",
+        getClass().getSimpleName(), processingTime, newProcessingTime);
+    processingTime = newProcessingTime;
+  }
+
+  /** Advances synchronized processing time to the given value. */
+  public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime)
+      throws Exception {
+    checkNotNull(newSynchronizedProcessingTime);
+    checkState(
+        !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
+        "Cannot move processing time backwards from %s to %s",
+        synchronizedProcessingTime,
+        newSynchronizedProcessingTime);
+    WindowTracing.trace(
+        "{}.advanceProcessingTime: from {} to {}",
+        getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime);
+    synchronizedProcessingTime = newSynchronizedProcessingTime;
+  }
+
+  /** Returns the next eligible event time timer, if none returns null. */
+  @Nullable
+  public TimerData removeNextEventTimer() {
+    TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME);
+    if (timer != null) {
+      WindowTracing.trace(
+          "{}.removeNextEventTimer: firing {} at {}",
+          getClass().getSimpleName(), timer, inputWatermarkTime);
+    }
+    return timer;
+  }
+
+  /** Returns the next eligible processing time timer, if none returns null. */
+  @Nullable
+  public TimerData removeNextProcessingTimer() {
+    TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME);
+    if (timer != null) {
+      WindowTracing.trace(
+          "{}.removeNextProcessingTimer: firing {} at {}",
+          getClass().getSimpleName(), timer, processingTime);
+    }
+    return timer;
+  }
+
+  /** Returns the next eligible synchronized processing time timer, if none returns null. */
+  @Nullable
+  public TimerData removeNextSynchronizedProcessingTimer() {
+    TimerData timer = removeNextTimer(
+        synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    if (timer != null) {
+      WindowTracing.trace(
+          "{}.removeNextSynchronizedProcessingTimer: firing {} at {}",
+          getClass().getSimpleName(), timer, synchronizedProcessingTime);
+    }
+    return timer;
+  }
+
+  @Nullable
+  private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) {
+    PriorityQueue<TimerData> queue = queue(domain);
+    if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) {
+      TimerData timer = queue.remove();
+      existingTimers.remove(timer);
+      return timer;
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/45ed5c70/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java
new file mode 100644
index 0000000..4a2763c
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.state;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link InMemoryTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class InMemoryTimerInternalsTest {
+
+  private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
+
+  @Test
+  public void testFiringTimers() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    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(new Instant(20));
+    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+
+    // Advancing just a little shouldn't refire
+    underTest.advanceProcessingTime(new Instant(21));
+    assertNull(underTest.removeNextProcessingTimer());
+
+    // Adding the timer and advancing a little should refire
+    underTest.setTimer(processingTime1);
+    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+
+    // And advancing the rest of the way should still have the other timer
+    underTest.advanceProcessingTime(new Instant(30));
+    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+  }
+
+  @Test
+  public void testFiringTimersWithCallback() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    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(new Instant(20));
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
+
+    // Advancing just a little shouldn't refire
+    underTest.advanceProcessingTime(new Instant(21));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
+
+    // Adding the timer and advancing a little should fire again
+    underTest.setTimer(processingTime1);
+    underTest.advanceProcessingTime(new Instant(21));
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
+
+    // And advancing the rest of the way should still have the other timer
+    underTest.advanceProcessingTime(new Instant(30));
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
+  }
+
+  @Test
+  public void testTimerOrdering() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    TimerData eventTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+    TimerData synchronizedProcessingTime1 = TimerData.of(
+        NS1, new Instant(19), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+    TimerData eventTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
+    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+    TimerData synchronizedProcessingTime2 = TimerData.of(
+        NS1, new Instant(29), TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+    underTest.setTimer(processingTime1);
+    underTest.setTimer(eventTime1);
+    underTest.setTimer(synchronizedProcessingTime1);
+    underTest.setTimer(processingTime2);
+    underTest.setTimer(eventTime2);
+    underTest.setTimer(synchronizedProcessingTime2);
+
+    assertNull(underTest.removeNextEventTimer());
+    underTest.advanceInputWatermark(new Instant(30));
+    assertEquals(eventTime1, underTest.removeNextEventTimer());
+    assertEquals(eventTime2, underTest.removeNextEventTimer());
+    assertNull(underTest.removeNextEventTimer());
+
+    assertNull(underTest.removeNextProcessingTimer());
+    underTest.advanceProcessingTime(new Instant(30));
+    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
+    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+
+    assertNull(underTest.removeNextSynchronizedProcessingTimer());
+    underTest.advanceSynchronizedProcessingTime(new Instant(30));
+    assertEquals(synchronizedProcessingTime1, underTest.removeNextSynchronizedProcessingTimer());
+    assertEquals(synchronizedProcessingTime2, underTest.removeNextSynchronizedProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+  }
+
+  @Test
+  public void testDeduplicate() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    TimerData eventTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+    TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+    underTest.setTimer(eventTime);
+    underTest.setTimer(eventTime);
+    underTest.setTimer(processingTime);
+    underTest.setTimer(processingTime);
+    underTest.advanceProcessingTime(new Instant(20));
+    underTest.advanceInputWatermark(new Instant(20));
+
+    assertEquals(processingTime, underTest.removeNextProcessingTimer());
+    assertNull(underTest.removeNextProcessingTimer());
+    assertEquals(eventTime, underTest.removeNextEventTimer());
+    assertNull(underTest.removeNextEventTimer());
+  }
+}


[30/50] incubator-beam git commit: [BEAM-1165] Fix unexpected file creation when checking dependencies

Posted by ke...@apache.org.
[BEAM-1165] Fix unexpected file creation when checking dependencies

This error happens because maven-dependency-plugin asumes the property output
used by the flink tests as the export file for the depedency:tree
command.

Ref.
https://maven.apache.org/plugins/maven-dependency-plugin/tree-mojo.html#output


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

Branch: refs/heads/gearpump-runner
Commit: 9bd4288f13ac3f279b795e93adcfa3f897ef266c
Parents: 1e148cd
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Thu Dec 15 22:06:18 2016 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Dec 19 10:09:10 2016 +0100

----------------------------------------------------------------------
 runners/flink/examples/pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9bd4288f/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index 3145848..c50a293 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -34,9 +34,9 @@
 
   <properties>
     <!-- Default parameters for mvn exec:java -->
-    <input>kinglear.txt</input>
-    <output>wordcounts.txt</output>
-    <parallelism>-1</parallelism>
+    <flink.examples.input>kinglear.txt</flink.examples.input>
+    <flink.examples.output>wordcounts.txt</flink.examples.output>
+    <flink.examples.parallelism>-1</flink.examples.parallelism>
   </properties>
 
   <profiles>
@@ -112,9 +112,9 @@
           <executable>java</executable>
           <arguments>
             <argument>--runner=org.apache.beam.runners.flink.FlinkRunner</argument>
-            <argument>--parallelism=${parallelism}</argument>
-            <argument>--input=${input}</argument>
-            <argument>--output=${output}</argument>
+            <argument>--parallelism=${flink.examples.parallelism}</argument>
+            <argument>--input=${flink.examples.input}</argument>
+            <argument>--output=${flink.examples.output}</argument>
           </arguments>
         </configuration>
       </plugin>


[45/50] incubator-beam git commit: Disable automatic archiving of Maven builds

Posted by ke...@apache.org.
Disable automatic archiving of Maven builds

>From the Web UI:

> If checked, Jenkins will not automatically archive all artifacts generated by this project. If you wish to archive the results of this build within Jenkins, you will need to use the "Archive the artifacts" post-build action below.


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

Branch: refs/heads/gearpump-runner
Commit: 85422f999752fc827113609be0ba72cc64a9d3b3
Parents: 28d7913
Author: Daniel Halperin <dh...@users.noreply.github.com>
Authored: Mon Dec 19 11:13:49 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Dec 19 16:16:48 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/85422f99/.jenkins/common_job_properties.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy
index 3880236..e1688ec 100644
--- a/.jenkins/common_job_properties.groovy
+++ b/.jenkins/common_job_properties.groovy
@@ -140,6 +140,10 @@ class common_job_properties {
     context.rootPOM('pom.xml')
     // Use a repository local to the workspace for better isolation of jobs.
     context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE)
+    // Disable archiving the built artifacts by default, as this is slow and flaky.
+    // We can usually recreate them easily, and we can also opt-in individual jobs
+    // to artifact archiving.
+    context.archivingDisabled(true)
   }
 
   // Sets common config for PreCommit jobs.


[06/50] incubator-beam git commit: Revert "Moves DoFnAdapters to runners-core"

Posted by ke...@apache.org.
Revert "Moves DoFnAdapters to runners-core"

This reverts commit 33ed3238e2b3899cff061be3056c5cc29fc60a04.


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

Branch: refs/heads/gearpump-runner
Commit: 954e57d7696fd14f7d1015f4e40f025ef8538802
Parents: 4aa0ee1
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 16 15:37:02 2016 -0800
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Fri Dec 16 16:39:20 2016 -0800

----------------------------------------------------------------------
 .../apex/translation/WindowBoundTranslator.java |   2 +-
 .../operators/ApexGroupByKeyOperator.java       |   2 +-
 .../operators/ApexParDoOperator.java            |   2 +-
 .../apache/beam/runners/core/DoFnAdapters.java  | 508 -------------------
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   4 +-
 .../core/GroupAlsoByWindowsProperties.java      |   2 +-
 .../functions/FlinkDoFnFunction.java            |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   2 +-
 .../functions/FlinkProcessContextBase.java      |   2 +-
 .../wrappers/streaming/DoFnOperator.java        |   2 +-
 .../sdk/transforms/AggregatorRetriever.java     |  13 +-
 .../beam/sdk/transforms/DoFnAdapters.java       | 504 ++++++++++++++++++
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   2 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |   2 +-
 14 files changed, 518 insertions(+), 531 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
index ef049e1..33b9269 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
@@ -22,8 +22,8 @@ import java.util.Collections;
 import org.apache.beam.runners.apex.ApexPipelineOptions;
 import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator;
 import org.apache.beam.runners.core.AssignWindowsDoFn;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Window;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
index 4af7ff0..48ac177 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
@@ -413,7 +413,7 @@ public class ApexGroupByKeyOperator<K, V> implements Operator {
     }
 
     @Override
-    public <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
       throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
index 4538fb5..a3d3a97 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
@@ -38,7 +38,6 @@ import org.apache.beam.runners.apex.translation.utils.NoOpStepContext;
 import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions;
 import org.apache.beam.runners.apex.translation.utils.ValueAndCoderKryoSerializable;
 import org.apache.beam.runners.core.AggregatorFactory;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.DoFnRunners.OutputManager;
@@ -49,6 +48,7 @@ import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.NullSideInputReader;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
deleted file mode 100644
index fc5847c..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
+++ /dev/null
@@ -1,508 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.IOException;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AggregatorRetriever;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFn.Context;
-import org.apache.beam.sdk.transforms.DoFn.OnTimerContext;
-import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
-import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
-import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.Timer;
-import org.apache.beam.sdk.util.WindowingInternals;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TypeDescriptor;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Utility class containing adapters to/from {@link DoFn} and {@link OldDoFn}.
- *
- * @deprecated This class will go away when we start running {@link DoFn}'s directly (using {@link
- *     DoFnInvoker}) rather than via {@link OldDoFn}.
- */
-@Deprecated
-public class DoFnAdapters {
-  /** Should not be instantiated. */
-  private DoFnAdapters() {}
-
-  /**
-   * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the
-   * original {@link DoFn}, otherwise returns {@code fn.getClass()}.
-   */
-  public static Class<?> getDoFnClass(OldDoFn<?, ?> fn) {
-    if (fn instanceof SimpleDoFnAdapter) {
-      return ((SimpleDoFnAdapter<?, ?>) fn).fn.getClass();
-    } else {
-      return fn.getClass();
-    }
-  }
-
-  /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public static <InputT, OutputT> OldDoFn<InputT, OutputT> toOldDoFn(DoFn<InputT, OutputT> fn) {
-    DoFnSignature signature = DoFnSignatures.getSignature((Class) fn.getClass());
-    if (signature.processElement().observesWindow()) {
-      return new WindowDoFnAdapter<>(fn);
-    } else {
-      return new SimpleDoFnAdapter<>(fn);
-    }
-  }
-
-  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
-  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.ProcessContext adaptProcessContext(
-      OldDoFn<InputT, OutputT> fn,
-      final DoFn<InputT, OutputT>.ProcessContext c,
-      final DoFnInvoker.ArgumentProvider<InputT, OutputT> extra) {
-    return fn.new ProcessContext() {
-      @Override
-      public InputT element() {
-        return c.element();
-      }
-
-      @Override
-      public <T> T sideInput(PCollectionView<T> view) {
-        return c.sideInput(view);
-      }
-
-      @Override
-      public Instant timestamp() {
-        return c.timestamp();
-      }
-
-      @Override
-      public BoundedWindow window() {
-        return extra.window();
-      }
-
-      @Override
-      public PaneInfo pane() {
-        return c.pane();
-      }
-
-      @Override
-      public WindowingInternals<InputT, OutputT> windowingInternals() {
-        return extra.windowingInternals();
-      }
-
-      @Override
-      public PipelineOptions getPipelineOptions() {
-        return c.getPipelineOptions();
-      }
-
-      @Override
-      public void output(OutputT output) {
-        c.output(output);
-      }
-
-      @Override
-      public void outputWithTimestamp(OutputT output, Instant timestamp) {
-        c.outputWithTimestamp(output, timestamp);
-      }
-
-      @Override
-      public <T> void sideOutput(TupleTag<T> tag, T output) {
-        c.sideOutput(tag, output);
-      }
-
-      @Override
-      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-        c.sideOutputWithTimestamp(tag, output, timestamp);
-      }
-
-      @Override
-      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
-          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-        return c.createAggregator(name, combiner);
-      }
-    };
-  }
-
-  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
-  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.Context adaptContext(
-      OldDoFn<InputT, OutputT> fn,
-      final DoFn<InputT, OutputT>.Context c) {
-    return fn.new Context() {
-      @Override
-      public PipelineOptions getPipelineOptions() {
-        return c.getPipelineOptions();
-      }
-
-      @Override
-      public void output(OutputT output) {
-        c.output(output);
-      }
-
-      @Override
-      public void outputWithTimestamp(OutputT output, Instant timestamp) {
-        c.outputWithTimestamp(output, timestamp);
-      }
-
-      @Override
-      public <T> void sideOutput(TupleTag<T> tag, T output) {
-        c.sideOutput(tag, output);
-      }
-
-      @Override
-      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-        c.sideOutputWithTimestamp(tag, output, timestamp);
-      }
-
-      @Override
-      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
-          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-        return c.createAggregator(name, combiner);
-      }
-    };
-  }
-
-  /**
-   * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise,
-   * returns {@code null}.
-   */
-  @Nullable
-  public static <InputT, OutputT> DoFn<InputT, OutputT> getDoFn(OldDoFn<InputT, OutputT> fn) {
-    if (fn instanceof SimpleDoFnAdapter) {
-      return ((SimpleDoFnAdapter<InputT, OutputT>) fn).fn;
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link
-   * OldDoFn}.
-   */
-  private static class SimpleDoFnAdapter<InputT, OutputT> extends OldDoFn<InputT, OutputT> {
-    private final DoFn<InputT, OutputT> fn;
-    private transient DoFnInvoker<InputT, OutputT> invoker;
-
-    SimpleDoFnAdapter(DoFn<InputT, OutputT> fn) {
-      super(AggregatorRetriever.getDelegatingAggregators(fn));
-      this.fn = fn;
-      this.invoker = DoFnInvokers.invokerFor(fn);
-    }
-
-    @Override
-    public void setup() throws Exception {
-      this.invoker.invokeSetup();
-    }
-
-    @Override
-    public void startBundle(Context c) throws Exception {
-      fn.prepareForProcessing();
-      invoker.invokeStartBundle(new ContextAdapter<>(fn, c));
-    }
-
-    @Override
-    public void finishBundle(Context c) throws Exception {
-      invoker.invokeFinishBundle(new ContextAdapter<>(fn, c));
-    }
-
-    @Override
-    public void teardown() throws Exception {
-      this.invoker.invokeTeardown();
-    }
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      ProcessContextAdapter<InputT, OutputT> adapter = new ProcessContextAdapter<>(fn, c);
-      invoker.invokeProcessElement(adapter);
-    }
-
-    @Override
-    protected TypeDescriptor<InputT> getInputTypeDescriptor() {
-      return fn.getInputTypeDescriptor();
-    }
-
-    @Override
-    protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
-      return fn.getOutputTypeDescriptor();
-    }
-
-    @Override
-    Collection<Aggregator<?, ?>> getAggregators() {
-      return fn.getAggregators();
-    }
-
-    @Override
-    public Duration getAllowedTimestampSkew() {
-      return fn.getAllowedTimestampSkew();
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      builder.delegate(fn);
-    }
-
-    private void readObject(java.io.ObjectInputStream in)
-        throws IOException, ClassNotFoundException {
-      in.defaultReadObject();
-      this.invoker = DoFnInvokers.invokerFor(fn);
-    }
-  }
-
-  /** Wraps a {@link DoFn} that requires access to {@link BoundedWindow} as an {@link OldDoFn}. */
-  private static class WindowDoFnAdapter<InputT, OutputT> extends SimpleDoFnAdapter<InputT, OutputT>
-      implements OldDoFn.RequiresWindowAccess {
-
-    WindowDoFnAdapter(DoFn<InputT, OutputT> fn) {
-      super(fn);
-    }
-  }
-
-  /**
-   * Wraps an {@link OldDoFn.Context} as a {@link DoFnInvoker.ArgumentProvider} inside a {@link
-   * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is
-   * unavailable.
-   */
-  private static class ContextAdapter<InputT, OutputT> extends DoFn<InputT, OutputT>.Context
-      implements DoFnInvoker.ArgumentProvider<InputT, OutputT> {
-
-    private OldDoFn<InputT, OutputT>.Context context;
-
-    private ContextAdapter(DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.Context context) {
-      fn.super();
-      this.context = context;
-      super.setupDelegateAggregators();
-    }
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return context.getPipelineOptions();
-    }
-
-    @Override
-    public void output(OutputT output) {
-      context.output(output);
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      context.outputWithTimestamp(output, timestamp);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      context.sideOutput(tag, output);
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      context.sideOutputWithTimestamp(tag, output, timestamp);
-    }
-
-    @Override
-    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
-        String name,
-        CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return context.createAggregatorInternal(name, combiner);
-    }
-
-    @Override
-    public BoundedWindow window() {
-      // The OldDoFn doesn't allow us to ask for these outside processElement, so this
-      // should be unreachable.
-      throw new UnsupportedOperationException(
-          "Can only get the window in processElement; elsewhere there is no defined window.");
-    }
-
-    @Override
-    public Context context(DoFn<InputT, OutputT> doFn) {
-      return this;
-    }
-
-    @Override
-    public ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
-      throw new UnsupportedOperationException(
-          "Can only get a ProcessContext in processElement");
-    }
-
-    @Override
-    public OnTimerContext onTimerContext(DoFn<InputT, OutputT> doFn) {
-      throw new UnsupportedOperationException(
-          "Timers are not supported for OldDoFn");
-    }
-
-    @Override
-    public WindowingInternals<InputT, OutputT> windowingInternals() {
-      // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this
-      // should be unreachable.
-      throw new UnsupportedOperationException(
-          "Can only get WindowingInternals in processElement");
-    }
-
-    @Override
-    public DoFn.InputProvider<InputT> inputProvider() {
-      throw new UnsupportedOperationException("inputProvider() exists only for testing");
-    }
-
-    @Override
-    public DoFn.OutputReceiver<OutputT> outputReceiver() {
-      throw new UnsupportedOperationException("outputReceiver() exists only for testing");
-    }
-
-    @Override
-    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
-      throw new UnsupportedOperationException("This is a non-splittable DoFn");
-    }
-
-    @Override
-    public State state(String stateId) {
-      throw new UnsupportedOperationException("State is not supported by this runner");
-    }
-
-    @Override
-    public Timer timer(String timerId) {
-      throw new UnsupportedOperationException("Timers are not supported by this runner");
-    }
-  }
-
-  /**
-   * Wraps an {@link OldDoFn.ProcessContext} as a {@link DoFnInvoker.ArgumentProvider} method.
-   */
-  private static class ProcessContextAdapter<InputT, OutputT>
-      extends DoFn<InputT, OutputT>.ProcessContext
-      implements DoFnInvoker.ArgumentProvider<InputT, OutputT> {
-
-    private OldDoFn<InputT, OutputT>.ProcessContext context;
-
-    private ProcessContextAdapter(
-        DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.ProcessContext context) {
-      fn.super();
-      this.context = context;
-    }
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return context.getPipelineOptions();
-    }
-
-    @Override
-    public <T> T sideInput(PCollectionView<T> view) {
-      return context.sideInput(view);
-    }
-
-    @Override
-    public void output(OutputT output) {
-      context.output(output);
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      context.outputWithTimestamp(output, timestamp);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      context.sideOutput(tag, output);
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      context.sideOutputWithTimestamp(tag, output, timestamp);
-    }
-
-    @Override
-    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
-        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return context.createAggregatorInternal(name, combiner);
-    }
-
-    @Override
-    public InputT element() {
-      return context.element();
-    }
-
-    @Override
-    public Instant timestamp() {
-      return context.timestamp();
-    }
-
-    @Override
-    public PaneInfo pane() {
-      return context.pane();
-    }
-
-    @Override
-    public BoundedWindow window() {
-      return context.window();
-    }
-
-    @Override
-    public Context context(DoFn<InputT, OutputT> doFn) {
-      return this;
-    }
-
-    @Override
-    public ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
-      return this;
-    }
-
-    @Override
-    public OnTimerContext onTimerContext(DoFn<InputT, OutputT> doFn) {
-      throw new UnsupportedOperationException("Timers are not supported for OldDoFn");
-    }
-
-    @Override
-    public WindowingInternals<InputT, OutputT> windowingInternals() {
-      return context.windowingInternals();
-    }
-
-    @Override
-    public DoFn.InputProvider<InputT> inputProvider() {
-      throw new UnsupportedOperationException("inputProvider() exists only for testing");
-    }
-
-    @Override
-    public DoFn.OutputReceiver<OutputT> outputReceiver() {
-      throw new UnsupportedOperationException("outputReceiver() exists only for testing");
-    }
-
-    @Override
-    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
-      throw new UnsupportedOperationException("This is a non-splittable DoFn");
-    }
-
-    @Override
-    public State state(String stateId) {
-      throw new UnsupportedOperationException("State is not supported by this runner");
-    }
-
-    @Override
-    public Timer timer(String timerId) {
-      throw new UnsupportedOperationException("Timers are not supported by this runner");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
index 7d93200..1048fdc 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
@@ -322,7 +322,7 @@ class SimpleOldDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT
     }
 
     @Override
-    public <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+    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);
@@ -504,7 +504,7 @@ class SimpleOldDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT
     }
 
     @Override
-    public <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
+    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/954e57d7/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
index ef01106..97b67c6 100644
--- 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
@@ -744,7 +744,7 @@ public class GroupAlsoByWindowsProperties {
     }
 
     @Override
-    public <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
       throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index 2a4a68e..ed200d5 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -18,10 +18,10 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import java.util.Map;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
index a97bd46..7f6a436 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
@@ -18,10 +18,10 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import java.util.Map;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java
index 53b9803..6afca38 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java
@@ -252,7 +252,7 @@ abstract class FlinkProcessContextBase<InputT, OutputT>
   public abstract <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp);
 
   @Override
-  public <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
+  protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
   createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
     @SuppressWarnings("unchecked")
     SerializableFnAggregatorWrapper<AggInputT, AggOutputT> result =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/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 001e3b6..8704308 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
@@ -29,7 +29,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.runners.core.AggregatorFactory;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
@@ -42,6 +41,7 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
index b1d3ead..ce47e22 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java
@@ -18,10 +18,9 @@
 package org.apache.beam.sdk.transforms;
 
 import java.util.Collection;
-import java.util.Map;
 
 /**
- * An internal class for extracting {@link Aggregator Aggregators} from {@link DoFn DoFns}.
+ * An internal class for extracting {@link Aggregator Aggregators} from {@link OldDoFn DoFns}.
  */
 public final class AggregatorRetriever {
   private AggregatorRetriever() {
@@ -29,17 +28,9 @@ public final class AggregatorRetriever {
   }
 
   /**
-   * Returns the {@link Aggregator Aggregators} created by the provided {@link DoFn}.
+   * Returns the {@link Aggregator Aggregators} created by the provided {@link OldDoFn}.
    */
   public static Collection<Aggregator<?, ?>> getAggregators(DoFn<?, ?> fn) {
     return fn.getAggregators();
   }
-
-  /**
-   * Returns the {@link DelegatingAggregator delegating aggregators} created by the provided {@link
-   * DoFn}.
-   */
-  public static Map<String, DelegatingAggregator<?, ?>> getDelegatingAggregators(DoFn<?, ?> fn) {
-    return fn.aggregators;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/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
new file mode 100644
index 0000000..e15b08b
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
@@ -0,0 +1,504 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.io.IOException;
+import java.util.Collection;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn.Context;
+import org.apache.beam.sdk.transforms.DoFn.OnTimerContext;
+import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Utility class containing adapters to/from {@link DoFn} and {@link OldDoFn}.
+ *
+ * @deprecated This class will go away when we start running {@link DoFn}'s directly (using {@link
+ *     DoFnInvoker}) rather than via {@link OldDoFn}.
+ */
+@Deprecated
+public class DoFnAdapters {
+  /** Should not be instantiated. */
+  private DoFnAdapters() {}
+
+  /**
+   * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the
+   * original {@link DoFn}, otherwise returns {@code fn.getClass()}.
+   */
+  public static Class<?> getDoFnClass(OldDoFn<?, ?> fn) {
+    if (fn instanceof SimpleDoFnAdapter) {
+      return ((SimpleDoFnAdapter<?, ?>) fn).fn.getClass();
+    } else {
+      return fn.getClass();
+    }
+  }
+
+  /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public static <InputT, OutputT> OldDoFn<InputT, OutputT> toOldDoFn(DoFn<InputT, OutputT> fn) {
+    DoFnSignature signature = DoFnSignatures.getSignature((Class) fn.getClass());
+    if (signature.processElement().observesWindow()) {
+      return new WindowDoFnAdapter<>(fn);
+    } else {
+      return new SimpleDoFnAdapter<>(fn);
+    }
+  }
+
+  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
+  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.ProcessContext adaptProcessContext(
+      OldDoFn<InputT, OutputT> fn,
+      final DoFn<InputT, OutputT>.ProcessContext c,
+      final DoFnInvoker.ArgumentProvider<InputT, OutputT> extra) {
+    return fn.new ProcessContext() {
+      @Override
+      public InputT element() {
+        return c.element();
+      }
+
+      @Override
+      public <T> T sideInput(PCollectionView<T> view) {
+        return c.sideInput(view);
+      }
+
+      @Override
+      public Instant timestamp() {
+        return c.timestamp();
+      }
+
+      @Override
+      public BoundedWindow window() {
+        return extra.window();
+      }
+
+      @Override
+      public PaneInfo pane() {
+        return c.pane();
+      }
+
+      @Override
+      public WindowingInternals<InputT, OutputT> windowingInternals() {
+        return extra.windowingInternals();
+      }
+
+      @Override
+      public PipelineOptions getPipelineOptions() {
+        return c.getPipelineOptions();
+      }
+
+      @Override
+      public void output(OutputT output) {
+        c.output(output);
+      }
+
+      @Override
+      public void outputWithTimestamp(OutputT output, Instant timestamp) {
+        c.outputWithTimestamp(output, timestamp);
+      }
+
+      @Override
+      public <T> void sideOutput(TupleTag<T> tag, T output) {
+        c.sideOutput(tag, output);
+      }
+
+      @Override
+      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+        c.sideOutputWithTimestamp(tag, output, timestamp);
+      }
+
+      @Override
+      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+        return c.createAggregator(name, combiner);
+      }
+    };
+  }
+
+  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
+  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.Context adaptContext(
+      OldDoFn<InputT, OutputT> fn,
+      final DoFn<InputT, OutputT>.Context c) {
+    return fn.new Context() {
+      @Override
+      public PipelineOptions getPipelineOptions() {
+        return c.getPipelineOptions();
+      }
+
+      @Override
+      public void output(OutputT output) {
+        c.output(output);
+      }
+
+      @Override
+      public void outputWithTimestamp(OutputT output, Instant timestamp) {
+        c.outputWithTimestamp(output, timestamp);
+      }
+
+      @Override
+      public <T> void sideOutput(TupleTag<T> tag, T output) {
+        c.sideOutput(tag, output);
+      }
+
+      @Override
+      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+        c.sideOutputWithTimestamp(tag, output, timestamp);
+      }
+
+      @Override
+      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+        return c.createAggregator(name, combiner);
+      }
+    };
+  }
+
+  /**
+   * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise,
+   * returns {@code null}.
+   */
+  @Nullable
+  public static <InputT, OutputT> DoFn<InputT, OutputT> getDoFn(OldDoFn<InputT, OutputT> fn) {
+    if (fn instanceof SimpleDoFnAdapter) {
+      return ((SimpleDoFnAdapter<InputT, OutputT>) fn).fn;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link
+   * OldDoFn}.
+   */
+  private static class SimpleDoFnAdapter<InputT, OutputT> extends OldDoFn<InputT, OutputT> {
+    private final DoFn<InputT, OutputT> fn;
+    private transient DoFnInvoker<InputT, OutputT> invoker;
+
+    SimpleDoFnAdapter(DoFn<InputT, OutputT> fn) {
+      super(fn.aggregators);
+      this.fn = fn;
+      this.invoker = DoFnInvokers.invokerFor(fn);
+    }
+
+    @Override
+    public void setup() throws Exception {
+      this.invoker.invokeSetup();
+    }
+
+    @Override
+    public void startBundle(Context c) throws Exception {
+      fn.prepareForProcessing();
+      invoker.invokeStartBundle(new ContextAdapter<>(fn, c));
+    }
+
+    @Override
+    public void finishBundle(Context c) throws Exception {
+      invoker.invokeFinishBundle(new ContextAdapter<>(fn, c));
+    }
+
+    @Override
+    public void teardown() throws Exception {
+      this.invoker.invokeTeardown();
+    }
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      ProcessContextAdapter<InputT, OutputT> adapter = new ProcessContextAdapter<>(fn, c);
+      invoker.invokeProcessElement(adapter);
+    }
+
+    @Override
+    protected TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return fn.getInputTypeDescriptor();
+    }
+
+    @Override
+    protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return fn.getOutputTypeDescriptor();
+    }
+
+    @Override
+    Collection<Aggregator<?, ?>> getAggregators() {
+      return fn.getAggregators();
+    }
+
+    @Override
+    public Duration getAllowedTimestampSkew() {
+      return fn.getAllowedTimestampSkew();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      builder.delegate(fn);
+    }
+
+    private void readObject(java.io.ObjectInputStream in)
+        throws IOException, ClassNotFoundException {
+      in.defaultReadObject();
+      this.invoker = DoFnInvokers.invokerFor(fn);
+    }
+  }
+
+  /** Wraps a {@link DoFn} that requires access to {@link BoundedWindow} as an {@link OldDoFn}. */
+  private static class WindowDoFnAdapter<InputT, OutputT> extends SimpleDoFnAdapter<InputT, OutputT>
+      implements OldDoFn.RequiresWindowAccess {
+
+    WindowDoFnAdapter(DoFn<InputT, OutputT> fn) {
+      super(fn);
+    }
+  }
+
+  /**
+   * Wraps an {@link OldDoFn.Context} as a {@link DoFnInvoker.ArgumentProvider} inside a {@link
+   * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is
+   * unavailable.
+   */
+  private static class ContextAdapter<InputT, OutputT> extends DoFn<InputT, OutputT>.Context
+      implements DoFnInvoker.ArgumentProvider<InputT, OutputT> {
+
+    private OldDoFn<InputT, OutputT>.Context context;
+
+    private ContextAdapter(DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.Context context) {
+      fn.super();
+      this.context = context;
+      super.setupDelegateAggregators();
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      context.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      context.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      context.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+        String name,
+        CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return context.createAggregatorInternal(name, combiner);
+    }
+
+    @Override
+    public BoundedWindow window() {
+      // The OldDoFn doesn't allow us to ask for these outside processElement, so this
+      // should be unreachable.
+      throw new UnsupportedOperationException(
+          "Can only get the window in processElement; elsewhere there is no defined window.");
+    }
+
+    @Override
+    public Context context(DoFn<InputT, OutputT> doFn) {
+      return this;
+    }
+
+    @Override
+    public ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          "Can only get a ProcessContext in processElement");
+    }
+
+    @Override
+    public OnTimerContext onTimerContext(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          "Timers are not supported for OldDoFn");
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this
+      // should be unreachable.
+      throw new UnsupportedOperationException(
+          "Can only get WindowingInternals in processElement");
+    }
+
+    @Override
+    public DoFn.InputProvider<InputT> inputProvider() {
+      throw new UnsupportedOperationException("inputProvider() exists only for testing");
+    }
+
+    @Override
+    public DoFn.OutputReceiver<OutputT> outputReceiver() {
+      throw new UnsupportedOperationException("outputReceiver() exists only for testing");
+    }
+
+    @Override
+    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+      throw new UnsupportedOperationException("This is a non-splittable DoFn");
+    }
+
+    @Override
+    public State state(String stateId) {
+      throw new UnsupportedOperationException("State is not supported by this runner");
+    }
+
+    @Override
+    public Timer timer(String timerId) {
+      throw new UnsupportedOperationException("Timers are not supported by this runner");
+    }
+  }
+
+  /**
+   * Wraps an {@link OldDoFn.ProcessContext} as a {@link DoFnInvoker.ArgumentProvider} method.
+   */
+  private static class ProcessContextAdapter<InputT, OutputT>
+      extends DoFn<InputT, OutputT>.ProcessContext
+      implements DoFnInvoker.ArgumentProvider<InputT, OutputT> {
+
+    private OldDoFn<InputT, OutputT>.ProcessContext context;
+
+    private ProcessContextAdapter(
+        DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.ProcessContext context) {
+      fn.super();
+      this.context = context;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      return context.sideInput(view);
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      context.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      context.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      context.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return context.createAggregatorInternal(name, combiner);
+    }
+
+    @Override
+    public InputT element() {
+      return context.element();
+    }
+
+    @Override
+    public Instant timestamp() {
+      return context.timestamp();
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return context.pane();
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return context.window();
+    }
+
+    @Override
+    public Context context(DoFn<InputT, OutputT> doFn) {
+      return this;
+    }
+
+    @Override
+    public ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
+      return this;
+    }
+
+    @Override
+    public OnTimerContext onTimerContext(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("Timers are not supported for OldDoFn");
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return context.windowingInternals();
+    }
+
+    @Override
+    public DoFn.InputProvider<InputT> inputProvider() {
+      throw new UnsupportedOperationException("inputProvider() exists only for testing");
+    }
+
+    @Override
+    public DoFn.OutputReceiver<OutputT> outputReceiver() {
+      throw new UnsupportedOperationException("outputReceiver() exists only for testing");
+    }
+
+    @Override
+    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+      throw new UnsupportedOperationException("This is a non-splittable DoFn");
+    }
+
+    @Override
+    public State state(String stateId) {
+      throw new UnsupportedOperationException("State is not supported by this runner");
+    }
+
+    @Override
+    public Timer timer(String timerId) {
+      throw new UnsupportedOperationException("Timers are not supported by this runner");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index d1bb42b..2d2c1fd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -208,7 +208,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
      *         context
      */
     @Experimental(Kind.AGGREGATOR)
-    public abstract <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
+    protected abstract <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
         createAggregatorInternal(String name, CombineFn<AggInputT, ?, AggOutputT> combiner);
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/954e57d7/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java
index 0db130d..504480b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java
@@ -63,7 +63,7 @@ class NoOpOldDoFn<InputT, OutputT> extends OldDoFn<InputT, OutputT> {
         Instant timestamp) {
     }
     @Override
-    public <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
         createAggregatorInternal(String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
       return null;
     }


[47/50] incubator-beam git commit: Merge branch 'master' of https://github.com/apache/incubator-beam into gearpump-runner

Posted by ke...@apache.org.
Merge branch 'master' of https://github.com/apache/incubator-beam into gearpump-runner


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

Branch: refs/heads/gearpump-runner
Commit: 81d94cfabc645a96a3dc37801599d95433e85172
Parents: b6e7bb6 b3de17b
Author: manuzhang <ow...@gmail.com>
Authored: Tue Dec 20 14:55:51 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Tue Dec 20 14:55:51 2016 +0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           |   7 +
 ...ostCommit_Java_RunnableOnService_Apex.groovy |  41 ++
 README.md                                       |   2 +-
 examples/java/pom.xml                           |  20 +-
 .../apache/beam/examples/WindowedWordCount.java | 177 +++---
 .../org/apache/beam/examples/WordCount.java     |   2 +-
 .../examples/common/WriteWindowedFilesDoFn.java |  77 +++
 .../beam/examples/complete/AutoComplete.java    |   6 +-
 .../apache/beam/examples/complete/TfIdf.java    |   6 +-
 .../examples/complete/TopWikipediaSessions.java |  15 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   4 +-
 .../beam/examples/complete/TrafficRoutes.java   |   4 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/WindowedWordCountIT.java      | 182 +++++-
 .../examples/complete/AutoCompleteTest.java     |   2 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |   2 +-
 .../examples/complete/game/LeaderBoard.java     |   4 +-
 .../beam/examples/complete/game/UserScore.java  |   2 +-
 .../complete/game/utils/WriteToBigQuery.java    |   2 +-
 .../game/utils/WriteWindowedToBigQuery.java     |   7 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         |   5 +-
 runners/apex/pom.xml                            |  66 ++-
 .../apache/beam/runners/apex/ApexRunner.java    | 111 ++--
 .../beam/runners/apex/ApexRunnerResult.java     |  50 +-
 .../beam/runners/apex/ApexYarnLauncher.java     | 395 +++++++++++++
 .../translation/ApexPipelineTranslator.java     |   2 +
 .../translation/ParDoBoundMultiTranslator.java  |  22 +-
 .../apex/translation/ParDoBoundTranslator.java  |  22 +-
 .../apex/translation/TranslationContext.java    |   4 +-
 .../apex/translation/WindowBoundTranslator.java |  78 +++
 .../operators/ApexGroupByKeyOperator.java       |   4 +-
 .../operators/ApexParDoOperator.java            |  27 +-
 .../beam/runners/apex/ApexYarnLauncherTest.java | 138 +++++
 .../FlattenPCollectionTranslatorTest.java       |  15 +-
 .../translation/GroupByKeyTranslatorTest.java   |  21 +-
 .../translation/ParDoBoundTranslatorTest.java   |  38 +-
 .../translation/ReadUnboundTranslatorTest.java  |  15 +-
 runners/core-java/pom.xml                       |   6 +-
 .../beam/runners/core/ActiveWindowSet.java      | 173 ++++++
 .../apache/beam/runners/core/AssignWindows.java |  46 --
 .../apache/beam/runners/core/DoFnRunner.java    |  24 +-
 .../apache/beam/runners/core/DoFnRunners.java   |   2 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  12 +-
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  48 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |   8 +-
 .../apache/beam/runners/core/KeyedWorkItem.java |  44 ++
 .../beam/runners/core/KeyedWorkItemCoder.java   | 130 +++++
 .../beam/runners/core/KeyedWorkItems.java       | 122 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  34 +-
 .../runners/core/MergingActiveWindowSet.java    | 428 ++++++++++++++
 .../runners/core/NonMergingActiveWindowSet.java |  91 +++
 .../beam/runners/core/PaneInfoTracker.java      |   4 +
 .../runners/core/PerKeyCombineFnRunner.java     | 149 +++++
 .../runners/core/PerKeyCombineFnRunners.java    |   1 -
 .../core/PushbackSideInputDoFnRunner.java       |  31 +-
 .../runners/core/ReduceFnContextFactory.java    |  26 +-
 .../beam/runners/core/ReduceFnRunner.java       | 496 ++++++++++------
 .../beam/runners/core/SimpleDoFnRunner.java     | 252 ++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   9 +-
 .../beam/runners/core/SplittableParDo.java      |  28 +-
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../apache/beam/runners/core/WatermarkHold.java |   5 +
 .../TriggerStateMachineContextFactory.java      |   2 +-
 .../triggers/TriggerStateMachineRunner.java     |  14 +-
 .../runners/core/KeyedWorkItemCoderTest.java    |  64 +++
 .../core/LateDataDroppingDoFnRunnerTest.java    |   3 +
 .../core/MergingActiveWindowSetTest.java        | 262 +++++++++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  55 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   | 133 ++---
 .../beam/runners/core/ReduceFnTester.java       |  77 ++-
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 301 ++++++++++
 .../beam/runners/core/SplittableParDoTest.java  |   2 -
 .../triggers/TriggerStateMachineTester.java     |  25 +-
 runners/direct-java/pom.xml                     |   3 +-
 .../runners/direct/AggregatorContainer.java     |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java     | 467 +++++++++++++++
 .../runners/direct/DirectExecutionContext.java  |   1 -
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |   9 +-
 .../beam/runners/direct/DirectGraphVisitor.java |  23 +-
 .../beam/runners/direct/DirectGroupByKey.java   |  46 +-
 .../direct/DirectGroupByKeyOverrideFactory.java |   3 +-
 .../beam/runners/direct/DirectRunner.java       |   7 +-
 .../beam/runners/direct/EvaluationContext.java  |   1 -
 .../direct/ExecutorServiceParallelExecutor.java |   4 +-
 .../runners/direct/ForwardingPTransform.java    |   6 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |   8 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   4 +-
 .../direct/KeyedPValueTrackingVisitor.java      |   4 +-
 .../direct/PTransformOverrideFactory.java       |  35 --
 .../beam/runners/direct/ParDoEvaluator.java     |   1 -
 .../runners/direct/ParDoEvaluatorFactory.java   |   2 +-
 .../direct/ParDoMultiOverrideFactory.java       |  29 +-
 .../ParDoSingleViaMultiOverrideFactory.java     |  17 +-
 ...littableProcessElementsEvaluatorFactory.java |   2 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   4 +-
 .../runners/direct/StepTransformResult.java     |   1 -
 .../direct/TestStreamEvaluatorFactory.java      |   7 +-
 .../beam/runners/direct/TransformResult.java    |   1 -
 .../runners/direct/ViewEvaluatorFactory.java    |   8 +-
 .../direct/WriteWithShardingFactory.java        |  10 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  18 +-
 .../runners/direct/CommittedResultTest.java     |   2 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 562 +++++++++++++++++++
 .../runners/direct/DirectGraphVisitorTest.java  |  71 ++-
 .../beam/runners/direct/DirectGraphs.java       |  35 ++
 .../beam/runners/direct/DirectRunnerTest.java   |  36 --
 .../runners/direct/EvaluationContextTest.java   |  83 +--
 .../direct/FlattenEvaluatorFactoryTest.java     |  15 +-
 .../direct/ForwardingPTransformTest.java        |   4 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   6 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |   7 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   2 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   4 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   3 +-
 .../StatefulParDoEvaluatorFactoryTest.java      |  11 +-
 .../runners/direct/StepTransformResultTest.java |   2 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  14 +-
 .../runners/direct/TransformExecutorTest.java   |   9 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |  24 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   4 +-
 .../direct/WatermarkCallbackExecutorTest.java   |   6 +-
 .../runners/direct/WatermarkManagerTest.java    | 237 ++++----
 .../direct/WriteWithShardingFactoryTest.java    |   4 +-
 runners/flink/examples/pom.xml                  |  14 +-
 .../beam/runners/flink/examples/TFIDF.java      |   6 +-
 .../beam/runners/flink/examples/WordCount.java  |   2 +-
 .../flink/examples/streaming/AutoComplete.java  |   6 +-
 runners/flink/pom.xml                           |   2 +-
 runners/flink/runner/pom.xml                    |  10 +-
 .../FlinkPipelineExecutionEnvironment.java      |  12 +
 .../runners/flink/FlinkPipelineOptions.java     |   5 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  24 +-
 .../FlinkBatchPipelineTranslator.java           |   4 +-
 .../FlinkBatchTransformTranslators.java         |  59 +-
 .../FlinkStreamingPipelineTranslator.java       |   7 +-
 .../FlinkStreamingTransformTranslators.java     |  77 ++-
 .../functions/FlinkDoFnFunction.java            |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   2 +-
 .../FlinkMergingPartialReduceFunction.java      |   2 +-
 .../functions/FlinkMergingReduceFunction.java   |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |  10 +-
 .../functions/FlinkPartialReduceFunction.java   |   2 +-
 .../functions/FlinkProcessContextBase.java      |  21 +-
 .../functions/FlinkReduceFunction.java          |   2 +-
 .../wrappers/streaming/DoFnOperator.java        |  46 +-
 .../streaming/SingletonKeyedWorkItem.java       |   2 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  14 +-
 .../wrappers/streaming/WorkItemKeySelector.java |   2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   6 +-
 .../flink/streaming/DoFnOperatorTest.java       |  13 +-
 .../flink/streaming/GroupByNullKeyTest.java     |  18 +-
 .../streaming/TopWikipediaSessionsITCase.java   |  10 +-
 runners/google-cloud-dataflow-java/pom.xml      |  12 +-
 .../beam/runners/dataflow/DataflowClient.java   | 140 +++++
 .../runners/dataflow/DataflowPipelineJob.java   |  31 +-
 .../dataflow/DataflowPipelineTranslator.java    |  15 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 136 +++--
 .../runners/dataflow/DataflowRunnerInfo.java    |  92 +++
 .../dataflow/internal/AssignWindows.java        |  10 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   2 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  57 +-
 .../options/DataflowProfilingOptions.java       |   8 +-
 .../dataflow/testing/TestDataflowRunner.java    |  35 +-
 .../dataflow/util/DataflowTemplateJob.java      |   2 +-
 .../runners/dataflow/util/MonitoringUtil.java   |  22 +-
 .../beam/runners/dataflow/dataflow.properties   |  23 +
 .../dataflow/DataflowPipelineJobTest.java       |  49 +-
 .../DataflowPipelineTranslatorTest.java         |  61 +-
 .../dataflow/DataflowRunnerInfoTest.java        |  51 ++
 .../runners/dataflow/DataflowRunnerTest.java    | 285 ++++++----
 .../options/DataflowProfilingOptionsTest.java   |   6 +-
 .../testing/TestDataflowRunnerTest.java         |  52 +-
 .../transforms/DataflowGroupByKeyTest.java      |  18 +-
 .../dataflow/transforms/DataflowViewTest.java   |  20 +-
 .../dataflow/util/MonitoringUtilTest.java       |  21 +-
 runners/pom.xml                                 |   2 +-
 runners/spark/README.md                         |  59 +-
 runners/spark/pom.xml                           |  55 +-
 .../beam/runners/spark/EvaluationResult.java    |  67 ---
 .../runners/spark/SparkPipelineOptions.java     |  10 +
 .../beam/runners/spark/SparkPipelineResult.java | 193 +++++++
 .../apache/beam/runners/spark/SparkRunner.java  | 130 +++--
 .../beam/runners/spark/TestSparkRunner.java     |  91 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   6 +-
 .../spark/aggregators/SparkAggregators.java     | 126 +++++
 .../metrics/WithNamedAggregatorsSupport.java    |   4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |  60 +-
 .../beam/runners/spark/examples/WordCount.java  |  15 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |   2 +-
 .../beam/runners/spark/io/CreateStream.java     |   2 +-
 .../beam/runners/spark/io/SourceDStream.java    |  21 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  17 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   4 +-
 .../runners/spark/translation/DoFnFunction.java | 110 ++--
 .../spark/translation/EvaluationContext.java    | 131 +----
 .../translation/GroupCombineFunctions.java      |  23 +-
 .../spark/translation/MultiDoFnFunction.java    | 135 ++---
 .../spark/translation/SparkAssignWindowFn.java  |  69 +++
 .../spark/translation/SparkContextFactory.java  |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   | 214 +++++++
 .../spark/translation/SparkProcessContext.java  | 385 +++----------
 .../spark/translation/SparkRuntimeContext.java  |  62 +-
 .../translation/StorageLevelPTransform.java     |   2 +-
 .../spark/translation/TransformTranslator.java  |  67 +--
 .../spark/translation/TranslationUtils.java     |  33 ++
 .../spark/translation/WindowingHelpers.java     |   3 +-
 .../streaming/StreamingTransformTranslator.java |  45 +-
 .../util/SinglePrimitiveOutputPTransform.java   |   2 +-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 ++++
 .../runners/spark/ProvidedSparkContextTest.java |   6 +-
 .../runners/spark/SparkPipelineStateTest.java   | 217 +++++++
 .../spark/aggregators/ClearAggregatorsRule.java |  37 ++
 .../metrics/sink/ClearAggregatorsRule.java      |  33 --
 .../metrics/sink/NamedAggregatorsTest.java      |   3 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |  57 ++
 .../beam/runners/spark/io/AvroPipelineTest.java |   2 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   2 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   2 +-
 .../spark/translation/SideEffectsTest.java      |  59 --
 .../streaming/EmptyStreamAssertionTest.java     |   4 +
 .../streaming/KafkaStreamingTest.java           |  34 +-
 .../ResumeFromCheckpointStreamingTest.java      |  32 +-
 .../streaming/utils/PAssertStreaming.java       |  31 +-
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml |  68 ++-
 sdks/java/core/pom.xml                          |   2 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   3 +-
 .../beam/sdk/annotations/Experimental.java      |   5 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |   2 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   4 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  16 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |  52 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |  17 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  77 ++-
 .../org/apache/beam/sdk/io/FileBasedSource.java |  12 +-
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 +
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 +
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 ++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |   4 +-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   6 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  12 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |   4 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   2 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   8 +-
 .../apache/beam/sdk/metrics/CounterCell.java    |  10 +-
 .../beam/sdk/metrics/DistributionCell.java      |  14 +-
 .../org/apache/beam/sdk/metrics/MetricCell.java |   2 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |   3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |   5 +-
 .../beam/sdk/metrics/MetricsEnvironment.java    |   8 +-
 .../sdk/options/PipelineOptionsFactory.java     | 110 +++-
 .../beam/sdk/options/StreamingOptions.java      |   2 +-
 .../apache/beam/sdk/options/ValueProvider.java  |  27 +-
 .../sdk/runners/PTransformOverrideFactory.java  |  41 ++
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    |  26 +-
 .../beam/sdk/testing/BigqueryMatcher.java       |  48 +-
 .../beam/sdk/testing/FileChecksumMatcher.java   | 114 +---
 .../apache/beam/sdk/testing/GatherAllPanes.java |   2 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  12 +-
 .../apache/beam/sdk/testing/TestPipeline.java   | 225 +++++++-
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    |  24 +
 .../beam/sdk/testing/UsesTimersInParDo.java     |  25 +
 .../sdk/transforms/AggregatorRetriever.java     |   2 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |  75 +--
 .../beam/sdk/transforms/ApproximateUnique.java  |   4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  14 +-
 .../beam/sdk/transforms/CombineWithContext.java |   2 +-
 .../org/apache/beam/sdk/transforms/Count.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Create.java  |   4 +-
 .../apache/beam/sdk/transforms/Distinct.java    |   4 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  22 +
 .../beam/sdk/transforms/DoFnAdapters.java       |  12 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  29 +-
 .../org/apache/beam/sdk/transforms/Filter.java  |   2 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   2 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   4 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   2 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |   4 +-
 .../apache/beam/sdk/transforms/MapElements.java |   2 +-
 .../apache/beam/sdk/transforms/PTransform.java  |   4 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 191 +------
 .../apache/beam/sdk/transforms/Partition.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  14 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Values.java  |   2 +-
 .../org/apache/beam/sdk/transforms/View.java    |  16 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../beam/sdk/transforms/WithTimestamps.java     |   2 +-
 .../sdk/transforms/display/DisplayData.java     |  22 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |   2 +-
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  11 +
 .../sdk/transforms/reflect/DoFnInvoker.java     |   8 +
 .../sdk/transforms/reflect/DoFnSignature.java   |  26 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |  96 +++-
 .../beam/sdk/transforms/windowing/Window.java   |   4 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   | 173 ------
 .../org/apache/beam/sdk/util/ApiSurface.java    |  52 +-
 .../apache/beam/sdk/util/ExecutionContext.java  |   8 +-
 .../beam/sdk/util/ExplicitShardedFile.java      | 120 ++++
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 107 +++-
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |  43 --
 .../beam/sdk/util/KeyedWorkItemCoder.java       | 128 -----
 .../apache/beam/sdk/util/KeyedWorkItems.java    | 121 ----
 .../beam/sdk/util/MergingActiveWindowSet.java   | 428 --------------
 .../sdk/util/NonMergingActiveWindowSet.java     |  91 ---
 .../beam/sdk/util/NumberedShardedFile.java      | 220 ++++++++
 .../beam/sdk/util/PerKeyCombineFnRunner.java    | 148 -----
 .../org/apache/beam/sdk/util/PropertyNames.java |   2 +
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   4 -
 .../org/apache/beam/sdk/util/Reshuffle.java     |   2 +-
 .../org/apache/beam/sdk/util/ShardedFile.java   |  42 ++
 .../CopyOnAccessInMemoryStateInternals.java     | 453 ---------------
 .../sdk/util/state/InMemoryStateInternals.java  |  33 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 126 +++--
 .../beam/sdk/util/state/StateInternals.java     |   2 +-
 .../beam/sdk/util/state/TimerCallback.java      |  35 --
 .../org/apache/beam/sdk/values/PCollection.java |   4 +-
 .../beam/sdk/values/PCollectionTuple.java       |   2 +-
 .../org/apache/beam/sdk/values/TypedPValue.java |   2 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |   8 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |   6 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   4 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  63 ++-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  10 +
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  11 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  26 +
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 ++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  19 +
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   2 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |   4 +-
 .../apache/beam/sdk/metrics/MetricMatchers.java |  47 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    |  63 ++-
 .../sdk/options/PipelineOptionsFactoryTest.java | 222 +++++++-
 .../beam/sdk/options/ValueProviderTest.java     |  21 +
 .../sdk/runners/TransformHierarchyTest.java     |  17 +-
 .../beam/sdk/runners/TransformTreeTest.java     |   4 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   |  82 +--
 .../sdk/testing/FileChecksumMatcherTest.java    |  77 ---
 .../beam/sdk/testing/TestPipelineTest.java      | 186 +++++-
 .../beam/sdk/transforms/DoFnTesterTest.java     |   2 +-
 .../sdk/transforms/FlatMapElementsTest.java     |   2 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |   4 +-
 .../beam/sdk/transforms/MapElementsTest.java    |   2 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |  72 ---
 .../apache/beam/sdk/transforms/OldDoFnTest.java | 125 ++---
 .../beam/sdk/transforms/PTransformTest.java     |   2 +-
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  28 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 342 ++++++-----
 .../beam/sdk/transforms/SplittableDoFnTest.java |  58 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   4 +-
 .../display/DisplayDataEvaluatorTest.java       |   2 +-
 .../display/DisplayDataMatchersTest.java        |   6 +-
 .../sdk/transforms/display/DisplayDataTest.java |  90 ++-
 .../DoFnSignaturesSplittableDoFnTest.java       |   3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  47 ++
 .../sdk/transforms/windowing/WindowingTest.java |   2 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  85 ++-
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |  62 --
 .../sdk/util/MergingActiveWindowSetTest.java    | 262 ---------
 .../beam/sdk/util/NumberedShardedFileTest.java  | 181 ++++++
 .../apache/beam/sdk/util/StringUtilsTest.java   |   6 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 552 ------------------
 .../util/state/InMemoryTimerInternalsTest.java  | 125 +++--
 .../org/apache/beam/sdk/values/PDoneTest.java   |   4 +-
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/extensions/sorter/README.md           |   2 +-
 sdks/java/extensions/sorter/pom.xml             |  10 +-
 .../sorter/BufferedExternalSorter.java          |   6 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |  15 +-
 .../sdk/extensions/sorter/InMemorySorter.java   |  26 +-
 .../beam/sdk/extensions/sorter/SortValues.java  |   2 +-
 .../sorter/BufferedExternalSorterTest.java      |  16 +
 .../extensions/sorter/ExternalSorterTest.java   |  16 +
 .../extensions/sorter/InMemorySorterTest.java   |   8 +
 sdks/java/io/google-cloud-platform/pom.xml      |   8 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 453 +++++++++++----
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   7 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 121 +++-
 .../io/gcp/bigquery/BigQueryTableInserter.java  | 217 -------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  28 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   6 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |   4 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 ++
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 ++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  |   2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 543 ++++++++++++++----
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 139 ++++-
 .../gcp/bigquery/BigQueryTableInserterTest.java | 245 --------
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  50 +-
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++
 sdks/java/io/hdfs/pom.xml                       |   8 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 +
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 ++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 ++
 sdks/java/io/jdbc/pom.xml                       |   2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |   4 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |   4 +-
 sdks/java/io/jms/pom.xml                        |  22 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 342 +++++++----
 .../org/apache/beam/sdk/io/jms/JmsRecord.java   |   8 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  20 +-
 sdks/java/io/kinesis/pom.xml                    |   2 +-
 sdks/java/io/mongodb/pom.xml                    |   2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |   4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |  16 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |   8 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |   8 +-
 sdks/java/io/pom.xml                            |   2 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../maven-archetypes/examples-java8/pom.xml     |  13 +-
 .../main/resources/archetype-resources/pom.xml  | 122 +++-
 sdks/java/maven-archetypes/examples/pom.xml     |  13 +-
 .../main/resources/archetype-resources/pom.xml  |  47 +-
 sdks/java/maven-archetypes/pom.xml              |   2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |   9 +-
 .../main/resources/archetype-resources/pom.xml  |  16 +-
 .../resources/projects/basic/reference/pom.xml  |  16 +-
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 438 files changed, 13536 insertions(+), 7591 deletions(-)
----------------------------------------------------------------------


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


[23/50] incubator-beam git commit: Empty TestPipeline need not be run

Posted by ke...@apache.org.
Empty TestPipeline need not be run


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

Branch: refs/heads/gearpump-runner
Commit: 0d54a4c6708516170cb045b5e0d1d33443973c6e
Parents: 443b25a
Author: Kenneth Knowles <kl...@google.com>
Authored: Sat Dec 17 13:57:07 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sat Dec 17 14:12:23 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/TestPipeline.java   | 20 +++++++++++++++++++-
 .../beam/sdk/testing/TestPipelineTest.java      |  3 +++
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0d54a4c6/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 49ac3af..b707a81 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -169,7 +169,12 @@ public class TestPipeline extends Pipeline implements TestRule {
           throw new AbandonedNodeException("The pipeline contains abandoned PTransform(s).");
         }
       } else if (runVisitedNodes == null && !enableAutoRunIfMissing) {
-        throw new PipelineRunMissingException("The pipeline has not been run.");
+        IsEmptyVisitor isEmptyVisitor = new IsEmptyVisitor();
+        pipeline.traverseTopologically(isEmptyVisitor);
+
+        if (!isEmptyVisitor.isEmpty()) {
+          throw new PipelineRunMissingException("The pipeline has not been run.");
+        }
       }
     }
 
@@ -389,4 +394,17 @@ public class TestPipeline extends Pipeline implements TestRule {
     }
     return firstInstanceAfterTestPipeline;
   }
+
+  private static class IsEmptyVisitor extends PipelineVisitor.Defaults {
+    private boolean empty = true;
+
+    public boolean isEmpty() {
+      return empty;
+    }
+
+    @Override
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+      empty = false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0d54a4c6/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
index d1797e7..f484566 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
@@ -74,6 +74,9 @@ public class TestPipelineTest implements Serializable {
   @Rule public transient RuleChain ruleOrder = RuleChain.outerRule(exception).around(pipeline);
 
   @Test
+  public void testNoTestPipelineUsed() { }
+
+  @Test
   public void testCreationUsingDefaults() {
     assertNotNull(TestPipeline.create());
   }


[17/50] incubator-beam git commit: This closes #1614: Spark runner: Force streaming execution on batch pipelines for testing

Posted by ke...@apache.org.
This closes #1614: Spark runner: Force streaming execution on batch pipelines for testing

  [BEAM-853] Force streaming execution on batch pipelines for testing. Expose the adapted source.


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

Branch: refs/heads/gearpump-runner
Commit: 4206408bf98898be3c37c1d9f518de8ea9acd170
Parents: ffe3ab3 5f41ded
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 16 22:03:59 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 22:03:59 2016 -0800

----------------------------------------------------------------------
 .../runners/spark/SparkPipelineOptions.java     |   5 +
 .../beam/runners/spark/TestSparkRunner.java     |  80 +++++++++++-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 +++++++++++++++++++
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  14 ++-
 4 files changed, 217 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-beam git commit: Add timer support to DoFnRunner(s)

Posted by ke...@apache.org.
Add timer support to DoFnRunner(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/8af13b01
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8af13b01
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8af13b01

Branch: refs/heads/gearpump-runner
Commit: 8af13b0102cda6c68601efa4119723900d12ca5c
Parents: c1e1017
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Nov 23 14:21:40 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 20:14:19 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunner.java    |   9 +
 .../core/LateDataDroppingDoFnRunner.java        |   7 +
 .../core/PushbackSideInputDoFnRunner.java       |   8 +
 .../beam/runners/core/SimpleDoFnRunner.java     | 236 +++++++++++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   8 +
 .../core/PushbackSideInputDoFnRunnerTest.java   |  41 +++
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 247 +++++++++++++++++++
 7 files changed, 555 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/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
index 501667e..7c73a34 100644
--- 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
@@ -20,8 +20,11 @@ package org.apache.beam.runners.core;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
+import org.joda.time.Instant;
 
 /**
  * An wrapper interface that represents the execution of a {@link DoFn}.
@@ -39,6 +42,12 @@ public interface DoFnRunner<InputT, OutputT> {
   void processElement(WindowedValue<InputT> elem);
 
   /**
+   * Calls a {@link DoFn DoFn's} {@link DoFn.OnTimer @OnTimer} method for the given timer
+   * in the given window.
+   */
+  void onTimer(String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain);
+
+  /**
    * Calls a {@link DoFn DoFn's} {@link DoFn.FinishBundle @FinishBundle} method and performs
    * additional tasks, such as flushing in-memory states.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/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
index 9bfe9ae..290171a 100644
--- 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
@@ -24,6 +24,7 @@ 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.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowTracing;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -73,6 +74,12 @@ public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWin
   }
 
   @Override
+  public void onTimer(String timerId, BoundedWindow window, Instant timestamp,
+      TimeDomain timeDomain) {
+    doFnRunner.onTimer(timerId, window, timestamp, timeDomain);
+  }
+
+  @Override
   public void finishBundle() {
     doFnRunner.finishBundle();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/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
index 0bb9153..2962832 100644
--- 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
@@ -25,8 +25,10 @@ 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.TimeDomain;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Instant;
 
 /**
  * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
@@ -109,6 +111,12 @@ public class PushbackSideInputDoFnRunner<InputT, OutputT> implements DoFnRunner<
     underlying.processElement(elem);
   }
 
+  @Override
+  public void onTimer(String timerId, BoundedWindow window, Instant timestamp,
+      TimeDomain timeDomain) {
+    underlying.onTimer(timerId, window, timestamp, timeDomain);
+  }
+
   /**
    * Call the underlying {@link DoFnRunner#finishBundle()}.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index 29ef3ef..a7d82bf 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -50,8 +50,10 @@ 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.TimeDomain;
 import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerSpec;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingInternals;
@@ -64,6 +66,7 @@ import org.apache.beam.sdk.util.state.StateSpec;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.format.PeriodFormat;
 
@@ -161,6 +164,35 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
     }
   }
 
+  @Override
+  public void onTimer(
+      String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {
+
+    // The effective timestamp is when derived elements will have their timestamp set, if not
+    // otherwise specified. If this is an event time timer, then they have the timestamp of the
+    // timer itself. Otherwise, they are set to the input timestamp, which is by definition
+    // non-late.
+    Instant effectiveTimestamp;
+    switch (timeDomain) {
+      case EVENT_TIME:
+        effectiveTimestamp = timestamp;
+        break;
+
+      case PROCESSING_TIME:
+      case SYNCHRONIZED_PROCESSING_TIME:
+        effectiveTimestamp = context.stepContext.timerInternals().currentInputWatermarkTime();
+        break;
+
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unknown time domain: %s", timeDomain));
+    }
+
+    OnTimerArgumentProvider<InputT, OutputT> argumentProvider =
+        new OnTimerArgumentProvider<>(fn, context, window, effectiveTimestamp, timeDomain);
+    invoker.invokeOnTimer(timerId, argumentProvider);
+  }
+
   private void invokeProcessElement(WindowedValue<InputT> elem) {
     final DoFnProcessContext<InputT, OutputT> processContext = createProcessContext(elem);
 
@@ -630,7 +662,13 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
 
     @Override
     public Timer timer(String timerId) {
-      throw new UnsupportedOperationException("Timer parameters are not supported.");
+      try {
+        TimerSpec spec =
+            (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn);
+        return new TimerInternalsTimer(getNamespace(), timerId, spec, stepContext.timerInternals());
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
     }
 
     @Override
@@ -682,5 +720,201 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
         }
       };
     }
+
+  }
+
+  /**
+   * A concrete implementation of {@link DoFnInvoker.ArgumentProvider} used for running a {@link
+   * DoFn} on a timer.
+   *
+   * @param <InputT> the type of the {@link DoFn} (main) input elements
+   * @param <OutputT> the type of the {@link DoFn} (main) output elements
+   */
+  private class OnTimerArgumentProvider<InputT, OutputT>
+      extends DoFn<InputT, OutputT>.OnTimerContext
+      implements DoFnInvoker.ArgumentProvider<InputT, OutputT> {
+
+    final DoFn<InputT, OutputT> fn;
+    final DoFnContext<InputT, OutputT> context;
+    private final BoundedWindow window;
+    private final Instant timestamp;
+    private final TimeDomain timeDomain;
+
+    /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */
+    private StateNamespace namespace;
+
+    /**
+     * The state namespace for this context.
+     *
+     * <p>Any call to {@link #getNamespace()} when more than one window is present will crash; this
+     * represents a bug in the runner or the {@link DoFnSignature}, since values must be in exactly
+     * one window when state or timers are relevant.
+     */
+    private StateNamespace getNamespace() {
+      if (namespace == null) {
+        namespace = StateNamespaces.window(windowCoder, window);
+      }
+      return namespace;
+    }
+
+    private OnTimerArgumentProvider(
+        DoFn<InputT, OutputT> fn,
+        DoFnContext<InputT, OutputT> context,
+        BoundedWindow window,
+        Instant timestamp,
+        TimeDomain timeDomain) {
+      fn.super();
+      this.fn = fn;
+      this.context = context;
+      this.window = window;
+      this.timestamp = timestamp;
+      this.timeDomain = timeDomain;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return window;
+    }
+
+    @Override
+    public TimeDomain timeDomain() {
+      return timeDomain;
+    }
+
+    @Override
+    public Context context(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("Context parameters are not supported.");
+    }
+
+    @Override
+    public ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("ProcessContext parameters are not supported.");
+    }
+
+    @Override
+    public OnTimerContext onTimerContext(DoFn<InputT, OutputT> doFn) {
+      return this;
+    }
+
+    @Override
+    public InputProvider<InputT> inputProvider() {
+      throw new UnsupportedOperationException("InputProvider parameters are not supported.");
+    }
+
+    @Override
+    public OutputReceiver<OutputT> outputReceiver() {
+      throw new UnsupportedOperationException("OutputReceiver parameters are not supported.");
+    }
+
+    @Override
+    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+      throw new UnsupportedOperationException("RestrictionTracker parameters are not supported.");
+    }
+
+    @Override
+    public State state(String stateId) {
+      try {
+        StateSpec<?, ?> spec =
+            (StateSpec<?, ?>) signature.stateDeclarations().get(stateId).field().get(fn);
+        return stepContext
+            .stateInternals()
+            .state(getNamespace(), StateTags.tagForSpec(stateId, (StateSpec) spec));
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public Timer timer(String timerId) {
+      try {
+        TimerSpec spec =
+            (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn);
+        return new TimerInternalsTimer(getNamespace(), timerId, spec, stepContext.timerInternals());
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      context.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      context.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      context.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+        String name,
+        CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      throw new UnsupportedOperationException("Cannot createAggregator in @OnTimer method");
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      throw new UnsupportedOperationException("WindowingInternals are unsupported.");
+    }
+  }
+
+  private static class TimerInternalsTimer implements Timer {
+    private final TimerInternals timerInternals;
+    private final String timerId;
+    private final TimerSpec spec;
+    private final StateNamespace namespace;
+
+    public TimerInternalsTimer(
+        StateNamespace namespace, String timerId, TimerSpec spec, TimerInternals timerInternals) {
+      this.namespace = namespace;
+      this.timerId = timerId;
+      this.spec = spec;
+      this.timerInternals = timerInternals;
+    }
+
+    @Override
+    public void setForNowPlus(Duration durationFromNow) {
+      timerInternals.setTimer(
+          namespace, timerId, getCurrentTime().plus(durationFromNow), spec.getTimeDomain());
+    }
+
+    @Override
+    public void cancel() {
+      timerInternals.deleteTimer(namespace, timerId);
+    }
+
+    private Instant getCurrentTime() {
+      switch(spec.getTimeDomain()) {
+        case EVENT_TIME:
+          return timerInternals.currentInputWatermarkTime();
+        case PROCESSING_TIME:
+          return timerInternals.currentProcessingTime();
+        case SYNCHRONIZED_PROCESSING_TIME:
+          return timerInternals.currentSynchronizedProcessingTime();
+        default:
+          throw new IllegalStateException(
+              String.format("Timer created for unknown time domain %s", spec.getTimeDomain()));
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
index 1048fdc..342a4a8 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
@@ -39,6 +39,7 @@ 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.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -107,6 +108,13 @@ class SimpleOldDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT
     }
   }
 
+  @Override
+  public void onTimer(String timerId, BoundedWindow window, Instant timestamp,
+      TimeDomain timeDomain) {
+    throw new UnsupportedOperationException(
+        String.format("Timers are not supported by %s", OldDoFn.class.getSimpleName()));
+  }
+
   private 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.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/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
index 176ab26..a1cdbf6 100644
--- 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
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.runners.core;
 
+import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
 import static org.hamcrest.Matchers.equalTo;
@@ -37,7 +38,10 @@ 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.TimeDomain;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateNamespaces;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.hamcrest.Matchers;
@@ -215,8 +219,33 @@ public class PushbackSideInputDoFnRunnerTest {
     assertThat(underlying.inputElems, containsInAnyOrder(multiWindow));
   }
 
+  /** Tests that a call to onTimer gets delegated. */
+  @Test
+  public void testOnTimerCalled() {
+    PushbackSideInputDoFnRunner<Integer, Integer> runner =
+        createRunner(ImmutableList.<PCollectionView<?>>of());
+
+    String timerId = "fooTimer";
+    IntervalWindow window = new IntervalWindow(new Instant(4), new Instant(16));
+    Instant timestamp = new Instant(72);
+
+    // Mocking is not easily compatible with annotation analysis, so we manually record
+    // the method call.
+    runner.onTimer(timerId, window, new Instant(timestamp), TimeDomain.EVENT_TIME);
+
+    assertThat(
+        underlying.firedTimers,
+        contains(
+            TimerData.of(
+                timerId,
+                StateNamespaces.window(IntervalWindow.getCoder(), window),
+                timestamp,
+                TimeDomain.EVENT_TIME)));
+  }
+
   private static class TestDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
     List<WindowedValue<InputT>> inputElems;
+    List<TimerData> firedTimers;
     private boolean started = false;
     private boolean finished = false;
 
@@ -224,6 +253,7 @@ public class PushbackSideInputDoFnRunnerTest {
     public void startBundle() {
       started = true;
       inputElems = new ArrayList<>();
+      firedTimers = new ArrayList<>();
     }
 
     @Override
@@ -232,6 +262,17 @@ public class PushbackSideInputDoFnRunnerTest {
     }
 
     @Override
+    public void onTimer(String timerId, BoundedWindow window, Instant timestamp,
+        TimeDomain timeDomain) {
+      firedTimers.add(
+          TimerData.of(
+              timerId,
+              StateNamespaces.window(IntervalWindow.getCoder(), (IntervalWindow) window),
+              timestamp,
+              timeDomain));
+    }
+
+    @Override
     public void finishBundle() {
       finished = true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8af13b01/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..f068c19
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.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 org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.DoFn;
+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.WindowFn;
+import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link SimpleDoFnRunner}. */
+@RunWith(JUnit4.class)
+public class SimpleDoFnRunnerTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Mock StepContext mockStepContext;
+
+  @Mock TimerInternals mockTimerInternals;
+
+  @Before
+  public void setup() {
+    MockitoAnnotations.initMocks(this);
+    when(mockStepContext.timerInternals()).thenReturn(mockTimerInternals);
+  }
+
+  @Test
+  public void testProcessElementExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(new GlobalWindows()));
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+  }
+
+  @Test
+  public void testOnTimerExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(new GlobalWindows()));
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.onTimer(
+        ThrowingDoFn.TIMER_ID,
+        GlobalWindow.INSTANCE,
+        new Instant(0),
+        TimeDomain.EVENT_TIME);
+  }
+
+  /**
+   * Tests that a users call to set a timer gets properly dispatched to the timer internals. From
+   * there on, it is the duty of the runner & step context to set it in whatever way is right for
+   * that runner.
+   */
+  @Test
+  public void testTimerSet() {
+    WindowFn<?, ?> windowFn = new GlobalWindows();
+    DoFnWithTimers<GlobalWindow> fn = new DoFnWithTimers(windowFn.windowCoder());
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(new GlobalWindows()));
+
+    // Setting the timer needs the current time, as it is set relative
+    Instant currentTime = new Instant(42);
+    when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(currentTime);
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+
+    verify(mockTimerInternals)
+        .setTimer(
+            StateNamespaces.window(new GlobalWindows().windowCoder(), GlobalWindow.INSTANCE),
+            DoFnWithTimers.TIMER_ID,
+            currentTime.plus(DoFnWithTimers.TIMER_OFFSET),
+            TimeDomain.EVENT_TIME);
+  }
+
+  /**
+   * Tests that {@link SimpleDoFnRunner#onTimer} properly dispatches to the underlying
+   * {@link DoFn}.
+   */
+  @Test
+  public void testOnTimerCalled() {
+    WindowFn<?, GlobalWindow> windowFn = new GlobalWindows();
+    DoFnWithTimers<GlobalWindow> fn = new DoFnWithTimers(windowFn.windowCoder());
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(windowFn));
+
+    Instant currentTime = new Instant(42);
+    Duration offset = Duration.millis(37);
+
+    // Mocking is not easily compatible with annotation analysis, so we manually record
+    // the method call.
+    runner.onTimer(
+        DoFnWithTimers.TIMER_ID,
+        GlobalWindow.INSTANCE,
+        currentTime.plus(offset),
+        TimeDomain.EVENT_TIME);
+
+    assertThat(
+        fn.onTimerInvocations,
+        contains(
+            TimerData.of(
+                DoFnWithTimers.TIMER_ID,
+                StateNamespaces.window(windowFn.windowCoder(), GlobalWindow.INSTANCE),
+                currentTime.plus(offset),
+                TimeDomain.EVENT_TIME)));
+  }
+
+  static class ThrowingDoFn extends DoFn<String, String> {
+    final Exception exceptionToThrow = new UnsupportedOperationException("Expected exception");
+
+    static final String TIMER_ID = "throwingTimerId";
+
+    @TimerId(TIMER_ID)
+    private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      throw exceptionToThrow;
+    }
+
+    @OnTimer(TIMER_ID)
+    public void onTimer(OnTimerContext context) throws Exception {
+      throw exceptionToThrow;
+    }
+  }
+
+  private static class DoFnWithTimers<W extends BoundedWindow> extends DoFn<String, String> {
+    static final String TIMER_ID = "testTimerId";
+
+    static final Duration TIMER_OFFSET = Duration.millis(100);
+
+    private final Coder<W> windowCoder;
+
+    // Mutable
+    List<TimerData> onTimerInvocations;
+
+    DoFnWithTimers(Coder<W> windowCoder) {
+      this.windowCoder = windowCoder;
+      this.onTimerInvocations = new ArrayList<>();
+    }
+
+    @TimerId(TIMER_ID)
+    private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+    @ProcessElement
+    public void process(ProcessContext context, @TimerId(TIMER_ID) Timer timer) {
+      timer.setForNowPlus(TIMER_OFFSET);
+    }
+
+    @OnTimer(TIMER_ID)
+    public void onTimer(OnTimerContext context) {
+      onTimerInvocations.add(
+          TimerData.of(
+              DoFnWithTimers.TIMER_ID,
+              StateNamespaces.window(windowCoder, (W) context.window()),
+              context.timestamp(),
+              context.timeDomain()));
+    }
+  }
+}


[33/50] incubator-beam git commit: Show timestamps on log lines in Jenkins

Posted by ke...@apache.org.
Show timestamps on log lines in Jenkins


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

Branch: refs/heads/gearpump-runner
Commit: 4d964734c719037dd6e7e2a3c256da574514758a
Parents: 1ad638e
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Dec 15 19:51:22 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:11:05 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d964734/.jenkins/common_job_properties.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy
index f3a8a07..9b1a5fb 100644
--- a/.jenkins/common_job_properties.groovy
+++ b/.jenkins/common_job_properties.groovy
@@ -24,6 +24,7 @@ class common_job_properties {
   static def setTopLevelJobProperties(def context,
                                       def default_branch = 'master',
                                       def default_timeout = 100) {
+
     // GitHub project.
     context.properties {
       githubProjectUrl('https://github.com/apache/incubator-beam/')
@@ -47,7 +48,7 @@ class common_job_properties {
         remote {
           url('https://github.com/apache/incubator-beam.git')
           refspec('+refs/heads/*:refs/remotes/origin/* ' +
-                  '+refs/pull/*:refs/remotes/origin/pr/*')
+                  '+refs/pull/*/head:refs/remotes/origin/pr/*')
         }
         branch('${sha1}')
         extensions {
@@ -134,6 +135,8 @@ class common_job_properties {
   // Sets common config for Maven jobs.
   static def setMavenConfig(def context) {
     context.mavenInstallation('Maven 3.3.3')
+    context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true')
+    context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS')
     context.rootPOM('pom.xml')
     // Use a repository local to the workspace for better isolation of jobs.
     context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE)


[07/50] incubator-beam git commit: Revert "Removes ArgumentProvider.windowingInternals"

Posted by ke...@apache.org.
Revert "Removes ArgumentProvider.windowingInternals"

This reverts commit f3e8a0383bf9cb3f9452e0364f7deba113cadff9.


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

Branch: refs/heads/gearpump-runner
Commit: 4aa0ee1436a8d94f7c1c75bd0151790d14635c64
Parents: a12fd8c
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 16 15:26:32 2016 -0800
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Fri Dec 16 16:39:20 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnAdapters.java  | 14 +++++
 .../beam/runners/core/SimpleDoFnRunner.java     | 57 ++++++++++++++++++++
 .../beam/runners/core/SplittableParDo.java      |  7 +++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  7 +++
 .../sdk/transforms/reflect/DoFnInvoker.java     | 20 +++++++
 .../transforms/reflect/DoFnInvokersTest.java    |  6 +++
 6 files changed, 111 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
index a4002da..fc5847c 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
@@ -38,6 +38,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.state.State;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
@@ -355,6 +356,14 @@ public class DoFnAdapters {
     }
 
     @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this
+      // should be unreachable.
+      throw new UnsupportedOperationException(
+          "Can only get WindowingInternals in processElement");
+    }
+
+    @Override
     public DoFn.InputProvider<InputT> inputProvider() {
       throw new UnsupportedOperationException("inputProvider() exists only for testing");
     }
@@ -467,6 +476,11 @@ public class DoFnAdapters {
     }
 
     @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return context.windowingInternals();
+    }
+
+    @Override
     public DoFn.InputProvider<InputT> inputProvider() {
       throw new UnsupportedOperationException("inputProvider() exists only for testing");
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index d504b40..29ef3ef 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -51,10 +51,13 @@ 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.Timer;
+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.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.StateSpec;
@@ -416,6 +419,11 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
     }
 
     @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      throw new UnsupportedOperationException("WindowingInternals are unsupported.");
+    }
+
+    @Override
     public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
       throw new UnsupportedOperationException(
           "Cannot access RestrictionTracker outside of @ProcessElement method.");
@@ -625,5 +633,54 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
       throw new UnsupportedOperationException("Timer parameters are not supported.");
     }
 
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return new WindowingInternals<InputT, OutputT>() {
+        @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 StateInternals<?> stateInternals() {
+          return stepContext.stateInternals();
+        }
+
+        @Override
+        public void outputWindowedValue(
+            OutputT output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {
+          throw new UnsupportedOperationException("A DoFn cannot output to a different window");
+        }
+
+        @Override
+        public <SideOutputT> void sideOutputWindowedValue(
+            TupleTag<SideOutputT> tag,
+            SideOutputT output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {
+          throw new UnsupportedOperationException(
+              "A DoFn cannot side output to a different window");
+        }
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow sideInputWindow) {
+          return context.sideInput(view, sideInputWindow);
+        }
+      };
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
index f8d12ec..e6a2466 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
@@ -53,6 +53,7 @@ import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals;
 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.State;
 import org.apache.beam.sdk.util.state.StateInternals;
@@ -684,6 +685,12 @@ public class SplittableParDo<InputT, OutputT, RestrictionT>
       }
 
       @Override
+      public WindowingInternals<InputT, OutputT> windowingInternals() {
+        // DoFnSignatures should have verified that this DoFn doesn't access extra context.
+        throw new IllegalStateException("Unexpected extra context access on a splittable DoFn");
+      }
+
+      @Override
       public TrackerT restrictionTracker() {
         return tracker;
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/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 b2c3fd5..2d8684a 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
@@ -48,6 +48,7 @@ import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.Timer;
 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.state.InMemoryStateInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
@@ -327,6 +328,12 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
             }
 
             @Override
+            public WindowingInternals<InputT, OutputT> windowingInternals() {
+              throw new UnsupportedOperationException(
+                  "Not expected to access WindowingInternals from a new DoFn");
+            }
+
+            @Override
             public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
               throw new UnsupportedOperationException(
                   "Not expected to access RestrictionTracker from a regular DoFn in DoFnTester");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
index 354578e..97ac9d3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
@@ -27,9 +27,11 @@ import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.DoFn.StartBundle;
 import org.apache.beam.sdk.transforms.DoFn.StateId;
 import org.apache.beam.sdk.transforms.DoFn.TimerId;
+import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.state.State;
 
 /**
@@ -120,6 +122,19 @@ public interface DoFnInvoker<InputT, OutputT> {
     OutputReceiver<OutputT> outputReceiver();
 
     /**
+     * For migration from {@link OldDoFn} to {@link DoFn}, provide a {@link WindowingInternals} so
+     * an {@link OldDoFn} can be run via {@link DoFnInvoker}.
+     *
+     * <p>This is <i>not</i> exposed via the reflective capabilities of {@link DoFn}.
+     *
+     * @deprecated Please port occurences of {@link OldDoFn} to {@link DoFn}. If they require state
+     *     and timers, they will need to wait for the arrival of those features. Do not introduce
+     *     new uses of this method.
+     */
+    @Deprecated
+    WindowingInternals<InputT, OutputT> windowingInternals();
+
+    /**
      * If this is a splittable {@link DoFn}, returns the {@link RestrictionTracker} associated with
      * the current {@link ProcessElement} call.
      */
@@ -165,6 +180,11 @@ public interface DoFnInvoker<InputT, OutputT> {
     }
 
     @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return null;
+    }
+
+    @Override
     public State state(String stateId) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4aa0ee14/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
index 4233b39..456a6eb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
@@ -41,6 +41,7 @@ import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation;
+import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker.FakeArgumentProvider;
 import org.apache.beam.sdk.transforms.reflect.testhelper.DoFnInvokersTestHelper;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
@@ -51,6 +52,7 @@ import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerSpec;
 import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.state.StateSpec;
 import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.ValueState;
@@ -76,14 +78,18 @@ public class DoFnInvokersTest {
   @Mock private IntervalWindow mockWindow;
   @Mock private DoFn.InputProvider<String> mockInputProvider;
   @Mock private DoFn.OutputReceiver<String> mockOutputReceiver;
+  @Mock private WindowingInternals<String, String> mockWindowingInternals;
   @Mock private DoFnInvoker.ArgumentProvider<String, String> mockArgumentProvider;
 
+  @Mock private OldDoFn<String, String> mockOldDoFn;
+
   @Before
   public void setUp() {
     MockitoAnnotations.initMocks(this);
     when(mockArgumentProvider.window()).thenReturn(mockWindow);
     when(mockArgumentProvider.inputProvider()).thenReturn(mockInputProvider);
     when(mockArgumentProvider.outputReceiver()).thenReturn(mockOutputReceiver);
+    when(mockArgumentProvider.windowingInternals()).thenReturn(mockWindowingInternals);
     when(mockArgumentProvider.processContext(Matchers.<DoFn>any())).thenReturn(mockProcessContext);
   }
 


[41/50] incubator-beam git commit: Change counter name in TestDataflowRunner

Posted by ke...@apache.org.
Change counter name in TestDataflowRunner


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

Branch: refs/heads/gearpump-runner
Commit: a2f44923c8a8db533ad0bb6f545a96cf5007cfa5
Parents: d13f11f
Author: Joshua Litt <jo...@google.com>
Authored: Sat Dec 17 11:12:12 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Mon Dec 19 12:28:19 2016 -0800

----------------------------------------------------------------------
 .../dataflow/testing/TestDataflowRunner.java    | 29 ++++++++++++++++----
 .../testing/TestDataflowRunnerTest.java         | 16 ++++++++++-
 2 files changed, 39 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a2f44923/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
index 4b0fcf2..0564448 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java
@@ -61,7 +61,12 @@ import org.slf4j.LoggerFactory;
  */
 public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   private static final String TENTATIVE_COUNTER = "tentative";
-  private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
+  // See https://issues.apache.org/jira/browse/BEAM-1170
+  // we need to either fix the API or pipe the DRAINED signal through
+  @VisibleForTesting
+  static final String LEGACY_WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
+  @VisibleForTesting
+  static final String WATERMARK_METRIC_SUFFIX = "DataWatermark";
   private static final long MAX_WATERMARK_VALUE = -2L;
   private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class);
 
@@ -248,6 +253,23 @@ public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   }
 
   /**
+   * Checks wether a metric is a streaming watermark.
+   *
+   * @return true if the metric is a watermark.
+   */
+  boolean isWatermark(MetricUpdate metric) {
+    if (metric.getName() == null || metric.getName().getName() == null) {
+      return false; // no name -> shouldn't happen, not the watermark
+    }
+    if (metric.getScalar() == null) {
+      return false; // no scalar value -> not the watermark
+    }
+    String name = metric.getName().getName();
+    return name.endsWith(LEGACY_WATERMARK_METRIC_SUFFIX)
+        || name.endsWith(WATERMARK_METRIC_SUFFIX);
+  }
+
+  /**
    * Check watermarks of the streaming job. At least one watermark metric must exist.
    *
    * @return true if all watermarks are at max, false otherwise.
@@ -256,10 +278,7 @@ public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   boolean atMaxWatermark(DataflowPipelineJob job, JobMetrics metrics) {
     boolean hasMaxWatermark = false;
     for (MetricUpdate metric : metrics.getMetrics()) {
-      if (metric.getName() == null
-          || metric.getName().getName() == null
-          || !metric.getName().getName().endsWith(WATERMARK_METRIC_SUFFIX)
-          || metric.getScalar() == null) {
+      if (!isWatermark(metric)) {
         continue;
       }
       BigDecimal watermark = (BigDecimal) metric.getScalar();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a2f44923/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
index 366c6a1..da5630b 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.dataflow.testing;
 
+import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.LEGACY_WATERMARK_METRIC_SUFFIX;
+import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.WATERMARK_METRIC_SUFFIX;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -95,7 +97,6 @@ public class TestDataflowRunnerTest {
   @Mock private MockLowLevelHttpRequest request;
   @Mock private GcsUtil mockGcsUtil;
 
-  private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark";
   private static final BigDecimal DEFAULT_MAX_WATERMARK = new BigDecimal(-2);
 
   private TestDataflowPipelineOptions options;
@@ -411,6 +412,19 @@ public class TestDataflowRunnerTest {
   }
 
   @Test
+  public void testCheckMaxWatermarkWithLegacyWatermarkAtMax() throws IOException {
+    DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null));
+    Pipeline p = TestPipeline.create(options);
+    p.apply(Create.of(1, 2, 3));
+
+    TestDataflowRunner runner = (TestDataflowRunner) p.getRunner();
+    JobMetrics metrics = buildJobMetrics(generateMockStreamingMetrics(
+        ImmutableMap.of(LEGACY_WATERMARK_METRIC_SUFFIX, DEFAULT_MAX_WATERMARK)));
+    doReturn(State.RUNNING).when(job).getState();
+    assertTrue(runner.atMaxWatermark(job, metrics));
+  }
+
+  @Test
   public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOException {
     DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null));
     Pipeline p = TestPipeline.create(options);


[26/50] incubator-beam git commit: [BEAM-716] Use AutoValue in JmsIO

Posted by ke...@apache.org.
[BEAM-716] Use AutoValue 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/caf1c720
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/caf1c720
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/caf1c720

Branch: refs/heads/gearpump-runner
Commit: caf1c720f66de4d502f79b6c11c64b49c53329b0
Parents: 1c9bf8d
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Sun Dec 11 07:43:41 2016 +0100
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Mon Dec 19 07:24:00 2016 +0100

----------------------------------------------------------------------
 sdks/java/io/jms/pom.xml                        |   7 +
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 321 +++++++++++++------
 2 files changed, 228 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/caf1c720/sdks/java/io/jms/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml
index bca0152..b88254e 100644
--- a/sdks/java/io/jms/pom.xml
+++ b/sdks/java/io/jms/pom.xml
@@ -81,6 +81,13 @@
       <artifactId>jsr305</artifactId>
     </dependency>
 
+    <!-- compile dependencies -->
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
     <!-- test dependencies -->
     <dependency>
       <groupId>org.apache.activemq</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/caf1c720/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 24fa67d..76dee67 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
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.io.jms;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -101,37 +102,148 @@ public class JmsIO {
   private static final Logger LOG = LoggerFactory.getLogger(JmsIO.class);
 
   public static Read read() {
-    return new Read(null, null, null, Long.MAX_VALUE, null);
+    return new AutoValue_JmsIO_Read.Builder().setMaxNumRecords(Long.MAX_VALUE).build();
   }
 
   public static Write write() {
-    return new Write(null, null, null);
+    return new AutoValue_JmsIO_Write.Builder().build();
   }
 
   /**
    * A {@link PTransform} to read from a JMS destination. See {@link JmsIO} for more
    * information on usage and configuration.
    */
-  public static class Read extends PTransform<PBegin, PCollection<JmsRecord>> {
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<JmsRecord>> {
 
+    /**
+     * NB: According to http://docs.oracle.com/javaee/1.4/api/javax/jms/ConnectionFactory.html
+     * "It is expected that JMS providers will provide the tools an administrator needs to create
+     * and configure administered objects in a JNDI namespace. JMS provider implementations of
+     * administered objects should be both javax.jndi.Referenceable and java.io.Serializable so
+     * that they can be stored in all JNDI naming contexts. In addition, it is recommended that
+     * these implementations follow the JavaBeansTM design patterns."
+     *
+     * <p>So, a {@link ConnectionFactory} implementation is serializable.
+     */
+    @Nullable abstract ConnectionFactory getConnectionFactory();
+    @Nullable abstract String getQueue();
+    @Nullable abstract String getTopic();
+    abstract long getMaxNumRecords();
+    @Nullable abstract Duration getMaxReadTime();
+
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setConnectionFactory(ConnectionFactory connectionFactory);
+      abstract Builder setQueue(String queue);
+      abstract Builder setTopic(String topic);
+      abstract Builder setMaxNumRecords(long maxNumRecords);
+      abstract Builder setMaxReadTime(Duration maxReadTime);
+      abstract Read build();
+    }
+
+    /**
+     * <p>Specify the JMS connection factory to connect to the JMS broker.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    pipeline.apply(JmsIO.read().withConnectionFactory(myConnectionFactory)
+     *   }
+     * </pre>
+     *
+     * @param connectionFactory The JMS {@link ConnectionFactory}.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Read withConnectionFactory(ConnectionFactory connectionFactory) {
-      return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime);
+      return builder().setConnectionFactory(connectionFactory).build();
     }
 
+    /**
+     * <p>Specify the JMS queue destination name where to read messages from. The
+     * {@link JmsIO.Read} acts as a consumer on the queue.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Read#withTopic(String)}. The user has to
+     * specify a destination: queue or topic.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    pipeline.apply(JmsIO.read().withQueue("my-queue")
+     *   }
+     * </pre>
+     *
+     * @param queue The JMS queue name where to read messages from.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Read withQueue(String queue) {
-      return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime);
+      return builder().setQueue(queue).build();
     }
 
+    /**
+     * <p>Specify the JMS topic destination name where to receive messages from. The
+     * {@link JmsIO.Read} acts as a subscriber on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Read#withQueue(String)}. The user has to
+     * specify a destination: queue or topic.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    pipeline.apply(JmsIO.read().withTopic("my-topic")
+     *   }
+     * </pre>
+     *
+     * @param topic The JMS topic name.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Read withTopic(String topic) {
-      return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime);
+      return builder().setTopic(topic).build();
     }
 
+    /**
+     * <p>Define the max number of records that the source will read. Using a max number of records
+     * different from {@code Long.MAX_VALUE} means the source will be {@code Bounded}, and will
+     * stop once the max number of records read is reached.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    pipeline.apply(JmsIO.read().withNumRecords(1000)
+     *   }
+     * </pre>
+     *
+     * @param maxNumRecords The max number of records to read from the JMS destination.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Read withMaxNumRecords(long maxNumRecords) {
-      return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime);
+      return builder().setMaxNumRecords(maxNumRecords).build();
     }
 
+    /**
+     * <p>Define the max read time that the source will read. Using a non null max read time
+     * duration means the source will be {@code Bounded}, and will stop once the max read time is
+     * reached.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    pipeline.apply(JmsIO.read().withMaxReadTime(Duration.minutes(10))
+     *   }
+     * </pre>
+     *
+     * @param maxReadTime The max read time duration.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Read withMaxReadTime(Duration maxReadTime) {
-      return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime);
+      return builder().setMaxReadTime(maxReadTime).build();
     }
 
     @Override
@@ -141,10 +253,10 @@ public class JmsIO {
 
       PTransform<PBegin, PCollection<JmsRecord>> transform = unbounded;
 
-      if (maxNumRecords != Long.MAX_VALUE) {
-        transform = unbounded.withMaxNumRecords(maxNumRecords);
-      } else if (maxReadTime != null) {
-        transform = unbounded.withMaxReadTime(maxReadTime);
+      if (getMaxNumRecords() != Long.MAX_VALUE) {
+        transform = unbounded.withMaxNumRecords(getMaxNumRecords());
+      } else if (getMaxReadTime() != null) {
+        transform = unbounded.withMaxReadTime(getMaxReadTime());
       }
 
       return input.getPipeline().apply(transform);
@@ -152,65 +264,29 @@ public class JmsIO {
 
     @Override
     public void validate(PBegin input) {
-      checkNotNull(connectionFactory, "ConnectionFactory not specified");
-      checkArgument((queue != null || topic != null), "Either queue or topic not specified");
+      checkNotNull(getConnectionFactory(), "ConnectionFactory not specified");
+      checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic not "
+          + "specified");
     }
 
     @Override
     public void populateDisplayData(DisplayData.Builder builder) {
       super.populateDisplayData(builder);
-
-      builder.addIfNotNull(DisplayData.item("queue", queue));
-      builder.addIfNotNull(DisplayData.item("topic", topic));
+      builder.addIfNotNull(DisplayData.item("queue", getQueue()));
+      builder.addIfNotNull(DisplayData.item("topic", getTopic()));
 
     }
 
     ///////////////////////////////////////////////////////////////////////////////////////
 
     /**
-     * NB: According to http://docs.oracle.com/javaee/1.4/api/javax/jms/ConnectionFactory.html
-     * "It is expected that JMS providers will provide the tools an administrator needs to create
-     * and configure administered objects in a JNDI namespace. JMS provider implementations of
-     * administered objects should be both javax.jndi.Referenceable and java.io.Serializable so
-     * that they can be stored in all JNDI naming contexts. In addition, it is recommended that
-     * these implementations follow the JavaBeansTM design patterns."
-     *
-     * <p>So, a {@link ConnectionFactory} implementation is serializable.
-     */
-    protected ConnectionFactory connectionFactory;
-    @Nullable
-    protected String queue;
-    @Nullable
-    protected String topic;
-    protected long maxNumRecords;
-    protected Duration maxReadTime;
-
-    private Read(
-        ConnectionFactory connectionFactory,
-        String queue,
-        String topic,
-        long maxNumRecords,
-        Duration maxReadTime) {
-      super("JmsIO.Read");
-
-      this.connectionFactory = connectionFactory;
-      this.queue = queue;
-      this.topic = topic;
-      this.maxNumRecords = maxNumRecords;
-      this.maxReadTime = maxReadTime;
-    }
-
-    /**
      * Creates an {@link UnboundedSource UnboundedSource&lt;JmsRecord, ?&gt;} with the configuration
      * in {@link Read}. Primary use case is unit tests, should not be used in an
      * application.
      */
     @VisibleForTesting
     UnboundedSource<JmsRecord, JmsCheckpointMark> createSource() {
-      return new UnboundedJmsSource(
-          connectionFactory,
-          queue,
-          topic);
+      return new UnboundedJmsSource(this);
     }
 
   }
@@ -219,17 +295,10 @@ public class JmsIO {
 
   private static class UnboundedJmsSource extends UnboundedSource<JmsRecord, JmsCheckpointMark> {
 
-    private final ConnectionFactory connectionFactory;
-    private final String queue;
-    private final String topic;
+    private final Read spec;
 
-    public UnboundedJmsSource(
-        ConnectionFactory connectionFactory,
-        String queue,
-        String topic) {
-      this.connectionFactory = connectionFactory;
-      this.queue = queue;
-      this.topic = topic;
+    public UnboundedJmsSource(Read spec) {
+      this.spec = spec;
     }
 
     @Override
@@ -237,7 +306,7 @@ public class JmsIO {
         int desiredNumSplits, PipelineOptions options) throws Exception {
       List<UnboundedJmsSource> sources = new ArrayList<>();
       for (int i = 0; i < desiredNumSplits; i++) {
-        sources.add(new UnboundedJmsSource(connectionFactory, queue, topic));
+        sources.add(new UnboundedJmsSource(spec));
       }
       return sources;
     }
@@ -250,8 +319,7 @@ public class JmsIO {
 
     @Override
     public void validate() {
-      checkNotNull(connectionFactory, "ConnectionFactory is not defined");
-      checkArgument((queue != null || topic != null), "Either queue or topic is not defined");
+      spec.validate(null);
     }
 
     @Override
@@ -291,15 +359,17 @@ public class JmsIO {
 
     @Override
     public boolean start() throws IOException {
-      ConnectionFactory connectionFactory = source.connectionFactory;
+      ConnectionFactory connectionFactory = source.spec.getConnectionFactory();
       try {
         this.connection = connectionFactory.createConnection();
         this.connection.start();
         this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
-        if (source.topic != null) {
-          this.consumer = this.session.createConsumer(this.session.createTopic(source.topic));
+        if (source.spec.getTopic() != null) {
+          this.consumer =
+              this.session.createConsumer(this.session.createTopic(source.spec.getTopic()));
         } else {
-          this.consumer = this.session.createConsumer(this.session.createQueue(source.queue));
+          this.consumer =
+              this.session.createConsumer(this.session.createQueue(source.spec.getQueue()));
         }
 
         return advance();
@@ -409,70 +479,122 @@ public class JmsIO {
    * A {@link PTransform} to write to a JMS queue. See {@link JmsIO} for
    * more information on usage and configuration.
    */
-  public static class Write extends PTransform<PCollection<String>, PDone> {
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
 
-    protected ConnectionFactory connectionFactory;
-    protected String queue;
-    protected String topic;
+    @Nullable abstract ConnectionFactory getConnectionFactory();
+    @Nullable abstract String getQueue();
+    @Nullable abstract String getTopic();
 
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setConnectionFactory(ConnectionFactory connectionFactory);
+      abstract Builder setQueue(String queue);
+      abstract Builder setTopic(String topic);
+      abstract Write build();
+    }
+
+    /**
+     * <p>Specify the JMS connection factory to connect to the JMS broker.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    .apply(JmsIO.write().withConnectionFactory(myConnectionFactory)
+     *   }
+     * </pre>
+     *
+     * @param connectionFactory The JMS {@link ConnectionFactory}.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Write withConnectionFactory(ConnectionFactory connectionFactory) {
-      return new Write(connectionFactory, queue, topic);
+      return builder().setConnectionFactory(connectionFactory).build();
     }
 
+    /**
+     * <p>Specify the JMS queue destination name where to send messages to. The
+     * {@link JmsIO.Write} acts as a producer on the queue.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withTopic(String)}. The user has to
+     * specify a destination: queue or topic.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    .apply(JmsIO.write().withQueue("my-queue")
+     *   }
+     * </pre>
+     *
+     * @param queue The JMS queue name where to send messages to.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Write withQueue(String queue) {
-      return new Write(connectionFactory, queue, topic);
+      return builder().setQueue(queue).build();
     }
 
+    /**
+     * <p>Specify the JMS topic destination name where to send messages to. The
+     * {@link JmsIO.Read} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String)}. The user has to
+     * specify a destination: queue or topic.
+     *
+     * <p>For instance:
+     *
+     * <pre>
+     *   {@code
+     *    .apply(JmsIO.write().withTopic("my-topic")
+     *   }
+     * </pre>
+     *
+     * @param topic The JMS topic name.
+     * @return The corresponding {@link JmsIO.Read}.
+     */
     public Write withTopic(String topic) {
-      return new Write(connectionFactory, queue, topic);
-    }
-
-    private Write(ConnectionFactory connectionFactory, String queue, String topic) {
-      this.connectionFactory = connectionFactory;
-      this.queue = queue;
-      this.topic = topic;
+      return builder().setTopic(topic).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
-      input.apply(ParDo.of(new JmsWriter(connectionFactory, queue, topic)));
+      input.apply(ParDo.of(new WriterFn(this)));
       return PDone.in(input.getPipeline());
     }
 
     @Override
     public void validate(PCollection<String> input) {
-      checkNotNull(connectionFactory, "ConnectionFactory is not defined");
-      checkArgument((queue != null || topic != null), "Either queue or topic is required");
+      checkNotNull(getConnectionFactory(), "ConnectionFactory is not defined");
+      checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic is "
+          + "required");
     }
 
-    private static class JmsWriter extends DoFn<String, Void> {
+    private static class WriterFn extends DoFn<String, Void> {
 
-      private ConnectionFactory connectionFactory;
-      private String queue;
-      private String topic;
+      private Write spec;
 
       private Connection connection;
       private Session session;
       private MessageProducer producer;
 
-      public JmsWriter(ConnectionFactory connectionFactory, String queue, String topic) {
-        this.connectionFactory = connectionFactory;
-        this.queue = queue;
-        this.topic = topic;
+      public WriterFn(Write spec) {
+        this.spec = spec;
       }
 
       @StartBundle
       public void startBundle(Context c) throws Exception {
         if (producer == null) {
-          this.connection = connectionFactory.createConnection();
+          this.connection = spec.getConnectionFactory().createConnection();
           this.connection.start();
           // false means we don't use JMS transaction.
           this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
           Destination destination;
-          if (queue != null) {
-            destination = session.createQueue(queue);
+          if (spec.getQueue() != null) {
+            destination = session.createQueue(spec.getQueue());
           } else {
-            destination = session.createTopic(topic);
+            destination = session.createTopic(spec.getTopic());
           }
           this.producer = this.session.createProducer(destination);
         }
@@ -481,7 +603,6 @@ public class JmsIO {
       @ProcessElement
       public void processElement(ProcessContext ctx) throws Exception {
         String value = ctx.element();
-
         try {
           TextMessage message = session.createTextMessage(value);
           producer.send(message);


[08/50] incubator-beam git commit: Revert "Removes code for wrapping DoFn as an OldDoFn"

Posted by ke...@apache.org.
Revert "Removes code for wrapping DoFn as an OldDoFn"

This reverts commit a22de15012c51e8b7e31143021f0a298e093bf51.


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

Branch: refs/heads/gearpump-runner
Commit: a12fd8c580d3b1ea46c5be951f39046bfa0dacf3
Parents: abdbee6
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 16 15:26:28 2016 -0800
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Fri Dec 16 16:39:20 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnAdapters.java  | 150 ++++++++++
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 295 ++++++++++++++++++-
 .../sdk/transforms/reflect/DoFnInvokers.java    | 141 ++++++++-
 .../transforms/reflect/DoFnInvokersTest.java    |  36 +++
 4 files changed, 611 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a12fd8c5/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
index 0f5624f..a4002da 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java
@@ -18,6 +18,8 @@
 package org.apache.beam.runners.core;
 
 import java.io.IOException;
+import java.util.Collection;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.AggregatorRetriever;
@@ -39,6 +41,7 @@ import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.state.State;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypeDescriptor;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -53,6 +56,18 @@ public class DoFnAdapters {
   /** Should not be instantiated. */
   private DoFnAdapters() {}
 
+  /**
+   * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the
+   * original {@link DoFn}, otherwise returns {@code fn.getClass()}.
+   */
+  public static Class<?> getDoFnClass(OldDoFn<?, ?> fn) {
+    if (fn instanceof SimpleDoFnAdapter) {
+      return ((SimpleDoFnAdapter<?, ?>) fn).fn.getClass();
+    } else {
+      return fn.getClass();
+    }
+  }
+
   /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */
   @SuppressWarnings({"unchecked", "rawtypes"})
   public static <InputT, OutputT> OldDoFn<InputT, OutputT> toOldDoFn(DoFn<InputT, OutputT> fn) {
@@ -64,6 +79,126 @@ public class DoFnAdapters {
     }
   }
 
+  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
+  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.ProcessContext adaptProcessContext(
+      OldDoFn<InputT, OutputT> fn,
+      final DoFn<InputT, OutputT>.ProcessContext c,
+      final DoFnInvoker.ArgumentProvider<InputT, OutputT> extra) {
+    return fn.new ProcessContext() {
+      @Override
+      public InputT element() {
+        return c.element();
+      }
+
+      @Override
+      public <T> T sideInput(PCollectionView<T> view) {
+        return c.sideInput(view);
+      }
+
+      @Override
+      public Instant timestamp() {
+        return c.timestamp();
+      }
+
+      @Override
+      public BoundedWindow window() {
+        return extra.window();
+      }
+
+      @Override
+      public PaneInfo pane() {
+        return c.pane();
+      }
+
+      @Override
+      public WindowingInternals<InputT, OutputT> windowingInternals() {
+        return extra.windowingInternals();
+      }
+
+      @Override
+      public PipelineOptions getPipelineOptions() {
+        return c.getPipelineOptions();
+      }
+
+      @Override
+      public void output(OutputT output) {
+        c.output(output);
+      }
+
+      @Override
+      public void outputWithTimestamp(OutputT output, Instant timestamp) {
+        c.outputWithTimestamp(output, timestamp);
+      }
+
+      @Override
+      public <T> void sideOutput(TupleTag<T> tag, T output) {
+        c.sideOutput(tag, output);
+      }
+
+      @Override
+      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+        c.sideOutputWithTimestamp(tag, output, timestamp);
+      }
+
+      @Override
+      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+        return c.createAggregator(name, combiner);
+      }
+    };
+  }
+
+  /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */
+  public static <InputT, OutputT> OldDoFn<InputT, OutputT>.Context adaptContext(
+      OldDoFn<InputT, OutputT> fn,
+      final DoFn<InputT, OutputT>.Context c) {
+    return fn.new Context() {
+      @Override
+      public PipelineOptions getPipelineOptions() {
+        return c.getPipelineOptions();
+      }
+
+      @Override
+      public void output(OutputT output) {
+        c.output(output);
+      }
+
+      @Override
+      public void outputWithTimestamp(OutputT output, Instant timestamp) {
+        c.outputWithTimestamp(output, timestamp);
+      }
+
+      @Override
+      public <T> void sideOutput(TupleTag<T> tag, T output) {
+        c.sideOutput(tag, output);
+      }
+
+      @Override
+      public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+        c.sideOutputWithTimestamp(tag, output, timestamp);
+      }
+
+      @Override
+      protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+          String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+        return c.createAggregator(name, combiner);
+      }
+    };
+  }
+
+  /**
+   * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise,
+   * returns {@code null}.
+   */
+  @Nullable
+  public static <InputT, OutputT> DoFn<InputT, OutputT> getDoFn(OldDoFn<InputT, OutputT> fn) {
+    if (fn instanceof SimpleDoFnAdapter) {
+      return ((SimpleDoFnAdapter<InputT, OutputT>) fn).fn;
+    } else {
+      return null;
+    }
+  }
+
   /**
    * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link
    * OldDoFn}.
@@ -106,6 +241,21 @@ public class DoFnAdapters {
     }
 
     @Override
+    protected TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return fn.getInputTypeDescriptor();
+    }
+
+    @Override
+    protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return fn.getOutputTypeDescriptor();
+    }
+
+    @Override
+    Collection<Aggregator<?, ?>> getAggregators() {
+      return fn.getAggregators();
+    }
+
+    @Override
     public Duration getAllowedTimestampSkew() {
       return fn.getAllowedTimestampSkew();
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a12fd8c5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index 7b04533..d1bb42b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -38,6 +38,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypeDescriptor;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -70,6 +71,21 @@ import org.joda.time.Instant;
  */
 @Deprecated
 public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDisplayData {
+
+  public DoFn<InputT, OutputT> toDoFn() {
+    DoFn<InputT, OutputT> doFn = DoFnAdapters.getDoFn(this);
+    if (doFn != null) {
+      return doFn;
+    }
+    if (this instanceof RequiresWindowAccess) {
+      // No parameters as it just accesses `this`
+      return new AdaptedRequiresWindowAccessDoFn();
+    } else {
+      // No parameters as it just accesses `this`
+      return new AdaptedDoFn();
+    }
+  }
+
   /**
    * Information accessible to all methods in this {@code OldDoFn}.
    * Used primarily to output elements.
@@ -318,7 +334,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     this(new HashMap<String, DelegatingAggregator<?, ?>>());
   }
 
-  public OldDoFn(Map<String, DelegatingAggregator<?, ?>> aggregators) {
+  OldDoFn(Map<String, DelegatingAggregator<?, ?>> aggregators) {
     this.aggregators = aggregators;
   }
 
@@ -403,6 +419,32 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
   /////////////////////////////////////////////////////////////////////////////
 
   /**
+   * Returns a {@link TypeDescriptor} capturing what is known statically
+   * about the input type of this {@code OldDoFn} instance's most-derived
+   * class.
+   *
+   * <p>See {@link #getOutputTypeDescriptor} for more discussion.
+   */
+  protected TypeDescriptor<InputT> getInputTypeDescriptor() {
+    return new TypeDescriptor<InputT>(getClass()) {};
+  }
+
+  /**
+   * Returns a {@link TypeDescriptor} capturing what is known statically
+   * about the output type of this {@code OldDoFn} instance's
+   * most-derived class.
+   *
+   * <p>In the normal case of a concrete {@code OldDoFn} subclass with
+   * no generic type parameters of its own (including anonymous inner
+   * classes), this will be a complete non-generic type, which is good
+   * for choosing a default output {@code Coder<OutputT>} for the output
+   * {@code PCollection<OutputT>}.
+   */
+  protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+    return new TypeDescriptor<OutputT>(getClass()) {};
+  }
+
+  /**
    * Returns an {@link Aggregator} with aggregation logic specified by the
    * {@link CombineFn} argument. The name provided must be unique across
    * {@link Aggregator}s created within the OldDoFn. Aggregators can only be created
@@ -462,4 +504,255 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
   Collection<Aggregator<?, ?>> getAggregators() {
     return Collections.<Aggregator<?, ?>>unmodifiableCollection(aggregators.values());
   }
+
+  /**
+   * A {@link Context} for an {@link OldDoFn} via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedContext extends Context {
+
+    private final DoFn<InputT, OutputT>.Context newContext;
+
+    public AdaptedContext(
+        DoFn<InputT, OutputT>.Context newContext) {
+      this.newContext = newContext;
+      super.setupDelegateAggregators();
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return newContext.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      newContext.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      newContext.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      newContext.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      newContext.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return newContext.createAggregator(name, combiner);
+    }
+  }
+
+  /**
+   * A {@link ProcessContext} for an {@link OldDoFn} via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedProcessContext extends ProcessContext {
+
+    private final DoFn<InputT, OutputT>.ProcessContext newContext;
+
+    public AdaptedProcessContext(DoFn<InputT, OutputT>.ProcessContext newContext) {
+      this.newContext = newContext;
+    }
+
+    @Override
+    public InputT element() {
+      return newContext.element();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      return newContext.sideInput(view);
+    }
+
+    @Override
+    public Instant timestamp() {
+      return newContext.timestamp();
+    }
+
+    @Override
+    public BoundedWindow window() {
+      throw new UnsupportedOperationException(String.format(
+          "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s",
+          OldDoFn.class.getSimpleName(),
+          OldDoFn.ProcessContext.class.getSimpleName(),
+          OldDoFn.class.getSimpleName(),
+          DoFn.class.getSimpleName()));
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return newContext.pane();
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      throw new UnsupportedOperationException(String.format(
+          "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s",
+          OldDoFn.class.getSimpleName(),
+          OldDoFn.ProcessContext.class.getSimpleName(),
+          OldDoFn.class.getSimpleName(),
+          DoFn.class.getSimpleName()));
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return newContext.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      newContext.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      newContext.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      newContext.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      newContext.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return newContext.createAggregator(name, combiner);
+    }
+  }
+
+  private class AdaptedDoFn extends DoFn<InputT, OutputT> {
+
+    @Setup
+    public void setup() throws Exception {
+      OldDoFn.this.setup();
+    }
+
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c));
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      OldDoFn.this.teardown();
+    }
+
+    @Override
+    public Duration getAllowedTimestampSkew() {
+      return OldDoFn.this.getAllowedTimestampSkew();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      OldDoFn.this.populateDisplayData(builder);
+    }
+
+    @Override
+    public TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return OldDoFn.this.getInputTypeDescriptor();
+    }
+
+    @Override
+    Collection<Aggregator<?, ?>> getAggregators() {
+      return OldDoFn.this.getAggregators();
+    }
+
+    @Override
+    public TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return OldDoFn.this.getOutputTypeDescriptor();
+    }
+  }
+
+  /**
+   * A {@link ProcessContext} for an {@link OldDoFn} that implements
+   * {@link RequiresWindowAccess}, via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedRequiresWindowAccessProcessContext extends AdaptedProcessContext {
+
+    private final BoundedWindow window;
+
+    public AdaptedRequiresWindowAccessProcessContext(
+        DoFn<InputT, OutputT>.ProcessContext newContext,
+        BoundedWindow window) {
+      super(newContext);
+      this.window = window;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return window;
+    }
+  }
+
+  private class AdaptedRequiresWindowAccessDoFn extends DoFn<InputT, OutputT> {
+
+    @Setup
+    public void setup() throws Exception {
+      OldDoFn.this.setup();
+    }
+
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) throws Exception {
+      OldDoFn.this.processElement(
+          OldDoFn.this.new AdaptedRequiresWindowAccessProcessContext(c, window));
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      OldDoFn.this.teardown();
+    }
+
+    @Override
+    public Duration getAllowedTimestampSkew() {
+      return OldDoFn.this.getAllowedTimestampSkew();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      OldDoFn.this.populateDisplayData(builder);
+    }
+
+    @Override
+    public TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return OldDoFn.this.getInputTypeDescriptor();
+    }
+
+    @Override
+    public TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return OldDoFn.this.getOutputTypeDescriptor();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a12fd8c5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
index b141d51..50a7082 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
@@ -18,7 +18,13 @@
 package org.apache.beam.sdk.transforms.reflect;
 
 import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnAdapters;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.util.UserCodeException;
 
 /** Static utilities for working with {@link DoFnInvoker}. */
 public class DoFnInvokers {
@@ -36,22 +42,137 @@ public class DoFnInvokers {
     return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn);
   }
 
+  private DoFnInvokers() {}
+
   /**
-   * Temporarily retained for compatibility with Dataflow worker.
-   * TODO: delete this when Dataflow worker is fixed to call {@link #invokerFor(DoFn)}.
+   * Returns a {@link DoFnInvoker} for the given {@link Object}, which should be either a {@link
+   * DoFn} or an {@link OldDoFn}. The expected use would be to deserialize a user's function as an
+   * {@link Object} and then pass it to this method, so there is no need to statically specify what
+   * sort of object it is.
    *
-   * @deprecated Use {@link #invokerFor(DoFn)}.
+   * @deprecated this is to be used only as a migration path for decoupling upgrades
    */
-  @SuppressWarnings("unchecked")
   @Deprecated
-  public static <InputT, OutputT> DoFnInvoker<InputT, OutputT> invokerFor(
-      Serializable deserializedFn) {
+  public static DoFnInvoker<?, ?> invokerFor(Serializable deserializedFn) {
     if (deserializedFn instanceof DoFn) {
-      return invokerFor((DoFn<InputT, OutputT>) deserializedFn);
+      return invokerFor((DoFn<?, ?>) deserializedFn);
+    } else if (deserializedFn instanceof OldDoFn) {
+      return new OldDoFnInvoker<>((OldDoFn<?, ?>) deserializedFn);
+    } else {
+      throw new IllegalArgumentException(
+          String.format(
+              "Cannot create a %s for %s; it should be either a %s or an %s.",
+              DoFnInvoker.class.getSimpleName(),
+              deserializedFn.toString(),
+              DoFn.class.getSimpleName(),
+              OldDoFn.class.getSimpleName()));
     }
-    throw new UnsupportedOperationException(
-        "Only DoFn supported, was: " + deserializedFn.getClass());
   }
 
-  private DoFnInvokers() {}
+  /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */
+  @Deprecated public static final DoFnInvokers INSTANCE = new DoFnInvokers();
+
+  /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */
+  @Deprecated
+  public <InputT, OutputT> DoFnInvoker<InputT, OutputT> invokerFor(Object deserializedFn) {
+    return (DoFnInvoker<InputT, OutputT>) DoFnInvokers.invokerFor((Serializable) deserializedFn);
+  }
+
+
+  static class OldDoFnInvoker<InputT, OutputT> implements DoFnInvoker<InputT, OutputT> {
+
+    private final OldDoFn<InputT, OutputT> fn;
+
+    public OldDoFnInvoker(OldDoFn<InputT, OutputT> fn) {
+      this.fn = fn;
+    }
+
+    @Override
+    public DoFn.ProcessContinuation invokeProcessElement(
+        ArgumentProvider<InputT, OutputT> extra) {
+      // The outer DoFn is immaterial - it exists only to avoid typing InputT and OutputT repeatedly
+      DoFn<InputT, OutputT>.ProcessContext newCtx =
+          extra.processContext(new DoFn<InputT, OutputT>() {});
+      OldDoFn<InputT, OutputT>.ProcessContext oldCtx =
+          DoFnAdapters.adaptProcessContext(fn, newCtx, extra);
+      try {
+        fn.processElement(oldCtx);
+        return DoFn.ProcessContinuation.stop();
+      } catch (Throwable exc) {
+        throw UserCodeException.wrap(exc);
+      }
+    }
+
+    @Override
+    public void invokeOnTimer(String timerId, ArgumentProvider<InputT, OutputT> arguments) {
+      throw new UnsupportedOperationException(
+          String.format("Timers are not supported for %s", OldDoFn.class.getSimpleName()));
+    }
+
+    @Override
+    public void invokeStartBundle(DoFn.Context c) {
+      OldDoFn<InputT, OutputT>.Context oldCtx = DoFnAdapters.adaptContext(fn, c);
+      try {
+        fn.startBundle(oldCtx);
+      } catch (Throwable exc) {
+        throw UserCodeException.wrap(exc);
+      }
+    }
+
+    @Override
+    public void invokeFinishBundle(DoFn.Context c) {
+      OldDoFn<InputT, OutputT>.Context oldCtx = DoFnAdapters.adaptContext(fn, c);
+      try {
+        fn.finishBundle(oldCtx);
+      } catch (Throwable exc) {
+        throw UserCodeException.wrap(exc);
+      }
+    }
+
+    @Override
+    public void invokeSetup() {
+      try {
+        fn.setup();
+      } catch (Throwable exc) {
+        throw UserCodeException.wrap(exc);
+      }
+    }
+
+    @Override
+    public void invokeTeardown() {
+      try {
+        fn.teardown();
+      } catch (Throwable exc) {
+        throw UserCodeException.wrap(exc);
+      }
+    }
+
+    @Override
+    public <RestrictionT> RestrictionT invokeGetInitialRestriction(InputT element) {
+      throw new UnsupportedOperationException("OldDoFn is not splittable");
+    }
+
+    @Override
+    public <RestrictionT> Coder<RestrictionT> invokeGetRestrictionCoder(
+        CoderRegistry coderRegistry) {
+      throw new UnsupportedOperationException("OldDoFn is not splittable");
+    }
+
+    @Override
+    public <RestrictionT> void invokeSplitRestriction(
+        InputT element, RestrictionT restriction, DoFn.OutputReceiver<RestrictionT> receiver) {
+      throw new UnsupportedOperationException("OldDoFn is not splittable");
+    }
+
+    @Override
+    public <RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
+        TrackerT invokeNewTracker(RestrictionT restriction) {
+      throw new UnsupportedOperationException("OldDoFn is not splittable");
+    }
+
+    @Override
+    public DoFn<InputT, OutputT> getFn() {
+      throw new UnsupportedOperationException("getFn is not supported for OldDoFn");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a12fd8c5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
index 4c6bee1..4233b39 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.same;
+import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
@@ -731,4 +732,39 @@ public class DoFnInvokersTest {
     invoker.invokeOnTimer(timerId, mockArgumentProvider);
     assertThat(fn.window, equalTo(testWindow));
   }
+
+  private class OldDoFnIdentity extends OldDoFn<String, String> {
+    public void processElement(ProcessContext c) {}
+  }
+
+  @Test
+  public void testOldDoFnProcessElement() throws Exception {
+    new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn)
+        .invokeProcessElement(mockArgumentProvider);
+    verify(mockOldDoFn).processElement(any(OldDoFn.ProcessContext.class));
+  }
+
+  @Test
+  public void testOldDoFnStartBundle() throws Exception {
+    new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeStartBundle(mockProcessContext);
+    verify(mockOldDoFn).startBundle(any(OldDoFn.Context.class));
+  }
+
+  @Test
+  public void testOldDoFnFinishBundle() throws Exception {
+    new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeFinishBundle(mockProcessContext);
+    verify(mockOldDoFn).finishBundle(any(OldDoFn.Context.class));
+  }
+
+  @Test
+  public void testOldDoFnSetup() throws Exception {
+    new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeSetup();
+    verify(mockOldDoFn).setup();
+  }
+
+  @Test
+  public void testOldDoFnTeardown() throws Exception {
+    new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeTeardown();
+    verify(mockOldDoFn).teardown();
+  }
 }


[40/50] incubator-beam git commit: This closes #1659: More escaping in Jenkins timestamp spec

Posted by ke...@apache.org.
This closes #1659: More escaping in Jenkins timestamp spec


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

Branch: refs/heads/gearpump-runner
Commit: d13f11f8ca564a1d96f1cb351edac0b910c9d095
Parents: a972b23 627ccb5
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 11:50:01 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:50:01 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[19/50] incubator-beam git commit: BigQueryIO: fix streaming write, typo in API

Posted by ke...@apache.org.
BigQueryIO: fix streaming write, typo in API

and improve 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/5fb4f5de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/5fb4f5de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/5fb4f5de

Branch: refs/heads/gearpump-runner
Commit: 5fb4f5de9515db717818f1e3ffd7ca3c6eba5614
Parents: 4206408
Author: Sam McVeety <sg...@google.com>
Authored: Fri Dec 16 18:10:28 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 16 23:53:49 2016 -0800

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 23 +++++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 72 ++++++++++++--------
 2 files changed, 63 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5fb4f5de/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 28049ed..7bb1e51 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
@@ -370,7 +370,8 @@ public class BigQueryIO {
     }
   }
 
-  private static class TableSpecToTableRef
+  @VisibleForTesting
+  static class TableSpecToTableRef
       implements SerializableFunction<String, TableReference> {
     @Override
     public TableReference apply(String from) {
@@ -807,6 +808,7 @@ public class BigQueryIO {
       /**
        * Returns the query to be read, or {@code null} if reading from a table instead.
        */
+      @Nullable
       public String getQuery() {
         return query == null ? null : query.get();
       }
@@ -814,7 +816,8 @@ public class BigQueryIO {
       /**
        * Returns the query to be read, or {@code null} if reading from a table instead.
        */
-      public ValueProvider<String> getQueryProivder() {
+      @Nullable
+      public ValueProvider<String> getQueryProvider() {
         return query;
       }
 
@@ -2813,7 +2816,8 @@ public class BigQueryIO {
    * a randomUUID is generated only once per bucket of data. The actual unique
    * id is created by concatenating this randomUUID with a sequential number.
    */
-  private static class TagWithUniqueIdsAndTable
+  @VisibleForTesting
+  static class TagWithUniqueIdsAndTable
       extends DoFn<TableRow, KV<ShardedKey<String>, TableRowInfo>> {
     /** TableSpec to write to. */
     private final ValueProvider<String> tableSpec;
@@ -2830,8 +2834,12 @@ public class BigQueryIO {
       checkArgument(table == null ^ tableRefFunction == null,
           "Exactly one of table or tableRefFunction should be set");
       if (table != null) {
-        if (table.isAccessible() && table.get().getProjectId() == null) {
-          table.get().setProjectId(options.as(BigQueryOptions.class).getProject());
+        if (table.isAccessible() && Strings.isNullOrEmpty(table.get().getProjectId())) {
+          TableReference tableRef = table.get()
+              .setProjectId(options.as(BigQueryOptions.class).getProject());
+          table = NestedValueProvider.of(
+              StaticValueProvider.of(toJsonString(tableRef)),
+              new JsonTableRefToTableRef());
         }
         this.tableSpec = NestedValueProvider.of(table, new TableRefToTableSpec());
       } else {
@@ -2870,6 +2878,11 @@ public class BigQueryIO {
       }
     }
 
+    @VisibleForTesting
+    ValueProvider<String> getTableSpec() {
+      return tableSpec;
+    }
+
     private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window) {
       if (tableSpec != null) {
         return tableSpec.get();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5fb4f5de/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 b78316f..dc566d2 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
@@ -26,6 +26,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -2242,43 +2243,60 @@ public class BigQueryIOTest implements Serializable {
   }
 
   @Test
-  public void testRuntimeOptionsNotCalledInApplyInputTable() throws IOException {
+  public void testRuntimeOptionsNotCalledInApplyInputTable() {
     RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class);
     BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
-    bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
-    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
-        .withJobService(new FakeJobService());
+    bqOptions.setTempLocation("gs://testbucket/testdir");
     Pipeline pipeline = TestPipeline.create(options);
-    pipeline
-        .apply(BigQueryIO.Read
-            .from(options.getInputTable()).withoutValidation()
-            .withTestServices(fakeBqServices))
-            .apply(BigQueryIO.Write
-            .to(options.getOutputTable())
-            .withSchema(NestedValueProvider.of(
-                options.getOutputSchema(), new JsonSchemaToTableSchema()))
-            .withTestServices(fakeBqServices)
-            .withoutValidation());
+    BigQueryIO.Read.Bound read = BigQueryIO.Read.from(
+        options.getInputTable()).withoutValidation();
+    pipeline.apply(read);
+    // Test that this doesn't throw.
+    DisplayData.from(read);
   }
 
   @Test
-  public void testRuntimeOptionsNotCalledInApplyInputQuery() throws IOException {
+  public void testRuntimeOptionsNotCalledInApplyInputQuery() {
     RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class);
     BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
-    bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
-    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
-        .withJobService(new FakeJobService());
+    bqOptions.setTempLocation("gs://testbucket/testdir");
     Pipeline pipeline = TestPipeline.create(options);
+    BigQueryIO.Read.Bound read = BigQueryIO.Read.fromQuery(
+        options.getInputQuery()).withoutValidation();
+    pipeline.apply(read);
+    // Test that this doesn't throw.
+    DisplayData.from(read);
+  }
+
+  @Test
+  public void testRuntimeOptionsNotCalledInApplyOutput() {
+    RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class);
+    BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
+    bqOptions.setTempLocation("gs://testbucket/testdir");
+    Pipeline pipeline = TestPipeline.create(options);
+    BigQueryIO.Write.Bound write = BigQueryIO.Write
+        .to(options.getOutputTable())
+        .withSchema(NestedValueProvider.of(
+            options.getOutputSchema(), new JsonSchemaToTableSchema()))
+        .withoutValidation();
     pipeline
-        .apply(BigQueryIO.Read
-            .fromQuery(options.getInputQuery()).withoutValidation()
-            .withTestServices(fakeBqServices))
-            .apply(BigQueryIO.Write
-            .to(options.getOutputTable())
-            .withSchema(NestedValueProvider.of(
-                options.getOutputSchema(), new JsonSchemaToTableSchema()))
-            .withTestServices(fakeBqServices)
-            .withoutValidation());
+        .apply(Create.<TableRow>of())
+        .apply(write);
+    // Test that this doesn't throw.
+    DisplayData.from(write);
+  }
+
+  @Test
+  public void testTagWithUniqueIdsAndTableProjectNotNullWithNvp() {
+    BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class);
+    bqOptions.setProject("project");
+    BigQueryIO.TagWithUniqueIdsAndTable tag =
+        new BigQueryIO.TagWithUniqueIdsAndTable(
+            bqOptions, NestedValueProvider.of(
+                StaticValueProvider.of("data_set.table_name"),
+                new BigQueryIO.TableSpecToTableRef()), null);
+    TableReference table = BigQueryIO.parseTableSpec(tag.getTableSpec().get());
+    assertNotNull(table.getProjectId());
   }
 
   private static void testNumFiles(File tempDir, int expectedNumFiles) {


[44/50] incubator-beam git commit: [BEAM-59] initial interfaces and classes of Beam FileSystem

Posted by ke...@apache.org.
[BEAM-59] initial interfaces and classes of Beam FileSystem

This closes #1558


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

Branch: refs/heads/gearpump-runner
Commit: 28d7913be5d9bba9d4cb23187c59c9dfd3ab0cae
Parents: 4f97efc 467f7d1
Author: Luke Cwik <lc...@google.com>
Authored: Mon Dec 19 15:21:02 2016 -0800
Committer: Luke Cwik <lc...@google.com>
Committed: Mon Dec 19 15:21:02 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 ++++
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++++++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++++++++++++++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 ++++
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 +++++
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 +++++++++++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++++++
 sdks/java/io/google-cloud-platform/pom.xml      |   6 +
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 ++++
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 +++++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +++
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++++++
 sdks/java/io/hdfs/pom.xml                       |   6 +
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 ++++
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 +++++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 +++++++
 16 files changed, 732 insertions(+)
----------------------------------------------------------------------



[29/50] incubator-beam git commit: [BEAM-1178] Make naming of logger objects consistent

Posted by ke...@apache.org.
[BEAM-1178] Make naming of logger objects consistent


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

Branch: refs/heads/gearpump-runner
Commit: 4cf59170546da3689fca4352ccde259cee408331
Parents: fbfea59
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Sun Dec 18 21:01:13 2016 +0100
Committer: Isma�l Mej�a <ie...@gmail.com>
Committed: Mon Dec 19 09:42:22 2016 +0100

----------------------------------------------------------------------
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |  4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |  8 +--
 .../beam/sdk/metrics/MetricsEnvironment.java    |  6 +--
 .../org/apache/beam/sdk/util/ApiSurface.java    | 52 ++++++++++----------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    | 24 ++++-----
 .../io/gcp/bigtable/BigtableServiceImpl.java    |  6 +--
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   | 12 ++---
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |  8 +--
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  8 +--
 10 files changed, 66 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
index e30f8af..0212e8a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java
@@ -69,7 +69,7 @@ import org.slf4j.LoggerFactory;
  * @see ByteKey
  */
 public final class ByteKeyRange implements Serializable {
-  private static final Logger logger = LoggerFactory.getLogger(ByteKeyRange.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ByteKeyRange.class);
 
   /** The range of all keys, with empty start and end keys. */
   public static final ByteKeyRange ALL_KEYS = ByteKeyRange.of(ByteKey.EMPTY, ByteKey.EMPTY);
@@ -191,7 +191,7 @@ public final class ByteKeyRange implements Serializable {
     // Keys are equal subject to padding by 0.
     BigInteger range = rangeEndInt.subtract(rangeStartInt);
     if (range.equals(BigInteger.ZERO)) {
-      logger.warn(
+      LOG.warn(
           "Using 0.0 as the default fraction for this near-empty range {} where start and end keys"
               + " differ only by trailing zeros.",
           this);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
index 7c0f1c0..99717a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
@@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory;
  * @see ByteKeyRange
  */
 public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
-  private static final Logger logger = LoggerFactory.getLogger(ByteKeyRangeTracker.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ByteKeyRangeTracker.class);
 
   /** Instantiates a new {@link ByteKeyRangeTracker} with the specified range. */
   public static ByteKeyRangeTracker of(ByteKeyRange range) {
@@ -89,7 +89,7 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
   public synchronized boolean trySplitAtPosition(ByteKey splitPosition) {
     // Unstarted.
     if (position == null) {
-      logger.warn(
+      LOG.warn(
           "{}: Rejecting split request at {} because no records have been returned.",
           this,
           splitPosition);
@@ -98,7 +98,7 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
 
     // Started, but not after current position.
     if (splitPosition.compareTo(position) <= 0) {
-      logger.warn(
+      LOG.warn(
           "{}: Rejecting split request at {} because it is not after current position {}.",
           this,
           splitPosition,
@@ -108,7 +108,7 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
 
     // Sanity check.
     if (!range.containsKey(splitPosition)) {
-      logger.warn(
+      LOG.warn(
           "{}: Rejecting split request at {} because it is not within the range.",
           this,
           splitPosition);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java
index 5d7cb0b..2942578 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  */
 public class MetricsEnvironment {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(MetricsContainer.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsContainer.class);
 
   private static final AtomicBoolean METRICS_SUPPORTED = new AtomicBoolean(false);
   private static final AtomicBoolean REPORTED_MISSING_CONTAINER = new AtomicBoolean(false);
@@ -107,11 +107,11 @@ public class MetricsEnvironment {
     MetricsContainer container = CONTAINER_FOR_THREAD.get();
     if (container == null && REPORTED_MISSING_CONTAINER.compareAndSet(false, true)) {
       if (METRICS_SUPPORTED.get()) {
-        LOGGER.error(
+        LOG.error(
             "Unable to update metrics on the current thread. "
                 + "Most likely caused by using metrics outside the managed work-execution thread.");
       } else {
-        LOGGER.warn("Reporting metrics are not supported in the current execution environment.");
+        LOG.warn("Reporting metrics are not supported in the current execution environment.");
       }
     }
     return container;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
index 3914bb0..2040161 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
@@ -75,13 +75,13 @@ import org.slf4j.LoggerFactory;
  */
 @SuppressWarnings("rawtypes")
 public class ApiSurface {
-  private static Logger logger = LoggerFactory.getLogger(ApiSurface.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ApiSurface.class);
 
   /**
    * Returns an empty {@link ApiSurface}.
    */
   public static ApiSurface empty() {
-    logger.debug("Returning an empty ApiSurface");
+    LOG.debug("Returning an empty ApiSurface");
     return new ApiSurface(Collections.<Class<?>>emptySet(), Collections.<Pattern>emptySet());
   }
 
@@ -113,7 +113,7 @@ public class ApiSurface {
         newRootClasses.add(clazz);
       }
     }
-    logger.debug("Including package {} and subpackages: {}", packageName, newRootClasses);
+    LOG.debug("Including package {} and subpackages: {}", packageName, newRootClasses);
     newRootClasses.addAll(rootClasses);
 
     return new ApiSurface(newRootClasses, patternsToPrune);
@@ -124,7 +124,7 @@ public class ApiSurface {
    */
   public ApiSurface includingClass(Class<?> clazz) {
     Set<Class<?>> newRootClasses = Sets.newHashSet();
-    logger.debug("Including class {}", clazz);
+    LOG.debug("Including class {}", clazz);
     newRootClasses.add(clazz);
     newRootClasses.addAll(rootClasses);
     return new ApiSurface(newRootClasses, patternsToPrune);
@@ -360,7 +360,7 @@ public class ApiSurface {
    * See {@link #addExposedTypes(Type, Class)}.
    */
   private void addExposedTypes(TypeToken type, Class<?> cause) {
-    logger.debug(
+    LOG.debug(
         "Adding exposed types from {}, which is the type in type token {}", type.getType(), type);
     addExposedTypes(type.getType(), cause);
   }
@@ -372,19 +372,19 @@ public class ApiSurface {
    */
   private void addExposedTypes(Type type, Class<?> cause) {
     if (type instanceof TypeVariable) {
-      logger.debug("Adding exposed types from {}, which is a type variable", type);
+      LOG.debug("Adding exposed types from {}, which is a type variable", type);
       addExposedTypes((TypeVariable) type, cause);
     } else if (type instanceof WildcardType) {
-      logger.debug("Adding exposed types from {}, which is a wildcard type", type);
+      LOG.debug("Adding exposed types from {}, which is a wildcard type", type);
       addExposedTypes((WildcardType) type, cause);
     } else if (type instanceof GenericArrayType) {
-      logger.debug("Adding exposed types from {}, which is a generic array type", type);
+      LOG.debug("Adding exposed types from {}, which is a generic array type", type);
       addExposedTypes((GenericArrayType) type, cause);
     } else if (type instanceof ParameterizedType) {
-      logger.debug("Adding exposed types from {}, which is a parameterized type", type);
+      LOG.debug("Adding exposed types from {}, which is a parameterized type", type);
       addExposedTypes((ParameterizedType) type, cause);
     } else if (type instanceof Class) {
-      logger.debug("Adding exposed types from {}, which is a class", type);
+      LOG.debug("Adding exposed types from {}, which is a class", type);
       addExposedTypes((Class) type, cause);
     } else {
       throw new IllegalArgumentException("Unknown implementation of Type");
@@ -402,7 +402,7 @@ public class ApiSurface {
     }
     visit(type);
     for (Type bound : type.getBounds()) {
-      logger.debug("Adding exposed types from {}, which is a type bound on {}", bound, type);
+      LOG.debug("Adding exposed types from {}, which is a type bound on {}", bound, type);
       addExposedTypes(bound, cause);
     }
   }
@@ -414,14 +414,14 @@ public class ApiSurface {
   private void addExposedTypes(WildcardType type, Class<?> cause) {
     visit(type);
     for (Type lowerBound : type.getLowerBounds()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is a type lower bound on wildcard type {}",
           lowerBound,
           type);
       addExposedTypes(lowerBound, cause);
     }
     for (Type upperBound : type.getUpperBounds()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is a type upper bound on wildcard type {}",
           upperBound,
           type);
@@ -439,7 +439,7 @@ public class ApiSurface {
       return;
     }
     visit(type);
-    logger.debug(
+    LOG.debug(
         "Adding exposed types from {}, which is the component type on generic array type {}",
         type.getGenericComponentType(),
         type);
@@ -467,13 +467,13 @@ public class ApiSurface {
     // The type parameters themselves may not be pruned,
     // for example with List<MyApiType> probably the
     // standard List is pruned, but MyApiType is not.
-    logger.debug(
+    LOG.debug(
         "Adding exposed types from {}, which is the raw type on parameterized type {}",
         type.getRawType(),
         type);
     addExposedTypes(type.getRawType(), cause);
     for (Type typeArg : type.getActualTypeArguments()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is a type argument on parameterized type {}",
           typeArg,
           type);
@@ -501,14 +501,14 @@ public class ApiSurface {
     TypeToken<?> token = TypeToken.of(clazz);
     for (TypeToken<?> superType : token.getTypes()) {
       if (!superType.equals(token)) {
-        logger.debug(
+        LOG.debug(
             "Adding exposed types from {}, which is a super type token on {}", superType, clazz);
         addExposedTypes(superType, clazz);
       }
     }
     for (Class innerClass : clazz.getDeclaredClasses()) {
       if (exposed(innerClass.getModifiers())) {
-        logger.debug(
+        LOG.debug(
             "Adding exposed types from {}, which is an exposed inner class of {}",
             innerClass,
             clazz);
@@ -517,12 +517,12 @@ public class ApiSurface {
     }
     for (Field field : clazz.getDeclaredFields()) {
       if (exposed(field.getModifiers())) {
-        logger.debug("Adding exposed types from {}, which is an exposed field on {}", field, clazz);
+        LOG.debug("Adding exposed types from {}, which is an exposed field on {}", field, clazz);
         addExposedTypes(field, clazz);
       }
     }
     for (Invokable invokable : getExposedInvokables(token)) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is an exposed invokable on {}", invokable, clazz);
       addExposedTypes(invokable, clazz);
     }
@@ -531,21 +531,21 @@ public class ApiSurface {
   private void addExposedTypes(Invokable<?, ?> invokable, Class<?> cause) {
     addExposedTypes(invokable.getReturnType(), cause);
     for (Annotation annotation : invokable.getAnnotations()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is an annotation on invokable {}",
           annotation,
           invokable);
      addExposedTypes(annotation.annotationType(), cause);
     }
     for (Parameter parameter : invokable.getParameters()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is a parameter on invokable {}",
           parameter,
           invokable);
       addExposedTypes(parameter, cause);
     }
     for (TypeToken<?> exceptionType : invokable.getExceptionTypes()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is an exception type on invokable {}",
           exceptionType,
           invokable);
@@ -554,13 +554,13 @@ public class ApiSurface {
   }
 
   private void addExposedTypes(Parameter parameter, Class<?> cause) {
-    logger.debug(
+    LOG.debug(
         "Adding exposed types from {}, which is the type of parameter {}",
         parameter.getType(),
         parameter);
     addExposedTypes(parameter.getType(), cause);
     for (Annotation annotation : parameter.getAnnotations()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is an annotation on parameter {}",
           annotation,
           parameter);
@@ -571,7 +571,7 @@ public class ApiSurface {
   private void addExposedTypes(Field field, Class<?> cause) {
     addExposedTypes(field.getGenericType(), cause);
     for (Annotation annotation : field.getDeclaredAnnotations()) {
-      logger.debug(
+      LOG.debug(
           "Adding exposed types from {}, which is an annotation on field {}", annotation, field);
       addExposedTypes(annotation.annotationType(), cause);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index a83784b..c27389d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -154,7 +154,7 @@ import org.slf4j.LoggerFactory;
  */
 @Experimental
 public class BigtableIO {
-  private static final Logger logger = LoggerFactory.getLogger(BigtableIO.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BigtableIO.class);
 
   /**
    * Creates an uninitialized {@link BigtableIO.Read}. Before use, the {@code Read} must be
@@ -307,7 +307,7 @@ public class BigtableIO {
             "Table %s does not exist",
             tableId);
       } catch (IOException e) {
-        logger.warn("Error checking whether table {} exists; proceeding.", tableId, e);
+        LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e);
       }
     }
 
@@ -521,7 +521,7 @@ public class BigtableIO {
             "Table %s does not exist",
             tableId);
       } catch (IOException e) {
-        logger.warn("Error checking whether table {} exists; proceeding.", tableId, e);
+        LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e);
       }
     }
 
@@ -612,7 +612,7 @@ public class BigtableIO {
       public void finishBundle(Context c) throws Exception {
         bigtableWriter.flush();
         checkForFailures();
-        logger.info("Wrote {} records", recordsWritten);
+        LOG.info("Wrote {} records", recordsWritten);
       }
 
       @Teardown
@@ -658,7 +658,7 @@ public class BigtableIO {
                 i + failures.size(),
                 i,
                 logEntry.toString());
-        logger.error(message);
+        LOG.error(message);
         throw new IOException(message);
       }
 
@@ -762,11 +762,11 @@ public class BigtableIO {
         long desiredBundleSizeBytes, List<SampleRowKeysResponse> sampleRowKeys) {
       // There are no regions, or no samples available. Just scan the entire range.
       if (sampleRowKeys.isEmpty()) {
-        logger.info("Not splitting source {} because no sample row keys are available.", this);
+        LOG.info("Not splitting source {} because no sample row keys are available.", this);
         return Collections.singletonList(this);
       }
 
-      logger.info(
+      LOG.info(
           "About to split into bundles of size {} with sampleRowKeys length {} first element {}",
           desiredBundleSizeBytes,
           sampleRowKeys.size(),
@@ -832,7 +832,7 @@ public class BigtableIO {
       }
 
       List<BigtableSource> ret = splits.build();
-      logger.info("Generated {} splits. First split: {}", ret.size(), ret.get(0));
+      LOG.info("Generated {} splits. First split: {}", ret.size(), ret.get(0));
       return ret;
     }
 
@@ -912,7 +912,7 @@ public class BigtableIO {
     private List<BigtableSource> splitKeyRangeIntoBundleSizedSubranges(
         long sampleSizeBytes, long desiredBundleSizeBytes, ByteKeyRange range) {
       // Catch the trivial cases. Split is small enough already, or this is the last region.
-      logger.debug(
+      LOG.debug(
           "Subsplit for sampleSizeBytes {} and desiredBundleSizeBytes {}",
           sampleSizeBytes,
           desiredBundleSizeBytes);
@@ -1010,7 +1010,7 @@ public class BigtableIO {
 
     @Override
     public void close() throws IOException {
-      logger.info("Closing reader after reading {} records.", recordsReturned);
+      LOG.info("Closing reader after reading {} records.", recordsReturned);
       if (reader != null) {
         reader.close();
         reader = null;
@@ -1033,11 +1033,11 @@ public class BigtableIO {
       try {
         splitKey = rangeTracker.getRange().interpolateKey(fraction);
       } catch (IllegalArgumentException e) {
-        logger.info(
+        LOG.info(
             "%s: Failed to interpolate key for fraction %s.", rangeTracker.getRange(), fraction);
         return null;
       }
-      logger.debug(
+      LOG.debug(
           "Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey);
       BigtableSource primary = source.withEndKey(splitKey);
       BigtableSource residual = source.withStartKey(splitKey);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
index 7ce4b4a..1a4937c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  * service.
  */
 class BigtableServiceImpl implements BigtableService {
-  private static final Logger logger = LoggerFactory.getLogger(BigtableService.class);
+  private static final Logger LOG = LoggerFactory.getLogger(BigtableService.class);
 
   public BigtableServiceImpl(BigtableOptions options) {
     this.options = options;
@@ -75,7 +75,7 @@ class BigtableServiceImpl implements BigtableService {
   @Override
   public boolean tableExists(String tableId) throws IOException {
     if (!BigtableSession.isAlpnProviderEnabled()) {
-      logger.info(
+      LOG.info(
           "Skipping existence check for table {} (BigtableOptions {}) because ALPN is not"
               + " configured.",
           tableId,
@@ -97,7 +97,7 @@ class BigtableServiceImpl implements BigtableService {
       String message =
           String.format(
               "Error checking whether table %s (BigtableOptions %s) exists", tableId, options);
-      logger.error(message, e);
+      LOG.error(message, e);
       throw new IOException(message, e);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
index d09929d..aa93a22 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
@@ -57,7 +57,7 @@ import org.slf4j.LoggerFactory;
  * Test on the JdbcIO.
  */
 public class JdbcIOTest implements Serializable {
-  private static final Logger LOGGER = LoggerFactory.getLogger(JdbcIOTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcIOTest.class);
 
   private static NetworkServerControl derbyServer;
   private static ClientDataSource dataSource;
@@ -70,7 +70,7 @@ public class JdbcIOTest implements Serializable {
     port = socket.getLocalPort();
     socket.close();
 
-    LOGGER.info("Starting Derby database on {}", port);
+    LOG.info("Starting Derby database on {}", port);
 
     System.setProperty("derby.stream.error.file", "target/derby.log");
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/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
index f539431..de2eb16 100644
--- 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
@@ -93,7 +93,7 @@ import org.slf4j.LoggerFactory;
  */
 public class MongoDbIO {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIO.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MongoDbIO.class);
 
   /** Read data from MongoDB. */
   public static Read read() {
@@ -253,19 +253,19 @@ public class MongoDbIO {
       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);
+      LOG.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<Document>> sources = new ArrayList<>();
       if (splitKeys.size() < 1) {
-        LOGGER.debug("Split keys is low, using an unique source");
+        LOG.debug("Split keys is low, using an unique source");
         sources.add(this);
         return sources;
       }
 
-      LOGGER.debug("Number of splits is {}", splitKeys.size());
+      LOG.debug("Number of splits is {}", splitKeys.size());
       for (String shardFilter : splitKeysToFilters(splitKeys, spec.filter())) {
         sources.add(new BoundedMongoDbSource(spec.withFilter(shardFilter)));
       }
@@ -392,12 +392,12 @@ public class MongoDbIO {
           cursor.close();
         }
       } catch (Exception e) {
-        LOGGER.warn("Error closing MongoDB cursor", e);
+        LOG.warn("Error closing MongoDB cursor", e);
       }
       try {
         client.close();
       } catch (Exception e) {
-        LOGGER.warn("Error closing MongoDB client", e);
+        LOG.warn("Error closing MongoDB client", e);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
index 5061487..df05c93 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
@@ -88,7 +88,7 @@ import org.slf4j.LoggerFactory;
  * Test on the MongoDbGridFSIO.
  */
 public class MongoDBGridFSIOTest implements Serializable {
-  private static final Logger LOGGER = LoggerFactory.getLogger(MongoDBGridFSIOTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MongoDBGridFSIOTest.class);
 
   private static final String MONGODB_LOCATION = "target/mongodb";
   private static final String DATABASE = "gridfs";
@@ -105,7 +105,7 @@ public class MongoDBGridFSIOTest implements Serializable {
     try (ServerSocket serverSocket = new ServerSocket(0)) {
       port = serverSocket.getLocalPort();
     }
-    LOGGER.info("Starting MongoDB embedded instance on {}", port);
+    LOG.info("Starting MongoDB embedded instance on {}", port);
     try {
       Files.forceDelete(new File(MONGODB_LOCATION));
     } catch (Exception e) {
@@ -127,7 +127,7 @@ public class MongoDBGridFSIOTest implements Serializable {
     mongodExecutable = mongodStarter.prepare(mongodConfig);
     mongodProcess = mongodExecutable.start();
 
-    LOGGER.info("Insert test data");
+    LOG.info("Insert test data");
 
     Mongo client = new Mongo("localhost", port);
     DB database = client.getDB(DATABASE);
@@ -174,7 +174,7 @@ public class MongoDBGridFSIOTest implements Serializable {
 
   @AfterClass
   public static void stop() throws Exception {
-    LOGGER.info("Stopping MongoDB instance");
+    LOG.info("Stopping MongoDB instance");
     mongodProcess.stop();
     mongodExecutable.stop();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4cf59170/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
index 129e81c..5faa618 100644
--- 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
@@ -67,7 +67,7 @@ import org.slf4j.LoggerFactory;
  */
 public class MongoDbIOTest implements Serializable {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIOTest.class);
+  private static final Logger LOG = LoggerFactory.getLogger(MongoDbIOTest.class);
 
   private static final String MONGODB_LOCATION = "target/mongodb";
   private static final String DATABASE = "beam";
@@ -92,7 +92,7 @@ public class MongoDbIOTest implements Serializable {
 
   @Before
   public void setup() throws Exception {
-    LOGGER.info("Starting MongoDB embedded instance on {}", port);
+    LOG.info("Starting MongoDB embedded instance on {}", port);
     try {
       Files.forceDelete(new File(MONGODB_LOCATION));
     } catch (Exception e) {
@@ -114,7 +114,7 @@ public class MongoDbIOTest implements Serializable {
     mongodExecutable = mongodStarter.prepare(mongodConfig);
     mongodProcess = mongodExecutable.start();
 
-    LOGGER.info("Insert test data");
+    LOG.info("Insert test data");
 
     MongoClient client = new MongoClient("localhost", port);
     MongoDatabase database = client.getDatabase(DATABASE);
@@ -135,7 +135,7 @@ public class MongoDbIOTest implements Serializable {
 
   @After
   public void stop() throws Exception {
-    LOGGER.info("Stopping MongoDB instance");
+    LOG.info("Stopping MongoDB instance");
     mongodProcess.stop();
     mongodExecutable.stop();
   }


[09/50] incubator-beam git commit: View.asMap: minor javadoc fixes

Posted by ke...@apache.org.
View.asMap: minor javadoc fixes


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

Branch: refs/heads/gearpump-runner
Commit: 6a4a699796fcf8a294ee0886658e6597bede0207
Parents: abdbee6
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 16 16:26:27 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 16 16:49:19 2016 -0800

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/beam/sdk/transforms/View.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6a4a6997/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
index 126679d..d18a0c6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
@@ -189,9 +189,9 @@ public class View {
 
   /**
    * Returns a {@link View.AsMap} transform that takes a
-   * {@link PCollection PCollection&lt;KV&lt;K V&gt;&gt;} as
+   * {@link PCollection PCollection&lt;KV&lt;K, V&gt;&gt;} as
    * input and produces a {@link PCollectionView} mapping each window to
-   * a {@link Map Map&gt;K, V&gt;}. It is required that each key of the input be
+   * a {@link Map Map&lt;K, V&gt;}. It is required that each key of the input be
    * associated with a single value, per window. If this is not the case, precede this
    * view with {@code Combine.perKey}, as in the example below, or alternatively
    * use {@link View#asMultimap()}.


[22/50] incubator-beam git commit: [BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage

Posted by ke...@apache.org.
[BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage


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

Branch: refs/heads/gearpump-runner
Commit: 443b25a4d11201fb88f40da437ec7aab4b3e273f
Parents: 33b7ca7
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 13 19:27:41 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sat Dec 17 14:11:39 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/TestPipeline.java   | 207 ++++++++++++++++---
 .../beam/sdk/testing/TestPipelineTest.java      | 183 ++++++++++++++--
 2 files changed, 344 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/443b25a4/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 493d4cc..49ac3af 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -23,12 +23,17 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Optional;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.base.Strings;
+import com.google.common.collect.FluentIterable;
 import com.google.common.collect.Iterators;
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map.Entry;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.Pipeline;
@@ -39,34 +44,39 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.TestCredential;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
 
 /**
- * A creator of test pipelines that can be used inside of tests that can be
- * configured to run locally or against a remote pipeline runner.
+ * A creator of test pipelines that can be used inside of tests that can be configured to run
+ * locally or against a remote pipeline runner.
  *
- * <p>It is recommended to tag hand-selected tests for this purpose using the
- * {@link RunnableOnService} {@link Category} annotation, as each test run against a pipeline runner
- * will utilize resources of that pipeline runner.
+ * <p>It is recommended to tag hand-selected tests for this purpose using the {@link
+ * RunnableOnService} {@link Category} annotation, as each test run against a pipeline runner will
+ * utilize resources of that pipeline runner.
  *
  * <p>In order to run tests on a pipeline runner, the following conditions must be met:
+ *
  * <ul>
- *   <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
- *   options. For example:
- *   <pre>{@code [
+ * <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
+ *     options. For example:
+ *     <pre>{@code [
  *     "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner",
  *     "--project=mygcpproject",
  *     "--stagingLocation=gs://mygcsbucket/path"
  *     ]}</pre>
  *     Note that the set of pipeline options required is pipeline runner specific.
- *   </li>
- *   <li>Jars containing the SDK and test classes must be available on the classpath.</li>
+ * <li>Jars containing the SDK and test classes must be available on the classpath.
  * </ul>
  *
  * <p>Use {@link PAssert} for tests, as it integrates with this test harness in both direct and
  * remote execution modes. For example:
+ *
  * <pre>{@code
  * Pipeline p = TestPipeline.create();
  * PCollection<Integer> output = ...
@@ -76,19 +86,136 @@ import org.junit.experimental.categories.Category;
  * p.run();
  * }</pre>
  *
- * <p>For pipeline runners, it is required that they must throw an {@link AssertionError}
- * containing the message from the {@link PAssert} that failed.
+ * <p>For pipeline runners, it is required that they must throw an {@link AssertionError} containing
+ * the message from the {@link PAssert} that failed.
  */
-public class TestPipeline extends Pipeline {
+public class TestPipeline extends Pipeline implements TestRule {
+
+  private static class PipelineRunEnforcement {
+
+    protected boolean enableAutoRunIfMissing;
+    protected final Pipeline pipeline;
+    private boolean runInvoked;
+
+    private PipelineRunEnforcement(final Pipeline pipeline) {
+      this.pipeline = pipeline;
+    }
+
+    private void enableAutoRunIfMissing(final boolean enable) {
+      enableAutoRunIfMissing = enable;
+    }
+
+    protected void beforePipelineExecution() {
+      runInvoked = true;
+    }
+
+    protected void afterTestCompletion() {
+      if (!runInvoked && enableAutoRunIfMissing) {
+        pipeline.run().waitUntilFinish();
+      }
+    }
+  }
+
+  private static class PipelineAbandonedNodeEnforcement extends PipelineRunEnforcement {
+
+    private List<TransformHierarchy.Node> runVisitedNodes;
+
+    private final Predicate<TransformHierarchy.Node> isPAssertNode =
+        new Predicate<TransformHierarchy.Node>() {
+
+          @Override
+          public boolean apply(final TransformHierarchy.Node node) {
+            return node.getTransform() instanceof PAssert.GroupThenAssert
+                || node.getTransform() instanceof PAssert.GroupThenAssertForSingleton
+                || node.getTransform() instanceof PAssert.OneSideInputAssert;
+          }
+        };
+
+    private static class NodeRecorder extends PipelineVisitor.Defaults {
+
+      private final List<TransformHierarchy.Node> visited = new LinkedList<>();
+
+      @Override
+      public void leaveCompositeTransform(final TransformHierarchy.Node node) {
+        visited.add(node);
+      }
+
+      @Override
+      public void visitPrimitiveTransform(final TransformHierarchy.Node node) {
+        visited.add(node);
+      }
+    }
+
+    private PipelineAbandonedNodeEnforcement(final TestPipeline pipeline) {
+      super(pipeline);
+    }
+
+    private List<TransformHierarchy.Node> recordPipelineNodes(final Pipeline pipeline) {
+      final NodeRecorder nodeRecorder = new NodeRecorder();
+      pipeline.traverseTopologically(nodeRecorder);
+      return nodeRecorder.visited;
+    }
+
+    private void verifyPipelineExecution() {
+      final List<TransformHierarchy.Node> pipelineNodes = recordPipelineNodes(pipeline);
+      if (runVisitedNodes != null && !runVisitedNodes.equals(pipelineNodes)) {
+        final boolean hasDanglingPAssert =
+            FluentIterable.from(pipelineNodes)
+                .filter(Predicates.not(Predicates.in(runVisitedNodes)))
+                .anyMatch(isPAssertNode);
+        if (hasDanglingPAssert) {
+          throw new AbandonedNodeException("The pipeline contains abandoned PAssert(s).");
+        } else {
+          throw new AbandonedNodeException("The pipeline contains abandoned PTransform(s).");
+        }
+      } else if (runVisitedNodes == null && !enableAutoRunIfMissing) {
+        throw new PipelineRunMissingException("The pipeline has not been run.");
+      }
+    }
+
+    @Override
+    protected void beforePipelineExecution() {
+      super.beforePipelineExecution();
+      runVisitedNodes = recordPipelineNodes(pipeline);
+    }
+
+    @Override
+    protected void afterTestCompletion() {
+      super.afterTestCompletion();
+      verifyPipelineExecution();
+    }
+  }
+
+  /**
+   * An exception thrown in case an abandoned {@link org.apache.beam.sdk.transforms.PTransform} is
+   * detected, that is, a {@link org.apache.beam.sdk.transforms.PTransform} that has not been run.
+   */
+  public static class AbandonedNodeException extends RuntimeException {
+
+    AbandonedNodeException(final String msg) {
+      super(msg);
+    }
+  }
+
+  /** An exception thrown in case a test finishes without invoking {@link Pipeline#run()}. */
+  public static class PipelineRunMissingException extends RuntimeException {
+
+    PipelineRunMissingException(final String msg) {
+      super(msg);
+    }
+  }
+
   static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions";
   static final String PROPERTY_USE_DEFAULT_DUMMY_RUNNER = "beamUseDummyRunner";
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
+  private PipelineRunEnforcement enforcement = new PipelineAbandonedNodeEnforcement(this);
+
   /**
    * Creates and returns a new test pipeline.
    *
-   * <p>Use {@link PAssert} to add tests, then call
-   * {@link Pipeline#run} to execute the pipeline and check the tests.
+   * <p>Use {@link PAssert} to add tests, then call {@link Pipeline#run} to execute the pipeline and
+   * check the tests.
    */
   public static TestPipeline create() {
     return fromOptions(testingPipelineOptions());
@@ -98,16 +225,30 @@ public class TestPipeline extends Pipeline {
     return new TestPipeline(PipelineRunner.fromOptions(options), options);
   }
 
-  private TestPipeline(PipelineRunner<? extends PipelineResult> runner, PipelineOptions options) {
+  private TestPipeline(
+      final PipelineRunner<? extends PipelineResult> runner, final PipelineOptions options) {
     super(runner, options);
   }
 
+  @Override
+  public Statement apply(final Statement statement, final Description description) {
+    return new Statement() {
+
+      @Override
+      public void evaluate() throws Throwable {
+        statement.evaluate();
+        enforcement.afterTestCompletion();
+      }
+    };
+  }
+
   /**
-   * Runs this {@link TestPipeline}, unwrapping any {@code AssertionError}
-   * that is raised during testing.
+   * Runs this {@link TestPipeline}, unwrapping any {@code AssertionError} that is raised during
+   * testing.
    */
   @Override
   public PipelineResult run() {
+    enforcement.beforePipelineExecution();
     try {
       return super.run();
     } catch (RuntimeException exc) {
@@ -120,18 +261,28 @@ public class TestPipeline extends Pipeline {
     }
   }
 
+  public TestPipeline enableAbandonedNodeEnforcement(final boolean enable) {
+    enforcement =
+        enable ? new PipelineAbandonedNodeEnforcement(this) : new PipelineRunEnforcement(this);
+
+    return this;
+  }
+
+  public TestPipeline enableAutoRunIfMissing(final boolean enable) {
+    enforcement.enableAutoRunIfMissing(enable);
+    return this;
+  }
+
   @Override
   public String toString() {
     return "TestPipeline#" + getOptions().as(ApplicationNameOptions.class).getAppName();
   }
 
-  /**
-   * Creates {@link PipelineOptions} for testing.
-   */
+  /** Creates {@link PipelineOptions} for testing. */
   public static PipelineOptions testingPipelineOptions() {
     try {
-      @Nullable String beamTestPipelineOptions =
-          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS);
+      @Nullable
+      String beamTestPipelineOptions = System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS);
 
       PipelineOptions options =
           Strings.isNullOrEmpty(beamTestPipelineOptions)
@@ -155,13 +306,15 @@ public class TestPipeline extends Pipeline {
       IOChannelUtils.registerIOFactoriesAllowOverride(options);
       return options;
     } catch (IOException e) {
-      throw new RuntimeException("Unable to instantiate test options from system property "
-          + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS + ":"
-          + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), e);
+      throw new RuntimeException(
+          "Unable to instantiate test options from system property "
+              + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS
+              + ":"
+              + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS),
+          e);
     }
   }
 
-
   public static String[] convertToArgs(PipelineOptions options) {
     try {
       byte[] opts = MAPPER.writeValueAsBytes(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/443b25a4/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
index 03563f3..d1797e7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java
@@ -24,30 +24,54 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.Serializable;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.UUID;
+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.coders.StringUtf8Coder;
+import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
+import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link TestPipeline}. */
 @RunWith(JUnit4.class)
-public class TestPipelineTest {
-  @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
-  @Rule public ExpectedException thrown = ExpectedException.none();
+public class TestPipelineTest implements Serializable {
+  private static final List<String> WORDS = Collections.singletonList("hi there");
+  private static final String DUMMY = "expected";
+
+  private final transient TestPipeline pipeline =
+      TestPipeline.fromOptions(pipelineOptions()).enableAbandonedNodeEnforcement(true);
+
+  private final transient ExpectedException exception = ExpectedException.none();
+
+  @Rule public transient TestRule restoreSystemProperties = new RestoreSystemProperties();
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+  @Rule public transient RuleChain ruleOrder = RuleChain.outerRule(exception).around(pipeline);
 
   @Test
   public void testCreationUsingDefaults() {
@@ -57,13 +81,13 @@ public class TestPipelineTest {
   @Test
   public void testCreationOfPipelineOptions() throws Exception {
     ObjectMapper mapper = new ObjectMapper();
-    String stringOptions = mapper.writeValueAsString(new String[]{
-      "--runner=org.apache.beam.sdk.testing.CrashingRunner",
-      "--project=testProject"
-    });
+    String stringOptions =
+        mapper.writeValueAsString(
+            new String[] {
+              "--runner=org.apache.beam.sdk.testing.CrashingRunner", "--project=testProject"
+            });
     System.getProperties().put("beamTestPipelineOptions", stringOptions);
-    GcpOptions options =
-        TestPipeline.testingPipelineOptions().as(GcpOptions.class);
+    GcpOptions options = TestPipeline.testingPipelineOptions().as(GcpOptions.class);
     assertEquals(CrashingRunner.class, options.getRunner());
     assertEquals(options.getProject(), "testProject");
   }
@@ -71,8 +95,10 @@ public class TestPipelineTest {
   @Test
   public void testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase() throws Exception {
     PipelineOptions options = TestPipeline.testingPipelineOptions();
-    assertThat(options.as(ApplicationNameOptions.class).getAppName(), startsWith(
-        "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase"));
+    assertThat(
+        options.as(ApplicationNameOptions.class).getAppName(),
+        startsWith(
+            "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase"));
   }
 
   @Test
@@ -96,13 +122,13 @@ public class TestPipelineTest {
 
   @Test
   public void testConvertToArgs() {
-    String[] args = new String[]{"--tempLocation=Test_Location"};
+    String[] args = new String[] {"--tempLocation=Test_Location"};
     PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
     String[] arr = TestPipeline.convertToArgs(options);
     List<String> lst = Arrays.asList(arr);
     assertEquals(lst.size(), 2);
-    assertThat(lst, containsInAnyOrder("--tempLocation=Test_Location",
-          "--appName=TestPipelineTest"));
+    assertThat(
+        lst, containsInAnyOrder("--tempLocation=Test_Location", "--appName=TestPipelineTest"));
   }
 
   @Test
@@ -131,8 +157,8 @@ public class TestPipelineTest {
     opts.setOnSuccessMatcher(m2);
 
     String[] arr = TestPipeline.convertToArgs(opts);
-    TestPipelineOptions newOpts = PipelineOptionsFactory.fromArgs(arr)
-        .as(TestPipelineOptions.class);
+    TestPipelineOptions newOpts =
+        PipelineOptionsFactory.fromArgs(arr).as(TestPipelineOptions.class);
 
     assertEquals(m1, newOpts.getOnCreateMatcher());
     assertEquals(m2, newOpts.getOnSuccessMatcher());
@@ -150,12 +176,11 @@ public class TestPipelineTest {
     pipeline.run();
   }
 
-  /**
-   * TestMatcher is a matcher designed for testing matcher serialization/deserialization.
-   */
+  /** TestMatcher is a matcher designed for testing matcher serialization/deserialization. */
   public static class TestMatcher extends BaseMatcher<PipelineResult>
       implements SerializableMatcher<PipelineResult> {
     private final UUID uuid = UUID.randomUUID();
+
     @Override
     public boolean matches(Object o) {
       return true;
@@ -180,4 +205,124 @@ public class TestPipelineTest {
       return uuid.hashCode();
     }
   }
+
+  private static class DummyRunner extends PipelineRunner<PipelineResult> {
+
+    @SuppressWarnings("unused") // used by reflection
+    public static DummyRunner fromOptions(final PipelineOptions opts) {
+      return new DummyRunner();
+    }
+
+    @Override
+    public PipelineResult run(final Pipeline pipeline) {
+      return new PipelineResult() {
+
+        @Override
+        public State getState() {
+          return null;
+        }
+
+        @Override
+        public State cancel() throws IOException {
+          return null;
+        }
+
+        @Override
+        public State waitUntilFinish(final Duration duration) {
+          return null;
+        }
+
+        @Override
+        public State waitUntilFinish() {
+          return null;
+        }
+
+        @Override
+        public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator)
+            throws AggregatorRetrievalException {
+          return null;
+        }
+
+        @Override
+        public MetricResults metrics() {
+          return null;
+        }
+      };
+    }
+  }
+
+  private static PipelineOptions pipelineOptions() {
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions.setRunner(DummyRunner.class);
+    return pipelineOptions;
+  }
+
+  private PCollection<String> pCollection() {
+    return addTransform(pipeline.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of())));
+  }
+
+  private PCollection<String> addTransform(final PCollection<String> pCollection) {
+    return pCollection.apply(
+        MapElements.via(
+            new SimpleFunction<String, String>() {
+
+              @Override
+              public String apply(final String input) {
+                return DUMMY;
+              }
+            }));
+  }
+
+  @Test
+  public void testPipelineRunMissing() throws Throwable {
+    exception.expect(TestPipeline.PipelineRunMissingException.class);
+    PAssert.that(pCollection()).containsInAnyOrder(DUMMY);
+    // missing pipeline#run
+  }
+
+  @Test
+  public void testPipelineHasAbandonedPAssertNode() throws Throwable {
+    exception.expect(TestPipeline.AbandonedNodeException.class);
+    exception.expectMessage("PAssert");
+
+    final PCollection<String> pCollection = pCollection();
+    PAssert.that(pCollection).containsInAnyOrder(DUMMY);
+    pipeline.run().waitUntilFinish();
+
+    // dangling PAssert
+    PAssert.that(pCollection).containsInAnyOrder(DUMMY);
+  }
+
+  @Test
+  public void testPipelineHasAbandonedPTransformNode() throws Throwable {
+    exception.expect(TestPipeline.AbandonedNodeException.class);
+    exception.expectMessage("PTransform");
+
+    final PCollection<String> pCollection = pCollection();
+    PAssert.that(pCollection).containsInAnyOrder(DUMMY);
+    pipeline.run().waitUntilFinish();
+
+    // dangling PTransform
+    addTransform(pCollection);
+  }
+
+  @Test
+  public void testNormalFlowWithPAssert() throws Throwable {
+    PAssert.that(pCollection()).containsInAnyOrder(DUMMY);
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testAutoAddMissingRunFlow() throws Throwable {
+    PAssert.that(pCollection()).containsInAnyOrder(DUMMY);
+    // missing pipeline#run, but have it auto-added.
+    pipeline.enableAutoRunIfMissing(true);
+  }
+
+  @Test
+  public void testDisableStrictPAssertFlow() throws Throwable {
+    pCollection();
+    // dangling PTransform, but ignore it
+    pipeline.enableAbandonedNodeEnforcement(false);
+  }
 }


[48/50] incubator-beam git commit: [BEAM-79] Update to latest Gearpump API

Posted by ke...@apache.org.
[BEAM-79] Update to latest Gearpump API


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

Branch: refs/heads/gearpump-runner
Commit: c2fb7c077243a51d821886e56bb21ceb78c792d2
Parents: 81d94cf
Author: manuzhang <ow...@gmail.com>
Authored: Tue Dec 20 15:57:21 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Tue Dec 20 15:57:21 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c2fb7c07/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
index 8477870..4083922 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java
@@ -107,7 +107,7 @@ public class GearpumpRunner extends PipelineRunner<GearpumpPipelineResult> {
     TranslationContext translationContext = new TranslationContext(streamApp, options);
     GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext);
     translator.translate(pipeline);
-    streamApp.run();
+    streamApp.submit();
 
     return null;
   }


[12/50] incubator-beam git commit: Make TimerSpec and StateSpec fields accessible

Posted by ke...@apache.org.
Make TimerSpec and StateSpec fields accessible


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

Branch: refs/heads/gearpump-runner
Commit: c1e1017d6e4d75aee7f32cc3d08b9e2a7c21dbb2
Parents: ffe3ab3
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Dec 7 20:09:06 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 20:10:37 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java   | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c1e1017d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
index e3ba966..d72cea4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
@@ -1040,6 +1040,8 @@ public class DoFnSignatures {
       ErrorReporter errors, Class<?> fnClazz) {
     Map<String, DoFnSignature.TimerDeclaration> declarations = new HashMap<>();
     for (Field field : declaredFieldsWithAnnotation(DoFn.TimerId.class, fnClazz, DoFn.class)) {
+      // TimerSpec fields may generally be private, but will be accessed via the signature
+      field.setAccessible(true);
       String id = field.getAnnotation(DoFn.TimerId.class).value();
       validateTimerField(errors, declarations, id, field);
       declarations.put(id, DoFnSignature.TimerDeclaration.create(id, field));
@@ -1205,6 +1207,8 @@ public class DoFnSignatures {
     Map<String, DoFnSignature.StateDeclaration> declarations = new HashMap<>();
 
     for (Field field : declaredFieldsWithAnnotation(DoFn.StateId.class, fnClazz, DoFn.class)) {
+      // StateSpec fields may generally be private, but will be accessed via the signature
+      field.setAccessible(true);
       String id = field.getAnnotation(DoFn.StateId.class).value();
 
       if (declarations.containsKey(id)) {


[42/50] incubator-beam git commit: This closes #1631: Change counter name in TestDataflowRunner

Posted by ke...@apache.org.
This closes #1631: Change counter name in TestDataflowRunner


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

Branch: refs/heads/gearpump-runner
Commit: 4f97efc114808cd40c6409e2d8714c028be201f5
Parents: a2f4492 6b055d2
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 13:47:19 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 13:47:19 2016 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[15/50] incubator-beam git commit: Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest

Posted by ke...@apache.org.
Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest


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

Branch: refs/heads/gearpump-runner
Commit: b78aa669831154f82266eb12ab795442c02f8977
Parents: 3b4c7d1
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 16 20:57:06 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 20:57:06 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/runners/core/SimpleDoFnRunnerTest.java | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b78aa669/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
index 837a162..ec5d375 100644
--- 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
@@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
+import org.apache.beam.sdk.util.NullSideInputReader;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals;
@@ -77,7 +78,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),
@@ -98,7 +99,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),
@@ -129,7 +130,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),
@@ -158,7 +159,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),
@@ -179,7 +180,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),
@@ -206,7 +207,7 @@ public class SimpleDoFnRunnerTest {
         new SimpleDoFnRunner<>(
             null,
             fn,
-            null,
+            NullSideInputReader.empty(),
             null,
             null,
             Collections.<TupleTag<?>>emptyList(),


[35/50] incubator-beam git commit: Fix seed job fetch spec

Posted by ke...@apache.org.
Fix seed job fetch spec


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

Branch: refs/heads/gearpump-runner
Commit: 2148adb824534214bd6c21fc3395abb5c9abdb57
Parents: d738c68
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 11:18:59 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:18:59 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2148adb8/.jenkins/common_job_properties.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy
index 9b1a5fb..7a88604 100644
--- a/.jenkins/common_job_properties.groovy
+++ b/.jenkins/common_job_properties.groovy
@@ -48,7 +48,7 @@ class common_job_properties {
         remote {
           url('https://github.com/apache/incubator-beam.git')
           refspec('+refs/heads/*:refs/remotes/origin/* ' +
-                  '+refs/pull/*/head:refs/remotes/origin/pr/*')
+                  '+refs/pull/*:refs/remotes/origin/pr/*')
         }
         branch('${sha1}')
         extensions {


[37/50] incubator-beam git commit: Add RunnableOnService test for Metrics

Posted by ke...@apache.org.
Add RunnableOnService test for Metrics

Add UsesMetrics interface and exclude from runners that don't yet
support Metrics

Add Serializability as needed for Metrics to be created during pipeline
construction

Remove test from DirectRunnerTest


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

Branch: refs/heads/gearpump-runner
Commit: 998cabc8bbbf8d08d7bfad71e9376707388f5c5c
Parents: 66318d8
Author: bchambers <bc...@google.com>
Authored: Thu Dec 15 17:04:59 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Mon Dec 19 11:29:39 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                            |  3 +-
 .../beam/runners/direct/DirectRunnerTest.java   | 39 ------------
 runners/flink/runner/pom.xml                    |  6 +-
 runners/google-cloud-dataflow-java/pom.xml      |  3 +-
 runners/spark/pom.xml                           |  3 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |  3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |  5 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    | 24 ++++++++
 .../apache/beam/sdk/metrics/MetricMatchers.java |  4 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    | 63 +++++++++++++++++++-
 10 files changed, 103 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/runners/apex/pom.xml
----------------------------------------------------------------------
diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml
index f71637c..d03964d 100644
--- a/runners/apex/pom.xml
+++ b/runners/apex/pom.xml
@@ -186,7 +186,8 @@
               <excludedGroups>
                 org.apache.beam.sdk.testing.UsesStatefulParDo,
                 org.apache.beam.sdk.testing.UsesTimersInParDo,
-                org.apache.beam.sdk.testing.UsesSplittableParDo
+                org.apache.beam.sdk.testing.UsesSplittableParDo,
+                org.apache.beam.sdk.testing.UsesMetrics
               </excludedGroups>
               <parallel>none</parallel>
               <failIfNoTests>true</failIfNoTests>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index eb0f344..eafb788 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -18,8 +18,6 @@
 package org.apache.beam.runners.direct;
 
 import static com.google.common.base.Preconditions.checkState;
-import static org.apache.beam.sdk.metrics.MetricMatchers.metricResult;
-import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.isA;
@@ -37,7 +35,6 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -48,13 +45,6 @@ import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.io.CountingSource;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Distribution;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.MetricNameFilter;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.metrics.MetricsFilter;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.runners.PipelineRunner;
@@ -467,35 +457,6 @@ public class DirectRunnerTest implements Serializable {
     }
   }
 
-  @Test
-  public void testMetrics() throws Exception {
-    Pipeline pipeline = getPipeline();
-    pipeline
-        .apply(Create.of(5, 8, 13))
-        .apply("MyStep", ParDo.of(new DoFn<Integer, Void>() {
-          @ProcessElement
-          public void processElement(ProcessContext c) {
-            Counter count = Metrics.counter(DirectRunnerTest.class, "count");
-            Distribution values = Metrics.distribution(DirectRunnerTest.class, "input");
-
-            count.inc();
-            values.update(c.element());
-          }
-        }));
-    PipelineResult result = pipeline.run();
-    MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder()
-        .addNameFilter(MetricNameFilter.inNamespace(DirectRunnerTest.class))
-        .build());
-
-    final String stepName = "MyStep/AnonymousParDo/AnonymousParMultiDo";
-    assertThat(metrics.counters(), contains(
-        metricResult(DirectRunnerTest.class.getName(), "count", stepName, 3L, 3L)));
-    assertThat(metrics.distributions(), contains(
-        metricResult(DirectRunnerTest.class.getName(), "input", stepName,
-            DistributionResult.create(26L, 3L, 5L, 13L),
-            DistributionResult.create(26L, 3L, 5L, 13L))));
-  }
-
   private static class MustSplitSource<T> extends BoundedSource<T>{
     public static <T> BoundedSource<T> of(BoundedSource<T> underlying) {
       return new MustSplitSource<>(underlying);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 09773e1..7f49372 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -56,7 +56,8 @@
                   <excludedGroups>
                     org.apache.beam.sdk.testing.UsesStatefulParDo,
                     org.apache.beam.sdk.testing.UsesTimersInParDo,
-                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                    org.apache.beam.sdk.testing.UsesSplittableParDo,
+                    org.apache.beam.sdk.testing.UsesMetrics
                   </excludedGroups>
                   <parallel>none</parallel>
                   <failIfNoTests>true</failIfNoTests>
@@ -86,7 +87,8 @@
                   <excludedGroups>
                     org.apache.beam.sdk.testing.UsesStatefulParDo,
                     org.apache.beam.sdk.testing.UsesTimersInParDo,
-                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                    org.apache.beam.sdk.testing.UsesSplittableParDo,
+                    org.apache.beam.sdk.testing.UsesMetrics
                   </excludedGroups>
                   <parallel>none</parallel>
                   <failIfNoTests>true</failIfNoTests>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 46ac7ef..0094791 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -80,7 +80,8 @@
               <excludedGroups>
                 org.apache.beam.sdk.testing.UsesStatefulParDo,
                 org.apache.beam.sdk.testing.UsesTimersInParDo,
-                org.apache.beam.sdk.testing.UsesSplittableParDo
+                org.apache.beam.sdk.testing.UsesSplittableParDo,
+                org.apache.beam.sdk.testing.UsesMetrics
               </excludedGroups>
               <excludes>
                 <exclude>org.apache.beam.sdk.transforms.FlattenTest</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 5a2fe87..309e1ff 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -75,7 +75,8 @@
                   <excludedGroups>
                     org.apache.beam.sdk.testing.UsesStatefulParDo,
                     org.apache.beam.sdk.testing.UsesTimersInParDo,
-                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                    org.apache.beam.sdk.testing.UsesSplittableParDo,
+                    org.apache.beam.sdk.testing.UsesMetrics
                   </excludedGroups>
                   <forkCount>1</forkCount>
                   <reuseForks>false</reuseForks>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java
index 843a885..3c77043 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.metrics;
 
 import com.google.auto.value.AutoValue;
+import java.io.Serializable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 
@@ -28,7 +29,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind;
  */
 @Experimental(Kind.METRICS)
 @AutoValue
-public abstract class MetricName {
+public abstract class MetricName implements Serializable {
 
   /** The namespace associated with this metric. */
   public abstract String namespace();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
index b72a0b2..045e076 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.metrics;
 
+import java.io.Serializable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 
@@ -58,7 +59,7 @@ public class Metrics {
   }
 
   /** Implementation of {@link Counter} that delegates to the instance for the current context. */
-  private static class DelegatingCounter implements Counter {
+  private static class DelegatingCounter implements Counter, Serializable {
     private final MetricName name;
 
     private DelegatingCounter(MetricName name) {
@@ -92,7 +93,7 @@ public class Metrics {
   /**
    * Implementation of {@link Distribution} that delegates to the instance for the current context.
    */
-  private static class DelegatingDistribution implements Distribution {
+  private static class DelegatingDistribution implements Distribution, Serializable {
     private final MetricName name;
 
     private DelegatingDistribution(MetricName name) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java
new file mode 100644
index 0000000..261354c
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.testing;
+
+/**
+ * Category tag for validation tests which utilize {@link org.apache.beam.sdk.metrics.Metrics}.
+ */
+public interface UsesMetrics {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java
index 6cd4c52..798d9d4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java
@@ -78,7 +78,7 @@ public class MetricMatchers {
       protected boolean matchesSafely(MetricResult<T> item) {
         return Objects.equals(namespace, item.name().namespace())
             && Objects.equals(name, item.name().name())
-            && Objects.equals(step, item.step())
+            && item.step().contains(step)
             && Objects.equals(committed, item.committed())
             && Objects.equals(attempted, item.attempted());
       }
@@ -109,7 +109,7 @@ public class MetricMatchers {
               .appendText(" != ").appendValue(item.name().name());
         }
 
-        if (!Objects.equals(step, item.step())) {
+        if (!item.step().contains(step)) {
           mismatchDescription
               .appendText("step: ").appendValue(step)
               .appendText(" != ").appendValue(item.step());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/998cabc8/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
index 732cb34..075df19 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
@@ -18,18 +18,30 @@
 
 package org.apache.beam.sdk.metrics;
 
+import static org.apache.beam.sdk.metrics.MetricMatchers.metricResult;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 
+import java.io.Serializable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesMetrics;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
 import org.hamcrest.CoreMatchers;
 import org.junit.After;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests for {@link Metrics}.
  */
-public class MetricsTest {
+public class MetricsTest implements Serializable {
 
   private static final String NS = "test";
   private static final String NAME = "name";
@@ -95,4 +107,53 @@ public class MetricsTest {
     counter.dec();
     assertThat(cell.getCumulative(), CoreMatchers.equalTo(42L));
   }
+
+  @Category({RunnableOnService.class, UsesMetrics.class})
+  @Test
+  public void metricsReportToQuery() {
+    final Counter count = Metrics.counter(MetricsTest.class, "count");
+    Pipeline pipeline = TestPipeline.create();
+    pipeline
+        .apply(Create.of(5, 8, 13))
+        .apply("MyStep1", ParDo.of(new DoFn<Integer, Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
+            Distribution values = Metrics.distribution(MetricsTest.class, "input");
+            count.inc();
+            values.update(c.element());
+
+            c.output(c.element());
+            c.output(c.element());
+          }
+        }))
+        .apply("MyStep2", ParDo.of(new DoFn<Integer, Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
+            Distribution values = Metrics.distribution(MetricsTest.class, "input");
+            count.inc();
+            values.update(c.element());
+          }
+        }));
+    PipelineResult result = pipeline.run();
+
+    result.waitUntilFinish();
+
+    MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder()
+      .addNameFilter(MetricNameFilter.inNamespace(MetricsTest.class))
+      .build());
+    // TODO: BEAM-1169: Metrics shouldn't verify the physical values tightly.
+    assertThat(metrics.counters(), hasItem(
+        metricResult(MetricsTest.class.getName(), "count", "MyStep1", 3L, 3L)));
+    assertThat(metrics.distributions(), hasItem(
+        metricResult(MetricsTest.class.getName(), "input", "MyStep1",
+            DistributionResult.create(26L, 3L, 5L, 13L),
+            DistributionResult.create(26L, 3L, 5L, 13L))));
+
+    assertThat(metrics.counters(), hasItem(
+        metricResult(MetricsTest.class.getName(), "count", "MyStep2", 6L, 6L)));
+    assertThat(metrics.distributions(), hasItem(
+        metricResult(MetricsTest.class.getName(), "input", "MyStep2",
+            DistributionResult.create(52L, 6L, 5L, 13L),
+            DistributionResult.create(52L, 6L, 5L, 13L))));
+  }
 }


[25/50] incubator-beam git commit: Fix grammar error (repeated for)

Posted by ke...@apache.org.
Fix grammar error (repeated for)


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

Branch: refs/heads/gearpump-runner
Commit: fbfea5953000bd77b07b6a1f1c7a192e24b88021
Parents: 1c9bf8d
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Sun Dec 18 16:02:41 2016 +0100
Committer: Isma�l Mej�a <ie...@gmail.com>
Committed: Sun Dec 18 16:02:41 2016 +0100

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fbfea595/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
index d72cea4..b6b764e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
@@ -306,7 +306,7 @@ public class DoFnSignatures {
       String id = onTimerMethod.getAnnotation(DoFn.OnTimer.class).value();
       errors.checkArgument(
           fnContext.getTimerDeclarations().containsKey(id),
-          "Callback %s is for for undeclared timer %s",
+          "Callback %s is for undeclared timer %s",
           onTimerMethod,
           id);
 


[11/50] incubator-beam git commit: This closes #1648: Undeletes DoFnInvokers.of(OldDoFn)

Posted by ke...@apache.org.
This closes #1648: Undeletes DoFnInvokers.of(OldDoFn)

  Revert "Move InMemoryTimerInternals to runners-core"
  Revert "Moves DoFnAdapters to runners-core"
  Revert "Removes ArgumentProvider.windowingInternals"
  Revert "Removes code for wrapping DoFn as an OldDoFn"


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

Branch: refs/heads/gearpump-runner
Commit: ffe3ab3d622c767871064dcdb8c5a953bbf0a1d2
Parents: 7d1976b 45ed5c7
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 16 18:46:17 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 18:46:17 2016 -0800

----------------------------------------------------------------------
 .../apex/translation/WindowBoundTranslator.java |   2 +-
 .../operators/ApexGroupByKeyOperator.java       |   2 +-
 .../operators/ApexParDoOperator.java            |   2 +-
 .../apache/beam/runners/core/DoFnAdapters.java  | 344 -------------
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   1 +
 .../runners/core/InMemoryTimerInternals.java    | 276 ----------
 .../beam/runners/core/SimpleDoFnRunner.java     |  57 +++
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   4 +-
 .../beam/runners/core/SplittableParDo.java      |   7 +
 .../core/GroupAlsoByWindowsProperties.java      |   2 +-
 .../core/InMemoryTimerInternalsTest.java        | 155 ------
 .../beam/runners/core/ReduceFnTester.java       |   1 +
 .../beam/runners/core/SplittableParDoTest.java  |  16 +-
 .../triggers/TriggerStateMachineTester.java     |   2 +-
 .../functions/FlinkDoFnFunction.java            |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   2 +-
 .../functions/FlinkProcessContextBase.java      |   2 +-
 .../wrappers/streaming/DoFnOperator.java        |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |   2 +-
 .../sdk/transforms/AggregatorRetriever.java     |  13 +-
 .../beam/sdk/transforms/DoFnAdapters.java       | 504 +++++++++++++++++++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  43 ++
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 297 ++++++++++-
 .../sdk/transforms/reflect/DoFnInvoker.java     |  20 +
 .../sdk/transforms/reflect/DoFnInvokers.java    | 141 +++++-
 .../sdk/util/state/InMemoryTimerInternals.java  | 275 ++++++++++
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |   2 +-
 .../transforms/reflect/DoFnInvokersTest.java    |  42 ++
 .../util/state/InMemoryTimerInternalsTest.java  | 153 ++++++
 29 files changed, 1547 insertions(+), 824 deletions(-)
----------------------------------------------------------------------



[18/50] incubator-beam git commit: Closes #1651

Posted by ke...@apache.org.
Closes #1651


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

Branch: refs/heads/gearpump-runner
Commit: 5255a33812758bbb9d081962675bd0180802c82b
Parents: 4206408 5fb4f5d
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 16 23:53:49 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 16 23:53:49 2016 -0800

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 23 +++++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 72 ++++++++++++--------
 2 files changed, 63 insertions(+), 32 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-beam git commit: This closes #1640: Show timestamps on log lines in Jenkins

Posted by ke...@apache.org.
This closes #1640: Show timestamps on log lines in Jenkins


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

Branch: refs/heads/gearpump-runner
Commit: d738c68b0d290968007120d1e3d1e1a99bdee90b
Parents: e68a70e 4d96473
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 11:12:57 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:12:57 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[43/50] incubator-beam git commit: [BEAM-59] initial interfaces and classes of Beam FileSystem.

Posted by ke...@apache.org.
[BEAM-59] initial interfaces and classes of Beam FileSystem.


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

Branch: refs/heads/gearpump-runner
Commit: 467f7d17c4c96bc57b0160c2d4768ceb303bc561
Parents: 4f97efc
Author: Pei He <pe...@google.com>
Authored: Wed Dec 7 17:35:23 2016 -0800
Committer: Luke Cwik <lc...@google.com>
Committed: Mon Dec 19 15:20:37 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 ++++
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++++++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++++++++++++++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 ++++
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 +++++
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 +++++++++++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++++++
 sdks/java/io/google-cloud-platform/pom.xml      |   6 +
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 ++++
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 +++++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +++
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++++++
 sdks/java/io/hdfs/pom.xml                       |   6 +
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 ++++
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 +++++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 +++++++
 16 files changed, 732 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java
new file mode 100644
index 0000000..d990403
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+/**
+ * File system interface in Beam.
+ *
+ * <p>It defines APIs for writing file systems agnostic code.
+ *
+ * <p>All methods are protected, and they are for file system providers to implement.
+ * Clients should use {@link FileSystems} utility.
+ */
+public abstract class FileSystem {
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java
new file mode 100644
index 0000000..1d81c1e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io;
+
+import com.google.auto.service.AutoService;
+import java.util.ServiceLoader;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * A registrar that creates {@link FileSystem} instances from {@link PipelineOptions}.
+ *
+ * <p>{@link FileSystem} creators have the ability to provide a registrar by creating
+ * a {@link ServiceLoader} entry and a concrete implementation of this interface.
+ *
+ * <p>It is optional but recommended to use one of the many build time tools such as
+ * {@link AutoService} to generate the necessary META-INF files automatically.
+ */
+public interface FileSystemRegistrar {
+  /**
+   * Create a {@link FileSystem} from the given {@link PipelineOptions}.
+   */
+  FileSystem fromOptions(@Nullable PipelineOptions options);
+
+  /**
+   * Get the URI scheme which defines the namespace of the {@link FileSystemRegistrar}.
+   *
+   * <p>The scheme is required to be unique among all
+   * {@link FileSystemRegistrar FileSystemRegistrars}.
+   *
+   * @see <a href="https://www.ietf.org/rfc/rfc2396.txt">RFC 2396</a>
+   */
+  String getScheme();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
new file mode 100644
index 0000000..d086ec6
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.Sets;
+import com.google.common.collect.TreeMultimap;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Pattern;
+import javax.annotation.Nonnull;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+
+/**
+ * Clients facing {@link FileSystem} utility.
+ */
+public class FileSystems {
+
+  public static final String DEFAULT_SCHEME = "default";
+
+  private static final Pattern URI_SCHEME_PATTERN = Pattern.compile("^[a-zA-Z][-a-zA-Z0-9+.]*$");
+
+  private static final Map<String, FileSystemRegistrar> SCHEME_TO_REGISTRAR =
+      new ConcurrentHashMap<>();
+
+  private static final Map<String, PipelineOptions> SCHEME_TO_DEFAULT_CONFIG =
+      new ConcurrentHashMap<>();
+
+  static {
+    loadFileSystemRegistrars();
+  }
+
+  /**
+   * Loads available {@link FileSystemRegistrar} services.
+   */
+  private static void loadFileSystemRegistrars() {
+    SCHEME_TO_REGISTRAR.clear();
+    Set<FileSystemRegistrar> registrars =
+        Sets.newTreeSet(ReflectHelpers.ObjectsClassComparator.INSTANCE);
+    registrars.addAll(Lists.newArrayList(
+        ServiceLoader.load(FileSystemRegistrar.class, ReflectHelpers.findClassLoader())));
+
+    verifySchemesAreUnique(registrars);
+
+    for (FileSystemRegistrar registrar : registrars) {
+      SCHEME_TO_REGISTRAR.put(registrar.getScheme().toLowerCase(), registrar);
+    }
+  }
+
+  /**
+   * Sets the default configuration to be used with a {@link FileSystemRegistrar} for the provided
+   * {@code scheme}.
+   *
+   * <p>Syntax: <pre>scheme = alpha *( alpha | digit | "+" | "-" | "." )</pre>
+   * Upper case letters are treated as the same as lower case letters.
+   */
+  public static void setDefaultConfig(String scheme, PipelineOptions options) {
+    String lowerCaseScheme = checkNotNull(scheme, "scheme").toLowerCase();
+    checkArgument(
+        URI_SCHEME_PATTERN.matcher(lowerCaseScheme).matches(),
+        String.format("Scheme: [%s] doesn't match URI syntax: %s",
+            lowerCaseScheme, URI_SCHEME_PATTERN.pattern()));
+    checkArgument(
+        SCHEME_TO_REGISTRAR.containsKey(lowerCaseScheme),
+        String.format("No FileSystemRegistrar found for scheme: [%s].", lowerCaseScheme));
+    SCHEME_TO_DEFAULT_CONFIG.put(lowerCaseScheme, checkNotNull(options, "options"));
+  }
+
+  @VisibleForTesting
+  static PipelineOptions getDefaultConfig(String scheme) {
+    return SCHEME_TO_DEFAULT_CONFIG.get(scheme.toLowerCase());
+  }
+
+  /**
+   * Internal method to get {@link FileSystem} for {@code spec}.
+   */
+  @VisibleForTesting
+  static FileSystem getFileSystemInternal(URI uri) {
+    String lowerCaseScheme = (uri.getScheme() != null
+        ? uri.getScheme().toLowerCase() : LocalFileSystemRegistrar.LOCAL_FILE_SCHEME);
+    return getRegistrarInternal(lowerCaseScheme).fromOptions(getDefaultConfig(lowerCaseScheme));
+  }
+
+  /**
+   * Internal method to get {@link FileSystemRegistrar} for {@code scheme}.
+   */
+  @VisibleForTesting
+  static FileSystemRegistrar getRegistrarInternal(String scheme) {
+    String lowerCaseScheme = scheme.toLowerCase();
+    if (SCHEME_TO_REGISTRAR.containsKey(lowerCaseScheme)) {
+      return SCHEME_TO_REGISTRAR.get(lowerCaseScheme);
+    } else if (SCHEME_TO_REGISTRAR.containsKey(DEFAULT_SCHEME)) {
+      return SCHEME_TO_REGISTRAR.get(DEFAULT_SCHEME);
+    } else {
+      throw new IllegalStateException("Unable to find registrar for " + scheme);
+    }
+  }
+
+  @VisibleForTesting
+  static void verifySchemesAreUnique(Set<FileSystemRegistrar> registrars) {
+    Multimap<String, FileSystemRegistrar> registrarsBySchemes =
+        TreeMultimap.create(Ordering.<String>natural(), Ordering.arbitrary());
+
+    for (FileSystemRegistrar registrar : registrars) {
+      registrarsBySchemes.put(registrar.getScheme().toLowerCase(), registrar);
+    }
+    for (Entry<String, Collection<FileSystemRegistrar>> entry
+        : registrarsBySchemes.asMap().entrySet()) {
+      if (entry.getValue().size() > 1) {
+        String conflictingRegistrars = Joiner.on(", ").join(
+            FluentIterable.from(entry.getValue())
+                .transform(new Function<FileSystemRegistrar, String>() {
+                  @Override
+                  public String apply(@Nonnull FileSystemRegistrar input) {
+                    return input.getClass().getName();
+                  }})
+                .toSortedList(Ordering.<String>natural()));
+        throw new IllegalStateException(String.format(
+            "Scheme: [%s] has conflicting registrars: [%s]",
+            entry.getKey(),
+            conflictingRegistrars));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
new file mode 100644
index 0000000..23c2a92
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+/**
+ * {@link FileSystem} implementation for local files.
+ */
+class LocalFileSystem extends FileSystem {
+
+  LocalFileSystem() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java
new file mode 100644
index 0000000..75a38e8
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io;
+
+import com.google.auto.service.AutoService;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * {@link AutoService} registrar for the {@link FileSystem}.
+ */
+@AutoService(FileSystemRegistrar.class)
+public class LocalFileSystemRegistrar implements FileSystemRegistrar {
+
+  static final String LOCAL_FILE_SCHEME = "file";
+
+  @Override
+  public FileSystem fromOptions(@Nullable PipelineOptions options) {
+    return new LocalFileSystem();
+  }
+
+  @Override
+  public String getScheme() {
+    return LOCAL_FILE_SCHEME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
new file mode 100644
index 0000000..9b41b98
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.Sets;
+import java.net.URI;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+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 FileSystems}.
+ */
+@RunWith(JUnit4.class)
+public class FileSystemsTest {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testSetDefaultConfig() throws Exception {
+    PipelineOptions first = PipelineOptionsFactory.create();
+    PipelineOptions second = PipelineOptionsFactory.create();
+    FileSystems.setDefaultConfig("file", first);
+    assertEquals(first, FileSystems.getDefaultConfig("file"));
+    assertEquals(first, FileSystems.getDefaultConfig("FILE"));
+
+    FileSystems.setDefaultConfig("FILE", second);
+    assertNotEquals(first, FileSystems.getDefaultConfig("file"));
+    assertNotEquals(first, FileSystems.getDefaultConfig("FILE"));
+    assertEquals(second, FileSystems.getDefaultConfig("file"));
+    assertEquals(second, FileSystems.getDefaultConfig("FILE"));
+  }
+
+  @Test
+  public void testSetDefaultConfigNotFound() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("No FileSystemRegistrar found for scheme: [gs-s3].");
+    FileSystems.setDefaultConfig("gs-s3", PipelineOptionsFactory.create());
+  }
+
+  @Test
+  public void testSetDefaultConfigInvalidScheme() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Scheme: [gs:] doesn't match URI syntax");
+    FileSystems.setDefaultConfig("gs:", PipelineOptionsFactory.create());
+  }
+
+  @Test
+  public void testGetLocalFileSystem() throws Exception {
+    assertTrue(
+        FileSystems.getFileSystemInternal(URI.create("~/home/")) instanceof LocalFileSystem);
+    assertTrue(
+        FileSystems.getFileSystemInternal(URI.create("file://home")) instanceof LocalFileSystem);
+    assertTrue(
+        FileSystems.getFileSystemInternal(URI.create("FILE://home")) instanceof LocalFileSystem);
+    assertTrue(
+        FileSystems.getFileSystemInternal(URI.create("File://home")) instanceof LocalFileSystem);
+  }
+
+  @Test
+  public void testVerifySchemesAreUnique() throws Exception {
+    thrown.expect(RuntimeException.class);
+    thrown.expectMessage("Scheme: [file] has conflicting registrars");
+    FileSystems.verifySchemesAreUnique(
+        Sets.<FileSystemRegistrar>newHashSet(
+            new LocalFileSystemRegistrar(),
+            new FileSystemRegistrar() {
+              @Override
+              public FileSystem fromOptions(@Nullable PipelineOptions options) {
+                return null;
+              }
+
+              @Override
+              public String getScheme() {
+                return "FILE";
+              }
+            }));
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index d3b5fed..76bdc45 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -100,6 +100,12 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <optional>true</optional>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.cloud.bigdataoss</groupId>
       <artifactId>util</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
new file mode 100644
index 0000000..4b03e27
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.storage;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.io.FileSystem;
+import org.apache.beam.sdk.options.GcsOptions;
+
+/**
+ * {@link FileSystem} implementation for Google Cloud Storage.
+ */
+class GcsFileSystem extends FileSystem {
+  private final GcsOptions options;
+
+  GcsFileSystem(GcsOptions options) {
+    this.options = checkNotNull(options, "options");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java
new file mode 100644
index 0000000..10452a1
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.storage;
+
+import com.google.auto.service.AutoService;
+import javax.annotation.Nonnull;
+import org.apache.beam.sdk.io.FileSystem;
+import org.apache.beam.sdk.io.FileSystemRegistrar;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * {@link AutoService} registrar for the {@link GcsFileSystem}.
+ */
+@AutoService(FileSystemRegistrar.class)
+public class GcsFileSystemRegistrar implements FileSystemRegistrar {
+
+  @Override
+  public FileSystem fromOptions(@Nonnull PipelineOptions options) {
+    return new GcsFileSystem(options.as(GcsOptions.class));
+  }
+
+  @Override
+  public String getScheme() {
+    return "gs";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java
new file mode 100644
index 0000000..b5378be
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 IO connectors for Google Cloud Storage.
+ */
+package org.apache.beam.sdk.io.gcp.storage;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java
new file mode 100644
index 0000000..ecac8f6
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.storage;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+import java.util.ServiceLoader;
+
+import org.apache.beam.sdk.io.FileSystemRegistrar;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link GcsFileSystemRegistrar}.
+ */
+@RunWith(JUnit4.class)
+public class GcsFileSystemRegistrarTest {
+
+  @Test
+  public void testServiceLoader() {
+    for (FileSystemRegistrar registrar
+        : Lists.newArrayList(ServiceLoader.load(FileSystemRegistrar.class).iterator())) {
+      if (registrar instanceof GcsFileSystemRegistrar) {
+        assertEquals("gs", registrar.getScheme());
+        assertTrue(registrar.fromOptions(PipelineOptionsFactory.create()) instanceof GcsFileSystem);
+        return;
+      }
+    }
+    fail("Expected to find " + GcsFileSystemRegistrar.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 772276b..b171cfe 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -64,6 +64,12 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.auto.service</groupId>
+      <artifactId>auto-service</artifactId>
+      <optional>true</optional>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java
new file mode 100644
index 0000000..b94a089
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hdfs;
+
+import org.apache.beam.sdk.io.FileSystem;
+
+/**
+ * Adapts {@link org.apache.hadoop.fs.FileSystem} connectors to be used as
+ * Apache Beam {@link FileSystem FileSystems}.
+ */
+class HadoopFileSystem extends FileSystem {
+
+  HadoopFileSystem() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
new file mode 100644
index 0000000..1471cb0
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hdfs;
+
+import com.google.auto.service.AutoService;
+import javax.annotation.Nonnull;
+import org.apache.beam.sdk.io.FileSystem;
+import org.apache.beam.sdk.io.FileSystemRegistrar;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * {@link AutoService} registrar for the {@link HadoopFileSystem}.
+ */
+@AutoService(FileSystemRegistrar.class)
+public class HadoopFileSystemRegistrar implements FileSystemRegistrar {
+
+  @Override
+  public FileSystem fromOptions(@Nonnull PipelineOptions options) {
+    return new HadoopFileSystem();
+  }
+
+  @Override
+  public String getScheme() {
+    return FileSystems.DEFAULT_SCHEME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/467f7d17/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
new file mode 100644
index 0000000..22a439a
--- /dev/null
+++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hdfs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+import java.util.ServiceLoader;
+import org.apache.beam.sdk.io.FileSystemRegistrar;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link HadoopFileSystemRegistrar}.
+ */
+@RunWith(JUnit4.class)
+public class HadoopFileSystemRegistrarTest {
+
+  @Test
+  public void testServiceLoader() {
+    for (FileSystemRegistrar registrar
+        : Lists.newArrayList(ServiceLoader.load(FileSystemRegistrar.class).iterator())) {
+      if (registrar instanceof HadoopFileSystemRegistrar) {
+        assertEquals(FileSystems.DEFAULT_SCHEME, registrar.getScheme());
+        assertTrue(
+            registrar.fromOptions(PipelineOptionsFactory.create()) instanceof HadoopFileSystem);
+        return;
+      }
+    }
+    fail("Expected to find " + HadoopFileSystemRegistrar.class);
+  }
+}



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

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


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

Branch: refs/heads/gearpump-runner
Commit: 203b7b174ad6e4194baae6965a1e714c4acadf8c
Parents: 1e148cd 9bd4288
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Dec 19 12:09:03 2016 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Dec 19 12:09:03 2016 +0100

----------------------------------------------------------------------
 runners/flink/examples/pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[13/50] incubator-beam git commit: Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle

Posted by ke...@apache.org.
Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle


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

Branch: refs/heads/gearpump-runner
Commit: 3b4c7d103c07e73d30b2ad534a17b3059232dbda
Parents: 8af13b0
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 16 13:43:54 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 20:14:19 2016 -0800

----------------------------------------------------------------------
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 53 ++++++++++++++++++++
 1 file changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3b4c7d10/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
index f068c19..837a162 100644
--- 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
@@ -151,6 +151,49 @@ public class SimpleDoFnRunnerTest {
             TimeDomain.EVENT_TIME);
   }
 
+  @Test
+  public void testStartBundleExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(new GlobalWindows()));
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.startBundle();
+  }
+
+  @Test
+  public void testFinishBundleExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner =
+        new SimpleDoFnRunner<>(
+            null,
+            fn,
+            null,
+            null,
+            null,
+            Collections.<TupleTag<?>>emptyList(),
+            mockStepContext,
+            null,
+            WindowingStrategy.of(new GlobalWindows()));
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.finishBundle();
+  }
+
+
   /**
    * Tests that {@link SimpleDoFnRunner#onTimer} properly dispatches to the underlying
    * {@link DoFn}.
@@ -200,6 +243,16 @@ public class SimpleDoFnRunnerTest {
     @TimerId(TIMER_ID)
     private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      throw exceptionToThrow;
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      throw exceptionToThrow;
+    }
+
     @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
       throw exceptionToThrow;


[50/50] incubator-beam git commit: This closes #1663: Merge master (b3de17b) into gearpump-runner

Posted by ke...@apache.org.
This closes #1663: Merge master (b3de17b) into gearpump-runner

Adjustments in gearpump-runner:

  [BEAM-79] Upgrade to beam-0.5.0-incubating-SNAPSHOT
  [BEAM-79] Update to latest Gearpump API

>From master:

  Disable automatic archiving of Maven builds
  [BEAM-59] initial interfaces and classes of Beam FileSystem.
  Change counter name in TestDataflowRunner
  More escaping in Jenkins timestamp spec
  Add RunnableOnService test for Metrics
  Fix seed job fetch spec
  Show timestamps on log lines in Jenkins
  [BEAM-1165] Fix unexpected file creation when checking dependencies
  [BEAM-1178] Make naming of logger objects consistent
  [BEAM-716] Fix javadoc on with* methods [BEAM-959] Improve check preconditions in JmsIO
  [BEAM-716] Use AutoValue in JmsIO
  Fix grammar error (repeated for)
  Empty TestPipeline need not be run
  [BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage
  Change counter name in TestDataflowRunner
  BigQueryIO: fix streaming write, typo in API
  [BEAM-853] Force streaming execution on batch pipelines for testing. Expose the adapted source.
  Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest
  Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle
  Add timer support to DoFnRunner(s)
  Make TimerSpec and StateSpec fields accessible
  View.asMap: minor javadoc fixes
  Revert "Move InMemoryTimerInternals to runners-core"
  Revert "Moves DoFnAdapters to runners-core"
  Revert "Removes ArgumentProvider.windowingInternals"
  Revert "Removes code for wrapping DoFn as an OldDoFn"
  checkstyle: missed newline in DistributionCell
  Make {Metric,Counter,Distribution}Cell public
  Add PTransformOverrideFactory to the Core SDK
  Move ActiveWindowSet and implementations to runners-core
  Update Dataflow worker to beam-master-20161216
  [BEAM-1108] Remove outdated language about experimental autoscaling
  [BEAM-450] Shade modules to separate paths
  [BEAM-362] Port runners to runners-core AggregatoryFactory
  Move InMemoryTimerInternals to runners-core
  Delete deprecated TimerCallback
  Remove deprecated methods of InMemoryTimerInternals
  Don't incorrectly log error in MetricsEnvironment
  Renames ParDo.getNewFn to getFn
  Moves DoFnAdapters to runners-core
  Removes unused code from NoOpOldDoFn
  Removes ArgumentProvider.windowingInternals
  Removes code for wrapping DoFn as an OldDoFn
  Removes OldDoFn from ParDo
  Pushes uses of OldDoFn deeper inside Flink runner
  Remove ParDo.of(OldDoFn) from Apex runner
  Converts all easy OldDoFns to DoFn
  [BEAM-1022] Add testing coverage for BigQuery streaming writes
  Fix mvn command args in Apex postcommit Jenkins job
  [BEAM-932] Enable findbugs validation (and fix existing issues)
  Fail to split in FileBasedSource if filePattern expands to empty.
  [BEAM-1154] Get side input from proper window in ReduceFn
  [BEAM-1153] GcsUtil: use non-batch API for single file size requests.
  Fix NPE in StatefulParDoEvaluatorFactoryTest mocking
  [BEAM-1033] Retry Bigquery Verifier when Query Fails
  Implement GetDefaultOutputCoder in DirectGroupByKey
  SimpleDoFnRunner observes window if SideInputReader is nonempty
  Better comments and cleanup
  Allow empty string value for ValueProvider types.
  starter: fix typo in pom.xml
  Revert "Allow stateful DoFn in DataflowRunner"
  Re-exclude UsesStatefulParDo tests for Dataflow
  Some minor changes and fixes for sorter module
  [BEAM-1149] Explode windows when fn uses side inputs
  Add Jenkins postcommit for RunnableOnService in Apex runner
  Update version from 0.5.0-SNAPSHOT to 0.5.0-incubating-SNAPSHOT
  Update Maven Archetype versions after cutting the release branch
  Move PerKeyCombineFnRunner to runners-core
  Update Dataflow worker to beam-master-20161212
  [maven-release-plugin] prepare for next development iteration
  [maven-release-plugin] prepare branch release-0.4.0-incubating
  Fix version of Kryo in examples/java jenkins-precommit profile
  Revert 91cc606 "This closes #1586": Kryo + UBRFBS
  [BEAM-909] improve starter archetype
  Fix JDom malformed comment in Apex runner.
  [BEAM-927] Fix findbugs and re-enable Maven plugin in JmsIO
  [BEAM-807] Replace OldDoFn with DoFn.
  [BEAM-757] Use DoFnRunner in the implementation of DoFn via FlatMapFunction.
  FileBasedSinkTest: fix tests in Windows OS by using IOChannelUtils.resolve().
  FileBasedSink: ignore exceptions when removing temp output files for issues in Windows OS.
  [BEAM-1142] Upgrade maven-invoker to address maven bug ARCHETYPE-488.
  Add Tests for Kryo Serialization of URFBS
  Add no-arg constructor for UnboundedReadFromBoundedSource
  Revise WindowedWordCount for runner and execution mode portability
  Factor out ShardedFile from FileChecksumMatcher
  Add IntervalWindow coder to the standard registry
  Stop expanding PValues in DirectRunner visitors
  Migrate AppliedPTransform to use AutoValue
  Enable and fix DirectRunnerTest case missing @Test
  [BEAM-1130] SparkRunner ResumeFromCheckpointStreamingTest Failing.
  [BEAM-1133] Add maxNumRecords per micro-batch for Spark runner options.
  BigQueryIO.Write: support runtime schema and table
  Fix handling of null ValueProviders in DisplayData
  [BEAM-551] Fix handling of default for VP
  [BEAM-1120] Move some DataflowRunner configurations from code to properties
  [BEAM-551] Fix toString for FileBasedSource
  [BEAM-921] spark-runner: register sources and coders to serialize with java serializer
  [BEAM-551] Fix handling of TextIO.Sink
  ...


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

Branch: refs/heads/gearpump-runner
Commit: 4c445dd0b6de0f5045c02579cb432da4fbc5d486
Parents: b6e7bb6 647034c
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Dec 21 12:20:11 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Dec 21 12:20:11 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           |   7 +
 ...ostCommit_Java_RunnableOnService_Apex.groovy |  41 ++
 README.md                                       |   2 +-
 examples/java/pom.xml                           |  20 +-
 .../apache/beam/examples/WindowedWordCount.java | 177 +++---
 .../org/apache/beam/examples/WordCount.java     |   2 +-
 .../examples/common/WriteWindowedFilesDoFn.java |  77 +++
 .../beam/examples/complete/AutoComplete.java    |   6 +-
 .../apache/beam/examples/complete/TfIdf.java    |   6 +-
 .../examples/complete/TopWikipediaSessions.java |  15 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   4 +-
 .../beam/examples/complete/TrafficRoutes.java   |   4 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/WindowedWordCountIT.java      | 182 +++++-
 .../examples/complete/AutoCompleteTest.java     |   2 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |   2 +-
 .../examples/complete/game/LeaderBoard.java     |   4 +-
 .../beam/examples/complete/game/UserScore.java  |   2 +-
 .../complete/game/utils/WriteToBigQuery.java    |   2 +-
 .../game/utils/WriteWindowedToBigQuery.java     |   7 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         |   5 +-
 runners/apex/pom.xml                            |  66 ++-
 .../apache/beam/runners/apex/ApexRunner.java    | 111 ++--
 .../beam/runners/apex/ApexRunnerResult.java     |  50 +-
 .../beam/runners/apex/ApexYarnLauncher.java     | 395 +++++++++++++
 .../translation/ApexPipelineTranslator.java     |   2 +
 .../translation/ParDoBoundMultiTranslator.java  |  22 +-
 .../apex/translation/ParDoBoundTranslator.java  |  22 +-
 .../apex/translation/TranslationContext.java    |   4 +-
 .../apex/translation/WindowBoundTranslator.java |  78 +++
 .../operators/ApexGroupByKeyOperator.java       |   4 +-
 .../operators/ApexParDoOperator.java            |  27 +-
 .../beam/runners/apex/ApexYarnLauncherTest.java | 138 +++++
 .../FlattenPCollectionTranslatorTest.java       |  15 +-
 .../translation/GroupByKeyTranslatorTest.java   |  21 +-
 .../translation/ParDoBoundTranslatorTest.java   |  38 +-
 .../translation/ReadUnboundTranslatorTest.java  |  15 +-
 runners/core-java/pom.xml                       |   6 +-
 .../beam/runners/core/ActiveWindowSet.java      | 173 ++++++
 .../apache/beam/runners/core/AssignWindows.java |  46 --
 .../apache/beam/runners/core/DoFnRunner.java    |  24 +-
 .../apache/beam/runners/core/DoFnRunners.java   |   2 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  12 +-
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  48 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |   8 +-
 .../apache/beam/runners/core/KeyedWorkItem.java |  44 ++
 .../beam/runners/core/KeyedWorkItemCoder.java   | 130 +++++
 .../beam/runners/core/KeyedWorkItems.java       | 122 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  34 +-
 .../runners/core/MergingActiveWindowSet.java    | 428 ++++++++++++++
 .../runners/core/NonMergingActiveWindowSet.java |  91 +++
 .../beam/runners/core/PaneInfoTracker.java      |   4 +
 .../runners/core/PerKeyCombineFnRunner.java     | 149 +++++
 .../runners/core/PerKeyCombineFnRunners.java    |   1 -
 .../core/PushbackSideInputDoFnRunner.java       |  31 +-
 .../runners/core/ReduceFnContextFactory.java    |  26 +-
 .../beam/runners/core/ReduceFnRunner.java       | 496 ++++++++++------
 .../beam/runners/core/SimpleDoFnRunner.java     | 252 ++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   9 +-
 .../beam/runners/core/SplittableParDo.java      |  28 +-
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../apache/beam/runners/core/WatermarkHold.java |   5 +
 .../TriggerStateMachineContextFactory.java      |   2 +-
 .../triggers/TriggerStateMachineRunner.java     |  14 +-
 .../runners/core/KeyedWorkItemCoderTest.java    |  64 +++
 .../core/LateDataDroppingDoFnRunnerTest.java    |   3 +
 .../core/MergingActiveWindowSetTest.java        | 262 +++++++++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  55 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   | 133 ++---
 .../beam/runners/core/ReduceFnTester.java       |  77 ++-
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 301 ++++++++++
 .../beam/runners/core/SplittableParDoTest.java  |   2 -
 .../triggers/TriggerStateMachineTester.java     |  25 +-
 runners/direct-java/pom.xml                     |   3 +-
 .../runners/direct/AggregatorContainer.java     |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java     | 467 +++++++++++++++
 .../runners/direct/DirectExecutionContext.java  |   1 -
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |   9 +-
 .../beam/runners/direct/DirectGraphVisitor.java |  23 +-
 .../beam/runners/direct/DirectGroupByKey.java   |  46 +-
 .../direct/DirectGroupByKeyOverrideFactory.java |   3 +-
 .../beam/runners/direct/DirectRunner.java       |   7 +-
 .../beam/runners/direct/EvaluationContext.java  |   1 -
 .../direct/ExecutorServiceParallelExecutor.java |   4 +-
 .../runners/direct/ForwardingPTransform.java    |   6 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |   8 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   4 +-
 .../direct/KeyedPValueTrackingVisitor.java      |   4 +-
 .../direct/PTransformOverrideFactory.java       |  35 --
 .../beam/runners/direct/ParDoEvaluator.java     |   1 -
 .../runners/direct/ParDoEvaluatorFactory.java   |   2 +-
 .../direct/ParDoMultiOverrideFactory.java       |  29 +-
 .../ParDoSingleViaMultiOverrideFactory.java     |  17 +-
 ...littableProcessElementsEvaluatorFactory.java |   2 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   4 +-
 .../runners/direct/StepTransformResult.java     |   1 -
 .../direct/TestStreamEvaluatorFactory.java      |   7 +-
 .../beam/runners/direct/TransformResult.java    |   1 -
 .../runners/direct/ViewEvaluatorFactory.java    |   8 +-
 .../direct/WriteWithShardingFactory.java        |  10 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  18 +-
 .../runners/direct/CommittedResultTest.java     |   2 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 562 +++++++++++++++++++
 .../runners/direct/DirectGraphVisitorTest.java  |  71 ++-
 .../beam/runners/direct/DirectGraphs.java       |  35 ++
 .../beam/runners/direct/DirectRunnerTest.java   |  36 --
 .../runners/direct/EvaluationContextTest.java   |  83 +--
 .../direct/FlattenEvaluatorFactoryTest.java     |  15 +-
 .../direct/ForwardingPTransformTest.java        |   4 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   6 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |   7 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   2 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   4 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   3 +-
 .../StatefulParDoEvaluatorFactoryTest.java      |  11 +-
 .../runners/direct/StepTransformResultTest.java |   2 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  14 +-
 .../runners/direct/TransformExecutorTest.java   |   9 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |  24 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   4 +-
 .../direct/WatermarkCallbackExecutorTest.java   |   6 +-
 .../runners/direct/WatermarkManagerTest.java    | 237 ++++----
 .../direct/WriteWithShardingFactoryTest.java    |   4 +-
 runners/flink/examples/pom.xml                  |  14 +-
 .../beam/runners/flink/examples/TFIDF.java      |   6 +-
 .../beam/runners/flink/examples/WordCount.java  |   2 +-
 .../flink/examples/streaming/AutoComplete.java  |   6 +-
 runners/flink/pom.xml                           |   2 +-
 runners/flink/runner/pom.xml                    |  10 +-
 .../FlinkPipelineExecutionEnvironment.java      |  12 +
 .../runners/flink/FlinkPipelineOptions.java     |   5 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  24 +-
 .../FlinkBatchPipelineTranslator.java           |   4 +-
 .../FlinkBatchTransformTranslators.java         |  59 +-
 .../FlinkStreamingPipelineTranslator.java       |   7 +-
 .../FlinkStreamingTransformTranslators.java     |  77 ++-
 .../functions/FlinkDoFnFunction.java            |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   2 +-
 .../FlinkMergingPartialReduceFunction.java      |   2 +-
 .../functions/FlinkMergingReduceFunction.java   |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |  10 +-
 .../functions/FlinkPartialReduceFunction.java   |   2 +-
 .../functions/FlinkProcessContextBase.java      |  21 +-
 .../functions/FlinkReduceFunction.java          |   2 +-
 .../wrappers/streaming/DoFnOperator.java        |  46 +-
 .../streaming/SingletonKeyedWorkItem.java       |   2 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  14 +-
 .../wrappers/streaming/WorkItemKeySelector.java |   2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   6 +-
 .../flink/streaming/DoFnOperatorTest.java       |  13 +-
 .../flink/streaming/GroupByNullKeyTest.java     |  18 +-
 .../streaming/TopWikipediaSessionsITCase.java   |  10 +-
 runners/gearpump/pom.xml                        |   2 +-
 .../beam/runners/gearpump/GearpumpRunner.java   |   2 +-
 .../gearpump/examples/StreamingWordCount.java   |  14 +-
 .../translators/ParDoBoundMultiTranslator.java  |   2 +-
 .../translators/ParDoBoundTranslator.java       |   2 +-
 .../translators/TranslationContext.java         |   3 +-
 .../translators/utils/DoFnRunnerFactory.java    |   2 +-
 .../utils/NoOpAggregatorFactory.java            |   2 +-
 runners/google-cloud-dataflow-java/pom.xml      |  12 +-
 .../beam/runners/dataflow/DataflowClient.java   | 140 +++++
 .../runners/dataflow/DataflowPipelineJob.java   |  31 +-
 .../dataflow/DataflowPipelineTranslator.java    |  15 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 136 +++--
 .../runners/dataflow/DataflowRunnerInfo.java    |  92 +++
 .../dataflow/internal/AssignWindows.java        |  10 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   2 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  57 +-
 .../options/DataflowProfilingOptions.java       |   8 +-
 .../dataflow/testing/TestDataflowRunner.java    |  35 +-
 .../dataflow/util/DataflowTemplateJob.java      |   2 +-
 .../runners/dataflow/util/MonitoringUtil.java   |  22 +-
 .../beam/runners/dataflow/dataflow.properties   |  23 +
 .../dataflow/DataflowPipelineJobTest.java       |  49 +-
 .../DataflowPipelineTranslatorTest.java         |  61 +-
 .../dataflow/DataflowRunnerInfoTest.java        |  51 ++
 .../runners/dataflow/DataflowRunnerTest.java    | 285 ++++++----
 .../options/DataflowProfilingOptionsTest.java   |   6 +-
 .../testing/TestDataflowRunnerTest.java         |  52 +-
 .../transforms/DataflowGroupByKeyTest.java      |  18 +-
 .../dataflow/transforms/DataflowViewTest.java   |  20 +-
 .../dataflow/util/MonitoringUtilTest.java       |  21 +-
 runners/pom.xml                                 |   2 +-
 runners/spark/README.md                         |  59 +-
 runners/spark/pom.xml                           |  55 +-
 .../beam/runners/spark/EvaluationResult.java    |  67 ---
 .../runners/spark/SparkPipelineOptions.java     |  10 +
 .../beam/runners/spark/SparkPipelineResult.java | 193 +++++++
 .../apache/beam/runners/spark/SparkRunner.java  | 130 +++--
 .../beam/runners/spark/TestSparkRunner.java     |  91 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   6 +-
 .../spark/aggregators/SparkAggregators.java     | 126 +++++
 .../metrics/WithNamedAggregatorsSupport.java    |   4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |  60 +-
 .../beam/runners/spark/examples/WordCount.java  |  15 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |   2 +-
 .../beam/runners/spark/io/CreateStream.java     |   2 +-
 .../beam/runners/spark/io/SourceDStream.java    |  21 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  17 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   4 +-
 .../runners/spark/translation/DoFnFunction.java | 110 ++--
 .../spark/translation/EvaluationContext.java    | 131 +----
 .../translation/GroupCombineFunctions.java      |  23 +-
 .../spark/translation/MultiDoFnFunction.java    | 135 ++---
 .../spark/translation/SparkAssignWindowFn.java  |  69 +++
 .../spark/translation/SparkContextFactory.java  |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   | 214 +++++++
 .../spark/translation/SparkProcessContext.java  | 385 +++----------
 .../spark/translation/SparkRuntimeContext.java  |  62 +-
 .../translation/StorageLevelPTransform.java     |   2 +-
 .../spark/translation/TransformTranslator.java  |  67 +--
 .../spark/translation/TranslationUtils.java     |  33 ++
 .../spark/translation/WindowingHelpers.java     |   3 +-
 .../streaming/StreamingTransformTranslator.java |  45 +-
 .../util/SinglePrimitiveOutputPTransform.java   |   2 +-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 ++++
 .../runners/spark/ProvidedSparkContextTest.java |   6 +-
 .../runners/spark/SparkPipelineStateTest.java   | 217 +++++++
 .../spark/aggregators/ClearAggregatorsRule.java |  37 ++
 .../metrics/sink/ClearAggregatorsRule.java      |  33 --
 .../metrics/sink/NamedAggregatorsTest.java      |   3 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |  57 ++
 .../beam/runners/spark/io/AvroPipelineTest.java |   2 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   2 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   2 +-
 .../spark/translation/SideEffectsTest.java      |  59 --
 .../streaming/EmptyStreamAssertionTest.java     |   4 +
 .../streaming/KafkaStreamingTest.java           |  34 +-
 .../ResumeFromCheckpointStreamingTest.java      |  32 +-
 .../streaming/utils/PAssertStreaming.java       |  31 +-
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml |  68 ++-
 sdks/java/core/pom.xml                          |   2 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   3 +-
 .../beam/sdk/annotations/Experimental.java      |   5 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |   2 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   4 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  16 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |  52 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |  17 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  77 ++-
 .../org/apache/beam/sdk/io/FileBasedSource.java |  12 +-
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 +
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 +
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 ++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |   4 +-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   6 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  12 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |   4 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   2 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   8 +-
 .../apache/beam/sdk/metrics/CounterCell.java    |  10 +-
 .../beam/sdk/metrics/DistributionCell.java      |  14 +-
 .../org/apache/beam/sdk/metrics/MetricCell.java |   2 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |   3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |   5 +-
 .../beam/sdk/metrics/MetricsEnvironment.java    |   8 +-
 .../sdk/options/PipelineOptionsFactory.java     | 110 +++-
 .../beam/sdk/options/StreamingOptions.java      |   2 +-
 .../apache/beam/sdk/options/ValueProvider.java  |  27 +-
 .../sdk/runners/PTransformOverrideFactory.java  |  41 ++
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    |  26 +-
 .../beam/sdk/testing/BigqueryMatcher.java       |  48 +-
 .../beam/sdk/testing/FileChecksumMatcher.java   | 114 +---
 .../apache/beam/sdk/testing/GatherAllPanes.java |   2 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  12 +-
 .../apache/beam/sdk/testing/TestPipeline.java   | 225 +++++++-
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    |  24 +
 .../beam/sdk/testing/UsesTimersInParDo.java     |  25 +
 .../sdk/transforms/AggregatorRetriever.java     |   2 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |  75 +--
 .../beam/sdk/transforms/ApproximateUnique.java  |   4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  14 +-
 .../beam/sdk/transforms/CombineWithContext.java |   2 +-
 .../org/apache/beam/sdk/transforms/Count.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Create.java  |   4 +-
 .../apache/beam/sdk/transforms/Distinct.java    |   4 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  22 +
 .../beam/sdk/transforms/DoFnAdapters.java       |  12 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  29 +-
 .../org/apache/beam/sdk/transforms/Filter.java  |   2 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   2 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   4 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   2 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |   4 +-
 .../apache/beam/sdk/transforms/MapElements.java |   2 +-
 .../apache/beam/sdk/transforms/PTransform.java  |   4 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 191 +------
 .../apache/beam/sdk/transforms/Partition.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  14 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Values.java  |   2 +-
 .../org/apache/beam/sdk/transforms/View.java    |  16 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../beam/sdk/transforms/WithTimestamps.java     |   2 +-
 .../sdk/transforms/display/DisplayData.java     |  22 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |   2 +-
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  11 +
 .../sdk/transforms/reflect/DoFnInvoker.java     |   8 +
 .../sdk/transforms/reflect/DoFnSignature.java   |  26 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |  96 +++-
 .../beam/sdk/transforms/windowing/Window.java   |   4 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   | 173 ------
 .../org/apache/beam/sdk/util/ApiSurface.java    |  52 +-
 .../apache/beam/sdk/util/ExecutionContext.java  |   8 +-
 .../beam/sdk/util/ExplicitShardedFile.java      | 120 ++++
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 107 +++-
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |  43 --
 .../beam/sdk/util/KeyedWorkItemCoder.java       | 128 -----
 .../apache/beam/sdk/util/KeyedWorkItems.java    | 121 ----
 .../beam/sdk/util/MergingActiveWindowSet.java   | 428 --------------
 .../sdk/util/NonMergingActiveWindowSet.java     |  91 ---
 .../beam/sdk/util/NumberedShardedFile.java      | 220 ++++++++
 .../beam/sdk/util/PerKeyCombineFnRunner.java    | 148 -----
 .../org/apache/beam/sdk/util/PropertyNames.java |   2 +
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   4 -
 .../org/apache/beam/sdk/util/Reshuffle.java     |   2 +-
 .../org/apache/beam/sdk/util/ShardedFile.java   |  42 ++
 .../CopyOnAccessInMemoryStateInternals.java     | 453 ---------------
 .../sdk/util/state/InMemoryStateInternals.java  |  33 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 126 +++--
 .../beam/sdk/util/state/StateInternals.java     |   2 +-
 .../beam/sdk/util/state/TimerCallback.java      |  35 --
 .../org/apache/beam/sdk/values/PCollection.java |   4 +-
 .../beam/sdk/values/PCollectionTuple.java       |   2 +-
 .../org/apache/beam/sdk/values/TypedPValue.java |   2 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |   8 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |   6 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   4 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  63 ++-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  10 +
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  11 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  26 +
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 ++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  19 +
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   2 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |   4 +-
 .../apache/beam/sdk/metrics/MetricMatchers.java |  47 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    |  63 ++-
 .../sdk/options/PipelineOptionsFactoryTest.java | 222 +++++++-
 .../beam/sdk/options/ValueProviderTest.java     |  21 +
 .../sdk/runners/TransformHierarchyTest.java     |  17 +-
 .../beam/sdk/runners/TransformTreeTest.java     |   4 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   |  82 +--
 .../sdk/testing/FileChecksumMatcherTest.java    |  77 ---
 .../beam/sdk/testing/TestPipelineTest.java      | 186 +++++-
 .../beam/sdk/transforms/DoFnTesterTest.java     |   2 +-
 .../sdk/transforms/FlatMapElementsTest.java     |   2 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |   4 +-
 .../beam/sdk/transforms/MapElementsTest.java    |   2 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |  72 ---
 .../apache/beam/sdk/transforms/OldDoFnTest.java | 125 ++---
 .../beam/sdk/transforms/PTransformTest.java     |   2 +-
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  28 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 342 ++++++-----
 .../beam/sdk/transforms/SplittableDoFnTest.java |  58 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   4 +-
 .../display/DisplayDataEvaluatorTest.java       |   2 +-
 .../display/DisplayDataMatchersTest.java        |   6 +-
 .../sdk/transforms/display/DisplayDataTest.java |  90 ++-
 .../DoFnSignaturesSplittableDoFnTest.java       |   3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  47 ++
 .../sdk/transforms/windowing/WindowingTest.java |   2 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  85 ++-
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |  62 --
 .../sdk/util/MergingActiveWindowSetTest.java    | 262 ---------
 .../beam/sdk/util/NumberedShardedFileTest.java  | 181 ++++++
 .../apache/beam/sdk/util/StringUtilsTest.java   |   6 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 552 ------------------
 .../util/state/InMemoryTimerInternalsTest.java  | 125 +++--
 .../org/apache/beam/sdk/values/PDoneTest.java   |   4 +-
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/extensions/sorter/README.md           |   2 +-
 sdks/java/extensions/sorter/pom.xml             |  10 +-
 .../sorter/BufferedExternalSorter.java          |   6 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |  15 +-
 .../sdk/extensions/sorter/InMemorySorter.java   |  26 +-
 .../beam/sdk/extensions/sorter/SortValues.java  |   2 +-
 .../sorter/BufferedExternalSorterTest.java      |  16 +
 .../extensions/sorter/ExternalSorterTest.java   |  16 +
 .../extensions/sorter/InMemorySorterTest.java   |   8 +
 sdks/java/io/google-cloud-platform/pom.xml      |   8 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 453 +++++++++++----
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   7 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 121 +++-
 .../io/gcp/bigquery/BigQueryTableInserter.java  | 217 -------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  28 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   6 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |   4 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 ++
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 ++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  |   2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 543 ++++++++++++++----
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 139 ++++-
 .../gcp/bigquery/BigQueryTableInserterTest.java | 245 --------
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  50 +-
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++
 sdks/java/io/hdfs/pom.xml                       |   8 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 +
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 ++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 ++
 sdks/java/io/jdbc/pom.xml                       |   2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |   4 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |   4 +-
 sdks/java/io/jms/pom.xml                        |  22 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 342 +++++++----
 .../org/apache/beam/sdk/io/jms/JmsRecord.java   |   8 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  20 +-
 sdks/java/io/kinesis/pom.xml                    |   2 +-
 sdks/java/io/mongodb/pom.xml                    |   2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |   4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |  16 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |   8 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |   8 +-
 sdks/java/io/pom.xml                            |   2 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../maven-archetypes/examples-java8/pom.xml     |  13 +-
 .../main/resources/archetype-resources/pom.xml  | 122 +++-
 sdks/java/maven-archetypes/examples/pom.xml     |  13 +-
 .../main/resources/archetype-resources/pom.xml  |  47 +-
 sdks/java/maven-archetypes/pom.xml              |   2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |   9 +-
 .../main/resources/archetype-resources/pom.xml  |  16 +-
 .../resources/projects/basic/reference/pom.xml  |  16 +-
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 446 files changed, 13550 insertions(+), 7606 deletions(-)
----------------------------------------------------------------------



[39/50] incubator-beam git commit: More escaping in Jenkins timestamp spec

Posted by ke...@apache.org.
More escaping in Jenkins timestamp spec


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

Branch: refs/heads/gearpump-runner
Commit: 627ccb522d7d818aebe2256246e81af2d3062d11
Parents: 66318d8
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Dec 19 11:39:29 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Dec 19 11:39:29 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/627ccb52/.jenkins/common_job_properties.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy
index 7a88604..3880236 100644
--- a/.jenkins/common_job_properties.groovy
+++ b/.jenkins/common_job_properties.groovy
@@ -136,7 +136,7 @@ class common_job_properties {
   static def setMavenConfig(def context) {
     context.mavenInstallation('Maven 3.3.3')
     context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true')
-    context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS')
+    context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\\\'T\\\'HH:mm:ss.SSS')
     context.rootPOM('pom.xml')
     // Use a repository local to the workspace for better isolation of jobs.
     context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE)


[21/50] incubator-beam git commit: This closes #1612: [BEAM-27] Support timer setting and receiving in SimpleDoFnRunner

Posted by ke...@apache.org.
This closes #1612: [BEAM-27] Support timer setting and receiving in SimpleDoFnRunner

  Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest
  Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle
  Add timer support to DoFnRunner(s)
  Make TimerSpec and StateSpec fields accessible


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

Branch: refs/heads/gearpump-runner
Commit: c28957d16fb0f63f82f578cf904df61bf7bb63e5
Parents: 5255a33 b78aa66
Author: Kenneth Knowles <kl...@google.com>
Authored: Sat Dec 17 12:59:25 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sat Dec 17 12:59:25 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunner.java    |   9 +
 .../core/LateDataDroppingDoFnRunner.java        |   7 +
 .../core/PushbackSideInputDoFnRunner.java       |   8 +
 .../beam/runners/core/SimpleDoFnRunner.java     | 236 ++++++++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   8 +
 .../core/PushbackSideInputDoFnRunnerTest.java   |  41 +++
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 301 +++++++++++++++++++
 .../sdk/transforms/reflect/DoFnSignatures.java  |   4 +
 8 files changed, 613 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[27/50] incubator-beam git commit: [BEAM-716] Fix javadoc on with* methods [BEAM-959] Improve check preconditions in JmsIO

Posted by ke...@apache.org.
[BEAM-716] Fix javadoc on with* methods
[BEAM-959] Improve check preconditions 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/30e14cfa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/30e14cfa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/30e14cfa

Branch: refs/heads/gearpump-runner
Commit: 30e14cfa63db50d567185599ea049c96229b48e2
Parents: caf1c72
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Tue Dec 13 21:55:46 2016 +0100
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Mon Dec 19 07:24:05 2016 +0100

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 45 +++++++++++++-------
 1 file changed, 30 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30e14cfa/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 76dee67..b6de26a 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
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.io.jms;
 
 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 com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
@@ -145,7 +145,7 @@ public class JmsIO {
     }
 
     /**
-     * <p>Specify the JMS connection factory to connect to the JMS broker.
+     * Specify the JMS connection factory to connect to the JMS broker.
      *
      * <p>For instance:
      *
@@ -159,11 +159,13 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Read withConnectionFactory(ConnectionFactory connectionFactory) {
+      checkArgument(connectionFactory != null, "withConnectionFactory(connectionFactory) called"
+          + " with null connectionFactory");
       return builder().setConnectionFactory(connectionFactory).build();
     }
 
     /**
-     * <p>Specify the JMS queue destination name where to read messages from. The
+     * Specify the JMS queue destination name where to read messages from. The
      * {@link JmsIO.Read} acts as a consumer on the queue.
      *
      * <p>This method is exclusive with {@link JmsIO.Read#withTopic(String)}. The user has to
@@ -181,11 +183,12 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Read withQueue(String queue) {
+      checkArgument(queue != null, "withQueue(queue) called with null queue");
       return builder().setQueue(queue).build();
     }
 
     /**
-     * <p>Specify the JMS topic destination name where to receive messages from. The
+     * Specify the JMS topic destination name where to receive messages from. The
      * {@link JmsIO.Read} acts as a subscriber on the topic.
      *
      * <p>This method is exclusive with {@link JmsIO.Read#withQueue(String)}. The user has to
@@ -203,11 +206,12 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Read withTopic(String topic) {
+      checkArgument(topic != null, "withTopic(topic) called with null topic");
       return builder().setTopic(topic).build();
     }
 
     /**
-     * <p>Define the max number of records that the source will read. Using a max number of records
+     * Define the max number of records that the source will read. Using a max number of records
      * different from {@code Long.MAX_VALUE} means the source will be {@code Bounded}, and will
      * stop once the max number of records read is reached.
      *
@@ -223,11 +227,13 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Read withMaxNumRecords(long maxNumRecords) {
+      checkArgument(maxNumRecords >= 0, "withMaxNumRecords(maxNumRecords) called with invalid "
+          + "maxNumRecords");
       return builder().setMaxNumRecords(maxNumRecords).build();
     }
 
     /**
-     * <p>Define the max read time that the source will read. Using a non null max read time
+     * Define the max read time that the source will read. Using a non null max read time
      * duration means the source will be {@code Bounded}, and will stop once the max read time is
      * reached.
      *
@@ -243,6 +249,8 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Read withMaxReadTime(Duration maxReadTime) {
+      checkArgument(maxReadTime != null, "withMaxReadTime(maxReadTime) called with null "
+          + "maxReadTime");
       return builder().setMaxReadTime(maxReadTime).build();
     }
 
@@ -264,9 +272,11 @@ public class JmsIO {
 
     @Override
     public void validate(PBegin input) {
-      checkNotNull(getConnectionFactory(), "ConnectionFactory not specified");
-      checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic not "
-          + "specified");
+      checkState(getConnectionFactory() != null, "JmsIO.read() requires a JMS connection "
+          + "factory to be set via withConnectionFactory(connectionFactory)");
+      checkState((getQueue() != null || getTopic() != null), "JmsIO.read() requires a JMS "
+          + "destination (queue or topic) to be set via withQueue(queueName) or withTopic"
+          + "(topicName)");
     }
 
     @Override
@@ -497,7 +507,7 @@ public class JmsIO {
     }
 
     /**
-     * <p>Specify the JMS connection factory to connect to the JMS broker.
+     * Specify the JMS connection factory to connect to the JMS broker.
      *
      * <p>For instance:
      *
@@ -511,11 +521,13 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Write withConnectionFactory(ConnectionFactory connectionFactory) {
+      checkArgument(connectionFactory != null, "withConnectionFactory(connectionFactory) called"
+          + " with null connectionFactory");
       return builder().setConnectionFactory(connectionFactory).build();
     }
 
     /**
-     * <p>Specify the JMS queue destination name where to send messages to. The
+     * Specify the JMS queue destination name where to send messages to. The
      * {@link JmsIO.Write} acts as a producer on the queue.
      *
      * <p>This method is exclusive with {@link JmsIO.Write#withTopic(String)}. The user has to
@@ -533,11 +545,12 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Write withQueue(String queue) {
+      checkArgument(queue != null, "withQueue(queue) called with null queue");
       return builder().setQueue(queue).build();
     }
 
     /**
-     * <p>Specify the JMS topic destination name where to send messages to. The
+     * Specify the JMS topic destination name where to send messages to. The
      * {@link JmsIO.Read} acts as a publisher on the topic.
      *
      * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String)}. The user has to
@@ -555,6 +568,7 @@ public class JmsIO {
      * @return The corresponding {@link JmsIO.Read}.
      */
     public Write withTopic(String topic) {
+      checkArgument(topic != null, "withTopic(topic) called with null topic");
       return builder().setTopic(topic).build();
     }
 
@@ -566,9 +580,10 @@ public class JmsIO {
 
     @Override
     public void validate(PCollection<String> input) {
-      checkNotNull(getConnectionFactory(), "ConnectionFactory is not defined");
-      checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic is "
-          + "required");
+      checkState(getConnectionFactory() != null, "JmsIO.write() requires a JMS connection "
+          + "factory to be set via withConnectionFactory(connectionFactory)");
+      checkState((getQueue() != null || getTopic() != null), "JmsIO.write() requires a JMS "
+          + "destination (queue or topic) to be set via withQueue(queue) or withTopic(topic)");
     }
 
     private static class WriterFn extends DoFn<String, Void> {


[24/50] incubator-beam git commit: This closes #1595, makes TestPipeline a JUnit rule

Posted by ke...@apache.org.
This closes #1595, makes TestPipeline a JUnit rule

  Empty TestPipeline need not be run
  [BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage


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

Branch: refs/heads/gearpump-runner
Commit: 1c9bf8d665979961478160c384b57ae42847e7b1
Parents: c28957d 0d54a4c
Author: Kenneth Knowles <kl...@google.com>
Authored: Sat Dec 17 14:12:37 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sat Dec 17 14:12:37 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/TestPipeline.java   | 225 ++++++++++++++++---
 .../beam/sdk/testing/TestPipelineTest.java      | 186 +++++++++++++--
 2 files changed, 365 insertions(+), 46 deletions(-)
----------------------------------------------------------------------



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

Posted by ke...@apache.org.
Closes #1650


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

Branch: refs/heads/gearpump-runner
Commit: 7d1976b2628e0d560df57610b8ed8a6b8443fb7b
Parents: abdbee6 6a4a699
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 16 17:41:51 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 16 17:41:51 2016 -0800

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/beam/sdk/transforms/View.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[16/50] incubator-beam git commit: [BEAM-853] Force streaming execution on batch pipelines for testing. Expose the adapted source.

Posted by ke...@apache.org.
[BEAM-853] Force streaming execution on batch pipelines for testing.
Expose the adapted source.

Force streaming execution, if set in PipelineOptions.

Added 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/5f41deda
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/5f41deda
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/5f41deda

Branch: refs/heads/gearpump-runner
Commit: 5f41deda509acbbbc6280323e583bb3c1af2dad2
Parents: 1ad638e
Author: Sela <an...@paypal.com>
Authored: Wed Dec 14 12:20:08 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 16 22:03:44 2016 -0800

----------------------------------------------------------------------
 .../runners/spark/SparkPipelineOptions.java     |   5 +
 .../beam/runners/spark/TestSparkRunner.java     |  80 +++++++++++-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 +++++++++++++++++++
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  14 ++-
 4 files changed, 217 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f41deda/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
index a2cd887..04c559e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
@@ -100,4 +100,9 @@ public interface SparkPipelineOptions
   @Default.Boolean(false)
   boolean getUsesProvidedSparkContext();
   void setUsesProvidedSparkContext(boolean value);
+
+  @Description("A special flag that forces streaming in tests.")
+  @Default.Boolean(false)
+  boolean isForceStreaming();
+  void setForceStreaming(boolean forceStreaming);
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f41deda/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
index 2c26d84..798ca47 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
@@ -19,16 +19,26 @@
 package org.apache.beam.runners.spark;
 
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
 
+import org.apache.beam.runners.core.UnboundedReadFromBoundedSource;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource;
 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.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+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.apache.beam.sdk.values.POutput;
 
+
 /**
  * The SparkRunner translate operations defined on a pipeline to a representation executable
  * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a Beam
@@ -53,9 +63,12 @@ import org.apache.beam.sdk.values.POutput;
 public final class TestSparkRunner extends PipelineRunner<SparkPipelineResult> {
 
   private SparkRunner delegate;
+  private boolean isForceStreaming;
+  private int expectedNumberOfAssertions = 0;
 
   private TestSparkRunner(SparkPipelineOptions options) {
     this.delegate = SparkRunner.fromOptions(options);
+    this.isForceStreaming = options.isForceStreaming();
   }
 
   public static TestSparkRunner fromOptions(PipelineOptions options) {
@@ -65,19 +78,78 @@ public final class TestSparkRunner extends PipelineRunner<SparkPipelineResult> {
     return new TestSparkRunner(sparkOptions);
   }
 
+  /**
+   * Overrides for the test runner.
+   */
+  @SuppressWarnings("unchecked")
   @Override
-  public <OutputT extends POutput, InputT extends PInput>
-      OutputT apply(PTransform<InputT, OutputT> transform, InputT input) {
-    return delegate.apply(transform, input);
-  };
+  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+          PTransform<InputT, OutputT> transform, InputT input) {
+    // if the pipeline forces execution as a streaming pipeline,
+    // and the source is an adapted unbounded source (as bounded),
+    // read it as unbounded source via UnboundedReadFromBoundedSource.
+    if (isForceStreaming && transform instanceof BoundedReadFromUnboundedSource) {
+      return (OutputT) delegate.apply(new AdaptedBoundedAsUnbounded(
+          (BoundedReadFromUnboundedSource) transform), input);
+    } else {
+      // no actual override, simply counts asserting transforms in the pipeline.
+      if (transform instanceof PAssert.OneSideInputAssert
+          || transform instanceof PAssert.GroupThenAssert
+          || transform instanceof PAssert.GroupThenAssertForSingleton) {
+        expectedNumberOfAssertions += 1;
+      }
+
+      return delegate.apply(transform, input);
+    }
+  }
 
   @Override
   public SparkPipelineResult run(Pipeline pipeline) {
     TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class);
     SparkPipelineResult result = delegate.run(pipeline);
     result.waitUntilFinish();
+
+    // make sure the test pipeline finished successfully.
+    State resultState = result.getState();
+    assertThat(
+        String.format("Test pipeline result state was %s instead of %s", resultState, State.DONE),
+        resultState,
+        is(State.DONE));
     assertThat(result, testPipelineOptions.getOnCreateMatcher());
     assertThat(result, testPipelineOptions.getOnSuccessMatcher());
+
+    // if the pipeline was executed in streaming mode, validate aggregators.
+    if (isForceStreaming) {
+      // validate assertion succeeded (at least once).
+      int success = result.getAggregatorValue(PAssert.SUCCESS_COUNTER, Integer.class);
+      assertThat(
+          String.format(
+              "Expected %d successful assertions, but found %d.",
+              expectedNumberOfAssertions, success),
+          success,
+          is(expectedNumberOfAssertions));
+      // validate assertion didn't fail.
+      int failure = result.getAggregatorValue(PAssert.FAILURE_COUNTER, Integer.class);
+      assertThat("Failure aggregator should be zero.", failure, is(0));
+    }
     return result;
   }
+
+  private static class AdaptedBoundedAsUnbounded<T> extends PTransform<PBegin, PCollection<T>> {
+    private final BoundedReadFromUnboundedSource<T> source;
+
+    AdaptedBoundedAsUnbounded(BoundedReadFromUnboundedSource<T> source) {
+      this.source = source;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public PCollection<T> expand(PBegin input) {
+      PTransform<PBegin, ? extends PCollection<ValueWithRecordId<T>>> replacingTransform =
+          new UnboundedReadFromBoundedSource<>(source.getAdaptedSource());
+      return (PCollection<T>) input.apply(replacingTransform)
+          .apply("StripIds", ParDo.of(new ValueWithRecordId.StripIdsDoFn()));
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f41deda/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java
new file mode 100644
index 0000000..eb17eea
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.junit.Test;
+
+
+/**
+ * Test that we can "force streaming" on pipelines with {@link BoundedReadFromUnboundedSource}
+ * inputs using the {@link TestSparkRunner}.
+ *
+ * <p>The test validates that when a pipeline reads from a {@link BoundedReadFromUnboundedSource},
+ * with {@link SparkPipelineOptions#setStreaming(boolean)} true
+ * and using the {@link TestSparkRunner}; the {@link Read.Bounded} transform
+ * is replaced by an {@link Read.Unbounded} transform.
+ *
+ * <p>This test does not execute a pipeline.
+ */
+public class ForceStreamingTest {
+
+  @Test
+  public void test() throws IOException {
+    SparkPipelineOptions options = PipelineOptionsFactory.create().as(SparkPipelineOptions.class);
+    options.setRunner(TestSparkRunner.class);
+    // force streaming.
+    options.setForceStreaming(true);
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    // apply the BoundedReadFromUnboundedSource.
+    @SuppressWarnings("unchecked")
+    BoundedReadFromUnboundedSource boundedRead =
+        Read.from(new FakeUnboundedSource()).withMaxNumRecords(-1);
+    //noinspection unchecked
+    pipeline.apply(boundedRead);
+
+    UnboundedReadDetector unboundedReadDetector = new UnboundedReadDetector();
+    pipeline.traverseTopologically(unboundedReadDetector);
+
+    // assert that the applied BoundedReadFromUnboundedSource
+    // is being treated as an unbounded read.
+    assertThat("Expected to have an unbounded read.", unboundedReadDetector.isUnbounded);
+  }
+
+  /**
+   * Traverses the Pipeline to check if the input is indeed a {@link Read.Unbounded}.
+   */
+  private class UnboundedReadDetector extends Pipeline.PipelineVisitor.Defaults {
+    private boolean isUnbounded = false;
+
+    @Override
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+      Class<? extends PTransform> transformClass = node.getTransform().getClass();
+      if (transformClass == Read.Unbounded.class) {
+        isUnbounded = true;
+      }
+    }
+
+  }
+
+  /**
+   * A fake {@link UnboundedSource} to satisfy the compiler.
+   */
+  private static class FakeUnboundedSource extends UnboundedSource {
+
+    @Override
+    public List<? extends UnboundedSource> generateInitialSplits(
+        int desiredNumSplits,
+        PipelineOptions options) throws Exception {
+      return null;
+    }
+
+    @Override
+    public UnboundedReader createReader(
+        PipelineOptions options,
+        CheckpointMark checkpointMark) throws IOException {
+      return null;
+    }
+
+    @Override
+    public Coder getCheckpointMarkCoder() {
+      return null;
+    }
+
+    @Override
+    public void validate() { }
+
+    @Override
+    public Coder getDefaultOutputCoder() {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f41deda/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 f2ef358..84e3044 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
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Distinct;
@@ -50,6 +51,7 @@ public class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PColle
   private final UnboundedSource<T, ?> source;
   private final long maxNumRecords;
   private final Duration maxReadTime;
+  private final BoundedSource<ValueWithRecordId<T>> adaptedSource;
   private static final FluentBackoff BACKOFF_FACTORY =
       FluentBackoff.DEFAULT
           .withInitialBackoff(Duration.millis(10))
@@ -81,12 +83,22 @@ public class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PColle
     this.source = source;
     this.maxNumRecords = maxNumRecords;
     this.maxReadTime = maxReadTime;
+    this.adaptedSource = new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime);
+  }
+
+  /**
+   * Returns an adapted {@link BoundedSource} wrapping the underlying {@link UnboundedSource},
+   * with the specified bounds on number of records and read time.
+   */
+  @Experimental
+  public BoundedSource<ValueWithRecordId<T>> getAdaptedSource() {
+    return adaptedSource;
   }
 
   @Override
   public PCollection<T> expand(PBegin input) {
     PCollection<ValueWithRecordId<T>> read = Pipeline.applyTransform(input,
-        Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime)));
+        Read.from(getAdaptedSource()));
     if (source.requiresDeduping()) {
       read = read.apply(Distinct.withRepresentativeValueFn(
           new SerializableFunction<ValueWithRecordId<T>, byte[]>() {


[03/50] incubator-beam git commit: checkstyle: missed newline in DistributionCell

Posted by ke...@apache.org.
checkstyle: missed newline in DistributionCell


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

Branch: refs/heads/gearpump-runner
Commit: 92dbab8ecf2761893759c494d33aa8583e919a6f
Parents: 1ee191f
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 16 15:56:00 2016 -0800
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Fri Dec 16 15:56:00 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/metrics/DistributionCell.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/92dbab8e/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
index e095158..7f684a8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java
@@ -66,4 +66,5 @@ public class DistributionCell implements MetricCell<Distribution, DistributionDa
   public Distribution getInterface() {
     return this;
   }
-}
\ No newline at end of file
+}
+


[04/50] incubator-beam git commit: Closes #1649

Posted by ke...@apache.org.
Closes #1649


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

Branch: refs/heads/gearpump-runner
Commit: abdbee61c7fa2383343f86f3e88f134163177439
Parents: 1ee191f 92dbab8
Author: bchambers <bc...@google.com>
Authored: Fri Dec 16 16:26:43 2016 -0800
Committer: bchambers <bc...@google.com>
Committed: Fri Dec 16 16:26:43 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/metrics/DistributionCell.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------