[46/51] [abbrv] incubator-beam git commit: Allow setting timer by ID in DirectTimerInternals

2016-12-21 Thread kenn
Allow setting timer by ID in DirectTimerInternals


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

Branch: refs/heads/python-sdk
Commit: 7f14c463acd2ae5b86ac81a9528ac4aa7dff765f
Parents: 4d71924
Author: Kenneth Knowles 
Authored: Wed Dec 7 20:18:44 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../runners/direct/DirectTimerInternals.java|  2 +-
 .../beam/runners/direct/WatermarkManager.java   | 25 
 2 files changed, 26 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f14c463/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 5ca276d..80e0721 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -49,7 +49,7 @@ class DirectTimerInternals implements TimerInternals {
   @Override
   public void setTimer(StateNamespace namespace, String timerId, Instant 
target,
   TimeDomain timeDomain) {
-throw new UnsupportedOperationException("Setting timer by ID not yet 
supported.");
+timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, 
timeDomain));
   }
 
   @Deprecated

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f14c463/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index 7bed751..f7bafd1 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -23,11 +23,13 @@ import com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.SortedMultiset;
+import com.google.common.collect.Table;
 import com.google.common.collect.TreeMultiset;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -56,6 +58,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.joda.time.Instant;
@@ -210,6 +213,10 @@ public class WatermarkManager {
 private final SortedMultiset pendingElements;
 private final Map objectTimers;
 
+// Entries in this table represent the authoritative timestamp for which
+// a per-key-and-StateNamespace timer is set.
+private final Map> existingTimers;
+
 private AtomicReference currentWatermark;
 
 public AppliedPTransformInputWatermark(Collection 
inputWatermarks) {
@@ -222,6 +229,7 @@ public class WatermarkManager {
   this.pendingElements =
   TreeMultiset.create(pendingBundleComparator);
   this.objectTimers = new HashMap<>();
+  this.existingTimers = new HashMap<>();
   currentWatermark = new 
AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
 }
 
@@ -276,14 +284,31 @@ public class WatermarkManager {
 keyTimers = new TreeSet<>();
 objectTimers.put(update.key, keyTimers);
   }
+  Table existingTimersForKey =
+  existingTimers.get(update.key);
+  if (existingTimersForKey == null) {
+existingTimersForKey = HashBasedTable.create();
+existingTimers.put(update.key, existingTimersForKey);
+  }
+

[49/51] [abbrv] incubator-beam git commit: This closes #1669: Preliminaries for timers in the direct runner

2016-12-21 Thread kenn
This closes #1669: Preliminaries for timers in the direct runner

  Hold output watermark according to pending timers
  Allow setting timer by ID in DirectTimerInternals
  Add UsesTestStream for use with JUnit @Category
  Add static Window.withOutputTimeFn to match build method
  Use informative Instant formatter in WatermarkHold
  Add informative Instant formatter to BoundedWindow


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

Branch: refs/heads/python-sdk
Commit: 57d9bbd797edfcf32fdd9284b802fc4f9694e8d2
Parents: ff39516 dfe2e62
Author: Kenneth Knowles 
Authored: Wed Dec 21 13:46:34 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:46:34 2016 -0800

--
 .../apache/beam/runners/core/WatermarkHold.java |  4 +-
 .../runners/direct/DirectTimerInternals.java|  2 +-
 .../beam/runners/direct/WatermarkManager.java   | 78 ++--
 .../apache/beam/sdk/testing/UsesTestStream.java | 24 ++
 .../sdk/transforms/windowing/BoundedWindow.java | 31 
 .../beam/sdk/transforms/windowing/Window.java   |  9 +++
 .../apache/beam/sdk/testing/TestStreamTest.java | 12 +--
 7 files changed, 144 insertions(+), 16 deletions(-)
--




[50/51] [abbrv] incubator-beam git commit: Merge remote-tracking branch 'origin/master' into python-sdk

2016-12-21 Thread kenn
Merge remote-tracking branch 'origin/master' into python-sdk


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

Branch: refs/heads/python-sdk
Commit: 77742979463d0590969b186cee2af017a2bb1037
Parents: 3b4fd5c 57d9bbd
Author: Ahmet Altay 
Authored: Wed Dec 21 14:22:06 2016 -0800
Committer: Ahmet Altay 
Committed: Wed Dec 21 14:22:06 2016 -0800

--
 .gitignore  |   4 +
 .jenkins/common_job_properties.groovy   | 173 ++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 ++
 ...ostCommit_Java_RunnableOnService_Apex.groovy |  41 ++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 ++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 ++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 ++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 ++
 .../job_beam_PostCommit_Python_Verify.groovy|  37 ++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 ++
 .../job_beam_Release_NightlySnapshot.groovy |  46 ++
 .jenkins/job_seed.groovy|  47 ++
 .travis.yml |   2 +-
 .travis/README.md   |   2 +-
 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 +-
 .../org/apache/beam/examples/complete/README.md |  14 +-
 .../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/DeDupExample.java|  96 ---
 .../beam/examples/cookbook/DistinctExample.java |  96 +++
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java|   2 +-
 .../org/apache/beam/examples/cookbook/README.md |  16 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/WindowedWordCountIT.java  | 182 +-
 .../org/apache/beam/examples/WordCountIT.java   |   2 +-
 .../org/apache/beam/examples/WordCountTest.java |   7 +-
 .../examples/complete/AutoCompleteTest.java |  13 +-
 .../beam/examples/complete/TfIdfTest.java   |   6 +-
 .../complete/TopWikipediaSessionsTest.java  |   7 +-
 .../examples/cookbook/DistinctExampleTest.java  |   9 +-
 .../examples/cookbook/JoinExamplesTest.java |   6 +-
 .../examples/cookbook/TriggerExampleTest.java   |   6 +-
 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/MinimalWordCountJava8Test.java |   6 +-
 .../examples/complete/game/GameStatsTest.java   |   7 +-
 .../complete/game/HourlyTeamScoreTest.java  |   5 +-
 .../examples/complete/game/LeaderBoardTest.java |  11 +-
 .../examples/complete/game/UserScoreTest.java   |  10 +-
 examples/pom.xml|   2 +-
 pom.xml |  15 +-
 runners/apex/pom.xml|  69 ++-
 .../apache/beam/runners/apex/ApexRunner.java| 111 ++--
 .../beam/runners/apex/ApexRunnerResult.java |  50 +-
 .../beam/runners/apex/ApexYarnLauncher.java | 395 
 .../translation/ApexPipelineTranslator.java |  14 +-
 .../translation/ParDoBoundMultiTranslator.java  |  22 +-
 .../apex/translation/ParDoBoundTranslator.java  |  22 +-
 .../apex/translation/TranslationContext.java|  10 +-
 .../apex/translation/WindowBoundTranslator.java |  78 +++
 .../operators/ApexGroupByKeyOperator.java   |  26 +-
 .../operators/ApexParDoOperator.java|  29 +-
 .../apex/translation/utils/NoOpStepContext.java |   3 +-
 .../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 +-
 

[41/51] [abbrv] incubator-beam git commit: Fix windowing in direct runner Stateful ParDo

2016-12-21 Thread kenn
Fix windowing in direct runner Stateful ParDo


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

Branch: refs/heads/python-sdk
Commit: 4fb16e8fb9bb087c0975f38c54665634868cfed7
Parents: 7ee8c86
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:58:29 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:22 2016 -0800

--
 .../direct/ParDoMultiOverrideFactory.java   | 34 ++--
 1 file changed, 31 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4fb16e8f/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index 2cea999..b35df87 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -34,8 +34,13 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
@@ -92,9 +97,12 @@ class ParDoMultiOverrideFactory
 @Override
 public PCollectionTuple expand(PCollection> input) {
 
+  WindowingStrategy inputWindowingStrategy = 
input.getWindowingStrategy();
+
   // A KvCoder is required since this goes through GBK. Further, 
WindowedValueCoder
   // is not registered by default, so we explicitly set the relevant 
coders.
-  checkState(input.getCoder() instanceof KvCoder,
+  checkState(
+  input.getCoder() instanceof KvCoder,
   "Input to a %s using state requires a %s, but the coder was %s",
   ParDo.class.getSimpleName(),
   KvCoder.class.getSimpleName(),
@@ -102,14 +110,27 @@ class ParDoMultiOverrideFactory
   KvCoder kvCoder = (KvCoder) input.getCoder();
   Coder keyCoder = kvCoder.getKeyCoder();
   Coder windowCoder =
-  input.getWindowingStrategy().getWindowFn().windowCoder();
+  inputWindowingStrategy.getWindowFn().windowCoder();
 
-  PCollectionTuple outputs =
+  PCollection>> adjustedInput =
   input
   // Stash the original timestamps, etc, for when it is fed to the 
user's DoFn
   .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn()))
   .setCoder(KvCoder.of(keyCoder, 
WindowedValue.getFullCoder(kvCoder, windowCoder)))
 
+  // We are going to GBK to gather keys and windows but otherwise 
do not want
+  // to alter the flow of data. This entails:
+  //  - trigger as fast as possible
+  //  - maintain the full timestamps of elements
+  //  - ensure this GBK holds to the minimum of those timestamps 
(via OutputTimeFn)
+  //  - discard past panes as it is "just a stream" of elements
+  .apply(
+  Window.>>create())
 
@@ -117,6 +138,13 @@ class ParDoMultiOverrideFactory
   .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem()))
   

[44/51] [abbrv] incubator-beam git commit: Add static Window.withOutputTimeFn to match build method

2016-12-21 Thread kenn
Add static Window.withOutputTimeFn to match build method


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

Branch: refs/heads/python-sdk
Commit: 8188040d930b1fa49efd4ed7d5f821d05d6f28ef
Parents: fa4958a
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:57:55 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../org/apache/beam/sdk/transforms/windowing/Window.java| 9 +
 1 file changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8188040d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index 0c430d0..1241abe 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -223,6 +223,15 @@ public class Window {
   }
 
   /**
+   * (Experimental) Override the default {@link OutputTimeFn}, 
to control
+   * the output timestamp of values output from a {@link GroupByKey} operation.
+   */
+  @Experimental(Kind.OUTPUT_TIME)
+  public static  Bound withOutputTimeFn(OutputTimeFn outputTimeFn) {
+return new Bound(null).withOutputTimeFn(outputTimeFn);
+  }
+
+  /**
* A {@code PTransform} that windows the elements of a {@code 
PCollection},
* into finite windows according to a user-specified {@code WindowFn}.
*



[45/51] [abbrv] incubator-beam git commit: Add UsesTestStream for use with JUnit @Category

2016-12-21 Thread kenn
Add UsesTestStream for use with JUnit @Category


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

Branch: refs/heads/python-sdk
Commit: 4d71924ccda9dae97c7cc9535a9780df9457cc3f
Parents: 8188040
Author: Kenneth Knowles 
Authored: Tue Dec 20 14:20:07 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../apache/beam/sdk/testing/UsesTestStream.java | 24 
 .../apache/beam/sdk/testing/TestStreamTest.java | 12 +-
 2 files changed, 30 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d71924c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java
new file mode 100644
index 000..8debb46
--- /dev/null
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.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 tests that use {@link TestStream}, which is not a part of 
the Beam model
+ * but a special feature currently only implemented by the direct runner.
+ */
+public interface UsesTestStream {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d71924c/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index 64aeca3..c12e9f3 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -69,7 +69,7 @@ public class TestStreamTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testLateDataAccumulating() {
 Instant instant = new Instant(0);
 TestStream source = TestStream.create(VarIntCoder.of())
@@ -136,7 +136,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testProcessingTimeTrigger() {
 TestStream source = TestStream.create(VarLongCoder.of())
 .addElements(TimestampedValue.of(1L, new Instant(1000L)),
@@ -159,7 +159,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testDiscardingMode() {
 TestStream stream =
 TestStream.create(StringUtf8Coder.of())
@@ -208,7 +208,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testFirstElementLate() {
 Instant lateElementTimestamp = new Instant(-1_000_000);
 TestStream stream =
@@ -238,7 +238,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testElementsAtAlmostPositiveInfinity() {
 Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp();
 TestStream stream = TestStream.create(StringUtf8Coder.of())
@@ -261,7 +261,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  

[36/51] [abbrv] incubator-beam git commit: This closes #1673: Require TimeDomain to delete a timer

2016-12-21 Thread kenn
This closes #1673: Require TimeDomain to delete a timer


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

Branch: refs/heads/python-sdk
Commit: 4843dc59c6e87ea0be75f7abd1e312bf5bc5a529
Parents: 0d0a5e2 35a0274
Author: Kenneth Knowles 
Authored: Wed Dec 21 10:15:56 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 10:15:56 2016 -0800

--
 .../operators/ApexGroupByKeyOperator.java  |  8 
 .../beam/runners/core/InMemoryTimerInternals.java  |  8 
 .../beam/runners/direct/DirectTimerInternals.java  |  8 
 .../wrappers/streaming/WindowDoFnOperator.java |  9 +
 .../org/apache/beam/sdk/util/TimerInternals.java   | 17 +++--
 5 files changed, 48 insertions(+), 2 deletions(-)
--




[51/51] [abbrv] incubator-beam git commit: This closes #1676: Merge master (57d9bbd) into python-sdk

2016-12-21 Thread kenn
This closes #1676: Merge master (57d9bbd) into python-sdk

No fixups. Commit from master:

  Hold output watermark according to pending timers
  Allow setting timer by ID in DirectTimerInternals
  Add UsesTestStream for use with JUnit @Category
  Add static Window.withOutputTimeFn to match build method
  Use informative Instant formatter in WatermarkHold
  Add informative Instant formatter to BoundedWindow
  Actually propagate and commit state in direct runner
  Fix windowing in direct runner Stateful ParDo
  Support set and delete of timer by ID in InMemoryTimerInternals
  Move ExecutionContext and related classes to runners-core
  Update Dataflow worker to beam-master-20161221
  Require TimeDomain to delete a timer
  Provide local tags in PInput, POutput expansions
  Remove deprecated InMemoryTimerInternals from SDK
  Remove deprecated AggregatorFactory from SDK
  Update Dataflow worker to beam-master-20161220
  Fixup usage of canonical name with name since canonical name != name for 
inner classes.
  Provide a better error message for non-existing gcpTempLocation
  Restore SDK's InMemoryTimerInternals, deprecated
  Move InMemoryTimerInternals to runners-core
  Port direct runner StatefulParDo to KeyedWorkItem
  Propagate key through ParDo if DoFn is key-preserving
  Move responsibility for knowing about keyedness into EvaluationContext
  Add some key-preserving to KeyedPValueTrackingVisitor
  Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit 
rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.
  Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kafka module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline 
as a JUnit rule.
  Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a 
JUnit rule.
  Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as 
a JUnit rule.
  Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. 
Plus, fixed some checkstyle errors from previous modules' migration.
  Migrated the beam-runners-direct-java module to TestPipeline as a JUnit rule.
  Migrated the beam-runners-core module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java8 module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java module to TestPipeline as a JUnit rule.
  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 

[38/51] [abbrv] incubator-beam git commit: Support set and delete of timer by ID in InMemoryTimerInternals

2016-12-21 Thread kenn
Support set and delete of timer by ID in InMemoryTimerInternals


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

Branch: refs/heads/python-sdk
Commit: df2e540d7a7b8444b9ff3b404740d5a3394b7691
Parents: acd2196
Author: Kenneth Knowles 
Authored: Mon Dec 19 14:01:36 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 11:01:36 2016 -0800

--
 .../runners/core/InMemoryTimerInternals.java|  65 +++
 .../core/InMemoryTimerInternalsTest.java| 112 +--
 2 files changed, 120 insertions(+), 57 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/df2e540d/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
index 5fcd088..292ac23 100644
--- 
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
@@ -17,13 +17,15 @@
  */
 package org.apache.beam.runners.core;
 
+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.common.base.MoreObjects;
-import java.util.HashSet;
-import java.util.PriorityQueue;
-import java.util.Set;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Table;
+import java.util.NavigableSet;
+import java.util.TreeSet;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.TimeDomain;
@@ -35,17 +37,17 @@ import org.joda.time.Instant;
 /** {@link TimerInternals} with all watermarks and processing clock simulated 
in-memory. */
 public class InMemoryTimerInternals implements TimerInternals {
 
-  /** At most one timer per timestamp is kept. */
-  private Set existingTimers = new HashSet<>();
+  /** The current set timers by namespace and ID. */
+  Table existingTimers = 
HashBasedTable.create();
 
   /** Pending input watermark timers, in timestamp order. */
-  private PriorityQueue watermarkTimers = new PriorityQueue<>(11);
+  private NavigableSet watermarkTimers = new TreeSet<>();
 
   /** Pending processing time timers, in timestamp order. */
-  private PriorityQueue processingTimers = new PriorityQueue<>(11);
+  private NavigableSet processingTimers = new TreeSet<>();
 
   /** Pending synchronized processing time timers, in timestamp order. */
-  private PriorityQueue synchronizedProcessingTimers = new 
PriorityQueue<>(11);
+  private NavigableSet synchronizedProcessingTimers = new 
TreeSet<>();
 
   /** Current input watermark. */
   private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
@@ -74,13 +76,13 @@ public class InMemoryTimerInternals implements 
TimerInternals {
 final TimerData data;
 switch (domain) {
   case EVENT_TIME:
-data = watermarkTimers.peek();
+data = watermarkTimers.first();
 break;
   case PROCESSING_TIME:
-data = processingTimers.peek();
+data = processingTimers.first();
 break;
   case SYNCHRONIZED_PROCESSING_TIME:
-data = synchronizedProcessingTimers.peek();
+data = synchronizedProcessingTimers.first();
 break;
   default:
 throw new IllegalArgumentException("Unexpected time domain: " + 
domain);
@@ -88,7 +90,7 @@ public class InMemoryTimerInternals implements TimerInternals 
{
 return (data == null) ? null : data.getTimestamp();
   }
 
-  private PriorityQueue queue(TimeDomain domain) {
+  private NavigableSet timersForDomain(TimeDomain domain) {
 switch (domain) {
   case EVENT_TIME:
 return watermarkTimers;
@@ -104,27 +106,45 @@ public class InMemoryTimerInternals implements 
TimerInternals {
   @Override
   public void setTimer(StateNamespace namespace, String timerId, Instant 
target,
   TimeDomain timeDomain) {
-throw new UnsupportedOperationException("Setting a timer by ID is not yet 
supported.");
+setTimer(TimerData.of(timerId, namespace, target, timeDomain));
   }
 
   @Override
   public void setTimer(TimerData timerData) {
 WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), 
timerData);
- 

[35/51] [abbrv] incubator-beam git commit: Update Dataflow worker to beam-master-20161221

2016-12-21 Thread kenn
Update Dataflow worker to beam-master-20161221


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

Branch: refs/heads/python-sdk
Commit: 64336e40dd6a48b3b6b48634bb9204db0aa0c7ca
Parents: 0d0a5e2
Author: Kenneth Knowles 
Authored: Wed Dec 21 10:09:49 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 10:10:00 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/64336e40/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
 
b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
index bf08e83..2912f61 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
+++ 
b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
@@ -18,6 +18,6 @@
 
 environment.major.version=6
 
-worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161220
+worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161221
 
-worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161220
+worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161221



[47/51] [abbrv] incubator-beam git commit: Hold output watermark according to pending timers

2016-12-21 Thread kenn
Hold output watermark according to pending timers


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

Branch: refs/heads/python-sdk
Commit: dfe2e62d103595583e3ca4594cc03885fe1bba16
Parents: 7f14c46
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:37:40 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../beam/runners/direct/WatermarkManager.java   | 59 
 1 file changed, 48 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dfe2e62d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index f7bafd1..248fafd 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -211,12 +211,18 @@ public class WatermarkManager {
   private static class AppliedPTransformInputWatermark implements Watermark {
 private final Collection inputWatermarks;
 private final SortedMultiset pendingElements;
-private final Map objectTimers;
+
+// This tracks only the quantity of timers at each timestamp, for quickly 
getting the cross-key
+// minimum
+private final SortedMultiset pendingTimers;
 
 // Entries in this table represent the authoritative timestamp for which
 // a per-key-and-StateNamespace timer is set.
 private final Map> existingTimers;
 
+// This per-key sorted set allows quick retrieval of timers that should 
fire for a key
+private final Map objectTimers;
+
 private AtomicReference currentWatermark;
 
 public AppliedPTransformInputWatermark(Collection 
inputWatermarks) {
@@ -224,10 +230,13 @@ public class WatermarkManager {
   // The ordering must order elements by timestamp, and must not compare 
two distinct elements
   // as equal. This is built on the assumption that any element added as a 
pending element will
   // be consumed without modifications.
+  //
+  // The same logic is applied for pending timers
   Ordering pendingBundleComparator =
   new 
BundleByElementTimestampComparator().compound(Ordering.arbitrary());
   this.pendingElements =
   TreeMultiset.create(pendingBundleComparator);
+  this.pendingTimers = TreeMultiset.create();
   this.objectTimers = new HashMap<>();
   this.existingTimers = new HashMap<>();
   currentWatermark = new 
AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
@@ -278,6 +287,14 @@ public class WatermarkManager {
   pendingElements.remove(completed);
 }
 
+private synchronized Instant getEarliestTimerTimestamp() {
+  if (pendingTimers.isEmpty()) {
+return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  } else {
+return pendingTimers.firstEntry().getElement();
+  }
+}
+
 private synchronized void updateTimers(TimerUpdate update) {
   NavigableSet keyTimers = objectTimers.get(update.key);
   if (keyTimers == null) {
@@ -291,27 +308,43 @@ public class WatermarkManager {
 existingTimers.put(update.key, existingTimersForKey);
   }
 
-  for (TimerData timer : update.setTimers) {
+  for (TimerData timer : update.getSetTimers()) {
+if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+  @Nullable
+  TimerData existingTimer =
+  existingTimersForKey.get(timer.getNamespace(), 
timer.getTimerId());
+
+  if (existingTimer == null) {
+pendingTimers.add(timer.getTimestamp());
+keyTimers.add(timer);
+  } else if (!existingTimer.equals(timer)) {
+keyTimers.remove(existingTimer);
+keyTimers.add(timer);
+  } // else the timer is already set identically, so noop
+
+  existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), 
timer);
+}
+  }
+
+  for (TimerData timer : update.getDeletedTimers()) {
 if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
   @Nullable
   TimerData existingTimer =
   existingTimersForKey.get(timer.getNamespace(), 
timer.getTimerId());
 

[43/51] [abbrv] incubator-beam git commit: Add informative Instant formatter to BoundedWindow

2016-12-21 Thread kenn
Add informative Instant formatter to BoundedWindow


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

Branch: refs/heads/python-sdk
Commit: 280a6a8f729cb382616ad65f71860b61277cbd6f
Parents: ff39516
Author: Kenneth Knowles 
Authored: Mon Dec 19 20:40:11 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:36 2016 -0800

--
 .../beam/sdk/transforms/windowing/BoundedWindow.java| 12 
 1 file changed, 12 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/280a6a8f/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
index 3654074..6da2495 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
@@ -39,6 +39,18 @@ public abstract class BoundedWindow {
   public static final Instant TIMESTAMP_MAX_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
 
+  public static String formatTimestamp(Instant timestamp) {
+if (timestamp.equals(TIMESTAMP_MIN_VALUE)) {
+  return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)";
+} else if (timestamp.equals(TIMESTAMP_MAX_VALUE)) {
+  return timestamp.toString() + " (TIMESTAMP_MAX_VALUE)";
+} else if (timestamp.equals(GlobalWindow.INSTANCE.maxTimestamp())) {
+  return timestamp.toString() + " (end of global window)";
+} else {
+  return timestamp.toString();
+}
+  }
+
   /**
* Returns the inclusive upper bound of timestamps for values in this window.
*/



[28/51] [abbrv] incubator-beam git commit: This closes #1665: Remove deprecated AggregatorFactory from SDK

2016-12-21 Thread kenn
This closes #1665: Remove deprecated AggregatorFactory from SDK

  Update Dataflow worker to beam-master-20161220
  Remove deprecated AggregatorFactory from SDK


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

Branch: refs/heads/python-sdk
Commit: 6a05d7f17aab5cab202cdbf50b766b4fc86180b4
Parents: acd2196 aab46a0
Author: Kenneth Knowles 
Authored: Tue Dec 20 15:07:12 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 15:07:12 2016 -0800

--
 .../beam/runners/dataflow/dataflow.properties|  4 ++--
 .../apache/beam/sdk/transforms/Aggregator.java   | 19 ---
 2 files changed, 2 insertions(+), 21 deletions(-)
--




[37/51] [abbrv] incubator-beam git commit: This closes #1666: Move ExecutionContext and related classes to runners-core

2016-12-21 Thread kenn
This closes #1666: Move ExecutionContext and related classes to runners-core

  Move ExecutionContext and related classes to runners-core
  Update Dataflow worker to beam-master-20161221


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

Branch: refs/heads/python-sdk
Commit: a9447a2251f46496b7a773c8b07b3281dbc7a6fb
Parents: 4843dc5 9d2b8e0
Author: Kenneth Knowles 
Authored: Wed Dec 21 10:24:11 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 10:24:11 2016 -0800

--
 .../operators/ApexParDoOperator.java|   2 +-
 .../apex/translation/utils/NoOpStepContext.java |   3 +-
 .../beam/runners/core/AggregatorFactory.java|   1 -
 .../beam/runners/core/BaseExecutionContext.java | 176 +++
 .../apache/beam/runners/core/DoFnRunners.java   |   2 +-
 .../beam/runners/core/ExecutionContext.java | 102 +++
 .../beam/runners/core/SimpleDoFnRunner.java |   2 +-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   2 +-
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   2 +-
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   3 +-
 .../runners/direct/AggregatorContainer.java |   2 +-
 .../runners/direct/DirectExecutionContext.java  |   6 +-
 .../beam/runners/direct/EvaluationContext.java  |   2 +-
 .../runners/direct/AggregatorContainerTest.java |   2 +-
 .../wrappers/streaming/DoFnOperator.java|   2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   2 +-
 .../beam/runners/dataflow/dataflow.properties   |   4 +-
 .../spark/aggregators/SparkAggregators.java |   2 +-
 .../spark/translation/SparkProcessContext.java  |   2 +-
 .../beam/sdk/util/BaseExecutionContext.java | 174 --
 .../apache/beam/sdk/util/ExecutionContext.java  | 100 ---
 21 files changed, 297 insertions(+), 296 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a9447a22/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
--



[48/51] [abbrv] incubator-beam git commit: Use informative Instant formatter in WatermarkHold

2016-12-21 Thread kenn
Use informative Instant formatter in WatermarkHold


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

Branch: refs/heads/python-sdk
Commit: fa4958a6140eb00ceee08b2468f7d88f17538794
Parents: 280a6a8
Author: Kenneth Knowles 
Authored: Mon Dec 19 20:40:47 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../apache/beam/runners/core/WatermarkHold.java  |  4 +++-
 .../sdk/transforms/windowing/BoundedWindow.java  | 19 +++
 2 files changed, 22 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fa4958a6/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
index 7f1afcc..5e5f44d 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
@@ -207,7 +207,9 @@ class WatermarkHold implements 
Serializable {
 Instant shifted = 
windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window);
 checkState(!shifted.isBefore(timestamp),
 "OutputTimeFn moved element from %s to earlier time %s for window %s",
-timestamp, shifted, window);
+BoundedWindow.formatTimestamp(timestamp),
+BoundedWindow.formatTimestamp(shifted),
+window);
 checkState(timestamp.isAfter(window.maxTimestamp())
 || !shifted.isAfter(window.maxTimestamp()),
 "OutputTimeFn moved element from %s to %s which is beyond end of "

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fa4958a6/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
index 6da2495..74223b5 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
@@ -34,11 +34,30 @@ import org.joda.time.Instant;
 public abstract class BoundedWindow {
   // The min and max timestamps that won't overflow when they are converted to
   // usec.
+
+  /**
+   * The minimum value for any Beam timestamp. Often referred to as 
"-infinity".
+   *
+   * This value and {@link #TIMESTAMP_MAX_VALUE} are chosen so that their
+   * microseconds-since-epoch can be safely represented with a {@code long}.
+   */
   public static final Instant TIMESTAMP_MIN_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
+
+  /**
+   * The maximum value for any Beam timestamp. Often referred to as 
"+infinity".
+   *
+   * This value and {@link #TIMESTAMP_MIN_VALUE} are chosen so that their
+   * microseconds-since-epoch can be safely represented with a {@code long}.
+   */
   public static final Instant TIMESTAMP_MAX_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
 
+  /**
+   * Formats a {@link Instant} timestamp with additional Beam-specific 
metadata, such as indicating
+   * whether the timestamp is the end of the global window or one of the 
distinguished values {@link
+   * #TIMESTAMP_MIN_VALUE} or {@link #TIMESTAMP_MIN_VALUE}.
+   */
   public static String formatTimestamp(Instant timestamp) {
 if (timestamp.equals(TIMESTAMP_MIN_VALUE)) {
   return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)";



[42/51] [abbrv] incubator-beam git commit: This closes #1670: Fixes for direct runner expansion and evaluation of stateful ParDo

2016-12-21 Thread kenn
This closes #1670: Fixes for direct runner expansion and evaluation of stateful 
ParDo

  Actually propagate and commit state in direct runner
  Fix windowing in direct runner Stateful ParDo


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

Branch: refs/heads/python-sdk
Commit: ff395169993d84cd920be21f11d9af8f8d8b
Parents: 7ee8c86 55176c3
Author: Kenneth Knowles 
Authored: Wed Dec 21 13:11:54 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:54 2016 -0800

--
 .../direct/ParDoMultiOverrideFactory.java   | 34 ++--
 .../direct/StatefulParDoEvaluatorFactory.java   |  1 +
 2 files changed, 32 insertions(+), 3 deletions(-)
--




[40/51] [abbrv] incubator-beam git commit: Actually propagate and commit state in direct runner

2016-12-21 Thread kenn
Actually propagate and commit state in direct 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/55176c38
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/55176c38
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/55176c38

Branch: refs/heads/python-sdk
Commit: 55176c385cc802be42b5467fbb2dcc9a1c7467ea
Parents: 4fb16e8
Author: Kenneth Knowles 
Authored: Tue Dec 20 15:59:45 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:22 2016 -0800

--
 .../apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java   | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/55176c38/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
index 5f9d8f4..003df0f 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
@@ -233,6 +233,7 @@ final class StatefulParDoEvaluatorFactory implements Transfo
   StepTransformResult.>>withHold(
   delegateResult.getTransform(), 
delegateResult.getWatermarkHold())
   .withTimerUpdate(delegateResult.getTimerUpdate())
+  .withState(delegateResult.getState())
   .withAggregatorChanges(delegateResult.getAggregatorChanges())
   .withMetricUpdates(delegateResult.getLogicalMetricUpdates())
   
.addOutput(Lists.newArrayList(delegateResult.getOutputBundles()));



[10/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as a 
JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 24ad18319248a128a1c5db4f2bef8861f7361d9f
Parents: 75a4c91
Author: Stas Levin 
Authored: Tue Dec 20 00:01:31 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../beam/sdk/extensions/joinlibrary/InnerJoinTest.java|  9 ++---
 .../sdk/extensions/joinlibrary/OuterLeftJoinTest.java | 10 +++---
 .../sdk/extensions/joinlibrary/OuterRightJoinTest.java| 10 +++---
 3 files changed, 20 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/24ad1831/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java
--
diff --git 
a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java
 
b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java
index 423ab9c..1c120c2 100644
--- 
a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java
+++ 
b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java
@@ -19,13 +19,13 @@ package org.apache.beam.sdk.extensions.joinlibrary;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 /**
@@ -33,15 +33,16 @@ import org.junit.Test;
  */
 public class InnerJoinTest {
 
-  Pipeline p;
   List> leftListOfKv;
   List> listRightOfKv;
   List>> expectedResult;
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Before
   public void setup() {
 
-p = TestPipeline.create();
 leftListOfKv = new ArrayList<>();
 listRightOfKv = new ArrayList<>();
 
@@ -131,11 +132,13 @@ public class InnerJoinTest {
 
   @Test(expected = NullPointerException.class)
   public void testJoinLeftCollectionNull() {
+p.enableAbandonedNodeEnforcement(false);
 Join.innerJoin(null, p.apply(Create.of(listRightOfKv)));
   }
 
   @Test(expected = NullPointerException.class)
   public void testJoinRightCollectionNull() {
+p.enableAbandonedNodeEnforcement(false);
 Join.innerJoin(p.apply(Create.of(leftListOfKv)), null);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/24ad1831/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java
--
diff --git 
a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java
 
b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java
index c32163f..81f4fa3 100644
--- 
a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java
+++ 
b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java
@@ -19,13 +19,13 @@ package org.apache.beam.sdk.extensions.joinlibrary;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 
@@ -34,15 +34,16 @@ import org.junit.Test;
  */
 public class OuterLeftJoinTest {
 
-  Pipeline p;
   List> leftListOfKv;
   List> listRightOfKv;
   List>> expectedResult;
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Before
   public void setup() {
 
-p = TestPipeline.create();
 leftListOfKv = new ArrayList<>();
 listRightOfKv = new ArrayList<>();
 
@@ -133,16 +134,19 @@ public class OuterLeftJoinTest 

[09/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a JUnit 
rule.


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

Branch: refs/heads/python-sdk
Commit: 63331aa8aa6314e8469c23a4f4a89fbf287cbc5a
Parents: 24ad183
Author: Stas Levin 
Authored: Tue Dec 20 09:54:57 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../org/apache/beam/sdk/extensions/sorter/SortValuesTest.java | 7 ---
 1 file changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/63331aa8/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
--
diff --git 
a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
 
b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
index ebfbd0e..4f77100 100644
--- 
a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
+++ 
b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java
@@ -24,7 +24,6 @@ import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 
 import java.util.Arrays;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
@@ -35,6 +34,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.TypeSafeMatcher;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -43,10 +43,11 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class SortValuesTest {
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   public void testSecondaryKeySorting() throws Exception {
-Pipeline p = TestPipeline.create();
-
 // Create a PCollection of > pairs.
 PCollection>> input =
 p.apply(



[02/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. Plus, fixed some checkstyle errors from previous modules' migration.

2016-12-21 Thread kenn
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
index 2a89a18..3bc0a65 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
@@ -38,7 +38,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Random;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
@@ -76,12 +75,12 @@ import org.junit.runners.JUnit4;
 @SuppressWarnings("unchecked")
 public class CreateTest {
   @Rule public final ExpectedException thrown = ExpectedException.none();
+  @Rule public final TestPipeline p = TestPipeline.create();
+
 
   @Test
   @Category(RunnableOnService.class)
   public void testCreate() {
-Pipeline p = TestPipeline.create();
-
 PCollection output =
 p.apply(Create.of(LINES));
 
@@ -93,8 +92,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateEmpty() {
-Pipeline p = TestPipeline.create();
-
 PCollection output =
 p.apply(Create.of(NO_LINES)
 .withCoder(StringUtf8Coder.of()));
@@ -106,7 +103,7 @@ public class CreateTest {
 
   @Test
   public void testCreateEmptyInfersCoder() {
-Pipeline p = TestPipeline.create();
+p.enableAbandonedNodeEnforcement(false);
 
 PCollection output =
 p.apply(Create.of());
@@ -126,8 +123,6 @@ public class CreateTest {
 thrown.expectMessage(
 Matchers.containsString("Unable to infer a coder"));
 
-Pipeline p = TestPipeline.create();
-
 // Create won't infer a default coder in this case.
 p.apply(Create.of(new Record(), new Record2()));
 
@@ -137,8 +132,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateWithNullsAndValues() throws Exception {
-Pipeline p = TestPipeline.create();
-
 PCollection output =
 p.apply(Create.of(null, "test1", null, "test2", null)
 .withCoder(SerializableCoder.of(String.class)));
@@ -150,8 +143,6 @@ public class CreateTest {
   @Test
   @Category(NeedsRunner.class)
   public void testCreateParameterizedType() throws Exception {
-Pipeline p = TestPipeline.create();
-
 PCollection output =
 p.apply(Create.of(
 TimestampedValue.of("a", new Instant(0)),
@@ -216,7 +207,6 @@ public class CreateTest {
 Create.Values create =
 Create.of(elements).withCoder(new 
UnserializableRecord.UnserializableRecordCoder());
 
-TestPipeline p = TestPipeline.create();
 PAssert.that(p.apply(create))
 .containsInAnyOrder(
 new UnserializableRecord("foo"),
@@ -235,8 +225,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateTimestamped() {
-Pipeline p = TestPipeline.create();
-
 List data = Arrays.asList(
 TimestampedValue.of("a", new Instant(1L)),
 TimestampedValue.of("b", new Instant(2L)),
@@ -254,8 +242,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateTimestampedEmpty() {
-Pipeline p = TestPipeline.create();
-
 PCollection output = p
 .apply(Create.timestamped(new ArrayList())
 .withCoder(StringUtf8Coder.of()));
@@ -266,7 +252,7 @@ public class CreateTest {
 
   @Test
   public void testCreateTimestampedEmptyInfersCoder() {
-Pipeline p = TestPipeline.create();
+p.enableAbandonedNodeEnforcement(false);
 
 PCollection output = p
 .apply(Create.timestamped());
@@ -280,8 +266,6 @@ public class CreateTest {
 thrown.expectMessage(
 Matchers.containsString("Unable to infer a coder"));
 
-Pipeline p = TestPipeline.create();
-
 // Create won't infer a default coder in this case.
 PCollection c = p.apply(Create.timestamped(
 TimestampedValue.of(new Record(), new Instant(0)),
@@ -295,7 +279,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateWithVoidType() throws Exception {
-Pipeline p = TestPipeline.create();
 PCollection output = p.apply(Create.of((Void) null, (Void) null));
 PAssert.that(output).containsInAnyOrder((Void) null, (Void) null);
 p.run();
@@ -304,8 +287,6 @@ public class CreateTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCreateWithKVVoidType() throws Exception {
-Pipeline p = TestPipeline.create();
-
 PCollection

[08/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline as 
a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 6dea0992d9976b39232cf846906831feaa25ec43
Parents: 63331aa
Author: Stas Levin 
Authored: Tue Dec 20 13:26:07 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 25 +---
 .../sdk/io/gcp/bigtable/BigtableIOTest.java | 13 ++
 2 files changed, 25 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6dea0992/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 dc566d2..4ddfdea 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
@@ -643,6 +643,7 @@ public class BigQueryIOTest implements Serializable {
 }
   }
 
+  @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
   @Rule public transient ExpectedLogs logged = 
ExpectedLogs.none(BigQueryIO.class);
   @Rule public transient TemporaryFolder testFolder = new TemporaryFolder();
@@ -1370,7 +1371,7 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testBuildWriteWithoutTable() {
-Pipeline p = TestPipeline.create();
+
 thrown.expect(IllegalStateException.class);
 thrown.expectMessage("must set the table reference");
 p.apply(Create.of().withCoder(TableRowJsonCoder.of()))
@@ -1591,9 +1592,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsCreateNoSchema() {
+p.enableAbandonedNodeEnforcement(false);
+
 thrown.expect(IllegalArgumentException.class);
 thrown.expectMessage("no schema was provided");
-TestPipeline.create()
+p
 .apply(Create.of())
 .apply(BigQueryIO.Write
 .to("dataset.table")
@@ -1602,9 +1605,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsTableAndTableSpec() {
+p.enableAbandonedNodeEnforcement(false);
+
 thrown.expect(IllegalStateException.class);
 thrown.expectMessage("Cannot set both a table reference and a table 
function");
-TestPipeline.create()
+p
 .apply(Create.of())
 .apply(BigQueryIO.Write
 .to("dataset.table")
@@ -1618,9 +1623,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsNoTableAndNoTableSpec() {
+p.enableAbandonedNodeEnforcement(false);
+
 thrown.expect(IllegalStateException.class);
 thrown.expectMessage("must set the table reference of a BigQueryIO.Write 
transform");
-TestPipeline.create()
+p
 .apply(Create.of())
 .apply("name", BigQueryIO.Write.withoutValidation());
   }
@@ -1950,7 +1957,6 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPassThroughThenCleanup() throws Exception {
-Pipeline p = TestPipeline.create();
 
 PCollection output = p
 .apply(Create.of(1, 2, 3))
@@ -1968,7 +1974,6 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPassThroughThenCleanupExecuted() throws Exception {
-Pipeline p = TestPipeline.create();
 
 p.apply(Create.of())
 .apply(new PassThroughThenCleanup(new CleanupOperation() {
@@ -2025,6 +2030,8 @@ public class BigQueryIOTest implements Serializable {
 
   private void testWritePartition(long numFiles, long fileSize, long 
expectedNumPartitions)
   throws Exception {
+p.enableAbandonedNodeEnforcement(false);
+
 List expectedPartitionIds = Lists.newArrayList();
 for (long i = 1; i <= expectedNumPartitions; ++i) {
   expectedPartitionIds.add(i);
@@ -2044,7 +2051,7 @@ public class BigQueryIOTest implements Serializable {
 new TupleTag>("singlePartitionTag") 

[34/51] [abbrv] incubator-beam git commit: Move ExecutionContext and related classes to runners-core

2016-12-21 Thread kenn
Move ExecutionContext and related classes to runners-core


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

Branch: refs/heads/python-sdk
Commit: 9d2b8e09bcb5e04017b487e1a919d335875dbfc0
Parents: 64336e4
Author: Kenneth Knowles 
Authored: Thu Dec 15 20:20:34 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 10:10:00 2016 -0800

--
 .../operators/ApexParDoOperator.java|   2 +-
 .../apex/translation/utils/NoOpStepContext.java |   3 +-
 .../beam/runners/core/AggregatorFactory.java|   1 -
 .../beam/runners/core/BaseExecutionContext.java | 176 +++
 .../apache/beam/runners/core/DoFnRunners.java   |   2 +-
 .../beam/runners/core/ExecutionContext.java | 102 +++
 .../beam/runners/core/SimpleDoFnRunner.java |   2 +-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   2 +-
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   2 +-
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   3 +-
 .../runners/direct/AggregatorContainer.java |   2 +-
 .../runners/direct/DirectExecutionContext.java  |   6 +-
 .../beam/runners/direct/EvaluationContext.java  |   2 +-
 .../runners/direct/AggregatorContainerTest.java |   2 +-
 .../wrappers/streaming/DoFnOperator.java|   2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   2 +-
 .../spark/aggregators/SparkAggregators.java |   2 +-
 .../spark/translation/SparkProcessContext.java  |   2 +-
 .../beam/sdk/util/BaseExecutionContext.java | 174 --
 .../apache/beam/sdk/util/ExecutionContext.java  | 100 ---
 20 files changed, 295 insertions(+), 294 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/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 a3d3a97..c41cd45 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
@@ -41,6 +41,7 @@ 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.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import org.apache.beam.sdk.coders.Coder;
@@ -50,7 +51,6 @@ 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;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.UserCodeException;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java
--
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java
index 078f95f..f169ae6 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java
@@ -19,10 +19,9 @@ package org.apache.beam.runners.apex.translation.utils;
 
 import java.io.IOException;
 import java.io.Serializable;
-
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.StateInternals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java

[27/51] [abbrv] incubator-beam git commit: Update Dataflow worker to beam-master-20161220

2016-12-21 Thread kenn
Update Dataflow worker to beam-master-20161220


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

Branch: refs/heads/python-sdk
Commit: a3f68d343d018d99a04ac5c9134a11f1bc74935b
Parents: 2f4b803
Author: Kenneth Knowles 
Authored: Tue Dec 20 14:05:19 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 14:05:35 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a3f68d34/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
 
b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
index 27a518f..bf08e83 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
+++ 
b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties
@@ -18,6 +18,6 @@
 
 environment.major.version=6
 
-worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161216
+worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161220
 
-worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161216
+worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161220



[14/51] [abbrv] incubator-beam git commit: This closes #1664: [BEAM-1176] Migrating tests to use TestPipeline as a JUnit rule

2016-12-21 Thread kenn
This closes #1664: [BEAM-1176] Migrating tests to use TestPipeline as a JUnit 
rule

  Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit 
rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.
  Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kafka module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline 
as a JUnit rule.
  Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a 
JUnit rule.
  Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as 
a JUnit rule.
  Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. 
Plus, fixed some checkstyle errors from previous modules' migration.
  Migrated the beam-runners-direct-java module to TestPipeline as a JUnit rule.
  Migrated the beam-runners-core module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java8 module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 22e25a47e2edc7b7f702eaca12630f6de7195657
Parents: b3de17b 4b23d42
Author: Kenneth Knowles 
Authored: Tue Dec 20 09:55:57 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:57 2016 -0800

--
 .../org/apache/beam/examples/WordCountTest.java |   7 +-
 .../examples/complete/AutoCompleteTest.java |  11 +-
 .../beam/examples/complete/TfIdfTest.java   |   6 +-
 .../complete/TopWikipediaSessionsTest.java  |   7 +-
 .../examples/cookbook/DistinctExampleTest.java  |   9 +-
 .../examples/cookbook/JoinExamplesTest.java |   6 +-
 .../examples/cookbook/TriggerExampleTest.java   |   6 +-
 .../examples/MinimalWordCountJava8Test.java |   6 +-
 .../examples/complete/game/GameStatsTest.java   |   7 +-
 .../complete/game/HourlyTeamScoreTest.java  |   5 +-
 .../examples/complete/game/LeaderBoardTest.java |  11 +-
 .../examples/complete/game/UserScoreTest.java   |  10 +-
 .../core/PushbackSideInputDoFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  |  10 +-
 .../UnboundedReadFromBoundedSourceTest.java |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  13 +-
 .../direct/CloningBundleFactoryTest.java|   8 +-
 .../runners/direct/CommittedResultTest.java |   6 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |   7 +-
 .../runners/direct/DirectGraphVisitorTest.java  |   3 +-
 .../runners/direct/EvaluationContextTest.java   |   7 +-
 .../direct/FlattenEvaluatorFactoryTest.java |   6 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   5 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java |   5 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   4 +-
 .../ImmutabilityEnforcementFactoryTest.java |   3 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  14 ++-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   5 +-
 .../runners/direct/SideInputContainerTest.java  |   5 +-
 .../StatefulParDoEvaluatorFactoryTest.java  |   7 +-
 .../runners/direct/StepTransformResultTest.java |   5 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |   5 +-
 .../runners/direct/TransformExecutorTest.java   |   4 +-
 .../UnboundedReadEvaluatorFactoryTest.java  |   9 +-
 .../direct/ViewEvaluatorFactoryTest.java|   5 +-
 .../direct/WatermarkCallbackExecutorTest.java   |   5 +-
 .../runners/direct/WatermarkManagerTest.java|   6 +-
 .../direct/WindowEvaluatorFactoryTest.java  |   5 +-
 .../direct/WriteWithShardingFactoryTest.java|  14 +--
 .../java/org/apache/beam/sdk/PipelineTest.java  |  37 +++---
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  11 +-
 .../beam/sdk/coders/CoderRegistryTest.java  |   6 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   7 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  16 +--
 .../io/BoundedReadFromUnboundedSourceTest.java  |   6 +-
 .../beam/sdk/io/CompressedSourceTest.java   |  12 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  12 +-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  13 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |   4 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java|  10 +-
 

[39/51] [abbrv] incubator-beam git commit: This closes #1160: Support set and delete of timer by ID in InMemoryTimerInternals

2016-12-21 Thread kenn
This closes #1160: Support set and delete of timer by ID in 
InMemoryTimerInternals


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

Branch: refs/heads/python-sdk
Commit: 7ee8c86d3b0553d8cb7de60b0dc1a03103dfbbc5
Parents: a9447a2 df2e540
Author: Kenneth Knowles 
Authored: Wed Dec 21 11:02:02 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 11:02:02 2016 -0800

--
 .../runners/core/InMemoryTimerInternals.java|  65 +++
 .../core/InMemoryTimerInternalsTest.java| 112 +--
 2 files changed, 120 insertions(+), 57 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7ee8c86d/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
--
diff --cc 
runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
index 5ddd5a7,292ac23..2c3d78a
--- 
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
@@@ -104,10 -106,9 +106,10 @@@ public class InMemoryTimerInternals imp
@Override
public void setTimer(StateNamespace namespace, String timerId, Instant 
target,
TimeDomain timeDomain) {
- throw new UnsupportedOperationException("Setting a timer by ID is not yet 
supported.");
+ setTimer(TimerData.of(timerId, namespace, target, timeDomain));
}
  
 +  @Deprecated
@Override
public void setTimer(TimerData timerData) {
  WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), 
timerData);
@@@ -117,17 -133,13 +134,20 @@@
}
  
@Override
 +  public void deleteTimer(StateNamespace namespace, String timerId, 
TimeDomain timeDomain) {
 +throw new UnsupportedOperationException("Canceling a timer by ID is not 
yet supported.");
 +  }
 +
 +  @Deprecated
 +  @Override
public void deleteTimer(StateNamespace namespace, String timerId) {
- throw new UnsupportedOperationException("Canceling a timer by ID is not 
yet supported.");
+ TimerData existing = existingTimers.get(namespace, timerId);
+ if (existing != null) {
+   deleteTimer(existing);
+ }
}
  
 +  @Deprecated
@Override
public void deleteTimer(TimerData timer) {
  WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), 
timer);



[23/51] [abbrv] incubator-beam git commit: Fixup usage of canonical name with name since canonical name != name for inner classes.

2016-12-21 Thread kenn
Fixup usage of canonical name with name since canonical name != name for inner 
classes.


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

Branch: refs/heads/python-sdk
Commit: 96d393147c365d0911a091d0b3600fef621709f4
Parents: ef1a858
Author: Luke Cwik 
Authored: Tue Dec 20 11:47:42 2016 -0800
Committer: Luke Cwik 
Committed: Tue Dec 20 13:04:24 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96d39314/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
--
diff --git 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index b29c4cd..21d575a 100644
--- 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -267,7 +267,7 @@ public class DataflowRunnerTest {
 "--runner=DataflowRunner",
 "--tempLocation=/tmp/not/a/gs/path",
 "--project=test-project",
-"--credentialFactoryClass=" + 
NoopCredentialFactory.class.getCanonicalName(),
+"--credentialFactoryClass=" + NoopCredentialFactory.class.getName(),
 };
 
 try {
@@ -286,7 +286,7 @@ public class DataflowRunnerTest {
 "--runner=DataflowRunner",
 "--tempLocation=gs://does/not/exist",
 "--project=test-project",
-"--credentialFactoryClass=" + 
NoopCredentialFactory.class.getCanonicalName(),
+"--credentialFactoryClass=" + NoopCredentialFactory.class.getName(),
 };
 
 try {
@@ -306,8 +306,8 @@ public class DataflowRunnerTest {
 "--runner=DataflowRunner",
 "--tempLocation=/tmp/testing",
 "--project=test-project",
-"--credentialFactoryClass=" + 
NoopCredentialFactory.class.getCanonicalName(),
-"--pathValidatorClass=" + NoopPathValidator.class.getCanonicalName(),
+"--credentialFactoryClass=" + NoopCredentialFactory.class.getName(),
+"--pathValidatorClass=" + NoopPathValidator.class.getName(),
 };
 // Should not crash, because gcpTempLocation should get set from 
tempLocation
 TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create());



[05/51] [abbrv] incubator-beam git commit: Migrated the beam-runners-core module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-runners-core module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: b6710251d8bb5d1968aea2258ce5878b43368dd5
Parents: 7106e88
Author: Stas Levin 
Authored: Sun Dec 18 18:51:31 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../runners/core/PushbackSideInputDoFnRunnerTest.java |  5 -
 .../org/apache/beam/runners/core/SplittableParDoTest.java | 10 --
 .../runners/core/UnboundedReadFromBoundedSourceTest.java  |  5 +++--
 3 files changed, 15 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b6710251/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 a1cdbf6..251c7c2 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
@@ -47,6 +47,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -63,10 +64,12 @@ public class PushbackSideInputDoFnRunnerTest {
   private TestDoFnRunner underlying;
   private PCollectionView singletonView;
 
+  @Rule
+  public TestPipeline p = 
TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
 MockitoAnnotations.initMocks(this);
-TestPipeline p = TestPipeline.create();
 PCollection created = p.apply(Create.of(1, 2, 3));
 singletonView =
 created

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b6710251/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 cf96b66..0f0b106 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
@@ -57,6 +57,7 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -132,9 +133,13 @@ public class SplittableParDoTest {
 return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty());
   }
 
+  @Rule
+  public TestPipeline pipeline = TestPipeline.create();
+
   @Test
   public void testBoundednessForBoundedFn() {
-Pipeline pipeline = TestPipeline.create();
+pipeline.enableAbandonedNodeEnforcement(false);
+
 DoFn boundedFn = new BoundedFakeFn();
 assertEquals(
 "Applying a bounded SDF to a bounded collection produces a bounded 
collection",
@@ -154,7 +159,8 @@ public class SplittableParDoTest {
 
   @Test
   public void testBoundednessForUnboundedFn() {
-Pipeline pipeline = TestPipeline.create();
+pipeline.enableAbandonedNodeEnforcement(false);
+
 DoFn unboundedFn = new UnboundedFakeFn();
 assertEquals(
 "Applying an unbounded SDF to a bounded collection produces a bounded 
collection",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b6710251/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
index 7fd8807..86450f2 100644
--- 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
+++ 

[01/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. Plus, fixed some checkstyle errors from previous modules' migration.

2016-12-21 Thread kenn
Repository: incubator-beam
Updated Branches:
  refs/heads/python-sdk 3b4fd5c7d -> 3454d691f


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 3bf63fd..1d8b32c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -84,12 +84,14 @@ public class ViewTest implements Serializable {
   // anonymous inner classes inside the non-static test methods.
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(RunnableOnService.class)
   public void testSingletonSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("Create47", 
Create.of(47)).apply(View.asSingleton());
@@ -112,7 +114,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSingletonSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("Create47", Create.timestamped(
@@ -143,7 +144,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testEmptySingletonSideInput() throws Exception {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateEmptyIntegers", 
Create.of().withCoder(VarIntCoder.of()))
@@ -169,7 +169,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testNonSingletonSideInput() throws Exception {
-Pipeline pipeline = TestPipeline.create();
 
 PCollection oneTwoThree = pipeline.apply(Create.of(1, 2, 
3));
 final PCollectionView view = 
oneTwoThree.apply(View.asSingleton());
@@ -194,7 +193,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testListSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 
23)).apply(View.asList());
@@ -221,7 +219,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedListSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", Create.timestamped(
@@ -262,7 +259,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyListSideInput() throws Exception {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateEmptyView", 
Create.of().withCoder(VarIntCoder.of()))
@@ -289,7 +285,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testListSideInputIsImmutable() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", 
Create.of(11)).apply(View.asList());
@@ -335,7 +330,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testIterableSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 23))
@@ -361,7 +355,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedIterableSideInput() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", Create.timestamped(
@@ -401,7 +394,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyIterableSideInput() throws Exception {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateEmptyView", 
Create.of().withCoder(VarIntCoder.of()))
@@ -427,7 +419,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testIterableSideInputIsImmutable() {
-Pipeline pipeline = TestPipeline.create();
 
 final PCollectionView view =
 pipeline.apply("CreateSideInput", 
Create.of(11)).apply(View.asIterable());
@@ -459,7 +450,6 @@ public class ViewTest implements Serializable {
   @Test
   

[30/51] [abbrv] incubator-beam git commit: Provide local tags in PInput, POutput expansions

2016-12-21 Thread kenn
Provide local tags in PInput, POutput expansions

Output an ordered colleciton in PInput and POutput expansions.

This provides information that is necessary to reconstruct a PInput
or POutput from its expansion.

Implement PCollectionList.equals, PCollectionTuple.equals


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

Branch: refs/heads/python-sdk
Commit: 34373c21ed67696235d88ef40d50e31c77b84c33
Parents: 6a05d7f
Author: Thomas Groh 
Authored: Tue Dec 6 11:03:52 2016 -0800
Committer: Thomas Groh 
Committed: Tue Dec 20 15:18:55 2016 -0800

--
 .../beam/runners/direct/DirectGraphVisitor.java |  18 +--
 .../beam/runners/direct/EvaluationContext.java  |   7 +-
 .../direct/KeyedPValueTrackingVisitor.java  |  16 ++-
 .../beam/runners/direct/WatermarkManager.java   |  19 +--
 .../apache/beam/runners/spark/SparkRunner.java  |  13 ++-
 .../beam/sdk/runners/TransformHierarchy.java|  49 
 .../transforms/join/KeyedPCollectionTuple.java  |   9 +-
 .../java/org/apache/beam/sdk/values/PBegin.java |   4 +-
 .../apache/beam/sdk/values/PCollectionList.java |  65 +++
 .../beam/sdk/values/PCollectionTuple.java   |  28 -
 .../java/org/apache/beam/sdk/values/PDone.java  |   4 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   4 +-
 .../org/apache/beam/sdk/values/POutput.java |   4 +-
 .../java/org/apache/beam/sdk/values/PValue.java |  10 ++
 .../org/apache/beam/sdk/values/PValueBase.java  |  11 +-
 .../apache/beam/sdk/values/TaggedPValue.java|  42 +++
 .../sdk/runners/TransformHierarchyTest.java |  23 +++-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  34 ++
 .../beam/sdk/values/PCollectionListTest.java| 117 +++
 .../beam/sdk/values/PCollectionTupleTest.java   |  70 +++
 20 files changed, 449 insertions(+), 98 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
index 0283d03..425bbf1 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
@@ -35,6 +35,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 
 /**
  * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each 
{@link PValue} in the
@@ -79,14 +80,16 @@ class DirectGraphVisitor extends PipelineVisitor.Defaults {
 
   @Override
   public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-toFinalize.removeAll(node.getInputs());
+for (TaggedPValue consumed : node.getInputs()) {
+  toFinalize.remove(consumed.getValue());
+}
 AppliedPTransform appliedTransform = getAppliedTransform(node);
 stepNames.put(appliedTransform, genStepName());
 if (node.getInputs().isEmpty()) {
   rootTransforms.add(appliedTransform);
 } else {
-  for (PValue value : node.getInputs()) {
-primitiveConsumers.put(value, appliedTransform);
+  for (TaggedPValue value : node.getInputs()) {
+primitiveConsumers.put(value.getValue(), appliedTransform);
   }
 }
   }
@@ -96,15 +99,12 @@ class DirectGraphVisitor extends PipelineVisitor.Defaults {
 toFinalize.add(value);
 
 AppliedPTransform appliedTransform = 
getAppliedTransform(producer);
+if (value instanceof PCollectionView) {
+  views.add((PCollectionView) value);
+}
 if (!producers.containsKey(value)) {
   producers.put(value, appliedTransform);
 }
-   if (value instanceof PCollectionView) {
- views.add((PCollectionView) value);
-   }
-   if (!producers.containsKey(value)) {
- producers.put(value, appliedTransform);
-   }
   }
 
   private AppliedPTransform 
getAppliedTransform(TransformHierarchy.Node node) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
--
diff --git 

[31/51] [abbrv] incubator-beam git commit: This closes #1569

2016-12-21 Thread kenn
This closes #1569


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

Branch: refs/heads/python-sdk
Commit: aadcf3a1203b257961a1a474acf74e6bbca1e2ad
Parents: 6a05d7f 34373c2
Author: Thomas Groh 
Authored: Tue Dec 20 15:18:55 2016 -0800
Committer: Thomas Groh 
Committed: Tue Dec 20 15:18:55 2016 -0800

--
 .../beam/runners/direct/DirectGraphVisitor.java |  18 +--
 .../beam/runners/direct/EvaluationContext.java  |   7 +-
 .../direct/KeyedPValueTrackingVisitor.java  |  16 ++-
 .../beam/runners/direct/WatermarkManager.java   |  19 +--
 .../apache/beam/runners/spark/SparkRunner.java  |  13 ++-
 .../beam/sdk/runners/TransformHierarchy.java|  49 
 .../transforms/join/KeyedPCollectionTuple.java  |   9 +-
 .../java/org/apache/beam/sdk/values/PBegin.java |   4 +-
 .../apache/beam/sdk/values/PCollectionList.java |  65 +++
 .../beam/sdk/values/PCollectionTuple.java   |  28 -
 .../java/org/apache/beam/sdk/values/PDone.java  |   4 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   4 +-
 .../org/apache/beam/sdk/values/POutput.java |   4 +-
 .../java/org/apache/beam/sdk/values/PValue.java |  10 ++
 .../org/apache/beam/sdk/values/PValueBase.java  |  11 +-
 .../apache/beam/sdk/values/TaggedPValue.java|  42 +++
 .../sdk/runners/TransformHierarchyTest.java |  23 +++-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  34 ++
 .../beam/sdk/values/PCollectionListTest.java| 117 +++
 .../beam/sdk/values/PCollectionTupleTest.java   |  70 +++
 20 files changed, 449 insertions(+), 98 deletions(-)
--




[33/51] [abbrv] incubator-beam git commit: Require TimeDomain to delete a timer

2016-12-21 Thread kenn
Require TimeDomain to delete a timer


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

Branch: refs/heads/python-sdk
Commit: 35a02740748182ee52729d8bfb621a3c342b8312
Parents: 0d0a5e2
Author: Kenneth Knowles 
Authored: Tue Dec 20 20:09:25 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 08:20:28 2016 -0800

--
 .../operators/ApexGroupByKeyOperator.java  |  8 
 .../beam/runners/core/InMemoryTimerInternals.java  |  8 
 .../beam/runners/direct/DirectTimerInternals.java  |  8 
 .../wrappers/streaming/WindowDoFnOperator.java |  9 +
 .../org/apache/beam/sdk/util/TimerInternals.java   | 17 +++--
 5 files changed, 48 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/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 48ac177..49ec1c8 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
@@ -425,12 +425,19 @@ public class ApexGroupByKeyOperator implements 
Operator {
*/
   public class ApexTimerInternals implements TimerInternals {
 
+@Deprecated
 @Override
 public void setTimer(TimerData timerData) {
   registerActiveTimer(context.element().key(), timerData);
 }
 
 @Override
+public void deleteTimer(StateNamespace namespace, String timerId, 
TimeDomain timeDomain) {
+  throw new UnsupportedOperationException("Canceling of timer by ID is not 
yet supported.");
+}
+
+@Deprecated
+@Override
 public void deleteTimer(TimerData timerKey) {
   unregisterActiveTimer(context.element().key(), timerKey);
 }
@@ -463,6 +470,7 @@ public class ApexGroupByKeyOperator implements 
Operator {
   throw new UnsupportedOperationException("Setting timer by ID not yet 
supported.");
 }
 
+@Deprecated
 @Override
 public void deleteTimer(StateNamespace namespace, String timerId) {
   throw new UnsupportedOperationException("Canceling of timer by ID is not 
yet supported.");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/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
index 5fcd088..5ddd5a7 100644
--- 
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
@@ -107,6 +107,7 @@ public class InMemoryTimerInternals implements 
TimerInternals {
 throw new UnsupportedOperationException("Setting a timer by ID is not yet 
supported.");
   }
 
+  @Deprecated
   @Override
   public void setTimer(TimerData timerData) {
 WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), 
timerData);
@@ -116,10 +117,17 @@ public class InMemoryTimerInternals implements 
TimerInternals {
   }
 
   @Override
+  public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain 
timeDomain) {
+throw new UnsupportedOperationException("Canceling a timer by ID is not 
yet supported.");
+  }
+
+  @Deprecated
+  @Override
   public void deleteTimer(StateNamespace namespace, String timerId) {
 throw new UnsupportedOperationException("Canceling a timer by ID is not 
yet supported.");
   }
 
+  @Deprecated
   @Override
   public void deleteTimer(TimerData timer) {
 WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), 
timer);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
 

[13/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 950aa7e1d9c50167933eb192a16e15700e483377
Parents: 12be8b1
Author: Stas Levin 
Authored: Tue Dec 20 17:44:15 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:46 2016 -0800

--
 .../org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java   | 7 +--
 .../java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java  | 6 --
 2 files changed, 9 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/950aa7e1/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
--
diff --git 
a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
 
b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
index f0ab46c..075805e 100644
--- 
a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
+++ 
b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java
@@ -22,19 +22,23 @@ import static com.google.common.collect.Lists.newArrayList;
 import 
com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
 import com.google.common.collect.Iterables;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.DateTime;
+import org.junit.Rule;
 import org.junit.Test;
 
 /**
  * Tests {@link AmazonKinesisMock}.
  */
 public class KinesisMockReadTest {
+
+@Rule
+public final transient TestPipeline p = TestPipeline.create();
+
 @Test
 public void readsDataFromMockKinesis() {
 int noOfShards = 3;
@@ -42,7 +46,6 @@ public class KinesisMockReadTest {
 List testData =
 provideTestData(noOfShards, noOfEventsPerShard);
 
-final Pipeline p = TestPipeline.create();
 PCollection result = p.
 apply(
 KinesisIO.Read.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/950aa7e1/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
--
diff --git 
a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
 
b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
index 73a2455..690cc11 100644
--- 
a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
+++ 
b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java
@@ -31,7 +31,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
@@ -43,6 +42,7 @@ import org.apache.commons.lang.RandomStringUtils;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 
 /**
@@ -53,6 +53,8 @@ public class KinesisReaderIT {
 private static final long PIPELINE_STARTUP_TIME = 
TimeUnit.SECONDS.toMillis(10);
 private ExecutorService singleThreadExecutor = newSingleThreadExecutor();
 
+@Rule
+public final transient TestPipeline p = TestPipeline.create();
 
 @Ignore
 @Test
@@ -76,7 +78,7 @@ public class KinesisReaderIT {
 
 private Future startTestPipeline(List testData, 
KinesisTestOptions options)
 throws InterruptedException {
-final Pipeline p = TestPipeline.create();
+
 PCollection result = p.
 apply(KinesisIO.Read.
 from(options.getAwsKinesisStream(), Instant.now()).



[24/51] [abbrv] incubator-beam git commit: Provide a better error message for non-existing gcpTempLocation

2016-12-21 Thread kenn
Provide a better error message for non-existing gcpTempLocation

gcpTempLocation will default to using the value for tmpLocation, as long
as the value is a valid GCP path. Non-valid GCP paths are silently
discarded.

This change removes existence validation from the default value logic
such that downstream validation can provide a better error message.


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

Branch: refs/heads/python-sdk
Commit: ef1a858347e475cd15f7dcd8873464f506527b2a
Parents: 2f4b803
Author: Scott Wegner 
Authored: Tue Dec 6 14:19:12 2016 -0800
Committer: Luke Cwik 
Committed: Tue Dec 20 13:04:24 2016 -0800

--
 .../beam/runners/dataflow/DataflowRunner.java   | 25 
 .../options/DataflowPipelineOptions.java| 19 -
 .../runners/dataflow/DataflowRunnerTest.java| 42 +++-
 .../options/DataflowPipelineOptionsTest.java| 20 ++
 .../org/apache/beam/sdk/options/GcpOptions.java | 19 +
 .../apache/beam/sdk/util/GcsPathValidator.java  |  3 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java | 32 +--
 .../beam/sdk/util/GcsPathValidatorTest.java | 15 +--
 8 files changed, 114 insertions(+), 61 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 711b1b0..1a15eaf 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -244,14 +244,23 @@ public class DataflowRunner extends 
PipelineRunner {
 }
 
 PathValidator validator = dataflowOptions.getPathValidator();
-checkArgument(
-!isNullOrEmpty(dataflowOptions.getGcpTempLocation()),
-"DataflowRunner requires gcpTempLocation, and it is missing in 
PipelineOptions.");
-
validator.validateOutputFilePrefixSupported(dataflowOptions.getGcpTempLocation());
-checkArgument(
-!isNullOrEmpty(dataflowOptions.getStagingLocation()),
-"DataflowRunner requires stagingLocation, and it is missing in 
PipelineOptions.");
-
validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
+String gcpTempLocation;
+try {
+  gcpTempLocation = dataflowOptions.getGcpTempLocation();
+} catch (Exception e) {
+  throw new IllegalArgumentException("DataflowRunner requires 
gcpTempLocation, "
+  + "but failed to retrieve a value from PipelineOptions", e);
+}
+validator.validateOutputFilePrefixSupported(gcpTempLocation);
+
+String stagingLocation;
+try {
+  stagingLocation = dataflowOptions.getStagingLocation();
+} catch (Exception e) {
+  throw new IllegalArgumentException("DataflowRunner requires 
stagingLocation, "
+  + "but failed to retrieve a value from PipelineOptions", e);
+}
+validator.validateOutputFilePrefixSupported(stagingLocation);
 
 if (!Strings.isNullOrEmpty(dataflowOptions.getSaveProfilesToGcs())) {
   
validator.validateOutputFilePrefixSupported(dataflowOptions.getSaveProfilesToGcs());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index 66632ad..5ddc5d0 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -17,9 +17,6 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Strings.isNullOrEmpty;
-
 import java.io.IOException;
 import 

[07/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 5ccbe6791af7e75c166ed877391e8c86bba5fe56
Parents: 6dea099
Author: Stas Levin 
Authored: Tue Dec 20 17:26:51 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java| 8 
 1 file changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ccbe679/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 aa93a22..eec7cb8 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
@@ -48,6 +48,7 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -64,6 +65,9 @@ public class JdbcIOTest implements Serializable {
 
   private static int port;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   @BeforeClass
   public static void startDatabase() throws Exception {
 ServerSocket socket = new ServerSocket(0);
@@ -207,7 +211,6 @@ public class JdbcIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testRead() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 PCollection> output = pipeline.apply(
 JdbcIO.>read()
@@ -245,7 +248,6 @@ public class JdbcIOTest implements Serializable {
@Test
@Category(NeedsRunner.class)
public void testReadWithSingleStringParameter() throws Exception {
- TestPipeline pipeline = TestPipeline.create();
 
  PCollection> output = pipeline.apply(
  JdbcIO.>read()
@@ -278,7 +280,6 @@ public class JdbcIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWrite() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 ArrayList> data = new ArrayList<>();
 for (int i = 0; i < 1000; i++) {
@@ -316,7 +317,6 @@ public class JdbcIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWriteWithEmptyPCollection() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 pipeline.apply(Create.of(new ArrayList>()))
 .apply(JdbcIO.>write()



[03/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. Plus, fixed some checkstyle errors from previous modules' migration.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule.
Plus, fixed some checkstyle errors from previous modules' migration.


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

Branch: refs/heads/python-sdk
Commit: 75a4c918346b5a04213a54bf7d1bf6507655342a
Parents: 09c404a
Author: Stas Levin 
Authored: Mon Dec 19 23:54:47 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../UnboundedReadFromBoundedSourceTest.java |   1 -
 .../direct/CloningBundleFactoryTest.java|   2 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |   6 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   2 +-
 .../direct/ImmutableListBundleFactoryTest.java  |   2 +-
 .../direct/WriteWithShardingFactoryTest.java|   2 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |  37 +++---
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  11 +-
 .../beam/sdk/coders/CoderRegistryTest.java  |   6 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   7 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  16 +--
 .../io/BoundedReadFromUnboundedSourceTest.java  |   6 +-
 .../beam/sdk/io/CompressedSourceTest.java   |  12 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  12 +-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  13 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |   4 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java|  10 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  12 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  29 +++--
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   2 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  10 +-
 .../sdk/options/ProxyInvocationHandlerTest.java |   5 +-
 .../sdk/runners/TransformHierarchyTest.java |   6 +-
 .../beam/sdk/runners/TransformTreeTest.java |  11 +-
 .../beam/sdk/testing/GatherAllPanesTest.java|   7 +-
 .../apache/beam/sdk/testing/PAssertTest.java|  32 ++---
 .../apache/beam/sdk/testing/TestStreamTest.java |   7 +-
 .../transforms/ApproximateQuantilesTest.java|  12 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |   6 +-
 .../beam/sdk/transforms/CombineFnsTest.java |   5 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  25 +---
 .../apache/beam/sdk/transforms/CountTest.java   |  13 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |  27 +
 .../beam/sdk/transforms/DistinctTest.java   |  12 +-
 .../apache/beam/sdk/transforms/DoFnTest.java|   4 +-
 .../beam/sdk/transforms/DoFnTesterTest.java |   6 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |  18 +--
 .../sdk/transforms/FlatMapElementsTest.java |  10 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |  35 +-
 .../beam/sdk/transforms/GroupByKeyTest.java |  30 ++---
 .../apache/beam/sdk/transforms/KeysTest.java|   9 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   9 +-
 .../apache/beam/sdk/transforms/LatestTest.java  |  12 +-
 .../beam/sdk/transforms/MapElementsTest.java|  14 +--
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  17 +--
 .../apache/beam/sdk/transforms/ParDoTest.java   | 118 +++
 .../beam/sdk/transforms/PartitionTest.java  |   8 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  25 +---
 .../apache/beam/sdk/transforms/SampleTest.java  |  34 +++---
 .../beam/sdk/transforms/SplittableDoFnTest.java |  12 +-
 .../org/apache/beam/sdk/transforms/TopTest.java |  15 ++-
 .../apache/beam/sdk/transforms/ValuesTest.java  |   7 +-
 .../apache/beam/sdk/transforms/ViewTest.java|  84 -
 .../beam/sdk/transforms/WithKeysTest.java   |   8 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |   9 +-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  11 +-
 .../sdk/transforms/windowing/WindowTest.java|  22 ++--
 .../sdk/transforms/windowing/WindowingTest.java |  11 +-
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  11 +-
 .../beam/sdk/values/PCollectionTupleTest.java   |  12 +-
 .../org/apache/beam/sdk/values/PDoneTest.java   |   9 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |  10 +-
 62 files changed, 353 insertions(+), 587 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
 

[22/51] [abbrv] incubator-beam git commit: This closes #1581: [BEAM-1117] Port direct runner StatefulParDo to KeyedWorkItem

2016-12-21 Thread kenn
This closes #1581: [BEAM-1117] Port direct runner StatefulParDo to KeyedWorkItem

  Port direct runner StatefulParDo to KeyedWorkItem
  Propagate key through ParDo if DoFn is key-preserving
  Move responsibility for knowing about keyedness into EvaluationContext
  Add some key-preserving to KeyedPValueTrackingVisitor


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

Branch: refs/heads/python-sdk
Commit: 2f4b80312c69da00df82aaa37d17cc2f6a742648
Parents: a526adb 1f018ab
Author: Kenneth Knowles 
Authored: Tue Dec 20 12:39:34 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 12:39:34 2016 -0800

--
 .../beam/runners/direct/DirectRunner.java   |  13 +-
 .../beam/runners/direct/EvaluationContext.java  |  26 +++-
 .../direct/ExecutorServiceParallelExecutor.java |   8 +-
 .../direct/KeyedPValueTrackingVisitor.java  |  44 --
 .../beam/runners/direct/ParDoEvaluator.java |  13 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |   3 +
 .../direct/ParDoMultiOverrideFactory.java   |  94 +++--
 ...littableProcessElementsEvaluatorFactory.java |   1 +
 .../direct/StatefulParDoEvaluatorFactory.java   |  36 ++---
 .../runners/direct/EvaluationContextTest.java   |   9 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  | 135 +--
 .../beam/runners/direct/ParDoEvaluatorTest.java |   1 +
 .../StatefulParDoEvaluatorFactoryTest.java  |  51 ---
 13 files changed, 281 insertions(+), 153 deletions(-)
--




[06/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 8d478c0f38c656d3533d590a65c6ed95da229f81
Parents: 5ccbe67
Author: Stas Levin 
Authored: Tue Dec 20 17:31:23 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:45 2016 -0800

--
 .../src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java | 9 -
 1 file changed, 4 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8d478c0f/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
--
diff --git 
a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java 
b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
index 4c3be6d..7259ce8 100644
--- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
+++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
@@ -28,7 +28,6 @@ import javax.jms.TextMessage;
 import org.apache.activemq.ActiveMQConnectionFactory;
 import org.apache.activemq.broker.BrokerService;
 import org.apache.activemq.store.memory.MemoryPersistenceAdapter;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -38,6 +37,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -55,6 +55,9 @@ public class JmsIOTest {
   private BrokerService broker;
   private ConnectionFactory connectionFactory;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   @Before
   public void startBroker() throws Exception {
 broker = new BrokerService();
@@ -92,8 +95,6 @@ public class JmsIOTest {
 session.close();
 connection.close();
 
-Pipeline pipeline = TestPipeline.create();
-
 // read from the queue
 PCollection output = pipeline.apply(
 JmsIO.read()
@@ -117,8 +118,6 @@ public class JmsIOTest {
   @Category(NeedsRunner.class)
   public void testWriteMessage() throws Exception {
 
-Pipeline pipeline = TestPipeline.create();
-
 ArrayList data = new ArrayList<>();
 for (int i = 0; i < 100; i++) {
   data.add("Message " + i);



[11/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit 
rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.


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

Branch: refs/heads/python-sdk
Commit: 4b23d42c31c95bed0d64bfc393fa193311e93498
Parents: fce4f65
Author: Stas Levin 
Authored: Tue Dec 20 18:57:57 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:46 2016 -0800

--
 .../apache/beam/sdk/transforms/CombineJava8Test.java  |  8 +++-
 .../apache/beam/sdk/transforms/DistinctJava8Test.java |  5 +++--
 .../apache/beam/sdk/transforms/FilterJava8Test.java   |  9 +++--
 .../beam/sdk/transforms/FlatMapElementsJava8Test.java |  7 ---
 .../beam/sdk/transforms/MapElementsJava8Test.java |  9 ++---
 .../beam/sdk/transforms/PartitionJava8Test.java   |  7 ---
 .../apache/beam/sdk/transforms/WithKeysJava8Test.java |  6 --
 .../beam/sdk/transforms/WithTimestampsJava8Test.java  | 14 ++
 8 files changed, 37 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
--
diff --git 
a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
 
b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
index 98d99ce..a0f7ce6 100644
--- 
a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
+++ 
b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java
@@ -23,7 +23,6 @@ import static org.hamcrest.Matchers.not;
 
 import com.google.common.collect.Iterables;
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.display.DisplayData;
@@ -45,6 +44,9 @@ import org.junit.runners.JUnit4;
 public class CombineJava8Test implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   /**
@@ -65,7 +67,6 @@ public class CombineJava8Test implements Serializable {
*/
   @Test
   public void testCombineGloballyLambda() {
-Pipeline pipeline = TestPipeline.create();
 
 PCollection output = pipeline
 .apply(Create.of(1, 2, 3, 4))
@@ -86,7 +87,6 @@ public class CombineJava8Test implements Serializable {
*/
   @Test
   public void testCombineGloballyInstanceMethodReference() {
-Pipeline pipeline = TestPipeline.create();
 
 PCollection output = pipeline
 .apply(Create.of(1, 2, 3, 4))
@@ -101,7 +101,6 @@ public class CombineJava8Test implements Serializable {
*/
   @Test
   public void testCombinePerKeyLambda() {
-Pipeline pipeline = TestPipeline.create();
 
 PCollection> output = pipeline
 .apply(Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 3), 
KV.of("c", 4)))
@@ -125,7 +124,6 @@ public class CombineJava8Test implements Serializable {
*/
   @Test
   public void testCombinePerKeyInstanceMethodReference() {
-Pipeline pipeline = TestPipeline.create();
 
 PCollection> output = pipeline
 .apply(Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 3), 
KV.of("c", 4)))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java
--
diff --git 
a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java
 
b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java
index 99ef232..790f51e 100644
--- 
a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java
+++ 
b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java
@@ -44,11 +44,13 @@ import org.junit.runners.JUnit4;
 public class DistinctJava8Test {
 
   @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
+  @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void withLambdaRepresentativeValuesFnAndTypeDescriptorShouldApplyFn() 
{
-TestPipeline p = TestPipeline.create();
 
 Multimap 

[18/51] [abbrv] incubator-beam git commit: Port direct runner StatefulParDo to KeyedWorkItem

2016-12-21 Thread kenn
Port direct runner StatefulParDo to KeyedWorkItem


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

Branch: refs/heads/python-sdk
Commit: 1f018ab69fdcc720a10e2aeb8ec1eea1c06e1cbc
Parents: d040b7f
Author: Kenneth Knowles 
Authored: Mon Dec 12 19:49:58 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:19:07 2016 -0800

--
 .../direct/KeyedPValueTrackingVisitor.java  | 13 ++-
 .../direct/ParDoMultiOverrideFactory.java   | 94 +---
 .../direct/StatefulParDoEvaluatorFactory.java   | 36 
 .../direct/KeyedPValueTrackingVisitorTest.java  | 69 --
 .../StatefulParDoEvaluatorFactoryTest.java  | 51 +++
 5 files changed, 205 insertions(+), 58 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index e91a768..65c41e0 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -31,6 +31,7 @@ import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PValue;
 
 /**
@@ -105,7 +106,15 @@ class KeyedPValueTrackingVisitor implements 
PipelineVisitor {
   }
 
   private static boolean isKeyPreserving(PTransform transform) {
-// There are currently no key-preserving transforms; this lays the 
infrastructure for them
-return false;
+// This is a hacky check for what is considered key-preserving to the 
direct runner.
+// The most obvious alternative would be a package-private marker 
interface, but
+// better to make this obviously hacky so it is less likely to 
proliferate. Meanwhile
+// we intend to allow explicit expression of key-preserving DoFn in the 
model.
+if (transform instanceof ParDo.BoundMulti) {
+  ParDo.BoundMulti parDo = (ParDo.BoundMulti) transform;
+  return parDo.getFn() instanceof 
ParDoMultiOverrideFactory.ToKeyedWorkItem;
+} else {
+  return false;
+}
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index c5bc069..2cea999 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -17,9 +17,15 @@
  */
 package org.apache.beam.runners.direct;
 
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItemCoder;
+import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -28,6 +34,8 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
@@ -84,16 +92,41 @@ class ParDoMultiOverrideFactory
 @Override
 public PCollectionTuple 

[17/51] [abbrv] incubator-beam git commit: Move responsibility for knowing about keyedness into EvaluationContext

2016-12-21 Thread kenn
Move responsibility for knowing about keyedness into EvaluationContext

This will allow transform evaluators to inquire about whether
various collections are keyed.


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

Branch: refs/heads/python-sdk
Commit: b26ceaa347c4bc50abfb4c3c138167a25a99cf57
Parents: 81702e6
Author: Kenneth Knowles 
Authored: Thu Dec 8 13:28:44 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:18:04 2016 -0800

--
 .../beam/runners/direct/DirectRunner.java   |  4 +--
 .../beam/runners/direct/EvaluationContext.java  | 26 +---
 .../direct/ExecutorServiceParallelExecutor.java |  8 +-
 .../runners/direct/EvaluationContextTest.java   |  9 ++-
 4 files changed, 34 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b26ceaa3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index afa43ff..7e6ea15 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -315,14 +315,14 @@ public class DirectRunner extends 
PipelineRunner {
 getPipelineOptions(),
 clockSupplier.get(),
 Enforcement.bundleFactoryFor(enabledEnforcements, graph),
-graph);
+graph,
+keyedPValueVisitor.getKeyedPValues());
 
 RootProviderRegistry rootInputProvider = 
RootProviderRegistry.defaultRegistry(context);
 TransformEvaluatorRegistry registry = 
TransformEvaluatorRegistry.defaultRegistry(context);
 PipelineExecutor executor =
 ExecutorServiceParallelExecutor.create(
 options.getTargetParallelism(), graph,
-keyedPValueVisitor.getKeyedPValues(),
 rootInputProvider,
 registry,
 Enforcement.defaultModelEnforcements(enabledEnforcements),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b26ceaa3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index 230d91b..cb9ddd8 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import javax.annotation.Nullable;
@@ -99,17 +100,28 @@ class EvaluationContext {
 
   private final DirectMetrics metrics;
 
+  private final Set keyedPValues;
+
   public static EvaluationContext create(
-  DirectOptions options, Clock clock, BundleFactory bundleFactory, 
DirectGraph graph) {
-return new EvaluationContext(options, clock, bundleFactory, graph);
+  DirectOptions options,
+  Clock clock,
+  BundleFactory bundleFactory,
+  DirectGraph graph,
+  Set keyedPValues) {
+return new EvaluationContext(options, clock, bundleFactory, graph, 
keyedPValues);
   }
 
   private EvaluationContext(
-  DirectOptions options, Clock clock, BundleFactory bundleFactory, 
DirectGraph graph) {
+  DirectOptions options,
+  Clock clock,
+  BundleFactory bundleFactory,
+  DirectGraph graph,
+  Set keyedPValues) {
 this.options = checkNotNull(options);
 this.clock = clock;
 this.bundleFactory = checkNotNull(bundleFactory);
 this.graph = checkNotNull(graph);
+this.keyedPValues = keyedPValues;
 
 this.watermarkManager = WatermarkManager.create(clock, graph);
 this.sideInputContainer = SideInputContainer.create(this, 
graph.getViews());
@@ -244,6 +256,14 @@ class EvaluationContext {
   }
 
   /**
+   * Indicate whether or not this {@link PCollection} has been determined to be
+   * keyed.
+   */
+  public  boolean isKeyed(PValue pValue) {
+return keyedPValues.contains(pValue);
+  }
+
+  /**
* Create a {@link 

[26/51] [abbrv] incubator-beam git commit: Remove deprecated AggregatorFactory from SDK

2016-12-21 Thread kenn
Remove deprecated AggregatorFactory from SDK


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

Branch: refs/heads/python-sdk
Commit: aab46a0ec6e0e45208f64de7aabb9af643acd0ec
Parents: a3f68d3
Author: Kenneth Knowles 
Authored: Thu Dec 15 20:13:25 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 14:05:35 2016 -0800

--
 .../apache/beam/sdk/transforms/Aggregator.java   | 19 ---
 1 file changed, 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aab46a0e/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
index 43f53a8..4119c53 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.transforms;
 
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.util.ExecutionContext;
 
 /**
  * An {@code Aggregator} enables monitoring of values of type {@code 
InputT},
@@ -68,22 +67,4 @@ public interface Aggregator {
* aggregator.
*/
   CombineFn getCombineFn();
-
-  /**
-   * @deprecated this is for use only by runners and exists only for a 
migration period. Please
-   * use the identical interface in org.apache.beam.runners.core
-   */
-  @Deprecated
-  interface AggregatorFactory {
-/**
- * Create an aggregator with the given {@code name} and {@link CombineFn}.
- *
- *  This method is called to create an aggregator for a {@link DoFn}. 
It receives the
- *  class of the {@link DoFn} being executed and the context of the step 
it is being
- *  executed in.
- */
- Aggregator 
createAggregatorForDoFn(
-Class fnClass, ExecutionContext.StepContext stepContext,
-String aggregatorName, CombineFn combine);
-  }
 }



[32/51] [abbrv] incubator-beam git commit: This closes #1668: Remove deprecated InMemoryTimerInternals from SDK

2016-12-21 Thread kenn
This closes #1668: Remove deprecated InMemoryTimerInternals from SDK


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

Branch: refs/heads/python-sdk
Commit: 0d0a5e2872aeba7a1069927408b3a9607709cf11
Parents: aadcf3a 9f1d3d1
Author: Kenneth Knowles 
Authored: Wed Dec 21 08:16:00 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 08:16:00 2016 -0800

--
 .../sdk/util/state/InMemoryTimerInternals.java  | 275 ---
 1 file changed, 275 deletions(-)
--




[15/51] [abbrv] incubator-beam git commit: Add some key-preserving to KeyedPValueTrackingVisitor

2016-12-21 Thread kenn
Add some key-preserving to KeyedPValueTrackingVisitor


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

Branch: refs/heads/python-sdk
Commit: 81702e67b92a23849cbc8f4a16b2a619e4b477a1
Parents: 22e25a4
Author: Kenneth Knowles 
Authored: Thu Dec 8 11:49:15 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:18:02 2016 -0800

--
 .../beam/runners/direct/DirectRunner.java   |  9 +--
 .../direct/KeyedPValueTrackingVisitor.java  | 35 +---
 .../direct/KeyedPValueTrackingVisitorTest.java  | 84 +++-
 3 files changed, 37 insertions(+), 91 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/81702e67/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 78163c0..afa43ff 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -31,8 +31,6 @@ import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.core.SplittableParDo;
-import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
-import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
 import 
org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory;
 import org.apache.beam.runners.direct.ViewEvaluatorFactory.ViewOverrideFactory;
@@ -306,12 +304,7 @@ public class DirectRunner extends 
PipelineRunner {
 graphVisitor.finishSpecifyingRemainder();
 
 @SuppressWarnings("rawtypes")
-KeyedPValueTrackingVisitor keyedPValueVisitor =
-KeyedPValueTrackingVisitor.create(
-ImmutableSet.of(
-SplittableParDo.GBKIntoKeyedWorkItems.class,
-DirectGroupByKeyOnly.class,
-DirectGroupAlsoByWindow.class));
+KeyedPValueTrackingVisitor keyedPValueVisitor = 
KeyedPValueTrackingVisitor.create();
 pipeline.traverseTopologically(keyedPValueVisitor);
 
 DisplayDataValidator.validatePipeline(pipeline);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/81702e67/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 7f85169..e91a768 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -18,9 +18,15 @@
 package org.apache.beam.runners.direct;
 
 import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Predicates.in;
+import static com.google.common.collect.Iterables.all;
 
+import com.google.common.collect.ImmutableSet;
 import java.util.HashSet;
 import java.util.Set;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -38,19 +44,21 @@ import org.apache.beam.sdk.values.PValue;
 // TODO: Handle Key-preserving transforms when appropriate and more 
aggressively make PTransforms
 // unkeyed
 class KeyedPValueTrackingVisitor implements PipelineVisitor {
-  @SuppressWarnings("rawtypes")
-  private final Set producesKeyedOutputs;
+
+  private static final Set PRODUCES_KEYED_OUTPUTS 
=
+  ImmutableSet.of(
+  SplittableParDo.GBKIntoKeyedWorkItems.class,
+  DirectGroupByKeyOnly.class,
+  DirectGroupAlsoByWindow.class);
+
   private final Set keyedValues;
   private boolean finalized;
 
-  public static KeyedPValueTrackingVisitor create(
-  @SuppressWarnings("rawtypes") Set 
producesKeyedOutputs) {
-return new 

[20/51] [abbrv] incubator-beam git commit: Move InMemoryTimerInternals to runners-core

2016-12-21 Thread kenn
Move InMemoryTimerInternals to runners-core


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

Branch: refs/heads/python-sdk
Commit: 445c120510948fb23e6d35b502da1e5a4f0ffdfb
Parents: 22e25a4
Author: Kenneth Knowles 
Authored: Thu Dec 15 20:45:56 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:21:52 2016 -0800

--
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   1 -
 .../runners/core/InMemoryTimerInternals.java| 273 ++
 .../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, 443 insertions(+), 471 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/445c1205/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 9189191..efcd771 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,7 +27,6 @@ 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/445c1205/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
new file mode 100644
index 000..5fcd088
--- /dev/null
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/** {@link TimerInternals} with all watermarks and processing clock simulated 
in-memory. */
+public class InMemoryTimerInternals implements TimerInternals {
+
+  /** At most one timer per timestamp is kept. */
+  private Set existingTimers = new HashSet<>();
+
+  /** Pending input watermark timers, in timestamp order. */
+  private PriorityQueue 

[25/51] [abbrv] incubator-beam git commit: [BEAM-1097] Provide a better error message for non-existing gcpTempLocation

2016-12-21 Thread kenn
[BEAM-1097] Provide a better error message for non-existing gcpTempLocation

This closes #1522


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

Branch: refs/heads/python-sdk
Commit: acd2196cf54e30e18e69c4dd30b57e6179909ecf
Parents: 2f4b803 96d3931
Author: Luke Cwik 
Authored: Tue Dec 20 13:04:31 2016 -0800
Committer: Luke Cwik 
Committed: Tue Dec 20 13:04:31 2016 -0800

--
 .../beam/runners/dataflow/DataflowRunner.java   | 25 ++
 .../options/DataflowPipelineOptions.java| 19 
 .../runners/dataflow/DataflowRunnerTest.java| 48 ++--
 .../options/DataflowPipelineOptionsTest.java| 20 +---
 .../org/apache/beam/sdk/options/GcpOptions.java | 19 
 .../apache/beam/sdk/util/GcsPathValidator.java  |  3 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java | 32 +++--
 .../beam/sdk/util/GcsPathValidatorTest.java | 15 +-
 8 files changed, 117 insertions(+), 64 deletions(-)
--




[19/51] [abbrv] incubator-beam git commit: Restore SDK's InMemoryTimerInternals, deprecated

2016-12-21 Thread kenn
Restore SDK's InMemoryTimerInternals, deprecated


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

Branch: refs/heads/python-sdk
Commit: 69d2c47b6a476099535e9cefe62d4cce5ccafbc1
Parents: 445c120
Author: Kenneth Knowles 
Authored: Fri Dec 16 20:22:59 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:21:52 2016 -0800

--
 .../sdk/util/state/InMemoryTimerInternals.java  | 275 +++
 1 file changed, 275 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/69d2c47b/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 000..a910d64
--- /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;
+
+/**
+ * @deprecated use {@code org.apache.beam.runners.core.InMemoryTimerInternals}.
+ */
+@Deprecated
+public class InMemoryTimerInternals implements TimerInternals {
+
+  /** At most one timer per timestamp is kept. */
+  private Set existingTimers = new HashSet<>();
+
+  /** Pending input watermark timers, in timestamp order. */
+  private PriorityQueue watermarkTimers = new PriorityQueue<>(11);
+
+  /** Pending processing time timers, in timestamp order. */
+  private PriorityQueue processingTimers = new PriorityQueue<>(11);
+
+  /** Pending synchronized processing time timers, in timestamp order. */
+  private PriorityQueue 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 queue(TimeDomain domain) {
+switch (domain) {
+  case EVENT_TIME:
+return watermarkTimers;
+  case 

[21/51] [abbrv] incubator-beam git commit: This closes #1652: Move InMemoryTimerInternals to runners-core

2016-12-21 Thread kenn
This closes #1652: Move InMemoryTimerInternals to runners-core

* github/pr/1652:
  Restore SDK's InMemoryTimerInternals, deprecated
  Move InMemoryTimerInternals to runners-core


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

Branch: refs/heads/python-sdk
Commit: a526adb33c1eeece866f464e3dfd2cdbc3be6dea
Parents: 22e25a4 69d2c47
Author: Kenneth Knowles 
Authored: Tue Dec 20 11:22:24 2016 -0800
Committer: Kenneth Knowles 
Committed: Tue Dec 20 11:22:24 2016 -0800

--
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   1 -
 .../runners/core/InMemoryTimerInternals.java| 273 +++
 .../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  |   4 +-
 .../util/state/InMemoryTimerInternalsTest.java  | 153 ---
 10 files changed, 445 insertions(+), 198 deletions(-)
--




[12/51] [abbrv] incubator-beam git commit: Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.

2016-12-21 Thread kenn
Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: fce4f6584ca2fd3c2c258405b9f3014be9da3514
Parents: 950aa7e
Author: Stas Levin 
Authored: Tue Dec 20 18:09:30 2016 +0200
Committer: Kenneth Knowles 
Committed: Tue Dec 20 09:55:46 2016 -0800

--
 .../org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java | 9 -
 .../java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java  | 7 ---
 2 files changed, 8 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fce4f658/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 df05c93..994be87 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
@@ -55,7 +55,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.Scanner;
 
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -79,6 +78,7 @@ import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -100,6 +100,9 @@ public class MongoDBGridFSIOTest implements Serializable {
 
   private static int port;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   @BeforeClass
   public static void setup() throws Exception {
 try (ServerSocket serverSocket = new ServerSocket(0)) {
@@ -182,7 +185,6 @@ public class MongoDBGridFSIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFullRead() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 PCollection output = pipeline.apply(
 MongoDbGridFSIO.read()
@@ -212,7 +214,6 @@ public class MongoDBGridFSIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReadWithParser() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 PCollection> output = pipeline.apply(
 MongoDbGridFSIO.>read()
@@ -297,8 +298,6 @@ public class MongoDBGridFSIOTest implements Serializable {
   @Category(NeedsRunner.class)
   public void testWriteMessage() throws Exception {
 
-Pipeline pipeline = TestPipeline.create();
-
 ArrayList data = new ArrayList<>(100);
 ArrayList intData = new ArrayList<>(100);
 for (int i = 0; i < 1000; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fce4f658/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 5faa618..e7ff712 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
@@ -57,6 +57,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -80,6 +81,9 @@ public class MongoDbIOTest implements Serializable {
 
   private static int port;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   /**
* Looking for an available network port.
*/
@@ -143,7 +147,6 @@ public class MongoDbIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFullRead() throws Exception {
-TestPipeline pipeline = TestPipeline.create();
 
 PCollection output = pipeline.apply(
 MongoDbIO.read()
@@ -177,7 +180,6 @@ public class MongoDbIOTest implements Serializable {
   @Test
   

[5/7] incubator-beam git commit: Add static Window.withOutputTimeFn to match build method

2016-12-21 Thread kenn
Add static Window.withOutputTimeFn to match build method


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

Branch: refs/heads/master
Commit: 8188040d930b1fa49efd4ed7d5f821d05d6f28ef
Parents: fa4958a
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:57:55 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../org/apache/beam/sdk/transforms/windowing/Window.java| 9 +
 1 file changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8188040d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index 0c430d0..1241abe 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -223,6 +223,15 @@ public class Window {
   }
 
   /**
+   * (Experimental) Override the default {@link OutputTimeFn}, 
to control
+   * the output timestamp of values output from a {@link GroupByKey} operation.
+   */
+  @Experimental(Kind.OUTPUT_TIME)
+  public static  Bound withOutputTimeFn(OutputTimeFn outputTimeFn) {
+return new Bound(null).withOutputTimeFn(outputTimeFn);
+  }
+
+  /**
* A {@code PTransform} that windows the elements of a {@code 
PCollection},
* into finite windows according to a user-specified {@code WindowFn}.
*



[6/7] incubator-beam git commit: Add UsesTestStream for use with JUnit @Category

2016-12-21 Thread kenn
Add UsesTestStream for use with JUnit @Category


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

Branch: refs/heads/master
Commit: 4d71924ccda9dae97c7cc9535a9780df9457cc3f
Parents: 8188040
Author: Kenneth Knowles 
Authored: Tue Dec 20 14:20:07 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../apache/beam/sdk/testing/UsesTestStream.java | 24 
 .../apache/beam/sdk/testing/TestStreamTest.java | 12 +-
 2 files changed, 30 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d71924c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java
new file mode 100644
index 000..8debb46
--- /dev/null
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.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 tests that use {@link TestStream}, which is not a part of 
the Beam model
+ * but a special feature currently only implemented by the direct runner.
+ */
+public interface UsesTestStream {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d71924c/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index 64aeca3..c12e9f3 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -69,7 +69,7 @@ public class TestStreamTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testLateDataAccumulating() {
 Instant instant = new Instant(0);
 TestStream source = TestStream.create(VarIntCoder.of())
@@ -136,7 +136,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testProcessingTimeTrigger() {
 TestStream source = TestStream.create(VarLongCoder.of())
 .addElements(TimestampedValue.of(1L, new Instant(1000L)),
@@ -159,7 +159,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testDiscardingMode() {
 TestStream stream =
 TestStream.create(StringUtf8Coder.of())
@@ -208,7 +208,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testFirstElementLate() {
 Instant lateElementTimestamp = new Instant(-1_000_000);
 TestStream stream =
@@ -238,7 +238,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  @Category({NeedsRunner.class, UsesTestStream.class})
   public void testElementsAtAlmostPositiveInfinity() {
 Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp();
 TestStream stream = TestStream.create(StringUtf8Coder.of())
@@ -261,7 +261,7 @@ public class TestStreamTest implements Serializable {
   }
 
   @Test
-  @Category(NeedsRunner.class)
+  

[2/7] incubator-beam git commit: Hold output watermark according to pending timers

2016-12-21 Thread kenn
Hold output watermark according to pending timers


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

Branch: refs/heads/master
Commit: dfe2e62d103595583e3ca4594cc03885fe1bba16
Parents: 7f14c46
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:37:40 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../beam/runners/direct/WatermarkManager.java   | 59 
 1 file changed, 48 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dfe2e62d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index f7bafd1..248fafd 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -211,12 +211,18 @@ public class WatermarkManager {
   private static class AppliedPTransformInputWatermark implements Watermark {
 private final Collection inputWatermarks;
 private final SortedMultiset pendingElements;
-private final Map objectTimers;
+
+// This tracks only the quantity of timers at each timestamp, for quickly 
getting the cross-key
+// minimum
+private final SortedMultiset pendingTimers;
 
 // Entries in this table represent the authoritative timestamp for which
 // a per-key-and-StateNamespace timer is set.
 private final Map> existingTimers;
 
+// This per-key sorted set allows quick retrieval of timers that should 
fire for a key
+private final Map objectTimers;
+
 private AtomicReference currentWatermark;
 
 public AppliedPTransformInputWatermark(Collection 
inputWatermarks) {
@@ -224,10 +230,13 @@ public class WatermarkManager {
   // The ordering must order elements by timestamp, and must not compare 
two distinct elements
   // as equal. This is built on the assumption that any element added as a 
pending element will
   // be consumed without modifications.
+  //
+  // The same logic is applied for pending timers
   Ordering pendingBundleComparator =
   new 
BundleByElementTimestampComparator().compound(Ordering.arbitrary());
   this.pendingElements =
   TreeMultiset.create(pendingBundleComparator);
+  this.pendingTimers = TreeMultiset.create();
   this.objectTimers = new HashMap<>();
   this.existingTimers = new HashMap<>();
   currentWatermark = new 
AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
@@ -278,6 +287,14 @@ public class WatermarkManager {
   pendingElements.remove(completed);
 }
 
+private synchronized Instant getEarliestTimerTimestamp() {
+  if (pendingTimers.isEmpty()) {
+return BoundedWindow.TIMESTAMP_MAX_VALUE;
+  } else {
+return pendingTimers.firstEntry().getElement();
+  }
+}
+
 private synchronized void updateTimers(TimerUpdate update) {
   NavigableSet keyTimers = objectTimers.get(update.key);
   if (keyTimers == null) {
@@ -291,27 +308,43 @@ public class WatermarkManager {
 existingTimers.put(update.key, existingTimersForKey);
   }
 
-  for (TimerData timer : update.setTimers) {
+  for (TimerData timer : update.getSetTimers()) {
+if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+  @Nullable
+  TimerData existingTimer =
+  existingTimersForKey.get(timer.getNamespace(), 
timer.getTimerId());
+
+  if (existingTimer == null) {
+pendingTimers.add(timer.getTimestamp());
+keyTimers.add(timer);
+  } else if (!existingTimer.equals(timer)) {
+keyTimers.remove(existingTimer);
+keyTimers.add(timer);
+  } // else the timer is already set identically, so noop
+
+  existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), 
timer);
+}
+  }
+
+  for (TimerData timer : update.getDeletedTimers()) {
 if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
   @Nullable
   TimerData existingTimer =
   existingTimersForKey.get(timer.getNamespace(), 
timer.getTimerId());
 
   

[3/7] incubator-beam git commit: Use informative Instant formatter in WatermarkHold

2016-12-21 Thread kenn
Use informative Instant formatter in WatermarkHold


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

Branch: refs/heads/master
Commit: fa4958a6140eb00ceee08b2468f7d88f17538794
Parents: 280a6a8
Author: Kenneth Knowles 
Authored: Mon Dec 19 20:40:47 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../apache/beam/runners/core/WatermarkHold.java  |  4 +++-
 .../sdk/transforms/windowing/BoundedWindow.java  | 19 +++
 2 files changed, 22 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fa4958a6/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
index 7f1afcc..5e5f44d 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
@@ -207,7 +207,9 @@ class WatermarkHold implements 
Serializable {
 Instant shifted = 
windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window);
 checkState(!shifted.isBefore(timestamp),
 "OutputTimeFn moved element from %s to earlier time %s for window %s",
-timestamp, shifted, window);
+BoundedWindow.formatTimestamp(timestamp),
+BoundedWindow.formatTimestamp(shifted),
+window);
 checkState(timestamp.isAfter(window.maxTimestamp())
 || !shifted.isAfter(window.maxTimestamp()),
 "OutputTimeFn moved element from %s to %s which is beyond end of "

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fa4958a6/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
index 6da2495..74223b5 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
@@ -34,11 +34,30 @@ import org.joda.time.Instant;
 public abstract class BoundedWindow {
   // The min and max timestamps that won't overflow when they are converted to
   // usec.
+
+  /**
+   * The minimum value for any Beam timestamp. Often referred to as 
"-infinity".
+   *
+   * This value and {@link #TIMESTAMP_MAX_VALUE} are chosen so that their
+   * microseconds-since-epoch can be safely represented with a {@code long}.
+   */
   public static final Instant TIMESTAMP_MIN_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
+
+  /**
+   * The maximum value for any Beam timestamp. Often referred to as 
"+infinity".
+   *
+   * This value and {@link #TIMESTAMP_MIN_VALUE} are chosen so that their
+   * microseconds-since-epoch can be safely represented with a {@code long}.
+   */
   public static final Instant TIMESTAMP_MAX_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
 
+  /**
+   * Formats a {@link Instant} timestamp with additional Beam-specific 
metadata, such as indicating
+   * whether the timestamp is the end of the global window or one of the 
distinguished values {@link
+   * #TIMESTAMP_MIN_VALUE} or {@link #TIMESTAMP_MIN_VALUE}.
+   */
   public static String formatTimestamp(Instant timestamp) {
 if (timestamp.equals(TIMESTAMP_MIN_VALUE)) {
   return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)";



[7/7] incubator-beam git commit: This closes #1669: Preliminaries for timers in the direct runner

2016-12-21 Thread kenn
This closes #1669: Preliminaries for timers in the direct runner

  Hold output watermark according to pending timers
  Allow setting timer by ID in DirectTimerInternals
  Add UsesTestStream for use with JUnit @Category
  Add static Window.withOutputTimeFn to match build method
  Use informative Instant formatter in WatermarkHold
  Add informative Instant formatter to BoundedWindow


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

Branch: refs/heads/master
Commit: 57d9bbd797edfcf32fdd9284b802fc4f9694e8d2
Parents: ff39516 dfe2e62
Author: Kenneth Knowles 
Authored: Wed Dec 21 13:46:34 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:46:34 2016 -0800

--
 .../apache/beam/runners/core/WatermarkHold.java |  4 +-
 .../runners/direct/DirectTimerInternals.java|  2 +-
 .../beam/runners/direct/WatermarkManager.java   | 78 ++--
 .../apache/beam/sdk/testing/UsesTestStream.java | 24 ++
 .../sdk/transforms/windowing/BoundedWindow.java | 31 
 .../beam/sdk/transforms/windowing/Window.java   |  9 +++
 .../apache/beam/sdk/testing/TestStreamTest.java | 12 +--
 7 files changed, 144 insertions(+), 16 deletions(-)
--




[4/7] incubator-beam git commit: Allow setting timer by ID in DirectTimerInternals

2016-12-21 Thread kenn
Allow setting timer by ID in DirectTimerInternals


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

Branch: refs/heads/master
Commit: 7f14c463acd2ae5b86ac81a9528ac4aa7dff765f
Parents: 4d71924
Author: Kenneth Knowles 
Authored: Wed Dec 7 20:18:44 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:37 2016 -0800

--
 .../runners/direct/DirectTimerInternals.java|  2 +-
 .../beam/runners/direct/WatermarkManager.java   | 25 
 2 files changed, 26 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f14c463/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 5ca276d..80e0721 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -49,7 +49,7 @@ class DirectTimerInternals implements TimerInternals {
   @Override
   public void setTimer(StateNamespace namespace, String timerId, Instant 
target,
   TimeDomain timeDomain) {
-throw new UnsupportedOperationException("Setting timer by ID not yet 
supported.");
+timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, 
timeDomain));
   }
 
   @Deprecated

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f14c463/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index 7bed751..f7bafd1 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -23,11 +23,13 @@ import com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.SortedMultiset;
+import com.google.common.collect.Table;
 import com.google.common.collect.TreeMultiset;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -56,6 +58,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.joda.time.Instant;
@@ -210,6 +213,10 @@ public class WatermarkManager {
 private final SortedMultiset pendingElements;
 private final Map objectTimers;
 
+// Entries in this table represent the authoritative timestamp for which
+// a per-key-and-StateNamespace timer is set.
+private final Map> existingTimers;
+
 private AtomicReference currentWatermark;
 
 public AppliedPTransformInputWatermark(Collection 
inputWatermarks) {
@@ -222,6 +229,7 @@ public class WatermarkManager {
   this.pendingElements =
   TreeMultiset.create(pendingBundleComparator);
   this.objectTimers = new HashMap<>();
+  this.existingTimers = new HashMap<>();
   currentWatermark = new 
AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
 }
 
@@ -276,14 +284,31 @@ public class WatermarkManager {
 keyTimers = new TreeSet<>();
 objectTimers.put(update.key, keyTimers);
   }
+  Table existingTimersForKey =
+  existingTimers.get(update.key);
+  if (existingTimersForKey == null) {
+existingTimersForKey = HashBasedTable.create();
+existingTimers.put(update.key, existingTimersForKey);
+  }
+

[1/7] incubator-beam git commit: Add informative Instant formatter to BoundedWindow

2016-12-21 Thread kenn
Repository: incubator-beam
Updated Branches:
  refs/heads/master ff3951699 -> 57d9bbd79


Add informative Instant formatter to BoundedWindow


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

Branch: refs/heads/master
Commit: 280a6a8f729cb382616ad65f71860b61277cbd6f
Parents: ff39516
Author: Kenneth Knowles 
Authored: Mon Dec 19 20:40:11 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:45:36 2016 -0800

--
 .../beam/sdk/transforms/windowing/BoundedWindow.java| 12 
 1 file changed, 12 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/280a6a8f/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
index 3654074..6da2495 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java
@@ -39,6 +39,18 @@ public abstract class BoundedWindow {
   public static final Instant TIMESTAMP_MAX_VALUE =
   new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
 
+  public static String formatTimestamp(Instant timestamp) {
+if (timestamp.equals(TIMESTAMP_MIN_VALUE)) {
+  return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)";
+} else if (timestamp.equals(TIMESTAMP_MAX_VALUE)) {
+  return timestamp.toString() + " (TIMESTAMP_MAX_VALUE)";
+} else if (timestamp.equals(GlobalWindow.INSTANCE.maxTimestamp())) {
+  return timestamp.toString() + " (end of global window)";
+} else {
+  return timestamp.toString();
+}
+  }
+
   /**
* Returns the inclusive upper bound of timestamps for values in this window.
*/



[1/3] incubator-beam git commit: Actually propagate and commit state in direct runner

2016-12-21 Thread kenn
Repository: incubator-beam
Updated Branches:
  refs/heads/master 7ee8c86d3 -> ff3951699


Actually propagate and commit state in direct 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/55176c38
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/55176c38
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/55176c38

Branch: refs/heads/master
Commit: 55176c385cc802be42b5467fbb2dcc9a1c7467ea
Parents: 4fb16e8
Author: Kenneth Knowles 
Authored: Tue Dec 20 15:59:45 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:22 2016 -0800

--
 .../apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java   | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/55176c38/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
index 5f9d8f4..003df0f 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
@@ -233,6 +233,7 @@ final class StatefulParDoEvaluatorFactory implements Transfo
   StepTransformResult.>>withHold(
   delegateResult.getTransform(), 
delegateResult.getWatermarkHold())
   .withTimerUpdate(delegateResult.getTimerUpdate())
+  .withState(delegateResult.getState())
   .withAggregatorChanges(delegateResult.getAggregatorChanges())
   .withMetricUpdates(delegateResult.getLogicalMetricUpdates())
   
.addOutput(Lists.newArrayList(delegateResult.getOutputBundles()));



[3/3] incubator-beam git commit: This closes #1670: Fixes for direct runner expansion and evaluation of stateful ParDo

2016-12-21 Thread kenn
This closes #1670: Fixes for direct runner expansion and evaluation of stateful 
ParDo

  Actually propagate and commit state in direct runner
  Fix windowing in direct runner Stateful ParDo


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

Branch: refs/heads/master
Commit: ff395169993d84cd920be21f11d9af8f8d8b
Parents: 7ee8c86 55176c3
Author: Kenneth Knowles 
Authored: Wed Dec 21 13:11:54 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:54 2016 -0800

--
 .../direct/ParDoMultiOverrideFactory.java   | 34 ++--
 .../direct/StatefulParDoEvaluatorFactory.java   |  1 +
 2 files changed, 32 insertions(+), 3 deletions(-)
--




[2/3] incubator-beam git commit: Fix windowing in direct runner Stateful ParDo

2016-12-21 Thread kenn
Fix windowing in direct runner Stateful ParDo


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

Branch: refs/heads/master
Commit: 4fb16e8fb9bb087c0975f38c54665634868cfed7
Parents: 7ee8c86
Author: Kenneth Knowles 
Authored: Tue Dec 20 13:58:29 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Dec 21 13:11:22 2016 -0800

--
 .../direct/ParDoMultiOverrideFactory.java   | 34 ++--
 1 file changed, 31 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4fb16e8f/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
--
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index 2cea999..b35df87 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -34,8 +34,13 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
@@ -92,9 +97,12 @@ class ParDoMultiOverrideFactory
 @Override
 public PCollectionTuple expand(PCollection> input) {
 
+  WindowingStrategy inputWindowingStrategy = 
input.getWindowingStrategy();
+
   // A KvCoder is required since this goes through GBK. Further, 
WindowedValueCoder
   // is not registered by default, so we explicitly set the relevant 
coders.
-  checkState(input.getCoder() instanceof KvCoder,
+  checkState(
+  input.getCoder() instanceof KvCoder,
   "Input to a %s using state requires a %s, but the coder was %s",
   ParDo.class.getSimpleName(),
   KvCoder.class.getSimpleName(),
@@ -102,14 +110,27 @@ class ParDoMultiOverrideFactory
   KvCoder kvCoder = (KvCoder) input.getCoder();
   Coder keyCoder = kvCoder.getKeyCoder();
   Coder windowCoder =
-  input.getWindowingStrategy().getWindowFn().windowCoder();
+  inputWindowingStrategy.getWindowFn().windowCoder();
 
-  PCollectionTuple outputs =
+  PCollection>> adjustedInput =
   input
   // Stash the original timestamps, etc, for when it is fed to the 
user's DoFn
   .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn()))
   .setCoder(KvCoder.of(keyCoder, 
WindowedValue.getFullCoder(kvCoder, windowCoder)))
 
+  // We are going to GBK to gather keys and windows but otherwise 
do not want
+  // to alter the flow of data. This entails:
+  //  - trigger as fast as possible
+  //  - maintain the full timestamps of elements
+  //  - ensure this GBK holds to the minimum of those timestamps 
(via OutputTimeFn)
+  //  - discard past panes as it is "just a stream" of elements
+  .apply(
+  Window.>>create())
 
@@ -117,6 +138,13 @@ class ParDoMultiOverrideFactory
   .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem()))
   

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

2016-12-21 Thread kenn
[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 
Authored: Wed Dec 21 09:32:35 2016 +0800
Committer: manuzhang 
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 @@
   
 org.apache.beam
 beam-runners-parent
-0.4.0-incubating-SNAPSHOT
+0.5.0-incubating-SNAPSHOT
 ../pom.xml
   
 

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 {
+  static class ExtractWordsFn extends DoFn {
 
-@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, String> {
+  static class FormatAsStringFn extends DoFn, 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 
implements
 JavaStream>> outputStream = 
inputStream.flatMap(
 new DoFnMultiFunction<>(
 context.getPipelineOptions(),
-transform.getNewFn(),
+transform.getFn(),
 

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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 11:50:01 2016 -0800
Committer: Kenneth Knowles 
Committed: Mon Dec 19 11:50:01 2016 -0800

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




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

2016-12-21 Thread kenn
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 
Authored: Wed Dec 14 15:46:50 2016 -0800
Committer: bchambers 
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;
  *
  * 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 {
+public class CounterCell implements MetricCell, 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.
+ *
+ * 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 {
+public class DistributionCell implements MetricCell, Distribution {
 
   private final DirtyState dirty = new DirtyState();
   private final AtomicReference value =
   new AtomicReference(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, 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 

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

2016-12-21 Thread kenn
[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 
Authored: Thu Dec 15 22:06:18 2016 +0100
Committer: Aljoscha Krettek 
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 @@
 
   
 
-kinglear.txt
-wordcounts.txt
--1
+kinglear.txt
+wordcounts.txt
+-1
   
 
   
@@ -112,9 +112,9 @@
   java
   
 
--runner=org.apache.beam.runners.flink.FlinkRunner
---parallelism=${parallelism}
---input=${input}
---output=${output}
+--parallelism=${flink.examples.parallelism}
+--input=${flink.examples.input}
+--output=${flink.examples.output}
   
 
   



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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 20:57:06 2016 -0800
Committer: Kenneth Knowles 
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.emptyList(),
@@ -98,7 +99,7 @@ public class SimpleDoFnRunnerTest {
 new SimpleDoFnRunner<>(
 null,
 fn,
-null,
+NullSideInputReader.empty(),
 null,
 null,
 Collections.emptyList(),
@@ -129,7 +130,7 @@ public class SimpleDoFnRunnerTest {
 new SimpleDoFnRunner<>(
 null,
 fn,
-null,
+NullSideInputReader.empty(),
 null,
 null,
 Collections.emptyList(),
@@ -158,7 +159,7 @@ public class SimpleDoFnRunnerTest {
 new SimpleDoFnRunner<>(
 null,
 fn,
-null,
+NullSideInputReader.empty(),
 null,
 null,
 Collections.emptyList(),
@@ -179,7 +180,7 @@ public class SimpleDoFnRunnerTest {
 new SimpleDoFnRunner<>(
 null,
 fn,
-null,
+NullSideInputReader.empty(),
 null,
 null,
 Collections.emptyList(),
@@ -206,7 +207,7 @@ public class SimpleDoFnRunnerTest {
 new SimpleDoFnRunner<>(
 null,
 fn,
-null,
+NullSideInputReader.empty(),
 null,
 null,
 Collections.emptyList(),



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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 14:26:35 2016 -0800
Committer: bchambers 
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(-)
--




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

2016-12-21 Thread kenn
[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 
Authored: Mon Dec 19 15:21:02 2016 -0800
Committer: Luke Cwik 
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(+)
--




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

2016-12-21 Thread kenn
[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 
Authored: Wed Dec 7 17:35:23 2016 -0800
Committer: Luke Cwik 
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 000..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.
+ *
+ * It defines APIs for writing file systems agnostic code.
+ *
+ * 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 000..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 

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

2016-12-21 Thread kenn
[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é 
Authored: Mon Dec 19 07:40:39 2016 +0100
Committer: Jean-Baptiste Onofré 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Thu Dec 15 19:51:22 2016 -0800
Committer: Kenneth Knowles 
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=-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)



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

2016-12-21 Thread kenn
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 
Authored: Sat Dec 17 11:12:12 2016 -0800
Committer: Joshua Litt 
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 {
   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 {
   }
 
   /**
+   * 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 {
   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 

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

2016-12-21 Thread kenn
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 
Authored: Thu Dec 15 17:04:59 2016 -0800
Committer: bchambers 
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 @@
   
 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
   
   none
   true

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() {
-  @ProcessElement
-  public void processElement(ProcessContext c) {
-Counter count = Metrics.counter(DirectRunnerTest.class, "count");
-Distribution values = Metrics.distribution(DirectRunnerTest.class, 
"input");
-
-

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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 16:24:09 2016 -0800
Committer: Dan Halperin 
Committed: Mon Dec 19 16:24:09 2016 -0800

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




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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 23:53:49 2016 -0800
Committer: Dan Halperin 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 12:09:03 2016 +0100
Committer: Aljoscha Krettek 
Committed: Mon Dec 19 12:09:03 2016 +0100

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




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

2016-12-21 Thread kenn
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 

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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 11:18:59 2016 -0800
Committer: Kenneth Knowles 
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 {



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

2016-12-21 Thread kenn
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 
Authored: Sat Dec 17 11:12:12 2016 -0800
Committer: bchambers 
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 {
   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 {
   }
 
   /**
+   * 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 {
   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 

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

2016-12-21 Thread kenn
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 
Authored: Wed Nov 23 14:21:40 2016 -0800
Committer: Kenneth Knowles 
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 {
   void processElement(WindowedValue 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

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

2016-12-21 Thread kenn
[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 
Authored: Wed Dec 14 12:20:08 2016 +0200
Committer: Kenneth Knowles 
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 
{
 
   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 {
 return new TestSparkRunner(sparkOptions);
   }
 
+  /**
+   * Overrides for the test runner.
+   */
+  @SuppressWarnings("unchecked")
   @Override
-  public 
-  OutputT apply(PTransform transform, InputT input) {
-return delegate.apply(transform, input);
-  };
+  public  OutputT apply(
+  PTransform transform, InputT input) {
+// if the pipeline forces execution as a streaming pipeline,
+// and the source is an adapted 

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

2016-12-21 Thread kenn
[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 
Authored: Sun Dec 18 21:01:13 2016 +0100
Committer: Ismaël Mejía 
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 {
-  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 {
   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 {
 
 // 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 {
 
 // Sanity check.
 if (!range.containsKey(splitPosition)) {
-  logger.warn(
+  LOG.warn(
   "{}: Rejecting split request at {} because it is not within the 

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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 16:26:27 2016 -0800
Committer: Dan Halperin 
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 PCollectionKVK V} as
+   * {@link PCollection PCollectionKVK, V} as
* input and produces a {@link PCollectionView} mapping each window to
-   * a {@link Map MapK, V}. It is required that each key of the input 
be
+   * a {@link Map MapK, V}. 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()}.



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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 17:41:51 2016 -0800
Committer: Dan Halperin 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 15:26:28 2016 -0800
Committer: Eugene Kirpichov 
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  OldDoFn 
toOldDoFn(DoFn fn) {
@@ -64,6 +79,126 @@ public class DoFnAdapters {
 }
   }
 
+  /** Creates a {@link OldDoFn.ProcessContext} from a {@link 
DoFn.ProcessContext}. */
+  public static  OldDoFn.ProcessContext 
adaptProcessContext(
+  OldDoFn fn,
+  final DoFn.ProcessContext c,
+  final DoFnInvoker.ArgumentProvider extra) {
+return fn.new ProcessContext() {
+  @Override
+  public InputT element() {
+return c.element();
+  }
+
+  @Override
+  public  T sideInput(PCollectionView 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 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  void sideOutput(TupleTag tag, T output) {
+c.sideOutput(tag, output);
+  }
+
+  @Override
+  public  void sideOutputWithTimestamp(TupleTag tag, T output, 
Instant timestamp) {
+c.sideOutputWithTimestamp(tag, output, timestamp);
+  }
+
+  @Override
+  protected  Aggregator 
createAggregatorInternal(
+  String name, 

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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 11:25:48 2016 -0800
Committer: Kenneth Knowles 
Committed: Mon Dec 19 11:25:48 2016 -0800

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




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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 16:26:43 2016 -0800
Committer: bchambers 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 11:39:29 2016 -0800
Committer: Kenneth Knowles 
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=-MM-dd\'T\'HH:mm:ss.SSS')
+
context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=-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)



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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 09:44:13 2016 -0800
Committer: Kenneth Knowles 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 11:30:20 2016 -0800
Committer: bchambers 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 16:33:51 2016 -0800
Committer: Eugene Kirpichov 
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..000
--- 
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 

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

2016-12-21 Thread kenn
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 
Authored: Sun Dec 18 16:02:41 2016 +0100
Committer: Ismaël Mejía 
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);
 



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

2016-12-21 Thread kenn
[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é 
Authored: Sun Dec 11 07:43:41 2016 +0100
Committer: Jean-Baptiste Onofré 
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 @@
   jsr305
 
 
+
+
+  com.google.auto.value
+  auto-value
+  provided
+
+
 
 
   org.apache.activemq

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 {
+  @AutoValue
+  public abstract static class Read extends PTransform {
 
+/**
+ * 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."
+ *
+ * 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();
+}
+
+/**
+ * Specify the JMS connection factory to connect to the JMS broker.
+ *
+ * For instance:
+ *
+ * 
+ *   {@code
+ *
pipeline.apply(JmsIO.read().withConnectionFactory(myConnectionFactory)
+ *   }
+ * 
+ *
+ * @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 

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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 18:46:17 2016 -0800
Committer: Kenneth Knowles 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Wed Dec 7 20:09:06 2016 -0800
Committer: Kenneth Knowles 
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 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 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)) {



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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 18:10:28 2016 -0800
Committer: Dan Halperin 
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 {
 @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 getQueryProivder() {
+  @Nullable
+  public ValueProvider 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> {
 /** TableSpec to write to. */
 private final ValueProvider 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 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 

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

2016-12-21 Thread kenn
[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 
Authored: Tue Dec 13 19:27:41 2016 +0200
Committer: Kenneth Knowles 
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.
  *
- * 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.
+ * 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.
  *
  * In order to run tests on a pipeline runner, the following conditions 
must be met:
+ *
  * 
- *   System property "beamTestPipelineOptions" must contain a JSON 
delimited list of pipeline
- *   options. For example:
- *   {@code [
+ * System property "beamTestPipelineOptions" must contain a JSON delimited 
list of pipeline
+ * options. For example:
+ * {@code [
  * "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner",
  * "--project=mygcpproject",
  * "--stagingLocation=gs://mygcsbucket/path"
  * ]}
  * Note that the set of pipeline options required is pipeline runner 
specific.
- *   
- *   Jars containing the SDK and test classes must be available on the 
classpath.
+ * Jars containing the SDK and test classes must be available on the 
classpath.
  * 
  *
  * Use {@link PAssert} for tests, as it integrates with this test harness 
in both direct and
  * remote execution modes. For example:
+ *
  * {@code
  * Pipeline p = TestPipeline.create();
  * PCollection output = ...
@@ -76,19 +86,136 @@ import org.junit.experimental.categories.Category;
  * p.run();
  * }
  *
- * For pipeline runners, it is required that they must throw an {@link 
AssertionError}
- * containing the message from the {@link PAssert} that failed.
+ * For pipeline runners, it is required that they must throw an {@link 
AssertionError} containing
+ 

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

2016-12-21 Thread kenn
[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 
Authored: Tue Dec 20 15:57:21 2016 +0800
Committer: manuzhang 
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 {
 TranslationContext translationContext = new TranslationContext(streamApp, 
options);
 GearpumpPipelineTranslator translator = new 
GearpumpPipelineTranslator(translationContext);
 translator.translate(pipeline);
-streamApp.run();
+streamApp.submit();
 
 return null;
   }



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

2016-12-21 Thread kenn
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 
Authored: Mon Dec 19 13:47:19 2016 -0800
Committer: Kenneth Knowles 
Committed: Mon Dec 19 13:47:19 2016 -0800

--

--




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

2016-12-21 Thread kenn
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 
Authored: Tue Dec 20 14:55:51 2016 +0800
Committer: manuzhang 
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 +
 

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

2016-12-21 Thread kenn
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 
Authored: Sat Dec 17 12:59:25 2016 -0800
Committer: Kenneth Knowles 
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(-)
--




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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 13:43:54 2016 -0800
Committer: Kenneth Knowles 
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 runner =
+new SimpleDoFnRunner<>(
+null,
+fn,
+null,
+null,
+null,
+Collections.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 runner =
+new SimpleDoFnRunner<>(
+null,
+fn,
+null,
+null,
+null,
+Collections.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;



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

2016-12-21 Thread kenn
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 
Authored: Fri Dec 16 15:56:00 2016 -0800
Committer: Eugene Kirpichov 
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

  1   2   3   4   5   6   7   8   9   10   >