[1/2] beam git commit: Drop late data in Flink runner

2017-02-03 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 4ccbdbc38 -> 72fef99a6


Drop late data in Flink runner


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

Branch: refs/heads/master
Commit: 0454a1897c645f674754bc9ef69dc7bab2b3c3ba
Parents: 7da5a2c
Author: Kenneth Knowles 
Authored: Wed Feb 1 18:25:42 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 1 18:25:42 2017 -0800

--
 .../wrappers/streaming/DoFnOperator.java| 20 ++--
 1 file changed, 18 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/0454a189/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
--
diff --git 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index de0264a..c1d33f7 100644
--- 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -33,6 +33,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.ExecutionContext;
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import 
org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
@@ -234,6 +235,8 @@ public class DoFnOperator
 
 doFnInvoker.invokeSetup();
 
+ExecutionContext.StepContext stepContext = createStepContext();
+
 DoFnRunner doFnRunner = DoFnRunners.simpleRunner(
 serializedOptions.getPipelineOptions(),
 doFn,
@@ -241,13 +244,26 @@ public class DoFnOperator
 outputManager,
 mainOutputTag,
 sideOutputTags,
-createStepContext(),
+stepContext,
 aggregatorFactory,
 windowingStrategy);
 
+if (doFn instanceof GroupAlsoByWindowViaWindowSetNewDoFn) {
+  // When the doFn is this, we know it came from WindowDoFnOperator and
+  //   InputT = KeyedWorkItem
+  //   OutputT = KV
+  //
+  // for some K, V
+
+  doFnRunner = DoFnRunners.lateDataDroppingRunner(
+  (DoFnRunner) doFnRunner,
+  stepContext,
+  windowingStrategy,
+  ((GroupAlsoByWindowViaWindowSetNewDoFn) 
doFn).getDroppedDueToLatenessAggregator());
+}
+
 pushbackDoFnRunner =
 PushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, 
sideInputHandler);
-
   }
 
   @Override



[1/2] beam git commit: Gives the runner access to RestrictionTracker

2017-02-01 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 168d6a18f -> a27d3cfa2


Gives the runner access to RestrictionTracker

Changes the SplittableParDo transform so that ProcessFn uses a
runner-supplied hook to run the @ProcessElement method, giving
it, among other things, the RestrictionTracker, so the runner
can initiate checkpointing/splitting with it at will.

Introduces a default implementation of said hook, which limits
the number of outputs and duration of the call. This implementation
is used in tests and in Direct runner. Dataflow Streaming runner
will also use this implementation, while Dataflow Batch runner
will be more sophisticated.


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

Branch: refs/heads/master
Commit: 7dc9e86fa76e5117bfc6825e120ed74ba3d2f910
Parents: 38208ea
Author: Eugene Kirpichov 
Authored: Fri Nov 18 11:21:19 2016 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 1 19:43:29 2017 -0800

--
 .../apache/beam/runners/core/DoFnAdapters.java  |   4 +-
 ...eBoundedSplittableProcessElementInvoker.java | 285 +++
 .../beam/runners/core/SimpleDoFnRunner.java |   6 +-
 .../beam/runners/core/SplittableParDo.java  | 190 ++---
 .../core/SplittableProcessElementInvoker.java   |  65 +
 ...ndedSplittableProcessElementInvokerTest.java | 146 ++
 .../beam/runners/core/SplittableParDoTest.java  | 147 +++---
 ...littableProcessElementsEvaluatorFactory.java |  66 +++--
 .../apache/beam/sdk/transforms/DoFnTester.java  |  15 +-
 .../sdk/transforms/reflect/DoFnInvoker.java |   4 +-
 .../transforms/splittabledofn/OffsetRange.java  |  71 +
 .../splittabledofn/OffsetRangeTracker.java  |  75 +
 .../splittabledofn/RestrictionTracker.java  |   2 +-
 .../beam/sdk/transforms/SplittableDoFnTest.java |  68 +
 .../transforms/reflect/DoFnInvokersTest.java|   2 +-
 .../splittabledofn/OffsetRangeTrackerTest.java  | 111 
 16 files changed, 956 insertions(+), 301 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/7dc9e86f/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 dcd7969..693cb2f 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
@@ -204,7 +204,7 @@ public class DoFnAdapters {
 }
 
 @Override
-public  RestrictionTracker 
restrictionTracker() {
+public RestrictionTracker restrictionTracker() {
   throw new UnsupportedOperationException("This is a non-splittable DoFn");
 }
 
@@ -306,7 +306,7 @@ public class DoFnAdapters {
 }
 
 @Override
-public  RestrictionTracker 
restrictionTracker() {
+public RestrictionTracker restrictionTracker() {
   throw new UnsupportedOperationException("This is a non-splittable DoFn");
 }
 

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

[1/2] beam git commit: Removes inputProvider() and outputReceiver()

2017-01-31 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 77b136603 -> 38208eaa4


Removes inputProvider() and outputReceiver()

Removes InputProvider itself too.
Does not remove OutputReceiver because it's used in @SplitRestriction
method.
Cleans up tests that looked at InputProvider/OutputReceiver parameters -
instead now they look at DoFn.ProcessContext parameter, and I improved
the formatting of parameter types too.


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

Branch: refs/heads/master
Commit: 7d787bddc80fa832576407c313b26d5436dfa393
Parents: 34b4a6d
Author: Eugene Kirpichov 
Authored: Fri Jan 27 14:56:56 2017 -0800
Committer: Eugene Kirpichov 
Committed: Fri Jan 27 15:00:19 2017 -0800

--
 .../apache/beam/runners/core/DoFnAdapters.java  | 20 -
 .../beam/runners/core/SimpleDoFnRunner.java | 32 
 .../beam/runners/core/SplittableParDo.java  | 12 ---
 .../org/apache/beam/sdk/transforms/DoFn.java|  6 --
 .../apache/beam/sdk/transforms/DoFnTester.java  | 12 ---
 .../reflect/ByteBuddyDoFnInvokerFactory.java| 12 ---
 .../sdk/transforms/reflect/DoFnInvoker.java | 18 -
 .../sdk/transforms/reflect/DoFnSignature.java   | 56 -
 .../sdk/transforms/reflect/DoFnSignatures.java  | 82 +---
 .../beam/sdk/util/common/ReflectHelpers.java| 16 ++--
 .../transforms/reflect/DoFnInvokersTest.java| 26 ---
 .../DoFnSignaturesProcessElementTest.java   | 40 --
 .../DoFnSignaturesSplittableDoFnTest.java   |  3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  6 +-
 14 files changed, 51 insertions(+), 290 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/7d787bdd/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 23aba58..dcd7969 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
@@ -204,16 +204,6 @@ public class DoFnAdapters {
 }
 
 @Override
-public DoFn.InputProvider inputProvider() {
-  throw new UnsupportedOperationException("inputProvider() exists only for 
testing");
-}
-
-@Override
-public DoFn.OutputReceiver outputReceiver() {
-  throw new UnsupportedOperationException("outputReceiver() exists only 
for testing");
-}
-
-@Override
 public  RestrictionTracker 
restrictionTracker() {
   throw new UnsupportedOperationException("This is a non-splittable DoFn");
 }
@@ -316,16 +306,6 @@ public class DoFnAdapters {
 }
 
 @Override
-public DoFn.InputProvider inputProvider() {
-  throw new UnsupportedOperationException("inputProvider() exists only for 
testing");
-}
-
-@Override
-public DoFn.OutputReceiver outputReceiver() {
-  throw new UnsupportedOperationException("outputReceiver() exists only 
for testing");
-}
-
-@Override
 public  RestrictionTracker 
restrictionTracker() {
   throw new UnsupportedOperationException("This is a non-splittable DoFn");
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/7d787bdd/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index df5f3f6..d54daf6 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -34,9 +34,7 @@ import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.Context;
-import org.apache.beam.sdk.transforms.DoFn.InputProvider;
 import org.apache.beam.sdk.transforms.DoFn.OnTimerContext;
-import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
 import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
@@ -439,16 +437,6 @@ public class SimpleDoFnRunner implements 
DoFnRunner

[2/2] beam git commit: This closes #1860: Removes inputProvider() and outputReceiver()

2017-01-31 Thread kenn
This closes #1860: Removes inputProvider() and outputReceiver()


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

Branch: refs/heads/master
Commit: 38208eaa4620f5dfb9bbec9b73c25cc50fd95274
Parents: 77b1366 7d787bd
Author: Kenneth Knowles 
Authored: Tue Jan 31 18:16:00 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Jan 31 18:16:00 2017 -0800

--
 .../apache/beam/runners/core/DoFnAdapters.java  | 20 -
 .../beam/runners/core/SimpleDoFnRunner.java | 32 
 .../beam/runners/core/SplittableParDo.java  | 12 ---
 .../org/apache/beam/sdk/transforms/DoFn.java|  6 --
 .../apache/beam/sdk/transforms/DoFnTester.java  | 12 ---
 .../reflect/ByteBuddyDoFnInvokerFactory.java| 12 ---
 .../sdk/transforms/reflect/DoFnInvoker.java | 18 -
 .../sdk/transforms/reflect/DoFnSignature.java   | 56 -
 .../sdk/transforms/reflect/DoFnSignatures.java  | 82 +---
 .../beam/sdk/util/common/ReflectHelpers.java| 16 ++--
 .../transforms/reflect/DoFnInvokersTest.java| 26 ---
 .../DoFnSignaturesProcessElementTest.java   | 40 --
 .../DoFnSignaturesSplittableDoFnTest.java   |  3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  6 +-
 14 files changed, 51 insertions(+), 290 deletions(-)
--




[2/2] beam git commit: This closes #1748: Add some more RunnableOnService tests for stateful ParDo

2017-02-06 Thread kenn
This closes #1748: Add some more RunnableOnService tests for stateful ParDo


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

Branch: refs/heads/master
Commit: a26fd1ff34bb5a51d6cc82ab9eb3c7e40cef49d3
Parents: 0c24286 6c00e91
Author: Kenneth Knowles 
Authored: Mon Feb 6 15:38:43 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 15:38:43 2017 -0800

--
 .../apache/beam/sdk/transforms/ParDoTest.java   | 138 +++
 1 file changed, 138 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/a26fd1ff/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
--



[1/2] beam git commit: Add some more RunnableOnService tests for stateful ParDo

2017-02-06 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 0c24286e1 -> a26fd1ff3


Add some more RunnableOnService tests for stateful ParDo


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

Branch: refs/heads/master
Commit: 6c00e9121e6572fc06d0379802883c118acbed9f
Parents: f4e1097
Author: Kenneth Knowles 
Authored: Fri Jan 6 12:03:11 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 15:21:43 2017 -0800

--
 .../apache/beam/sdk/transforms/ParDoTest.java   | 138 +++
 1 file changed, 138 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/6c00e912/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 2e3fb85..7381e06 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -80,6 +80,7 @@ import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerSpec;
 import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.StateSpec;
 import org.apache.beam.sdk.util.state.StateSpecs;
@@ -1502,6 +1503,104 @@ public class ParDoTest implements Serializable {
 
   @Test
   @Category({RunnableOnService.class, UsesStatefulParDo.class})
+  public void testValueStateFixedWindows() {
+final String stateId = "foo";
+
+DoFn, Integer> fn =
+new DoFn, Integer>() {
+
+  @StateId(stateId)
+  private final StateSpec intState =
+  StateSpecs.value(VarIntCoder.of());
+
+  @ProcessElement
+  public void processElement(
+  ProcessContext c, @StateId(stateId) ValueState state) {
+Integer currentValue = MoreObjects.firstNonNull(state.read(), 0);
+c.output(currentValue);
+state.write(currentValue + 1);
+  }
+};
+
+IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new 
Instant(10));
+IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new 
Instant(20));
+
+PCollection output =
+pipeline
+.apply(
+Create.timestamped(
+// first window
+TimestampedValue.of(KV.of("hello", 7), new Instant(1)),
+TimestampedValue.of(KV.of("hello", 14), new Instant(2)),
+TimestampedValue.of(KV.of("hello", 21), new Instant(3)),
+
+// second window
+TimestampedValue.of(KV.of("hello", 28), new Instant(11)),
+TimestampedValue.of(KV.of("hello", 35), new Instant(13
+.apply(Window.>into(FixedWindows.of(Duration.millis(10
+.apply("Stateful ParDo", ParDo.of(fn));
+
+PAssert.that(output).inWindow(firstWindow).containsInAnyOrder(0, 1, 2);
+PAssert.that(output).inWindow(secondWindow).containsInAnyOrder(0, 1);
+pipeline.run();
+  }
+
+  /**
+   * Tests that there is no state bleeding between adjacent stateful {@link 
ParDo} transforms,
+   * which may (or may not) be executed in similar contexts after runner 
optimizations.
+   */
+  @Test
+  @Category({RunnableOnService.class, UsesStatefulParDo.class})
+  public void testValueStateSameId() {
+final String stateId = "foo";
+
+DoFn, KV> fn =
+new DoFn, KV>() {
+
+  @StateId(stateId)
+  private final StateSpec intState =
+  StateSpecs.value(VarIntCoder.of());
+
+  @ProcessElement
+  public void processElement(
+  ProcessContext c, @StateId(stateId) ValueState state) {
+Integer currentValue = MoreObjects.firstNonNull(state.read(), 0);
+c.output(KV.of("sizzle", currentValue));
+state.write(currentValue + 1);
+  }
+};
+
+DoFn, Integer> fn2 =
+new DoFn, Integer>() {
+
+  @StateId(stateId)
+  private final StateSpec intState =
+  StateSpecs.value(VarIntCoder.of());
+
+ 

[1/2] beam git commit: Add wrapping of lambda in a SimpleFunction

2017-02-07 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master ed7b82e7e -> 144bffd40


Add wrapping of lambda in a SimpleFunction


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

Branch: refs/heads/master
Commit: 23152178f81e635db65a7aae71f47fa67b3dc065
Parents: ed7b82e
Author: Kenneth Knowles 
Authored: Thu Jan 26 11:19:42 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 7 08:16:39 2017 -0800

--
 .../beam/sdk/transforms/SimpleFunction.java | 44 +++--
 .../beam/sdk/transforms/SimpleFunctionTest.java | 43 
 .../sdk/transforms/MapElementsJava8Test.java| 24 ++-
 .../sdk/transforms/SimpleFunctionJava8Test.java | 69 
 4 files changed, 170 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/23152178/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
index 8604659..db44380 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import java.lang.reflect.Method;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.values.TypeDescriptor;
@@ -29,6 +31,40 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 public abstract class SimpleFunction
 implements SerializableFunction, HasDisplayData {
 
+  @Nullable
+  private final SerializableFunction fn;
+
+  protected SimpleFunction() {
+this.fn = null;
+// A subclass must override apply if using this constructor. Check that via
+// reflection.
+try {
+  Method methodThatMustBeOverridden =
+  SimpleFunction.class.getDeclaredMethod("apply", new Class[] 
{Object.class});
+  Method methodOnSubclass =
+  getClass().getMethod("apply", new Class[] {Object.class});
+
+  if (methodOnSubclass.equals(methodThatMustBeOverridden)) {
+throw new IllegalStateException(
+"Subclass of SimpleFunction must override 'apply' method"
++ " or pass a SerializableFunction to the constructor,"
++ " usually via a lambda or method reference.");
+  }
+
+} catch (NoSuchMethodException exc) {
+  throw new RuntimeException("Impossible state: missing 'apply' method 
entirely", exc);
+}
+  }
+
+  protected SimpleFunction(SerializableFunction fn) {
+this.fn = fn;
+  }
+
+  @Override
+  public OutputT apply(InputT input) {
+return fn.apply(input);
+  }
+
   public static 
   SimpleFunction fromSerializableFunctionWithOutputType(
   SerializableFunction fn, TypeDescriptor 
outputType) {
@@ -77,23 +113,17 @@ public abstract class SimpleFunction
   private static class SimpleFunctionWithOutputType
   extends SimpleFunction {
 
-private final SerializableFunction fn;
 private final TypeDescriptor outputType;
 
 public SimpleFunctionWithOutputType(
 SerializableFunction fn,
 TypeDescriptor outputType) {
-  this.fn = fn;
+  super(fn);
   this.outputType = outputType;
 }
 
 
 @Override
-public OutputT apply(InputT input) {
-  return fn.apply(input);
-}
-
-@Override
 public TypeDescriptor getOutputTypeDescriptor() {
   return outputType;
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/23152178/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleFunctionTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleFunctionTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleFunctionTest.java
new file mode 100644
index 000..bcfb558
--- /dev/null
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleFunctionTest.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this 

[2/2] beam git commit: This closes #1855: Add wrapping of lambda in a SimpleFunction

2017-02-07 Thread kenn
This closes #1855: Add wrapping of lambda in a SimpleFunction


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

Branch: refs/heads/master
Commit: 144bffd4024decbc586bf16efe05caeda9427d77
Parents: ed7b82e 2315217
Author: Kenneth Knowles 
Authored: Tue Feb 7 08:18:16 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 7 08:18:16 2017 -0800

--
 .../beam/sdk/transforms/SimpleFunction.java | 44 +++--
 .../beam/sdk/transforms/SimpleFunctionTest.java | 43 
 .../sdk/transforms/MapElementsJava8Test.java| 24 ++-
 .../sdk/transforms/SimpleFunctionJava8Test.java | 69 
 4 files changed, 170 insertions(+), 10 deletions(-)
--




[1/2] beam git commit: [BEAM-65] ProcessFn: support setup/teardown

2017-02-06 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 224e44765 -> f4e109767


[BEAM-65] ProcessFn: support setup/teardown

Previously, ProcessFn did not explicitly invoke the underlying fn's
@Setup and @Teardown methods - it was assuming that those methods would
get invoked on that fn externally. This was true in direct runner,
but is not necessarily true in other runners: e.g., Dataflow runner
will serialize the whole ProcessFn and treat it mostly as a regular
DoFn, so it makes more sense to have lifecycle methods of ProcessFn
delegate to the underlying fn.

Also, adds a getter for fn (a runner may need it to create a proper
ProcessContext when creating the SplittableProcessElementInvoker).


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

Branch: refs/heads/master
Commit: b6877ce1739650e7e7593c1d2c2858d60d2393fb
Parents: 4ccbdbc
Author: Eugene Kirpichov 
Authored: Wed Feb 1 16:07:44 2017 -0800
Committer: Eugene Kirpichov 
Committed: Fri Feb 3 10:41:47 2017 -0800

--
 .../beam/runners/core/SplittableParDo.java  | 16 -
 .../beam/runners/core/SplittableParDoTest.java  | 73 +++-
 .../runners/direct/ParDoEvaluatorFactory.java   |  5 +-
 ...littableProcessElementsEvaluatorFactory.java | 11 ++-
 4 files changed, 96 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/b6877ce1/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
index 78acb19..664f334 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
@@ -375,7 +375,6 @@ public class SplittableParDo
 Coder restrictionCoder,
 Coder windowCoder) {
   this.fn = fn;
-  this.invoker = DoFnInvokers.invokerFor(fn);
   this.windowCoder = windowCoder;
   this.elementTag =
   StateTags.value("element", WindowedValue.getFullCoder(elementCoder, 
this.windowCoder));
@@ -395,6 +394,21 @@ public class SplittableParDo
   this.processElementInvoker = invoker;
 }
 
+public DoFn getFn() {
+  return fn;
+}
+
+@Setup
+public void setup() throws Exception {
+  invoker = DoFnInvokers.invokerFor(fn);
+  invoker.invokeSetup();
+}
+
+@Teardown
+public void tearDown() throws Exception {
+  invoker.invokeTeardown();
+}
+
 @StartBundle
 public void startBundle(Context c) throws Exception {
   invoker.invokeStartBundle(wrapContext(c));

http://git-wip-us.apache.org/repos/asf/beam/blob/b6877ce1/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 bb7fd8c..96d65ae 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
@@ -200,7 +200,8 @@ public class SplittableParDoTest {
* {@link DoFn.ProcessElement} calls).
*/
   private static class ProcessFnTester<
-  InputT, OutputT, RestrictionT, TrackerT extends 
RestrictionTracker> {
+  InputT, OutputT, RestrictionT, TrackerT extends 
RestrictionTracker>
+  implements AutoCloseable {
 private final DoFnTester<
 KeyedWorkItem>, OutputT>
 tester;
@@ -270,6 +271,11 @@ public class SplittableParDoTest {
   this.currentProcessingTime = currentProcessingTime;
 }
 
+@Override
+public void close() throws Exception {
+  tester.close();
+}
+
 /** Performs a seed {@link DoFn.ProcessElement} call feeding the element 
and restriction. */
 void startElement(InputT element, RestrictionT restriction) throws 
Exception {
   startElement(
@@ -633,4 +639,69 @@ public class SplittableParDoTest {
 Instant.now().getMillis() - base.getMillis(),
 greaterThanOrEqualTo(maxBundleDuration.getMillis()));
   }
+
+  private static class LifecycleVerifyingFn extends 

[2/2] beam git commit: This closes #1895: ProcessFn: support setup/teardown

2017-02-06 Thread kenn
This closes #1895: ProcessFn: support setup/teardown


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

Branch: refs/heads/master
Commit: f4e10976789aa5fe4b350be21df29544fa626251
Parents: 224e447 b6877ce
Author: Kenneth Knowles 
Authored: Mon Feb 6 14:50:36 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 14:50:36 2017 -0800

--
 .../beam/runners/core/SplittableParDo.java  | 16 -
 .../beam/runners/core/SplittableParDoTest.java  | 73 +++-
 .../runners/direct/ParDoEvaluatorFactory.java   |  5 +-
 ...littableProcessElementsEvaluatorFactory.java | 11 ++-
 4 files changed, 96 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/f4e10976/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
--

http://git-wip-us.apache.org/repos/asf/beam/blob/f4e10976/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
--

http://git-wip-us.apache.org/repos/asf/beam/blob/f4e10976/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
--



[1/2] beam git commit: Allow absolute timers

2017-02-06 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master f4e109767 -> 0c24286e1


Allow absolute timers


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

Branch: refs/heads/master
Commit: a93c5c0594dcd4519fcf4b842f2fe0b8244a81a3
Parents: f4e1097
Author: Kenneth Knowles 
Authored: Mon Jan 23 20:50:50 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 15:17:23 2017 -0800

--
 .../beam/runners/core/SimpleDoFnRunner.java | 83 +++--
 .../java/org/apache/beam/sdk/util/Timer.java| 11 +++
 .../apache/beam/sdk/transforms/ParDoTest.java   | 93 
 3 files changed, 178 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/a93c5c05/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index 8c9b8b7..7a89389 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.runners.core;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.common.collect.Iterables;
@@ -94,6 +95,8 @@ public class SimpleDoFnRunner implements 
DoFnRunner implements 
DoFnRunner(
@@ -182,7 +186,8 @@ public class SimpleDoFnRunner implements 
DoFnRunner argumentProvider =
-new OnTimerArgumentProvider<>(fn, context, window, effectiveTimestamp, 
timeDomain);
+new OnTimerArgumentProvider<>(
+fn, context, window, allowedLateness, effectiveTimestamp, 
timeDomain);
 invoker.invokeOnTimer(timerId, argumentProvider);
   }
 
@@ -210,7 +215,7 @@ public class SimpleDoFnRunner implements 
DoFnRunner 
createProcessContext(WindowedValue elem) {
-return new DoFnProcessContext(fn, context, elem);
+return new DoFnProcessContext(fn, context, elem, 
allowedLateness);
   }
 
   private RuntimeException wrapUserCodeException(Throwable t) {
@@ -465,6 +470,7 @@ public class SimpleDoFnRunner implements 
DoFnRunner fn;
 final DoFnContext context;
 final WindowedValue windowedValue;
+private final Duration allowedLateness;
 
 /** Lazily initialized; should only be accessed via {@link 
#getNamespace()}. */
 @Nullable private StateNamespace namespace;
@@ -486,11 +492,13 @@ public class SimpleDoFnRunner implements 
DoFnRunner fn,
 DoFnContext context,
-WindowedValue windowedValue) {
+WindowedValue windowedValue,
+Duration allowedLateness) {
   fn.super();
   this.fn = fn;
   this.context = context;
   this.windowedValue = windowedValue;
+  this.allowedLateness = allowedLateness;
 }
 
 @Override
@@ -633,7 +641,8 @@ public class SimpleDoFnRunner implements 
DoFnRunner

[2/2] beam git commit: This closes #1827: Allow absolute timers

2017-02-06 Thread kenn
This closes #1827: Allow absolute timers


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

Branch: refs/heads/master
Commit: 0c24286e1bd586b9202b170dbca74188383aa13d
Parents: f4e1097 a93c5c0
Author: Kenneth Knowles 
Authored: Mon Feb 6 15:17:45 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 15:17:45 2017 -0800

--
 .../beam/runners/core/SimpleDoFnRunner.java | 83 +++--
 .../java/org/apache/beam/sdk/util/Timer.java| 11 +++
 .../apache/beam/sdk/transforms/ParDoTest.java   | 93 
 3 files changed, 178 insertions(+), 9 deletions(-)
--




[1/2] beam git commit: [BEAM-1434] Add new beam-sdk-java-core test deps to test scope for DataflowRunner

2017-02-08 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master df2774cd8 -> cf1925fba


[BEAM-1434] Add new beam-sdk-java-core test deps to test scope for 
DataflowRunner

Since test dependencies are not transitively inherited, this is needed to 
prevent test-time crashes.

Missed in #1948.


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

Branch: refs/heads/master
Commit: f3c099b6609ee751da9affc2343129bfbebdbd96
Parents: df2774c
Author: Dan Halperin 
Authored: Wed Feb 8 01:37:02 2017 -0800
Committer: Dan Halperin 
Committed: Wed Feb 8 01:37:14 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 14 ++
 1 file changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/f3c099b6/runners/google-cloud-dataflow-java/pom.xml
--
diff --git a/runners/google-cloud-dataflow-java/pom.xml 
b/runners/google-cloud-dataflow-java/pom.xml
index 0125e2e..ecfbc9a 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -333,5 +333,19 @@
   datastore-v1-protos
   test
 
+
+
+
+  com.fasterxml.jackson.dataformat
+  jackson-dataformat-yaml
+  test
+
+
+
+  org.apache.beam
+  beam-sdks-common-fn-api
+  test-jar
+  test
+
   
 



[2/2] beam git commit: This closes #1949: Add new beam-sdk-java-core test deps to test scope for DataflowRunner

2017-02-08 Thread kenn
This closes #1949: Add new beam-sdk-java-core test deps to test scope for 
DataflowRunner


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

Branch: refs/heads/master
Commit: cf1925fba4a419c8dd7aff11394468243535cb5c
Parents: df2774c f3c099b
Author: Kenneth Knowles 
Authored: Wed Feb 8 06:55:31 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 8 06:55:31 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 14 ++
 1 file changed, 14 insertions(+)
--




[2/2] beam git commit: This closes #1912: Add license header to website Jenkins job descriptions

2017-02-02 Thread kenn
This closes #1912: Add license header to website Jenkins job descriptions


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

Branch: refs/heads/master
Commit: 08e58e1dfd4cc21a24755d21b8241ca4488b0e83
Parents: 96a05a4 838ef04
Author: Kenneth Knowles 
Authored: Thu Feb 2 20:16:19 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 2 20:16:19 2017 -0800

--
 .jenkins/job_beam_PreCommit_Website_Stage.groovy | 18 ++
 .jenkins/job_beam_PreCommit_Website_Test.groovy  | 18 ++
 2 files changed, 36 insertions(+)
--




[1/2] beam git commit: Add license header to website Jenkins job descriptions

2017-02-02 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 96a05a4b4 -> 08e58e1df


Add license header to website Jenkins job descriptions


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

Branch: refs/heads/master
Commit: 838ef04da912091db6f83be2fe48a9a3525e8279
Parents: 96a05a4
Author: Kenneth Knowles 
Authored: Thu Feb 2 19:21:46 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 2 19:21:46 2017 -0800

--
 .jenkins/job_beam_PreCommit_Website_Stage.groovy | 18 ++
 .jenkins/job_beam_PreCommit_Website_Test.groovy  | 18 ++
 2 files changed, 36 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/838ef04d/.jenkins/job_beam_PreCommit_Website_Stage.groovy
--
diff --git a/.jenkins/job_beam_PreCommit_Website_Stage.groovy 
b/.jenkins/job_beam_PreCommit_Website_Stage.groovy
index b63d40a..69be64d 100644
--- a/.jenkins/job_beam_PreCommit_Website_Stage.groovy
+++ b/.jenkins/job_beam_PreCommit_Website_Stage.groovy
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 import common_job_properties
 
 // Defines a job.

http://git-wip-us.apache.org/repos/asf/beam/blob/838ef04d/.jenkins/job_beam_PreCommit_Website_Test.groovy
--
diff --git a/.jenkins/job_beam_PreCommit_Website_Test.groovy 
b/.jenkins/job_beam_PreCommit_Website_Test.groovy
index ada089f..2b55374 100644
--- a/.jenkins/job_beam_PreCommit_Website_Test.groovy
+++ b/.jenkins/job_beam_PreCommit_Website_Test.groovy
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 import common_job_properties
 
 // Defines a job.



[2/4] beam git commit: Autoformat GroupAlsoByWindowsViaOutputBufferDoFn

2017-02-06 Thread kenn
Autoformat GroupAlsoByWindowsViaOutputBufferDoFn


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

Branch: refs/heads/master
Commit: c2925feb11ce26308e82209929ea478a93e06c84
Parents: 4800a3e
Author: Kenneth Knowles 
Authored: Mon Feb 6 08:30:31 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 08:30:31 2017 -0800

--
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  | 37 ++--
 1 file changed, 19 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/c2925feb/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 d83060f..bc02679 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
@@ -35,7 +35,7 @@ import org.joda.time.Instant;
  */
 @SystemDoFnInternal
 public class GroupAlsoByWindowsViaOutputBufferDoFn
-   extends GroupAlsoByWindowsDoFn {
+extends GroupAlsoByWindowsDoFn {
 
   private final WindowingStrategy strategy;
   private final StateInternalsFactory stateInternalsFactory;
@@ -90,25 +90,26 @@ public class GroupAlsoByWindowsViaOutputBufferDoFn reduceFnRunner) throws Exception {
+  private void fireEligibleTimers(
+  InMemoryTimerInternals timerInternals, ReduceFnRunner reduceFnRunner)
+  throws Exception {
 List timers = new ArrayList<>();
 while (true) {
-TimerInternals.TimerData timer;
-while ((timer = timerInternals.removeNextEventTimer()) != null) {
-  timers.add(timer);
-}
-while ((timer = timerInternals.removeNextProcessingTimer()) != null) {
-  timers.add(timer);
-}
-while ((timer = 
timerInternals.removeNextSynchronizedProcessingTimer()) != null) {
-  timers.add(timer);
-}
-if (timers.isEmpty()) {
-  break;
-}
-reduceFnRunner.onTimers(timers);
-timers.clear();
+  TimerInternals.TimerData timer;
+  while ((timer = timerInternals.removeNextEventTimer()) != null) {
+timers.add(timer);
+  }
+  while ((timer = timerInternals.removeNextProcessingTimer()) != null) {
+timers.add(timer);
+  }
+  while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) 
!= null) {
+timers.add(timer);
+  }
+  if (timers.isEmpty()) {
+break;
+  }
+  reduceFnRunner.onTimers(timers);
+  timers.clear();
 }
   }
 }



[1/4] beam git commit: Remove extraneous chunking from GroupAlsoByWindowsViaOutputBufferDoFn

2017-02-06 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master b3d962df2 -> 7564486f5


Remove extraneous chunking from GroupAlsoByWindowsViaOutputBufferDoFn


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

Branch: refs/heads/master
Commit: 4800a3eb42e1c92a307578a7b42fb6a4024eb27f
Parents: b3d962d
Author: Kenneth Knowles 
Authored: Sun Feb 5 20:42:03 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 08:30:16 2017 -0800

--
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  | 23 ++--
 1 file changed, 2 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/4800a3eb/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..d83060f 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
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.runners.core;
 
-import com.google.common.collect.Iterables;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
@@ -25,7 +24,6 @@ import 
org.apache.beam.runners.core.triggers.TriggerStateMachines;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 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.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
@@ -77,25 +75,8 @@ public class GroupAlsoByWindowsViaOutputBufferDoFn> chunks =
-Iterables.partition(c.element().getValue(), 1000);
-for (Iterable chunk : chunks) {
-  // Process the chunk of elements.
-  reduceFnRunner.processElements(chunk);
-
-  // Then, since elements are sorted by their timestamp, advance the input 
watermark
-  // to the first element.
-  
timerInternals.advanceInputWatermark(chunk.iterator().next().getTimestamp());
-  // Advance the processing times.
-  timerInternals.advanceProcessingTime(Instant.now());
-  timerInternals.advanceSynchronizedProcessingTime(Instant.now());
-
-  // Fire all the eligible timers.
-  fireEligibleTimers(timerInternals, reduceFnRunner);
-
-  // Leave the output watermark undefined. Since there's no late data in 
batch mode
-  // there's really no need to track it as we do for streaming.
-}
+// Process the elements.
+reduceFnRunner.processElements(c.element().getValue());
 
 // Finish any pending windows by advancing the input watermark to infinity.
 timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);



[4/4] beam git commit: This closes #1924: Remove extraneous chunking from GroupAlsoByWindowsViaOutputBufferDoFn

2017-02-06 Thread kenn
This closes #1924: Remove extraneous chunking from 
GroupAlsoByWindowsViaOutputBufferDoFn

  Remove inaccurate pluralization from GABWViaOutputBufferDoFn
  Autoformat GroupAlsoByWindowsViaOutputBufferDoFn
  Remove extraneous chunking from GroupAlsoByWindowsViaOutputBufferDoFn


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

Branch: refs/heads/master
Commit: 7564486f57a34aa5e9049b6ed995626ac251f172
Parents: b3d962d a27bb24
Author: Kenneth Knowles 
Authored: Mon Feb 6 08:38:42 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 08:38:42 2017 -0800

--
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   | 115 
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  | 133 ---
 ...roupAlsoByWindowViaOutputBufferDoFnTest.java | 110 +++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java | 110 ---
 .../translation/SparkGroupAlsoByWindowFn.java   |   3 +-
 5 files changed, 227 insertions(+), 244 deletions(-)
--




[12/15] beam git commit: Upgrade Dataflow container version to beam-master-20170203

2017-02-06 Thread kenn
Upgrade Dataflow container version to beam-master-20170203


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

Branch: refs/heads/master
Commit: fd7e6149a038a6480b6d2f4f76a738bb83c47476
Parents: e77e7f0
Author: Kenneth Knowles 
Authored: Fri Feb 3 19:08:52 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:56 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/fd7e6149/runners/google-cloud-dataflow-java/pom.xml
--
diff --git a/runners/google-cloud-dataflow-java/pom.xml 
b/runners/google-cloud-dataflow-java/pom.xml
index 35f31b9..0125e2e 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -33,7 +33,7 @@
   jar
 
   
-
beam-master-20170126
+
beam-master-20170203
 6
   
 



[15/15] beam git commit: This closes #1858: Move state-oriented remnants of OldDoFn to runners/core-java

2017-02-06 Thread kenn
This closes #1858: Move state-oriented remnants of OldDoFn to runners/core-java

  Upgrade Dataflow container version to beam-master-20170203
  Move StateNamespace and related to runners/core-java
  Move StateInternals to runners/core-java
  Move StateTable to runners/core-java
  Move TimerInternals to runners/core-java
  Move TimerInternalsFactory to runners/core-java
  Move StateInternalsFactory to runners/core-java
  Remove comment-only uses that block runners/core-java migrations
  Move WindowingInternals to runners/core-java
  Delete long-deprecated StateContexts methods
  Move InMemoryStateInternals to runners/core-java
  Eliminate use of InMemoryStateInternals in the Java SDK


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

Branch: refs/heads/master
Commit: 224e447653e5fef60982a230127b974dcb700161
Parents: e0189f3 fd7e614
Author: Kenneth Knowles 
Authored: Mon Feb 6 14:30:06 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 14:30:06 2017 -0800

--
 .../apex/translation/TranslationContext.java|   2 +-
 .../operators/ApexGroupByKeyOperator.java   |  10 +-
 .../operators/ApexParDoOperator.java|   4 +-
 .../translation/utils/ApexStateInternals.java   |   8 +-
 .../apex/translation/utils/NoOpStepContext.java |   4 +-
 .../utils/ApexStateInternalsTest.java   |   6 +-
 .../beam/runners/core/BaseExecutionContext.java |   2 -
 .../beam/runners/core/ExecutionContext.java |   2 -
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   |   3 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   3 -
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   |   4 -
 .../runners/core/InMemoryStateInternals.java| 439 +++
 .../runners/core/InMemoryTimerInternals.java|   2 -
 .../apache/beam/runners/core/KeyedWorkItem.java |   2 +-
 .../beam/runners/core/KeyedWorkItemCoder.java   |   4 +-
 .../beam/runners/core/KeyedWorkItems.java   |   2 +-
 .../core/LateDataDroppingDoFnRunner.java|   1 -
 .../runners/core/MergingActiveWindowSet.java|   2 -
 .../org/apache/beam/runners/core/OldDoFn.java   |   1 -
 .../beam/runners/core/PaneInfoTracker.java  |   1 -
 .../runners/core/ReduceFnContextFactory.java|   8 +-
 .../beam/runners/core/ReduceFnRunner.java   |   6 +-
 .../beam/runners/core/SideInputHandler.java |   2 -
 .../beam/runners/core/SimpleDoFnRunner.java |   3 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   3 -
 .../beam/runners/core/SplittableParDo.java  |   6 -
 .../beam/runners/core/StateInternals.java   |  60 +++
 .../runners/core/StateInternalsFactory.java |  35 ++
 .../beam/runners/core/StateNamespace.java   |  56 +++
 .../runners/core/StateNamespaceForTest.java |  65 +++
 .../beam/runners/core/StateNamespaces.java  | 278 
 .../apache/beam/runners/core/StateTable.java|  85 
 .../core/TestInMemoryStateInternals.java|  64 +++
 .../beam/runners/core/TimerInternals.java   | 286 
 .../runners/core/TimerInternalsFactory.java |  35 ++
 .../apache/beam/runners/core/WatermarkHold.java |   1 -
 .../beam/runners/core/WindowingInternals.java   |  81 
 .../core/WindowingInternalsAdapters.java|   1 -
 .../TriggerStateMachineContextFactory.java  |   6 +-
 ...roupAlsoByWindowViaOutputBufferDoFnTest.java |   1 -
 .../core/GroupAlsoByWindowsProperties.java  |   5 -
 .../core/InMemoryStateInternalsTest.java| 357 +++
 .../core/InMemoryTimerInternalsTest.java|   4 +-
 .../runners/core/KeyedWorkItemCoderTest.java|   3 +-
 .../core/LateDataDroppingDoFnRunnerTest.java|   1 -
 .../core/MergingActiveWindowSetTest.java|   2 -
 .../core/PushbackSideInputDoFnRunnerTest.java   |   3 +-
 .../beam/runners/core/ReduceFnTester.java   |   7 +-
 .../beam/runners/core/SideInputHandlerTest.java |   1 -
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   4 +-
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   1 -
 .../beam/runners/core/SplittableParDoTest.java  |   8 +-
 .../beam/runners/core/StateNamespacesTest.java  | 130 ++
 .../beam/runners/core/TimerInternalsTest.java   | 105 +
 .../core/triggers/NeverStateMachineTest.java|   1 -
 .../triggers/TriggerStateMachineTester.java |  14 +-
 .../CopyOnAccessInMemoryStateInternals.java |  18 +-
 .../runners/direct/DirectExecutionContext.java  |   2 +-
 .../beam/runners/direct/DirectRunner.java   |   2 +-
 .../runners/direct/DirectTimerInternals.java|   4 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../beam/runners/direct/EvaluationContext.java  |   2 +-
 .../direct/ExecutorServiceParallelExecutor.java |   2 +-
 

[02/15] beam git commit: Move InMemoryStateInternals to runners/core-java

2017-02-06 Thread kenn
Move InMemoryStateInternals to runners/core-java


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

Branch: refs/heads/master
Commit: af391b88cfc38659f594799bb58b6090a7bcd3a4
Parents: 2b6698d
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:03:42 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../runners/core/InMemoryStateInternals.java| 442 +++
 .../core/TestInMemoryStateInternals.java|  65 +++
 .../core/GroupAlsoByWindowsProperties.java  |   1 -
 .../core/InMemoryStateInternalsTest.java| 359 +++
 .../core/MergingActiveWindowSetTest.java|   1 -
 .../beam/runners/core/ReduceFnTester.java   |   1 -
 .../beam/runners/core/SideInputHandlerTest.java |   1 -
 .../beam/runners/core/SplittableParDoTest.java  |   1 -
 .../triggers/TriggerStateMachineTester.java |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java |  12 +-
 .../spark/translation/TranslationUtils.java |   3 +-
 .../sdk/util/state/InMemoryStateInternals.java  | 430 --
 .../util/state/TestInMemoryStateInternals.java  |  61 ---
 .../util/state/InMemoryStateInternalsTest.java  | 348 ---
 14 files changed, 874 insertions(+), 853 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/af391b88/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
new file mode 100644
index 000..059e32d
--- /dev/null
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
@@ -0,0 +1,442 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import 
org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.CombineFnUtil;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateTable;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTag.StateBinder;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.joda.time.Instant;
+
+/**
+ * In-memory implementation of {@link StateInternals}. Used in {@code 
BatchModeExecutionContext}
+ * and for running tests that need state.
+ */
+@Experimental(Kind.STATE)
+public class InMemoryStateInternals implements StateInternals {
+
+  public static  InMemoryStateInternals forKey(K key) {
+return new InMemoryStateInternals<>(key);
+  }
+
+  private final K key;
+
+  protected InMemoryStateInternals(K key) {
+this.key = key;
+  }
+
+  @Override
+  public K getKey() {
+

[03/15] beam git commit: Eliminate use of InMemoryStateInternals in the Java SDK

2017-02-06 Thread kenn
Eliminate use of InMemoryStateInternals in the Java SDK


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

Branch: refs/heads/master
Commit: 2b6698d1b136f251436aa8048cf145695ffb5f16
Parents: 7564486
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:00:15 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../apache/beam/runners/core/SplittableParDoTest.java|  5 -
 .../java/org/apache/beam/sdk/transforms/DoFnTester.java  | 11 ---
 2 files changed, 4 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/2b6698d1/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 bb7fd8c..ad0b01d 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
@@ -54,6 +54,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.util.state.TimerInternalsFactory;
@@ -207,6 +208,7 @@ public class SplittableParDoTest {
 private Instant currentProcessingTime;
 
 private InMemoryTimerInternals timerInternals;
+private InMemoryStateInternals stateInternals;
 
 ProcessFnTester(
 Instant currentProcessingTime,
@@ -221,11 +223,12 @@ public class SplittableParDoTest {
   fn, inputCoder, restrictionCoder, IntervalWindow.getCoder());
   this.tester = DoFnTester.of(processFn);
   this.timerInternals = new InMemoryTimerInternals();
+  this.stateInternals = InMemoryStateInternals.forKey("dummy");
   processFn.setStateInternalsFactory(
   new StateInternalsFactory() {
 @Override
 public StateInternals stateInternalsForKey(String key) {
-  return tester.getStateInternals();
+  return stateInternals;
 }
   });
   processFn.setTimerInternalsFactory(

http://git-wip-us.apache.org/repos/asf/beam/blob/2b6698d1/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 87ae1f5..cc5281c 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -48,8 +48,6 @@ import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.state.InMemoryStateInternals;
-import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
@@ -135,11 +133,6 @@ public class DoFnTester implements 
AutoCloseable {
 windowValues.put(window, value);
   }
 
-  @SuppressWarnings("unchecked")
-  public  StateInternals getStateInternals() {
-return (StateInternals) stateInternals;
-  }
-
   public PipelineOptions getPipelineOptions() {
 return options;
   }
@@ -228,8 +221,6 @@ public class DoFnTester implements 
AutoCloseable {
 }
 TestContext context = new TestContext();
 context.setupDelegateAggregators();
-// State and timer internals are per-bundle.
-stateInternals = InMemoryStateInternals.forKey(new Object());
 try {
   fnInvoker.invokeStartBundle(context);
 } catch (UserCodeException e) {
@@ -768,8 +759,6 @@ public class DoFnTester implements 
AutoCloseable {
   /** The outputs from the {@link DoFn} under test. */
   private Map> outputs;
 
-  private InMemoryStateInternals stateInternals;
-
   /** The state of processing of the 

[07/15] beam git commit: Remove comment-only uses that block runners/core-java migrations

2017-02-06 Thread kenn
Remove comment-only uses that block runners/core-java migrations


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

Branch: refs/heads/master
Commit: 1d7c6b0638d521cf221bb7456a0e976371ca1b07
Parents: 949ab3a
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:09:41 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java  | 2 --
 .../main/java/org/apache/beam/sdk/util/state/ReadableState.java  | 4 ++--
 2 files changed, 2 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/1d7c6b06/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
index c9d178a..de5b1e1 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
@@ -65,8 +65,6 @@ public final class PaneInfo {
* produces a final pane, it will not be merged into any new windows.
*
* The predictions above are made using the mechanism of watermarks.
-   * See {@link org.apache.beam.sdk.util.TimerInternals} for more information
-   * about watermarks.
*
* We can state some properties of {@code LATE} and {@code ON_TIME} 
panes, but first need some
* definitions:

http://git-wip-us.apache.org/repos/asf/beam/blob/1d7c6b06/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
index 3b4cb7b..c3e9936 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
@@ -35,14 +35,14 @@ public interface ReadableState {
*
* If there will be many calls to {@link #read} for different state in 
short succession,
* you should first call {@link #readLater} for all of them so the reads can 
potentially be
-   * batched (depending on the underlying {@link StateInternals} 
implementation}.
+   * batched (depending on the underlying implementation}.
*/
   T read();
 
   /**
* Indicate that the value will be read later.
*
-   * This allows a {@link StateInternals} implementation to start an 
asynchronous prefetch or
+   * This allows an implementation to start an asynchronous prefetch or
* to include this state in the next batch of reads.
*
* @return this for convenient chaining



[11/15] beam git commit: Move TimerInternals to runners/core-java

2017-02-06 Thread kenn
Move TimerInternals to runners/core-java


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

Branch: refs/heads/master
Commit: b086d2fdf1dba2b0e4a3362bd3ee1a92bf21c56a
Parents: 92b33bc
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:17:18 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:56 2017 -0800

--
 .../operators/ApexGroupByKeyOperator.java   |   2 +-
 .../apex/translation/utils/NoOpStepContext.java |   2 +-
 .../beam/runners/core/BaseExecutionContext.java |   1 -
 .../beam/runners/core/ExecutionContext.java |   1 -
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   |   1 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   1 -
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   |   2 -
 .../runners/core/InMemoryTimerInternals.java|   1 -
 .../apache/beam/runners/core/KeyedWorkItem.java |   2 +-
 .../beam/runners/core/KeyedWorkItemCoder.java   |   4 +-
 .../beam/runners/core/KeyedWorkItems.java   |   2 +-
 .../core/LateDataDroppingDoFnRunner.java|   1 -
 .../beam/runners/core/PaneInfoTracker.java  |   1 -
 .../runners/core/ReduceFnContextFactory.java|   3 +-
 .../beam/runners/core/ReduceFnRunner.java   |   3 +-
 .../beam/runners/core/SimpleDoFnRunner.java |   1 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   1 -
 .../beam/runners/core/SplittableParDo.java  |   1 -
 .../beam/runners/core/TimerInternals.java   | 288 +++
 .../runners/core/TimerInternalsFactory.java |   1 -
 .../apache/beam/runners/core/WatermarkHold.java |   1 -
 .../beam/runners/core/WindowingInternals.java   |   1 -
 .../core/GroupAlsoByWindowsProperties.java  |   1 -
 .../core/InMemoryTimerInternalsTest.java|   2 +-
 .../runners/core/KeyedWorkItemCoderTest.java|   2 +-
 .../core/LateDataDroppingDoFnRunnerTest.java|   1 -
 .../core/PushbackSideInputDoFnRunnerTest.java   |   2 +-
 .../beam/runners/core/ReduceFnTester.java   |   3 +-
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   3 +-
 .../beam/runners/core/SplittableParDoTest.java  |   1 -
 .../beam/runners/core/TimerInternalsTest.java   | 107 +++
 .../triggers/TriggerStateMachineTester.java |   2 +-
 .../runners/direct/DirectExecutionContext.java  |   2 +-
 .../beam/runners/direct/DirectRunner.java   |   2 +-
 .../runners/direct/DirectTimerInternals.java|   2 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../beam/runners/direct/EvaluationContext.java  |   2 +-
 .../direct/ExecutorServiceParallelExecutor.java |   2 +-
 .../GroupAlsoByWindowEvaluatorFactory.java  |   2 +-
 .../beam/runners/direct/ParDoEvaluator.java |   2 +-
 ...littableProcessElementsEvaluatorFactory.java |   2 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   2 +-
 .../beam/runners/direct/WatermarkManager.java   |   4 +-
 .../direct/DirectTimerInternalsTest.java|   2 +-
 ...leManagerRemovingTransformEvaluatorTest.java |   2 +-
 .../runners/direct/EvaluationContextTest.java   |   2 +-
 .../runners/direct/WatermarkManagerTest.java|   2 +-
 .../functions/FlinkProcessContextBase.java  |   2 +-
 .../wrappers/streaming/DoFnOperator.java|   2 +-
 .../streaming/SingletonKeyedWorkItem.java   |   2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   4 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |   2 +-
 .../spark/translation/SparkProcessContext.java  |   2 +-
 .../apache/beam/sdk/util/TimerInternals.java| 286 --
 .../sdk/transforms/join/UnionCoderTest.java |   7 -
 .../beam/sdk/util/TimerInternalsTest.java   | 106 ---
 .../beam/fn/harness/fake/FakeStepContext.java   |   2 +-
 57 files changed, 433 insertions(+), 459 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/b086d2fd/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 6322796..7891b34 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
@@ -47,6 +47,7 @@ import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.OldDoFn;
 import org.apache.beam.runners.core.StateInternalsFactory;
 import 

[05/15] beam git commit: Move WindowingInternals to runners/core-java

2017-02-06 Thread kenn
Move WindowingInternals to runners/core-java


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

Branch: refs/heads/master
Commit: 949ab3ac6d654a310a513d2e64e8dbf39fd4f388
Parents: b12e5ff
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:06:10 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../operators/ApexGroupByKeyOperator.java   |  2 +-
 .../org/apache/beam/runners/core/OldDoFn.java   |  1 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  1 -
 .../beam/runners/core/WindowingInternals.java   | 83 
 .../core/WindowingInternalsAdapters.java|  1 -
 .../core/GroupAlsoByWindowsProperties.java  |  1 -
 .../beam/runners/core/ReduceFnTester.java   |  1 -
 .../functions/FlinkProcessContextBase.java  |  2 +-
 .../beam/sdk/util/WindowingInternals.java   | 82 ---
 9 files changed, 85 insertions(+), 89 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/949ab3ac/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 4c2b461..c5da368 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
@@ -46,6 +46,7 @@ import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.OldDoFn;
 import org.apache.beam.runners.core.SystemReduceFn;
+import org.apache.beam.runners.core.WindowingInternals;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.KvCoder;
@@ -59,7 +60,6 @@ import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;

http://git-wip-us.apache.org/repos/asf/beam/blob/949ab3ac/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
index b099721..4033260 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java
@@ -42,7 +42,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.joda.time.Duration;

http://git-wip-us.apache.org/repos/asf/beam/blob/949ab3ac/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
index 2fe9226..9f80bca 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
@@ -42,7 +42,6 @@ import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;


[08/15] beam git commit: Move StateNamespace and related to runners/core-java

2017-02-06 Thread kenn
http://git-wip-us.apache.org/repos/asf/beam/blob/e77e7f0d/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
deleted file mode 100644
index f546e56..000
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util.state;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link StateNamespaces}.
- */
-@RunWith(JUnit4.class)
-public class StateNamespacesTest {
-
-  private final Coder intervalCoder = 
IntervalWindow.getCoder();
-
-  private IntervalWindow intervalWindow(long start, long end) {
-return new IntervalWindow(new Instant(start), new Instant(end));
-  }
-
-  /**
-   * This test should not be changed. It verifies that the stringKey matches 
certain expectations.
-   * If this changes, the ability to reload any pipeline that has persisted 
these namespaces will
-   * be impacted.
-   */
-  @Test
-  public void testStability() {
-StateNamespace global = StateNamespaces.global();
-StateNamespace intervalWindow =
-StateNamespaces.window(intervalCoder, intervalWindow(1000, 87392));
-StateNamespace intervalWindowAndTrigger =
-StateNamespaces.windowAndTrigger(intervalCoder, intervalWindow(1000, 
87392), 57);
-StateNamespace globalWindow = StateNamespaces.window(
-GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
-StateNamespace globalWindowAndTrigger = StateNamespaces.windowAndTrigger(
-GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, 12);
-
-assertEquals("/", global.stringKey());
-assertEquals("/gAABVWD4ogU/", intervalWindow.stringKey());
-assertEquals("/gAABVWD4ogU/1L/", intervalWindowAndTrigger.stringKey());
-assertEquals("//", globalWindow.stringKey());
-assertEquals("//C/", globalWindowAndTrigger.stringKey());
-  }
-
-  /**
-   * Test that WindowAndTrigger namespaces are prefixed by the related Window 
namespace.
-   */
-  @Test
-  public void testIntervalWindowPrefixing() {
-StateNamespace window =
-StateNamespaces.window(intervalCoder, intervalWindow(1000, 87392));
-StateNamespace windowAndTrigger = StateNamespaces.windowAndTrigger(
-intervalCoder, intervalWindow(1000, 87392), 57);
-assertThat(windowAndTrigger.stringKey(), 
Matchers.startsWith(window.stringKey()));
-assertThat(StateNamespaces.global().stringKey(),
-Matchers.not(Matchers.startsWith(window.stringKey(;
-  }
-
-  /**
-   * Test that WindowAndTrigger namespaces are prefixed by the related Window 
namespace.
-   */
-  @Test
-  public void testGlobalWindowPrefixing() {
-StateNamespace window =
-StateNamespaces.window(GlobalWindow.Coder.INSTANCE, 
GlobalWindow.INSTANCE);
-StateNamespace windowAndTrigger = StateNamespaces.windowAndTrigger(
-GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, 57);
-assertThat(windowAndTrigger.stringKey(), 
Matchers.startsWith(window.stringKey()));
-assertThat(StateNamespaces.global().stringKey(),
-Matchers.not(Matchers.startsWith(window.stringKey(;
-  }
-
-  @Test
-  public void testFromStringGlobal() {
-assertStringKeyRoundTrips(intervalCoder, StateNamespaces.global());
-  }
-
-  @Test
-  public void testFromStringIntervalWindow() {
-assertStringKeyRoundTrips(
-intervalCoder, StateNamespaces.window(intervalCoder, 
intervalWindow(1000, 8000)));
-assertStringKeyRoundTrips(
-

[01/15] beam git commit: Move TimerInternalsFactory to runners/core-java

2017-02-06 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master e0189f352 -> 224e44765


Move TimerInternalsFactory to runners/core-java


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

Branch: refs/heads/master
Commit: 92b33bc7ec6d7c5c32a49c5750ac1d4381a478ce
Parents: 2e6c131
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:13:42 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../beam/runners/core/SplittableParDo.java  |  1 -
 .../runners/core/TimerInternalsFactory.java | 36 
 .../beam/runners/core/SplittableParDoTest.java  |  1 -
 ...littableProcessElementsEvaluatorFactory.java |  2 +-
 .../sdk/util/state/TimerInternalsFactory.java   | 36 
 5 files changed, 37 insertions(+), 39 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/92b33bc7/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
index e414430..7368b2f 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
@@ -57,7 +57,6 @@ import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.TimerInternalsFactory;
 import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/beam/blob/92b33bc7/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternalsFactory.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternalsFactory.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternalsFactory.java
new file mode 100644
index 000..e129aed
--- /dev/null
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternalsFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.util.TimerInternals;
+
+/**
+ * A factory for providing {@link TimerInternals} for a particular key.
+ *
+ * Because it will generally be embedded in a {@link 
org.apache.beam.sdk.transforms.DoFn DoFn},
+ * albeit at execution time, it is marked {@link Serializable}.
+ */
+@Experimental(Kind.STATE)
+public interface TimerInternalsFactory {
+
+  /** Returns {@link TimerInternals} for the provided key. */
+  TimerInternals timerInternalsForKey(K key);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/92b33bc7/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 b408d37..427e2f4 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
@@ -55,7 +55,6 @@ import org.apache.beam.sdk.util.SideInputReader;
 import 

[04/15] beam git commit: Move StateInternalsFactory to runners/core-java

2017-02-06 Thread kenn
Move StateInternalsFactory to runners/core-java


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

Branch: refs/heads/master
Commit: 2e6c131027caea3e4df5545ff1b65acc2a7f53a0
Parents: 1d7c6b0
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:13:14 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../apex/translation/TranslationContext.java|  2 +-
 .../operators/ApexGroupByKeyOperator.java   |  2 +-
 .../operators/ApexParDoOperator.java|  2 +-
 .../translation/utils/ApexStateInternals.java   |  4 +--
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   |  1 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  1 -
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   |  1 -
 .../beam/runners/core/SplittableParDo.java  |  1 -
 .../runners/core/StateInternalsFactory.java | 36 
 ...roupAlsoByWindowViaOutputBufferDoFnTest.java |  1 -
 .../core/GroupAlsoByWindowsProperties.java  |  1 -
 .../beam/runners/core/SplittableParDoTest.java  |  1 -
 ...littableProcessElementsEvaluatorFactory.java |  2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  3 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |  2 +-
 .../spark/translation/TranslationUtils.java |  2 +-
 .../sdk/util/state/StateInternalsFactory.java   | 35 ---
 17 files changed, 44 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/2e6c1310/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
--
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
index 8c30bc6..acd8ab1 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
@@ -33,11 +33,11 @@ import org.apache.beam.runners.apex.ApexPipelineOptions;
 import org.apache.beam.runners.apex.translation.utils.ApexStateInternals;
 import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple;
 import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec;
+import org.apache.beam.runners.core.StateInternalsFactory;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;

http://git-wip-us.apache.org/repos/asf/beam/blob/2e6c1310/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 c5da368..6322796 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
@@ -45,6 +45,7 @@ import 
org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.OldDoFn;
+import org.apache.beam.runners.core.StateInternalsFactory;
 import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.core.WindowingInternals;
 import org.apache.beam.sdk.coders.Coder;
@@ -62,7 +63,6 @@ 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.StateInternals;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;

http://git-wip-us.apache.org/repos/asf/beam/blob/2e6c1310/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java

[06/15] beam git commit: Delete long-deprecated StateContexts methods

2017-02-06 Thread kenn
Delete long-deprecated StateContexts methods


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

Branch: refs/heads/master
Commit: b12e5ffbaf9bacebc5d80b3ed24c37a22e37524f
Parents: af391b8
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:05:09 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:06 2017 -0800

--
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  1 -
 .../core/triggers/NeverStateMachineTest.java|  1 -
 .../beam/sdk/util/state/StateContexts.java  | 63 
 3 files changed, 65 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
index 97da9ee..28698ca 100644
--- 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
+++ 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
@@ -23,7 +23,6 @@ import static org.mockito.Mockito.mock;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.runners.core.BaseExecutionContext.StepContext;
-
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
index 6d8a344..79147d4 100644
--- 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
+++ 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
@@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.values.TimestampedValue;
-
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/beam/blob/b12e5ffb/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
index a9ce0b5..2ce9594 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java
@@ -17,10 +17,8 @@
  */
 package org.apache.beam.sdk.util.state;
 
-import javax.annotation.Nullable;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 
 /**
@@ -50,65 +48,4 @@ public class StateContexts {
   public static  StateContext nullContext() {
 return (StateContext) NULL_CONTEXT;
   }
-
-  /**
-   * @deprecated This exists only for temporary compatibility with Dataflow 
worker and should be
-   * deleted once a worker image is released that uses runners-core build 
after
-   * https://github.com/apache/beam/pull/1353.
-   */
-  @Deprecated
-  public static  StateContext windowOnly(final W 
window) {
-return new StateContext() {
-  @Override
-  public PipelineOptions getPipelineOptions() {
-throw new IllegalArgumentException(
-"cannot call getPipelineOptions() in a window only context");
-  }
-
-  @Override
-  public  T sideInput(PCollectionView view) {
-throw new IllegalArgumentException("cannot call sideInput() in a 
window only context");
-  }
-
-  @Override
-  public W window() {
-return window;
-  }
-};
-  }
-
-  /**
-   * @deprecated This exists only for temporary compatibility with Dataflow 
worker and should be
-   * deleted once a worker image is released that uses 

[14/15] beam git commit: Move StateInternals to runners/core-java

2017-02-06 Thread kenn
Move StateInternals to runners/core-java


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

Branch: refs/heads/master
Commit: 144b1df88176f32aec4991423eb225b844ff16c2
Parents: 4e4391c
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:22:38 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:56 2017 -0800

--
 .../operators/ApexGroupByKeyOperator.java   |  2 +-
 .../operators/ApexParDoOperator.java|  2 +-
 .../translation/utils/ApexStateInternals.java   |  2 +-
 .../apex/translation/utils/NoOpStepContext.java |  2 +-
 .../beam/runners/core/BaseExecutionContext.java |  1 -
 .../beam/runners/core/ExecutionContext.java |  1 -
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   |  1 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  1 -
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   |  1 -
 .../runners/core/InMemoryStateInternals.java|  1 -
 .../runners/core/MergingActiveWindowSet.java|  1 -
 .../runners/core/ReduceFnContextFactory.java|  1 -
 .../beam/runners/core/ReduceFnRunner.java   |  1 -
 .../beam/runners/core/SideInputHandler.java |  1 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  1 -
 .../beam/runners/core/SplittableParDo.java  |  1 -
 .../beam/runners/core/StateInternals.java   | 61 
 .../runners/core/StateInternalsFactory.java |  1 -
 .../beam/runners/core/WindowingInternals.java   |  1 -
 .../TriggerStateMachineContextFactory.java  |  2 +-
 .../core/GroupAlsoByWindowsProperties.java  |  1 -
 .../core/MergingActiveWindowSetTest.java|  1 -
 .../beam/runners/core/SplittableParDoTest.java  |  1 -
 .../triggers/TriggerStateMachineTester.java |  2 +-
 .../CopyOnAccessInMemoryStateInternals.java |  2 +-
 ...littableProcessElementsEvaluatorFactory.java |  2 +-
 .../StatefulParDoEvaluatorFactoryTest.java  |  2 +-
 .../functions/FlinkProcessContextBase.java  |  2 +-
 .../wrappers/streaming/DoFnOperator.java|  2 +-
 .../wrappers/streaming/FlinkStateInternals.java |  2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |  4 +-
 .../spark/translation/SparkProcessContext.java  |  2 +-
 .../spark/translation/TranslationUtils.java |  2 +-
 .../beam/sdk/util/state/StateInternals.java | 57 --
 .../beam/fn/harness/fake/FakeStepContext.java   |  2 +-
 36 files changed, 78 insertions(+), 93 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/144b1df8/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 7891b34..274e807 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
@@ -45,6 +45,7 @@ import 
org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.OldDoFn;
+import org.apache.beam.runners.core.StateInternals;
 import org.apache.beam.runners.core.StateInternalsFactory;
 import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.core.TimerInternals;
@@ -62,7 +63,6 @@ import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;

http://git-wip-us.apache.org/repos/asf/beam/blob/144b1df8/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 13db40a..7f2512a 100644
--- 

[09/15] beam git commit: Move StateNamespace and related to runners/core-java

2017-02-06 Thread kenn
Move StateNamespace and related to runners/core-java


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

Branch: refs/heads/master
Commit: e77e7f0d535eb8ad309f3210e3061566b55e7a21
Parents: 144b1df
Author: Kenneth Knowles 
Authored: Thu Jan 26 21:26:04 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 6 09:26:56 2017 -0800

--
 .../operators/ApexGroupByKeyOperator.java   |   2 +-
 .../translation/utils/ApexStateInternals.java   |   2 +-
 .../utils/ApexStateInternalsTest.java   |   6 +-
 .../runners/core/InMemoryStateInternals.java|   1 -
 .../runners/core/InMemoryTimerInternals.java|   1 -
 .../runners/core/MergingActiveWindowSet.java|   1 -
 .../runners/core/ReduceFnContextFactory.java|   4 +-
 .../beam/runners/core/ReduceFnRunner.java   |   2 +-
 .../beam/runners/core/SideInputHandler.java |   1 -
 .../beam/runners/core/SimpleDoFnRunner.java |   2 -
 .../beam/runners/core/SplittableParDo.java  |   2 -
 .../beam/runners/core/StateInternals.java   |   1 -
 .../beam/runners/core/StateNamespace.java   |  56 
 .../runners/core/StateNamespaceForTest.java |  65 +
 .../beam/runners/core/StateNamespaces.java  | 278 +++
 .../apache/beam/runners/core/StateTable.java|   1 -
 .../core/TestInMemoryStateInternals.java|   1 -
 .../beam/runners/core/TimerInternals.java   |   2 -
 .../TriggerStateMachineContextFactory.java  |   4 +-
 .../core/InMemoryStateInternalsTest.java|   2 -
 .../core/InMemoryTimerInternalsTest.java|   2 -
 .../runners/core/KeyedWorkItemCoderTest.java|   1 -
 .../core/PushbackSideInputDoFnRunnerTest.java   |   1 -
 .../beam/runners/core/ReduceFnTester.java   |   2 -
 .../beam/runners/core/SimpleDoFnRunnerTest.java |   1 -
 .../beam/runners/core/StateNamespacesTest.java  | 130 +
 .../beam/runners/core/TimerInternalsTest.java   |   2 -
 .../triggers/TriggerStateMachineTester.java |   8 +-
 .../CopyOnAccessInMemoryStateInternals.java |   2 +-
 .../runners/direct/DirectTimerInternals.java|   2 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   4 +-
 .../beam/runners/direct/WatermarkManager.java   |   2 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |   6 +-
 .../direct/DirectTimerInternalsTest.java|   2 +-
 ...leManagerRemovingTransformEvaluatorTest.java |   2 +-
 .../runners/direct/EvaluationContextTest.java   |   2 +-
 .../StatefulParDoEvaluatorFactoryTest.java  |   4 +-
 .../runners/direct/WatermarkManagerTest.java|   2 +-
 .../wrappers/streaming/FlinkStateInternals.java |   2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   2 +-
 .../streaming/FlinkStateInternalsTest.java  |   4 +-
 .../beam/sdk/util/state/StateNamespace.java |  56 
 .../sdk/util/state/StateNamespaceForTest.java   |  65 -
 .../beam/sdk/util/state/StateNamespaces.java| 278 ---
 .../sdk/util/state/StateNamespacesTest.java | 130 -
 45 files changed, 559 insertions(+), 587 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/e77e7f0d/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 274e807..3508c3e 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
@@ -47,6 +47,7 @@ import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.OldDoFn;
 import org.apache.beam.runners.core.StateInternals;
 import org.apache.beam.runners.core.StateInternalsFactory;
+import org.apache.beam.runners.core.StateNamespace;
 import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.core.TimerInternals;
 import org.apache.beam.runners.core.WindowingInternals;
@@ -63,7 +64,6 @@ import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;


[2/2] beam git commit: This closes #1780: Simplified API surface verifications

2017-01-23 Thread kenn
This closes #1780: Simplified API surface verifications


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

Branch: refs/heads/master
Commit: 26a2c47f48ddc1ea1389023a0fb72e449fb8f592
Parents: a1a022d 29ffaf3
Author: Kenneth Knowles 
Authored: Mon Jan 23 13:56:55 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Jan 23 13:56:55 2017 -0800

--
 .../org/apache/beam/sdk/util/ApiSurface.java| 420 ++-
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |  61 +++
 .../apache/beam/sdk/util/ApiSurfaceTest.java| 152 ++-
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  | 134 --
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java  |  76 
 5 files changed, 484 insertions(+), 359 deletions(-)
--




[1/2] beam git commit: [BEAM-882, BEAM-883, BEAM-878] Simplified API surface verifications.

2017-01-23 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master a1a022d6b -> 26a2c47f4


[BEAM-882,BEAM-883,BEAM-878] Simplified API surface verifications.


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

Branch: refs/heads/master
Commit: 29ffaf3859ba9b4d8ba8529efc96fd5e105e21a3
Parents: a1a022d
Author: Stas Levin 
Authored: Mon Jan 16 16:20:25 2017 +0200
Committer: Kenneth Knowles 
Committed: Mon Jan 23 13:56:45 2017 -0800

--
 .../org/apache/beam/sdk/util/ApiSurface.java| 420 ++-
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |  61 +++
 .../apache/beam/sdk/util/ApiSurfaceTest.java| 152 ++-
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  | 134 --
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java  |  76 
 5 files changed, 484 insertions(+), 359 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/29ffaf38/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
index 2040161..b6b0b32 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
@@ -17,12 +17,21 @@
  */
 package org.apache.beam.sdk.util;
 
+import static org.hamcrest.Matchers.anyOf;
+
+import com.google.common.base.Function;
 import com.google.common.base.Joiner;
+import com.google.common.base.Predicate;
 import com.google.common.base.Supplier;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
+import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.common.reflect.ClassPath;
 import com.google.common.reflect.ClassPath.ClassInfo;
@@ -45,15 +54,20 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
+import javax.annotation.Nonnull;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.StringDescription;
+import org.hamcrest.TypeSafeDiagnosingMatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Represents the API surface of a package prefix. Used for accessing public 
classes,
- * methods, and the types they reference, to control what dependencies are 
re-exported.
+ * Represents the API surface of a package prefix. Used for accessing public 
classes, methods, and
+ * the types they reference, to control what dependencies are re-exported.
  *
- * For the purposes of calculating the public API surface, exposure 
includes any public
- * or protected occurrence of:
+ * For the purposes of calculating the public API surface, exposure 
includes any public or
+ * protected occurrence of:
  *
  * 
  * superclasses
@@ -66,42 +80,272 @@ import org.slf4j.LoggerFactory;
  * wildcard bounds
  * 
  *
- * Exposure is a transitive property. The resulting map excludes primitives
- * and array classes themselves.
+ * Exposure is a transitive property. The resulting map excludes primitives 
and array classes
+ * themselves.
  *
- * It is prudent (though not required) to prune prefixes like "java" via 
the builder
- * method {@link #pruningPrefix} to halt the traversal so it does not 
uselessly catalog references
- * that are not interesting.
+ * It is prudent (though not required) to prune prefixes like "java" via 
the builder method
+ * {@link #pruningPrefix} to halt the traversal so it does not uselessly 
catalog references that are
+ * not interesting.
  */
 @SuppressWarnings("rawtypes")
 public class ApiSurface {
   private static final Logger LOG = LoggerFactory.getLogger(ApiSurface.class);
 
+  /** A factory method to create a {@link Class} matcher for classes residing 
in a given package. */
+  public static Matcher classesInPackage(final String packageName) {
+return new Matchers.ClassInPackage(packageName);
+  }
+
+  /**
+   * A factory method to create an {@link ApiSurface} matcher, producing a 
positive match if the
+   * queried api surface contains ONLY classes described by the provided 
matchers.
+   */
+  public static Matcher containsOnlyClassesMatching(
+  final Set> classMatchers) {
+return 

[2/2] beam git commit: This closes #1822: Revert "Simplified API surface verifications"

2017-01-23 Thread kenn
This closes #1822: Revert "Simplified API surface verifications"


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

Branch: refs/heads/master
Commit: cb6e0a80c57b056489d447cde092cffdd041eed5
Parents: 6ecbfb9 9248bef
Author: Kenneth Knowles 
Authored: Mon Jan 23 19:47:47 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Jan 23 19:47:47 2017 -0800

--
 .../org/apache/beam/sdk/util/ApiSurface.java| 420 +--
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |  61 ---
 .../apache/beam/sdk/util/ApiSurfaceTest.java| 152 +--
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  | 134 ++
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java  |  76 
 5 files changed, 359 insertions(+), 484 deletions(-)
--




[04/12] beam git commit: fix group by window

2017-01-23 Thread kenn
fix group by window


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

Branch: refs/heads/gearpump-runner
Commit: e63d42d1113728badc66285e7ce7a8ce204a82d9
Parents: ea633d2
Author: manuzhang 
Authored: Sat Jan 7 23:07:23 2017 +0800
Committer: manuzhang 
Committed: Sat Jan 14 13:35:31 2017 +0800

--
 .../beam/runners/gearpump/GearpumpRunner.java   |  3 ++-
 .../translators/GroupByKeyTranslator.java   |  4 +--
 .../translators/TranslationContext.java |  1 -
 .../translators/WindowBoundTranslator.java  | 27 ++--
 .../gearpump/translators/io/GearpumpSource.java |  4 +--
 5 files changed, 30 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/e63d42d1/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 9c44da3..01fdb3b 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
@@ -102,8 +102,9 @@ public class GearpumpRunner extends 
PipelineRunner {
 options.getSerializers());
 ClientContext clientContext = getClientContext(options, config);
 options.setClientContext(clientContext);
+UserConfig userConfig = UserConfig.empty();
 JavaStreamApp streamApp = new JavaStreamApp(
-appName, clientContext, UserConfig.empty());
+appName, clientContext, userConfig);
 TranslationContext translationContext = new TranslationContext(streamApp, 
options);
 GearpumpPipelineTranslator translator = new 
GearpumpPipelineTranslator(translationContext);
 translator.translate(pipeline);

http://git-wip-us.apache.org/repos/asf/beam/blob/e63d42d1/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
index 989957f..8e3ffe3 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
@@ -37,7 +37,7 @@ import org.apache.beam.sdk.values.KV;
 
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
-import org.apache.gearpump.streaming.dsl.window.api.Accumulating$;
+import org.apache.gearpump.streaming.dsl.window.api.Discarding$;
 import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$;
 import org.apache.gearpump.streaming.dsl.window.api.Window;
 import org.apache.gearpump.streaming.dsl.window.api.WindowFn;
@@ -60,7 +60,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator>>> outputStream = inputStream
 .window(Window.apply(new 
GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()),
-EventTimeTrigger$.MODULE$, Accumulating$.MODULE$), "assign_window")
+EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window")
 .groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window")
 .map(new ValueToIterable(), "map_value_to_iterable")
 .reduce(new MergeValue(), "merge_value");

http://git-wip-us.apache.org/repos/asf/beam/blob/e63d42d1/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
index 63fb619..b2cff8a 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
@@ -50,7 +50,6 @@ public class TranslationContext {
   public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions 
pipelineOptions) {
 this.streamApp = streamApp;
 this.pipelineOptions = 

[05/12] beam git commit: update to latest gearpump dsl function interface

2017-01-23 Thread kenn
update to latest gearpump dsl function interface


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

Branch: refs/heads/gearpump-runner
Commit: 3bf82638096ae7aa91c7d3c862c2994772bee51b
Parents: e63d42d
Author: manuzhang 
Authored: Sat Jan 14 13:36:07 2017 +0800
Committer: manuzhang 
Committed: Sat Jan 14 21:40:18 2017 +0800

--
 .../translators/GroupByKeyTranslator.java   | 12 
 .../translators/ParDoBoundMultiTranslator.java  | 29 ++--
 .../translators/WindowBoundTranslator.java  |  4 +--
 .../translators/functions/DoFnFunction.java | 21 +++---
 4 files changed, 46 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/3bf82638/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
index 8e3ffe3..4eaf755 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
@@ -36,15 +36,15 @@ import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.gearpump.streaming.dsl.api.functions.MapFunction;
+import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction;
 import org.apache.gearpump.streaming.dsl.javaapi.JavaStream;
+import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction;
 import org.apache.gearpump.streaming.dsl.window.api.Discarding$;
 import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$;
 import org.apache.gearpump.streaming.dsl.window.api.Window;
 import org.apache.gearpump.streaming.dsl.window.api.WindowFn;
 import org.apache.gearpump.streaming.dsl.window.impl.Bucket;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction;
-import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction;
 import scala.collection.JavaConversions;
 
 
@@ -122,7 +122,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator implements
+  private static class GroupByFn extends
   GroupByFunction>, K> {
 
 @Override
@@ -132,7 +132,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator
-  implements MapFunction>, WindowedValue>> {
+  extends MapFunction>, WindowedValue>> {
 
 @Override
 public WindowedValue> apply(WindowedValue> wv) 
{
@@ -141,7 +141,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator implements
+  private static class MergeValue extends
   ReduceFunction>> {
 
 @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/3bf82638/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 24f9734..0d5b8bc 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
@@ -33,6 +33,7 @@ import 
org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader;
 import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
@@ -41,10 +42,10 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
+import 

[02/12] beam git commit: update ROS configurations

2017-01-23 Thread kenn
update ROS configurations


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

Branch: refs/heads/gearpump-runner
Commit: cfdc971f45ff716b7bd88b3e054ca7077454ab07
Parents: 2155476
Author: manuzhang 
Authored: Thu Jan 5 13:47:42 2017 +0800
Committer: manuzhang 
Committed: Thu Jan 5 13:47:42 2017 +0800

--
 runners/gearpump/pom.xml | 6 ++
 1 file changed, 6 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/cfdc971f/runners/gearpump/pom.xml
--
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index 777ad34..4e3722c 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -64,6 +64,12 @@
 
 
   
org.apache.beam.sdk.testing.RunnableOnService
+  
+org.apache.beam.sdk.testing.UsesStatefulParDo,
+org.apache.beam.sdk.testing.UsesTimersInParDo,
+org.apache.beam.sdk.testing.UsesSplittableParDo,
+org.apache.beam.sdk.testing.UsesMetrics
+  
   none
   true
   



[10/12] beam git commit: Remove cache for Gearpump on travis

2017-01-23 Thread kenn
Remove cache for Gearpump on travis


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

Branch: refs/heads/gearpump-runner
Commit: 85dcfbd153acb4e450a4f0f94fc54b19b76507d3
Parents: 7613ec4
Author: manuzhang 
Authored: Fri Jan 20 08:33:04 2017 +0800
Committer: manuzhang 
Committed: Fri Jan 20 10:52:11 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/85dcfbd1/.travis.yml
--
diff --git a/.travis.yml b/.travis.yml
index a806477..7dcd5d1 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -59,6 +59,7 @@ before_install:
 install:
   # Removing this here protects from inadvertent caching
   - rm -rf "$HOME/.m2/repository/org/apache/beam"
+  - rm -rf "$HOME/.m2/repository/org/apache/gearpump"
 
 script:
   - travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates 
$MAVEN_OVERRIDE install && travis_retry bash -ex .travis/test_wordcount.sh



[01/12] beam git commit: [BEAM-1180] Implement GearpumpPipelineResult

2017-01-23 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/gearpump-runner 4c445dd0b -> 1ed16f11a


[BEAM-1180] Implement GearpumpPipelineResult


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

Branch: refs/heads/gearpump-runner
Commit: 21554764056c45ea18be1e844b4ca1bfb71e544a
Parents: 4c445dd
Author: manuzhang 
Authored: Tue Dec 20 10:39:56 2016 +0800
Committer: manuzhang 
Committed: Wed Jan 4 12:59:08 2017 +0800

--
 runners/gearpump/pom.xml|  1 +
 .../gearpump/GearpumpPipelineResult.java| 59 ++--
 .../beam/runners/gearpump/GearpumpRunner.java   |  4 +-
 .../runners/gearpump/TestGearpumpRunner.java|  4 ++
 .../translators/GroupByKeyTranslator.java   |  1 -
 5 files changed, 62 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/21554764/runners/gearpump/pom.xml
--
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index bb35ad7..777ad34 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -69,6 +69,7 @@
   
 
org.apache.beam:beam-sdks-java-core
   
+  -noverify
   
 
 

http://git-wip-us.apache.org/repos/asf/beam/blob/21554764/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
index ed1201d..9c8f7b3 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.gearpump;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.AggregatorValues;
@@ -26,31 +27,62 @@ import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.sdk.transforms.Aggregator;
 
+import org.apache.gearpump.cluster.MasterToAppMaster;
+import org.apache.gearpump.cluster.MasterToAppMaster.AppMasterData;
+import org.apache.gearpump.cluster.client.ClientContext;
 import org.joda.time.Duration;
 
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
 
 /**
  * Result of executing a {@link Pipeline} with Gearpump.
  */
 public class GearpumpPipelineResult implements PipelineResult {
+
+  private final ClientContext client;
+  private final int appId;
+  private final Duration defaultWaitDuration = Duration.standardSeconds(60);
+  private final Duration defaultWaitInterval = Duration.standardSeconds(10);
+
+  public GearpumpPipelineResult(ClientContext client, int appId) {
+this.client = client;
+this.appId = appId;
+  }
+
   @Override
   public State getState() {
-return null;
+return getGearpumpState();
   }
 
   @Override
   public State cancel() throws IOException {
-return null;
+client.shutdown(appId);
+return State.CANCELLED;
   }
 
   @Override
   public State waitUntilFinish(Duration duration) {
-return null;
+long start = System.currentTimeMillis();
+do {
+  try {
+Thread.sleep(defaultWaitInterval.getMillis());
+  } catch (InterruptedException e) {
+throw new RuntimeException(e);
+  }
+} while (State.RUNNING == getGearpumpState()
+&& (System.currentTimeMillis() - start) < duration.getMillis());
+
+if (State.RUNNING == getGearpumpState()) {
+  return State.DONE;
+} else {
+  return State.FAILED;
+}
   }
 
   @Override
   public State waitUntilFinish() {
-return null;
+return waitUntilFinish(defaultWaitDuration);
   }
 
   @Override
@@ -66,4 +98,23 @@ public class GearpumpPipelineResult implements 
PipelineResult {
 return null;
   }
 
+  private State getGearpumpState() {
+String status = null;
+List apps =
+JavaConverters.seqAsJavaListConverter(
+(Seq) client.listApps().appMasters()).asJava();
+for (AppMasterData app: apps) {
+  if (app.appId() == appId) {
+status = app.status();
+  }
+}
+if (null == status || 
status.equals(MasterToAppMaster.AppMasterNonExist())) {
+  return State.UNKNOWN;
+} else if 

[08/12] beam git commit: fix ParDo.BoundMulti translation

2017-01-23 Thread kenn
fix ParDo.BoundMulti translation


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

Branch: refs/heads/gearpump-runner
Commit: b2d326ff73afca5c8e941c8006e9d74261a6b9df
Parents: 364a3f0
Author: manuzhang 
Authored: Mon Jan 16 12:31:26 2017 +0800
Committer: manuzhang 
Committed: Mon Jan 16 12:31:26 2017 +0800

--
 .../gearpump/translators/ParDoBoundMultiTranslator.java| 6 --
 1 file changed, 4 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/b2d326ff/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 0d5b8bc..bf7073b 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
@@ -91,8 +91,7 @@ public class ParDoBoundMultiTranslator 
implements
 private final DoFnRunnerFactory doFnRunnerFactory;
 private DoFnRunner doFnRunner;
 private final DoFn doFn;
-private final List>> outputs 
= Lists
-.newArrayList();
+private List>> outputs;
 
 public DoFnMultiFunction(
 GearpumpPipelineOptions pipelineOptions,
@@ -127,6 +126,8 @@ public class ParDoBoundMultiTranslator 
implements
 
 @Override
 public Iterator>> 
apply(WindowedValue wv) {
+  outputs = Lists.newArrayList();
+
   if (null == doFnRunner) {
 doFnRunner = doFnRunnerFactory.createRunner();
   }
@@ -166,6 +167,7 @@ public class ParDoBoundMultiTranslator 
implements
 
 @Override
 public WindowedValue apply(WindowedValue> wv) {
+  // System.out.println(wv.getValue().getKey() + ":" + 
wv.getValue().getValue());
   return WindowedValue.of(wv.getValue().getValue(), wv.getTimestamp(),
   wv.getWindows(), wv.getPane());
 }



[11/12] beam git commit: note thread is interrupted on InterruptedException

2017-01-23 Thread kenn
note thread is interrupted on InterruptedException


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

Branch: refs/heads/gearpump-runner
Commit: d814857a6c372ba3f87106d49d3ce1ef7c3c7766
Parents: 85dcfbd
Author: manuzhang 
Authored: Fri Jan 20 13:21:24 2017 +0800
Committer: manuzhang 
Committed: Fri Jan 20 13:21:24 2017 +0800

--
 .../apache/beam/runners/gearpump/GearpumpPipelineResult.java | 8 +++-
 1 file changed, 7 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/d814857a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
index 3dd78de..9e53517 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
@@ -67,7 +67,13 @@ public class GearpumpPipelineResult implements 
PipelineResult {
 do {
   try {
 Thread.sleep(defaultWaitInterval.getMillis());
-  } catch (InterruptedException e) {
+  } catch (Exception e) {
+if (e instanceof InterruptedException) {
+  Thread.currentThread().interrupt();
+}
+if (e instanceof RuntimeException) {
+  throw (RuntimeException) e;
+}
 throw new RuntimeException(e);
   }
 } while (State.RUNNING == getGearpumpState()



[09/12] beam git commit: reduce timeout to wait for result

2017-01-23 Thread kenn
reduce timeout to wait for result


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

Branch: refs/heads/gearpump-runner
Commit: 7613ec44cedf12d1e7bf80e8bb6a505f09653c4f
Parents: b2d326f
Author: manuzhang 
Authored: Mon Jan 16 13:25:12 2017 +0800
Committer: manuzhang 
Committed: Mon Jan 16 13:25:12 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/7613ec44/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
index 9c8f7b3..3dd78de 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java
@@ -42,8 +42,8 @@ public class GearpumpPipelineResult implements PipelineResult 
{
 
   private final ClientContext client;
   private final int appId;
-  private final Duration defaultWaitDuration = Duration.standardSeconds(60);
-  private final Duration defaultWaitInterval = Duration.standardSeconds(10);
+  private final Duration defaultWaitDuration = Duration.standardSeconds(30);
+  private final Duration defaultWaitInterval = Duration.standardSeconds(5);
 
   public GearpumpPipelineResult(ClientContext client, int appId) {
 this.client = client;



[12/12] beam git commit: This closes #1661: Implement GearpumpPipelineResult

2017-01-23 Thread kenn
This closes #1661: Implement GearpumpPipelineResult

  note thread is interrupted on InterruptedException
  Remove cache for Gearpump on travis
  reduce timeout to wait for result
  fix ParDo.BoundMulti translation
  return encoded key for GroupByKey translation
  support OutputTimeFn
  update to latest gearpump dsl function interface
  fix group by window
  activate ROS on Gearpump by default
  update ROS configurations
  [BEAM-1180] Implement GearpumpPipelineResult


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

Branch: refs/heads/gearpump-runner
Commit: 1ed16f11a3fb24c3cc6773235651c4a9255d6fbc
Parents: 4c445dd d814857
Author: Kenneth Knowles 
Authored: Mon Jan 23 19:22:31 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Jan 23 19:22:31 2017 -0800

--
 .travis.yml |  1 +
 runners/gearpump/pom.xml|  9 +-
 .../gearpump/GearpumpPipelineResult.java| 65 -
 .../beam/runners/gearpump/GearpumpRunner.java   |  7 +-
 .../runners/gearpump/TestGearpumpRunner.java|  4 +
 .../translators/GroupByKeyTranslator.java   | 96 
 .../translators/ParDoBoundMultiTranslator.java  | 35 +--
 .../translators/TranslationContext.java |  1 -
 .../translators/WindowBoundTranslator.java  | 49 --
 .../translators/functions/DoFnFunction.java | 21 -
 .../gearpump/translators/io/GearpumpSource.java |  4 +-
 11 files changed, 238 insertions(+), 54 deletions(-)
--




[07/12] beam git commit: return encoded key for GroupByKey translation

2017-01-23 Thread kenn
return encoded key for GroupByKey translation


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

Branch: refs/heads/gearpump-runner
Commit: 364a3f089747ff4761cb5b54c963c8a8013574a0
Parents: f6aaf0d
Author: manuzhang 
Authored: Mon Jan 16 11:16:05 2017 +0800
Committer: manuzhang 
Committed: Mon Jan 16 11:16:05 2017 +0800

--
 .../translators/GroupByKeyTranslator.java   | 24 
 1 file changed, 20 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/364a3f08/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
--
diff --git 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
index e16a178..ac8e218 100644
--- 
a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
+++ 
b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java
@@ -22,17 +22,22 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import java.io.Serializable;
+import java.nio.ByteBuffer;
 import java.time.Instant;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 
@@ -56,6 +61,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator transform, TranslationContext 
context) {
 PCollection> input = context.getInput(transform);
+Coder inputKeyCoder = ((KvCoder) input.getCoder()).getKeyCoder();
 JavaStream>> inputStream =
 context.getInputStream(input);
 int parallelism = context.getPipelineOptions().getParallelism();
@@ -64,7 +70,7 @@ public class GroupByKeyTranslator implements 
TransformTranslator>>> outputStream = inputStream
 .window(Window.apply(new 
GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()),
 EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window")
-.groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window")
+.groupBy(new GroupByFn(inputKeyCoder), parallelism, 
"group_by_Key_and_Window")
 .map(new ValueToIterable(), "map_value_to_iterable")
 .map(new KeyedByTimestamp(), "keyed_by_timestamp")
 .reduce(new Merge(outputTimeFn), "merge")
@@ -128,11 +134,21 @@ public class GroupByKeyTranslator implements 
TransformTranslator extends
-  GroupByFunction>, K> {
+  GroupByFunction>, ByteBuffer> {
+
+private final Coder keyCoder;
+
+GroupByFn(Coder keyCoder) {
+  this.keyCoder = keyCoder;
+}
 
 @Override
-public K apply(WindowedValue> wv) {
-  return wv.getValue().getKey();
+public ByteBuffer apply(WindowedValue> wv) {
+  try {
+return ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, 
wv.getValue().getKey()));
+  } catch (CoderException e) {
+throw new RuntimeException(e);
+  }
 }
   }
 



[2/2] beam git commit: This closes #1833: Removes ReduceFnExecutor interface

2017-01-24 Thread kenn
This closes #1833: Removes ReduceFnExecutor interface


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

Branch: refs/heads/master
Commit: 11c3cd70b784650e8b60a5660449cfafdba84bbf
Parents: b333487 8989473
Author: Kenneth Knowles 
Authored: Tue Jan 24 13:48:23 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Jan 24 13:48:23 2017 -0800

--
 .../apache/beam/runners/core/DoFnRunner.java| 20 
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  5 +
 .../beam/runners/direct/ParDoEvaluator.java |  2 --
 .../runners/spark/translation/DoFnFunction.java |  2 --
 .../spark/translation/MultiDoFnFunction.java|  2 --
 5 files changed, 1 insertion(+), 30 deletions(-)
--




[2/2] beam git commit: This closes #1828: Fix PostCommit test confs for Gearpump runner

2017-01-25 Thread kenn
This closes #1828: Fix PostCommit test confs for Gearpump runner


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

Branch: refs/heads/gearpump-runner
Commit: 4001aeb191062c5a7f83ef919f781ec413be8cb2
Parents: 1ed16f1 4fd216b
Author: Kenneth Knowles 
Authored: Wed Jan 25 22:40:04 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Jan 25 22:40:04 2017 -0800

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




[1/4] beam git commit: DoFnSignature: deprecate isStateful in favor of usesState

2017-01-20 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master eecbcc48b -> 77c7505b8


DoFnSignature: deprecate isStateful in favor of usesState


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

Branch: refs/heads/master
Commit: ed434ecfdbcfc339815c85aa15852124bb2d3981
Parents: eecbcc4
Author: Kenneth Knowles 
Authored: Fri Jan 20 11:28:45 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Jan 20 11:35:10 2017 -0800

--
 .../beam/runners/dataflow/DataflowPipelineTranslator.java |  2 +-
 .../apache/beam/sdk/transforms/reflect/DoFnSignature.java | 10 +++---
 2 files changed, 8 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/ed434ecf/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index 7609745..cfb3d1a 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -943,7 +943,7 @@ public class DataflowPipelineTranslator {
 DoFnInfo.forFn(
 fn, windowingStrategy, sideInputs, inputCoder, mainOutput, 
outputMap;
 
-if (signature.isStateful()) {
+if (signature.usesState()) {
   stepContext.addInput(PropertyNames.USES_KEYED_STATE, "true");
 }
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/ed434ecf/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
index ccc9ac3..5255adc 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
@@ -105,13 +105,17 @@ public abstract class DoFnSignature {
   @Nullable
   public abstract Map onTimerMethods();
 
-  /**
-   * Whether the {@link DoFn} described by this signature uses state.
-   */
+  /** @deprecated use {@link #usesState()}, it's cleaner */
+  @Deprecated
   public boolean isStateful() {
 return stateDeclarations().size() > 0;
   }
 
+  /** Whether the {@link DoFn} described by this signature uses state. */
+  public boolean usesState() {
+return stateDeclarations().size() > 0;
+  }
+
   static Builder builder() {
 return new AutoValue_DoFnSignature.Builder();
   }



[3/4] beam git commit: Set USES_KEYED_STATE in Dataflow when DoFn uses timers

2017-01-20 Thread kenn
Set USES_KEYED_STATE in Dataflow when DoFn uses timers


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

Branch: refs/heads/master
Commit: f535d658b551f56ebe4f2c77948fef63be7f44dc
Parents: 978ff55
Author: Kenneth Knowles 
Authored: Fri Jan 20 11:36:52 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Jan 20 11:36:52 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/f535d658/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
--
diff --git 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index cfb3d1a..babbe69 100644
--- 
a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ 
b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -943,7 +943,7 @@ public class DataflowPipelineTranslator {
 DoFnInfo.forFn(
 fn, windowingStrategy, sideInputs, inputCoder, mainOutput, 
outputMap;
 
-if (signature.usesState()) {
+if (signature.usesState() || signature.usesTimers()) {
   stepContext.addInput(PropertyNames.USES_KEYED_STATE, "true");
 }
   }



[3/4] beam git commit: Point to new Dataflow worker

2017-01-20 Thread kenn
Point to new Dataflow worker


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

Branch: refs/heads/master
Commit: a2b94ecabd80ef68654ba3bc8ffa0a0c3759316b
Parents: 5f8b8c5
Author: Eugene Kirpichov 
Authored: Fri Jan 20 13:31:30 2017 -0800
Committer: Eugene Kirpichov 
Committed: Fri Jan 20 13:32:19 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/a2b94eca/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 161a897..9976ed9 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-20170119
+worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20170120
 
-worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20170119
+worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20170120



[2/4] beam git commit: Moves OldDoFn to runners-core

2017-01-20 Thread kenn
Moves OldDoFn to runners-core


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

Branch: refs/heads/master
Commit: 5f8b8c5b06cfd49c4293a20dff2eea99f1076444
Parents: 77c7505
Author: Eugene Kirpichov 
Authored: Tue Jan 17 16:12:39 2017 -0800
Committer: Eugene Kirpichov 
Committed: Fri Jan 20 13:31:58 2017 -0800

--
 .../apex/translation/WindowBoundTranslator.java |   2 +-
 .../operators/ApexGroupByKeyOperator.java   |   2 +-
 .../operators/ApexParDoOperator.java|   2 +-
 .../beam/runners/core/AssignWindowsDoFn.java|   3 +-
 .../apache/beam/runners/core/DoFnAdapters.java  |   1 -
 .../apache/beam/runners/core/DoFnRunner.java|   1 -
 .../apache/beam/runners/core/DoFnRunners.java   |   1 -
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   1 -
 .../runners/core/GroupAlsoByWindowsDoFn.java|   1 -
 .../core/LateDataDroppingDoFnRunner.java|   1 -
 .../org/apache/beam/runners/core/OldDoFn.java   | 472 +++
 .../beam/runners/core/SimpleOldDoFnRunner.java  |   3 +-
 .../core/DoFnDelegatingAggregatorTest.java  | 144 ++
 .../apache/beam/runners/core/NoOpOldDoFn.java   |  72 +++
 .../beam/runners/core/OldDoFnContextTest.java   |  72 +++
 .../apache/beam/runners/core/OldDoFnTest.java   | 192 
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   2 +-
 .../runners/flink/OldPerKeyCombineFnRunner.java |   2 +-
 .../flink/OldPerKeyCombineFnRunners.java|   2 +-
 .../functions/FlinkDoFnFunction.java|   2 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   2 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |   2 +-
 .../FlinkMultiOutputProcessContext.java |   2 +-
 .../functions/FlinkNoElementAssignContext.java  |   2 +-
 .../functions/FlinkPartialReduceFunction.java   |   2 +-
 .../functions/FlinkProcessContextBase.java  |   2 +-
 .../functions/FlinkReduceFunction.java  |   2 +-
 .../FlinkSingleOutputProcessContext.java|   2 +-
 .../wrappers/streaming/DoFnOperator.java|   2 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |   2 +-
 .../beam/runners/dataflow/util/DoFnInfo.java|   4 +-
 .../sdk/transforms/DelegatingAggregator.java|   2 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 465 --
 .../org/apache/beam/sdk/util/NameUtils.java |   2 +-
 .../DoFnDelegatingAggregatorTest.java   | 142 --
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |  71 ---
 .../beam/sdk/transforms/OldDoFnContextTest.java |  69 ---
 .../apache/beam/sdk/transforms/OldDoFnTest.java | 187 
 .../org/apache/beam/sdk/util/NameUtilsTest.java |  20 +-
 39 files changed, 982 insertions(+), 978 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/5f8b8c5b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
--
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
index ef049e1..50af81d 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java
@@ -23,8 +23,8 @@ import org.apache.beam.runners.apex.ApexPipelineOptions;
 import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator;
 import org.apache.beam.runners.core.AssignWindowsDoFn;
 import org.apache.beam.runners.core.DoFnAdapters;
+import org.apache.beam.runners.core.OldDoFn;
 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.transforms.windowing.Window;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/beam/blob/5f8b8c5b/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 173434f..4c2b461 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
+++ 

[1/2] beam git commit: Fix some DoFn javadoc literals

2017-02-15 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 2e766ced5 -> 5fe78440b


Fix some DoFn javadoc literals


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

Branch: refs/heads/master
Commit: f360f47f9ca4f4054e9fb583c2a0f5dda9ee19ea
Parents: 18f3767
Author: Kenneth Knowles 
Authored: Tue Feb 7 14:11:13 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 15 19:18:42 2017 -0800

--
 .../org/apache/beam/sdk/transforms/DoFn.java| 66 ++--
 1 file changed, 32 insertions(+), 34 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/f360f47f/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 2043ef0..6f88738 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -77,15 +77,15 @@ import org.joda.time.Instant;
  *
  * Example usage:
  *
- * {@code
- * PCollection lines = ... ;
- * PCollection words =
- * lines.apply(ParDo.of(new DoFn() {
- * {@literal @}ProcessElement
- * public void processElement(ProcessContext c, BoundedWindow window) {
- *
- * }}));
- * }
+ * 
+ * {@literal PCollection} lines = ... ;
+ * {@literal PCollection} words =
+ * {@literal lines.apply(ParDo.of(new DoFn())} {
+ * {@literal @ProcessElement}
+ *  public void processElement(ProcessContext c, BoundedWindow window) 
{
+ *...
+ *  }}));
+ * 
  *
  * @param  the type of the (main) input elements
  * @param  the type of the (main) output elements
@@ -385,21 +385,21 @@ public abstract class DoFn implements 
Serializable, HasDisplayD
* subclass to your {@link ProcessElement @ProcessElement} or {@link OnTimer 
@OnTimer} method, and
* annotate it with {@link StateId}. See the following code for an example:
*
-   * {@code
-   * new DoFn, Baz>() {
-   *   {@literal @}StateId("my-state-id")
-   *   private final StateSpec myStateSpec =
+   * {@literal new DoFn, Baz>()} {
+   *
+   *  {@literal @StateId("my-state-id")}
+   *  {@literal private final StateSpec} myStateSpec =
*   StateSpecs.value(new MyStateCoder());
*
-   *   {@literal @}ProcessElement
+   *  {@literal @ProcessElement}
*   public void processElement(
*   ProcessContext c,
-   *   {@literal @}StateId("my-state-id") ValueState myState) {
+   *  {@literal @StateId("my-state-id") ValueState myState}) {
* myState.read();
* myState.write(...);
*   }
* }
-   * }
+   * 
*
* State is subject to the following validity conditions:
*
@@ -429,24 +429,22 @@ public abstract class DoFn implements 
Serializable, HasDisplayD
* {@link ProcessElement @ProcessElement} or {@link OnTimer @OnTimer} 
method, and annotate it with
* {@link TimerId}. See the following code for an example:
*
-   * {@code
-   * new DoFn, Baz>() {
-   *   {@literal @}TimerId("my-timer-id")
-   *   private final TimerSpec myTimer = 
TimerSpecs.timerForDomain(TimeDomain.EVENT_TIME);
-   *
-   *   {@literal @}ProcessElement
-   *   public void processElement(
-   *   ProcessContext c,
-   *   {@literal @}TimerId("my-timer-id") Timer myTimer) {
-   * myTimer.setForNowPlus(Duration.standardSeconds(...));
-   *   }
-   *
-   *   {@literal @}OnTimer("my-timer-id")
-   *   public void onMyTimer() {
-   * ...
-   *   }
-   * }
-   * }
+   * {@literal new DoFn, Baz>()} {
+   *   {@literal @TimerId("my-timer-id")}
+   *private final TimerSpec myTimer = 
TimerSpecs.timerForDomain(TimeDomain.EVENT_TIME);
+   *
+   *   {@literal @ProcessElement}
+   *public void processElement(
+   *ProcessContext c,
+   *   {@literal @TimerId("my-timer-id") Timer myTimer}) {
+   *  myTimer.setForNowPlus(Duration.standardSeconds(...));
+   *}
+   *
+   *   {@literal @OnTimer("my-timer-id")}
+   *public void onMyTimer() {
+   *  ...
+   *}
+   * }
*
* Timers are subject to the following validity conditions:
*



[2/2] beam git commit: This closes #1940: Fix some DoFn javadoc literals

2017-02-15 Thread kenn
This closes #1940: Fix some DoFn javadoc literals


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

Branch: refs/heads/master
Commit: 5fe78440bd603702661595d0d92c33c38d38a51a
Parents: 2e766ce f360f47
Author: Kenneth Knowles 
Authored: Wed Feb 15 20:17:18 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 15 20:17:18 2017 -0800

--
 .../org/apache/beam/sdk/transforms/DoFn.java| 66 ++--
 1 file changed, 32 insertions(+), 34 deletions(-)
--




[1/2] beam git commit: Upgrade bytebuddy to 1.6.8 to jump past asm 5.0

2017-02-15 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 18f3767e3 -> 2e766ced5


Upgrade bytebuddy to 1.6.8 to jump past asm 5.0

There is a suspected bug in asm 5.0 that is considered the likely root cause of
a bug sbt-assembly [1] that carried over to Gearpump [2]. This commit upgrades
us to depend on asm 5.2 in which those derivative bugs have cleared up.

I have not found a direct reference to what the issue is, precisely, but
the dependency effect of this is extremely small and these are libraries
that are useful to keep current.

[1] https://github.com/sbt/sbt-assembly/issues/205#issuecomment-279964607
[2] https://issues.apache.org/jira/browse/GEARPUMP-236


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

Branch: refs/heads/master
Commit: 3e4c05c3449064e7f032a48b98551a73d71a5bbb
Parents: 00ea3f7
Author: Kenneth Knowles 
Authored: Wed Feb 15 14:02:38 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 15 14:15:36 2017 -0800

--
 pom.xml  | 2 +-
 .../beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java | 4 ++--
 .../sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java   | 4 ++--
 3 files changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/3e4c05c3/pom.xml
--
diff --git a/pom.xml b/pom.xml
index d53502e..cc6de11 100644
--- a/pom.xml
+++ b/pom.xml
@@ -871,7 +871,7 @@
   
 net.bytebuddy
 byte-buddy
-1.5.5
+1.6.8
   
 
   

http://git-wip-us.apache.org/repos/asf/beam/blob/3e4c05c3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
index 46b21d6..8e3a37c 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
@@ -427,7 +427,7 @@ public class ByteBuddyDoFnInvokerFactory implements 
DoFnInvokerFactory {
   // Push "this" (DoFnInvoker on top of the stack)
   MethodVariableAccess.REFERENCE.loadFrom(0),
   // Access this.delegate (DoFn on top of the stack)
-  FieldAccess.forField(delegateField).getter(),
+  FieldAccess.forField(delegateField).read(),
   // Cast it to the more precise type
   TypeCasting.to(doFnType),
   // Run the beforeDelegation manipulations.
@@ -637,7 +637,7 @@ public class ByteBuddyDoFnInvokerFactory implements 
DoFnInvokerFactory {
   StackManipulation pushDelegate =
   new StackManipulation.Compound(
   MethodVariableAccess.REFERENCE.loadFrom(0),
-  FieldAccess.forField(delegateField).getter());
+  FieldAccess.forField(delegateField).read());
 
   StackManipulation pushExtraContextFactory = 
MethodVariableAccess.REFERENCE.loadFrom(1);
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3e4c05c3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
index 786857a..123808c 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
@@ -239,7 +239,7 @@ class ByteBuddyOnTimerInvokerFactory implements 
OnTimerInvokerFactory {
   StackManipulation pushDelegate =
   new StackManipulation.Compound(
   MethodVariableAccess.REFERENCE.loadFrom(0),
-  FieldAccess.forField(delegateField).getter());
+  FieldAccess.forField(delegateField).read());
 
   StackManipulation pushExtraContextFactory = 
MethodVariableAccess.REFERENCE.loadFrom(1);
 
@@ -295,7 +295,7 @@ class ByteBuddyOnTimerInvokerFactory implements 
OnTimerInvokerFactory {
   

[2/2] beam git commit: This closes #2014: Upgrade bytebuddy to 1.6.8 to jump past asm 5.0

2017-02-15 Thread kenn
This closes #2014: Upgrade bytebuddy to 1.6.8 to jump past asm 5.0


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

Branch: refs/heads/master
Commit: 2e766ced5e1bf3a605e13da208091d170486f9a8
Parents: 18f3767 3e4c05c
Author: Kenneth Knowles 
Authored: Wed Feb 15 19:12:09 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 15 19:12:09 2017 -0800

--
 pom.xml  | 2 +-
 .../beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java | 4 ++--
 .../sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java   | 4 ++--
 3 files changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/2e766ced/pom.xml
--



[1/2] beam git commit: Add ability to bundle any message with components in Runner API

2017-02-16 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 763fb50b5 -> 9151676a5


Add ability to bundle any message with components in Runner API


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

Branch: refs/heads/master
Commit: 368ad236ca03bd011d2f207bcc34f5961290c0a0
Parents: cd4e6e4
Author: Kenneth Knowles 
Authored: Thu Feb 16 14:07:40 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 16 15:18:24 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto| 66 
 1 file changed, 54 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/368ad236/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index a9133ab..f5dc81d 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -30,17 +30,9 @@ option java_outer_classname = "RunnerApi";
 
 import "google/protobuf/any.proto";
 
-// A Pipeline is a hierarchical graph of PTransforms, linked
-// by PCollections.
-//
-// This is represented by a number of by-reference maps to nodes,
-// PCollections, SDK environments, UDF, etc., for
-// supporting compact reuse and arbitrary graph structure.
-//
-// All of the keys in the maps here are arbitrary strings that are only
-// required to be internally consistent within this proto message.
-message Pipeline {
-
+// A set of mappings from id to message. This is included as an optional field
+// on any proto message that may contain references needing resolution.
+message Components {
   // (Required) A map from pipeline-scoped id to GraphNode
   //
   // Each node is required to contain a PTransform specification.
@@ -61,9 +53,58 @@ message Pipeline {
   // (Required) A map from pipeline-scoped id to FunctionSpec,
   // a record for a particular user-defined function.
   map function_specs = 6;
+}
+
+// A disjoint union of all the things that may contain references
+// that require Components to resolve.
+message MessageWithComponents {
+
+  // (Optional) The by-reference components of the root message,
+  // enabling a standalone message.
+  //
+  // If this is absent, it is expected that there are no
+  // references.
+  Components components = 1;
+
+  // (Required) The root message that may contain pointers
+  // that should be resolved by looking inside components.
+  oneof root {
+Coder coder = 2;
+CombinePayload combine_payload = 3;
+FunctionSpec function_spec = 4;
+GraphNode graph_node = 5;
+ParDoPayload par_do_payload = 6;
+PTransform ptransform = 7;
+PCollection pcollection = 8;
+ReadPayload read_payload = 9;
+SdkFunctionSpec sdk_function_spec = 10;
+SideInput side_input = 11;
+WindowIntoPayload window_into_payload = 12;
+WindowingStrategy windowing_strategy = 13;
+UrnWithParameter urn_with_parameter = 14;
+  }
+}
+
+// A Pipeline is a hierarchical graph of PTransforms, linked
+// by PCollections.
+//
+// This is represented by a number of by-reference maps to nodes,
+// PCollections, SDK environments, UDF, etc., for
+// supporting compact reuse and arbitrary graph structure.
+//
+// All of the keys in the maps here are arbitrary strings that are only
+// required to be internally consistent within this proto message.
+message Pipeline {
+
+  // (Required) The coders, UDFs, graph nodes, etc, that make up
+  // this pipeline.
+  Components components = 1;
+
+  // (Required) The graph node that is the root of the graph.
+  string root_graph_node = 2;
 
   // (Required) Static display data for the pipeline.
-  DisplayData display_data = 7;
+  DisplayData display_data = 3;
 }
 
 // A generic node in a bipartite directed hierarchical graph.
@@ -270,6 +311,7 @@ message ReadPayload {
 
 // The payload for the WindowInto transform.
 message WindowIntoPayload {
+
   // (Required) The pipeline-scoped id for the FunctionSpec of the WindowFn.
   string fn_id = 1;
 }



[2/2] beam git commit: This closes #2023: Add ability to bundle any message with components in Runner API

2017-02-16 Thread kenn
This closes #2023: Add ability to bundle any message with components in Runner 
API


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

Branch: refs/heads/master
Commit: 9151676a53e74a330773f7bfdb82dd7ed16640df
Parents: 763fb50 368ad23
Author: Kenneth Knowles 
Authored: Thu Feb 16 15:19:59 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 16 15:19:59 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto| 66 
 1 file changed, 54 insertions(+), 12 deletions(-)
--




[1/2] beam git commit: Use strings for ids in Fn API

2017-02-23 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 4f56acbba -> 2c71354d0


Use strings for ids in Fn API


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

Branch: refs/heads/master
Commit: 9933f27140ddfe5b9ded4a0688a9c0506ef94113
Parents: 4f56acb
Author: Kenneth Knowles 
Authored: Sun Feb 12 22:23:32 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 23 17:35:22 2017 -0800

--
 .../fn-api/src/main/proto/beam_fn_api.proto | 48 +-
 .../fn/harness/control/BeamFnControlClient.java |  3 +-
 .../harness/control/ProcessBundleHandler.java   |  8 +-
 .../fn/harness/control/RegisterHandler.java |  8 +-
 .../BeamFnDataBufferingOutboundObserver.java|  4 +-
 .../beam/fn/harness/data/BeamFnDataClient.java  |  4 +-
 .../fn/harness/data/BeamFnDataGrpcClient.java   |  4 +-
 .../harness/data/BeamFnDataGrpcMultiplexer.java | 11 +--
 .../fn/harness/logging/BeamFnLoggingClient.java |  4 +-
 .../beam/runners/core/BeamFnDataReadRunner.java |  4 +-
 .../runners/core/BeamFnDataWriteRunner.java |  4 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |  8 +-
 .../control/BeamFnControlClientTest.java| 12 +--
 .../control/ProcessBundleHandlerTest.java   | 95 ++--
 .../fn/harness/control/RegisterHandlerTest.java | 18 ++--
 ...BeamFnDataBufferingOutboundObserverTest.java |  9 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  | 21 +++--
 .../data/BeamFnDataGrpcMultiplexerTest.java | 12 +--
 .../data/BeamFnDataInboundObserverTest.java |  4 +-
 .../runners/core/BeamFnDataReadRunnerTest.java  |  8 +-
 .../runners/core/BeamFnDataWriteRunnerTest.java |  8 +-
 21 files changed, 159 insertions(+), 138 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/9933f271/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
--
diff --git a/sdks/common/fn-api/src/main/proto/beam_fn_api.proto 
b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
index 3ac0fbf..80bae2e 100644
--- a/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
+++ b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
@@ -59,7 +59,7 @@ message Target {
   }
 
   // (Required) The id of the PrimitiveTransform which is the target.
-  int64 primitive_transform_reference = 1;
+  string primitive_transform_reference = 1;
 
   // (Required) The local name of an input or output defined on the primitive
   // transform.
@@ -69,7 +69,7 @@ message Target {
 // Information defining a PCollection
 message PCollection {
   // (Required) A reference to a coder.
-  int64 coder_reference = 1;
+  string coder_reference = 1;
 
   // TODO: Windowing strategy, ...
 }
@@ -78,7 +78,7 @@ message PCollection {
 message PrimitiveTransform {
   // (Required) A pipeline level unique id which can be used as a reference to
   // refer to this.
-  int64 id = 1;
+  string id = 1;
 
   // (Required) A function spec that is used by this primitive
   // transform to process data.
@@ -117,7 +117,7 @@ message PrimitiveTransform {
 message FunctionSpec {
   // (Required) A pipeline level unique id which can be used as a reference to
   // refer to this.
-  int64 id = 1;
+  string id = 1;
 
   // (Required) A globally unique name representing this user definable
   // function.
@@ -131,7 +131,7 @@ message FunctionSpec {
 
   // (Required) Reference to specification of execution environment required to
   // invoke this function.
-  int64 environment_reference = 3;
+  string environment_reference = 3;
 
   // Data used to parameterize this function. Depending on the urn, this may be
   // optional or required.
@@ -179,7 +179,7 @@ message Coder {
   //
   // TODO: Perhaps this is redundant with the data of the FunctionSpec
   // for known coders?
-  repeated int64 component_coder_reference = 2;
+  repeated string component_coder_reference = 2;
 }
 
 // A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -218,7 +218,7 @@ service BeamFnControl {
 message InstructionRequest {
   // (Required) An unique identifier provided by the runner which represents
   // this requests execution. The InstructionResponse MUST have the matching 
id.
-  int64 instruction_id = 1;
+  string instruction_id = 1;
 
   // (Required) A request that the SDK Harness needs to interpret.
   oneof request {
@@ -235,7 +235,7 @@ message InstructionResponse {
   // (Required) A reference provided by the runner which represents a requests
   // execution. The InstructionResponse MUST have the matching id when
   // responding to the runner.
-  int64 instruction_id = 1;
+  string instruction_id = 1;
 
   // If 

[2/2] beam git commit: This closes #1190: Use strings for ids in Fn API

2017-02-23 Thread kenn
This closes #1190: Use strings for ids in Fn API


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

Branch: refs/heads/master
Commit: 2c71354d0a337151fd53dc7eaeace439bcbd41ef
Parents: 4f56acb 9933f27
Author: Kenneth Knowles 
Authored: Thu Feb 23 18:27:28 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 23 18:27:28 2017 -0800

--
 .../fn-api/src/main/proto/beam_fn_api.proto | 48 +-
 .../fn/harness/control/BeamFnControlClient.java |  3 +-
 .../harness/control/ProcessBundleHandler.java   |  8 +-
 .../fn/harness/control/RegisterHandler.java |  8 +-
 .../BeamFnDataBufferingOutboundObserver.java|  4 +-
 .../beam/fn/harness/data/BeamFnDataClient.java  |  4 +-
 .../fn/harness/data/BeamFnDataGrpcClient.java   |  4 +-
 .../harness/data/BeamFnDataGrpcMultiplexer.java | 11 +--
 .../fn/harness/logging/BeamFnLoggingClient.java |  4 +-
 .../beam/runners/core/BeamFnDataReadRunner.java |  4 +-
 .../runners/core/BeamFnDataWriteRunner.java |  4 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |  8 +-
 .../control/BeamFnControlClientTest.java| 12 +--
 .../control/ProcessBundleHandlerTest.java   | 95 ++--
 .../fn/harness/control/RegisterHandlerTest.java | 18 ++--
 ...BeamFnDataBufferingOutboundObserverTest.java |  9 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  | 21 +++--
 .../data/BeamFnDataGrpcMultiplexerTest.java | 12 +--
 .../data/BeamFnDataInboundObserverTest.java |  4 +-
 .../runners/core/BeamFnDataReadRunnerTest.java  |  8 +-
 .../runners/core/BeamFnDataWriteRunnerTest.java |  8 +-
 21 files changed, 159 insertions(+), 138 deletions(-)
--




[1/5] beam git commit: Factory method best practice for ReshuffleTriggerStateMachine

2017-02-24 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 3d0fe8539 -> 25a014f40


Factory method best practice for ReshuffleTriggerStateMachine


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

Branch: refs/heads/master
Commit: bd8b72c6d2f2beaef516a1b11ddf18639c3e7fb1
Parents: bfdbebd
Author: Kenneth Knowles 
Authored: Thu Feb 23 19:41:17 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 07:16:35 2017 -0800

--
 .../runners/core/triggers/ReshuffleTriggerStateMachine.java  | 6 +-
 .../core/triggers/ReshuffleTriggerStateMachineTest.java  | 8 
 2 files changed, 9 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/bd8b72c6/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
index fc9f203..8a2b736 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java
@@ -25,7 +25,11 @@ import org.apache.beam.sdk.util.Reshuffle;
  */
 public class ReshuffleTriggerStateMachine extends TriggerStateMachine {
 
-  public ReshuffleTriggerStateMachine() {
+  public static ReshuffleTriggerStateMachine create() {
+return new ReshuffleTriggerStateMachine();
+  }
+
+  private ReshuffleTriggerStateMachine() {
 super(null);
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/bd8b72c6/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
index ef74bb5..f8b0324 100644
--- 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
+++ 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java
@@ -38,14 +38,14 @@ public class ReshuffleTriggerStateMachineTest {
 
   /** Public so that other tests can instantiate {@link 
ReshuffleTriggerStateMachine}. */
   public static  ReshuffleTriggerStateMachine 
forTest() {
-return new ReshuffleTriggerStateMachine();
+return ReshuffleTriggerStateMachine.create();
   }
 
   @Test
   public void testShouldFire() throws Exception {
 TriggerStateMachineTester tester =
 TriggerStateMachineTester.forTrigger(
-new ReshuffleTriggerStateMachine(), 
FixedWindows.of(Duration.millis(100)));
+ReshuffleTriggerStateMachine.create(), 
FixedWindows.of(Duration.millis(100)));
 IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(300), new 
Instant(400));
 assertTrue(tester.shouldFire(arbitraryWindow));
   }
@@ -54,7 +54,7 @@ public class ReshuffleTriggerStateMachineTest {
   public void testOnTimer() throws Exception {
 TriggerStateMachineTester tester =
 TriggerStateMachineTester.forTrigger(
-new ReshuffleTriggerStateMachine(), 
FixedWindows.of(Duration.millis(100)));
+ReshuffleTriggerStateMachine.create(), 
FixedWindows.of(Duration.millis(100)));
 IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(100), new 
Instant(200));
 tester.fireIfShouldFire(arbitraryWindow);
 assertFalse(tester.isMarkedFinished(arbitraryWindow));
@@ -62,7 +62,7 @@ public class ReshuffleTriggerStateMachineTest {
 
   @Test
   public void testToString() {
-TriggerStateMachine trigger = new ReshuffleTriggerStateMachine();
+TriggerStateMachine trigger = ReshuffleTriggerStateMachine.create();
 assertEquals("ReshuffleTriggerStateMachine()", trigger.toString());
   }
 }



[2/5] beam git commit: Translate ReshuffleTrigger into Always trigger proto

2017-02-24 Thread kenn
Translate ReshuffleTrigger into Always trigger proto


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

Branch: refs/heads/master
Commit: bfdbebdd6ecdc29d9eb12590399ebae45466a038
Parents: 9dbaeaf
Author: Kenneth Knowles 
Authored: Sat Feb 18 19:19:33 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 07:16:35 2017 -0800

--
 sdks/common/runner-api/src/main/proto/beam_runner_api.proto   | 6 ++
 .../org/apache/beam/sdk/transforms/windowing/Triggers.java| 7 +++
 2 files changed, 13 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/bfdbebdd/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index ce089f5..c030e73 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -540,6 +540,11 @@ message Trigger {
   message Never {
   }
 
+  // Always ready. This can also be expressed as ElementCount(1) but
+  // is more explicit.
+  message Always {
+  }
+
   // Ready whenever either of its subtriggers are ready, but finishes output
   // when the finally subtrigger fires.
   message OrFinally {
@@ -566,6 +571,7 @@ message Trigger {
 AfterEndOfWindow after_end_of_widow = 4;
 AfterProcessingTime after_processing_time = 5;
 AfterSynchronizedProcessingTime after_synchronized_processing_time = 6;
+Always always = 12;
 Default default = 7;
 ElementCount element_count = 8;
 Never never = 9;

http://git-wip-us.apache.org/repos/asf/beam/blob/bfdbebdd/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
index 8ac904c..d788ca2 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
@@ -28,6 +28,7 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi;
 import 
org.apache.beam.sdk.transforms.windowing.AfterWatermark.AfterWatermarkEarlyAndLate;
 import org.apache.beam.sdk.transforms.windowing.Never.NeverTrigger;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.ReshuffleTrigger;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -95,6 +96,12 @@ public class Triggers implements Serializable {
   .build();
 }
 
+private RunnerApi.Trigger convertSpecific(ReshuffleTrigger v) {
+  return RunnerApi.Trigger.newBuilder()
+  .setAlways(RunnerApi.Trigger.Always.getDefaultInstance())
+  .build();
+}
+
 private RunnerApi.Trigger convertSpecific(AfterSynchronizedProcessingTime 
v) {
   return RunnerApi.Trigger.newBuilder()
   .setAfterSynchronizedProcessingTime(



[3/5] beam git commit: Add dependency from runners-core-java to runner-api

2017-02-24 Thread kenn
Add dependency from runners-core-java to runner-api


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

Branch: refs/heads/master
Commit: 9dbaeaf5c33808a89b2caa9122de7c950e7a0f26
Parents: 4f56acb
Author: Kenneth Knowles 
Authored: Fri Feb 17 18:58:31 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 07:16:35 2017 -0800

--
 runners/core-java/pom.xml | 5 +
 1 file changed, 5 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/9dbaeaf5/runners/core-java/pom.xml
--
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index 4293035..28ab0ee 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -117,6 +117,11 @@
   beam-sdks-java-core
 
 
+
+  org.apache.beam
+  beam-sdks-common-runner-api
+
+
 
 
 



[4/5] beam git commit: Build trigger state machine from Runner API Trigger proto directly

2017-02-24 Thread kenn
Build trigger state machine from Runner API Trigger proto directly


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

Branch: refs/heads/master
Commit: 27a482baf498bab8c931670adec5134c0bdf08ac
Parents: bd8b72c
Author: Kenneth Knowles 
Authored: Fri Feb 17 16:05:13 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 07:16:36 2017 -0800

--
 .../GroupAlsoByWindowViaOutputBufferDoFn.java   |   4 +-
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   4 +-
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   |   4 +-
 .../core/triggers/AfterAllStateMachine.java |   2 +-
 .../core/triggers/AfterFirstStateMachine.java   |   2 +-
 .../triggers/AfterWatermarkStateMachine.java|  14 +-
 .../core/triggers/OrFinallyStateMachine.java|   2 +-
 .../core/triggers/TriggerStateMachine.java  |   2 +-
 .../core/triggers/TriggerStateMachines.java | 272 ++-
 .../beam/runners/core/ReduceFnTester.java   |   8 +-
 .../core/triggers/TriggerStateMachinesTest.java | 160 +++
 .../GroupAlsoByWindowEvaluatorFactory.java  |   4 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |  40 +--
 13 files changed, 243 insertions(+), 275 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/27a482ba/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java
index 5b2e051..e3ce1ef 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
 import org.apache.beam.runners.core.triggers.TriggerStateMachines;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.Triggers;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.joda.time.Instant;
@@ -63,7 +64,8 @@ public class GroupAlsoByWindowViaOutputBufferDoFn

[2/3] beam git commit: Condense FunctionSpec and remove SdkFunctionSpec, merging data and params

2017-02-24 Thread kenn
Condense FunctionSpec and remove SdkFunctionSpec, merging data and params


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

Branch: refs/heads/master
Commit: 3120fd9a499b01b08659edc4e045f5abbcc3ae07
Parents: 672d12a
Author: Kenneth Knowles 
Authored: Thu Feb 23 18:06:56 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 14:38:28 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto| 114 ++-
 .../beam/sdk/util/WindowingStrategies.java  | 110 --
 2 files changed, 134 insertions(+), 90 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/3120fd9a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index 32c53fb..989e4bb 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -74,7 +74,6 @@ message MessageWithComponents {
 PTransform ptransform = 7;
 PCollection pcollection = 8;
 ReadPayload read_payload = 9;
-SdkFunctionSpec sdk_function_spec = 10;
 SideInput side_input = 11;
 WindowIntoPayload window_into_payload = 12;
 WindowingStrategy windowing_strategy = 13;
@@ -100,7 +99,8 @@ message Pipeline {
   // (Required) The id of the PTransform that is the root of the pipeline
   string root_transform_id = 2;
 
-  // (Required) Static display data for the pipeline.
+  // (Optional) Static display data for the pipeline. If there is none,
+  // it may be omitted.
   DisplayData display_data = 3;
 }
 
@@ -116,7 +116,8 @@ message PTransform {
   // etc.
   //
   // If it is not stable, then the runner decides what will happen. But, most
-  // importantly, it must always be here, even if it is autogenerated.
+  // importantly, it must always be here and be unique, even if it is
+  // autogenerated.
   string unique_name = 5;
 
   // (Optional) A URN and payload that, together, fully defined the semantics
@@ -125,13 +126,11 @@ message PTransform {
   // If absent, this must be an "anonymous" composite transform.
   //
   // For primitive transform in the Runner API, this is required, and the
-  // payloads are as follows:
+  // payloads are well-defined messages. When the URN indicates ParDo it
+  // is a ParDoPayload, and so on.
   //
-  //  - when the URN is "urn:beam:transforms:pardo" it is a ParDoPayload
-  //  - when the URN is "urn:beam:transforms:read" it is a ReadPayload
-  //  - when the URN is "urn:beam:transforms:gbk" it is a GroupByKeyPayload
-  //  - when the URN is "urn:beam:transforms:window" it is a WindowPayload
-  //  - when the URN is "urn:beam:transforms:flatten" it is absent
+  // TODO: document the standardized URNs and payloads
+  // TODO: separate standardized payloads into a separate proto file
   //
   // For some special composite transforms, the payload is also officially
   // defined:
@@ -144,7 +143,9 @@ message PTransform {
   // transforms that it contains.
   repeated string subtransforms = 2;
 
-  // (Required) A map from local names of inputs to PCollection ids.
+  // (Required) A map from local names of inputs (unique only with this map, 
and
+  // likely embedded in the transform payload and serialized user code) to
+  // PCollection ids.
   //
   // The payload for this transform may clarify the relationship of these
   // inputs. For example:
@@ -157,7 +158,9 @@ message PTransform {
   //
   map inputs = 3;
 
-  // (Required) A map from local names of outputs to PCollection ids.
+  // (Required) A map from local names of outputs (unique only within this map,
+  // and likely embedded in the transform payload and serialized user code)
+  // to PCollection ids.
   //
   // The URN or payload for this transform node may clarify the type and
   // relationship of these outputs. For example:
@@ -167,7 +170,9 @@ message PTransform {
   //
   map outputs = 4;
 
-  // (Required) Static display data for this PTransform application.
+  // (Optional) Static display data for this PTransform application. If
+  // there is none, or it is not relevant (such as use by the Fn API)
+  // then it may be omitted.
   DisplayData display_data = 6;
 }
 
@@ -193,7 +198,9 @@ message PCollection {
   // (Required) The id of the windowing strategy for this PCollection.
   string windowing_strategy_id = 4;
 
-  // (Required) Static display data for this PCollection.
+  // (Optional) Static display data for this 

[1/2] beam git commit: Ignore HBaseIOTest temporarily

2017-02-24 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 49c9de1e8 -> 8a06134c6


Ignore HBaseIOTest temporarily


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

Branch: refs/heads/master
Commit: 916fe65e5d63981026c132454ffbc56c3d0f1ca8
Parents: 00b789d
Author: Kenneth Knowles 
Authored: Fri Feb 24 14:30:06 2017 -0800
Committer: Kenneth Knowles 
Committed: Fri Feb 24 14:30:06 2017 -0800

--
 .../src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/916fe65e/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
--
diff --git 
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
 
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
index d3bc4dc..1d49f9d 100644
--- 
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
+++ 
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
@@ -25,10 +25,8 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
 import com.google.protobuf.ByteString;
-
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.hbase.HBaseIO.HBaseSource;
@@ -68,6 +66,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.hamcrest.Matchers;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -78,6 +77,7 @@ import org.junit.runners.JUnit4;
 /**
  * Test HBaseIO.
  */
+@Ignore
 @RunWith(JUnit4.class)
 public class HBaseIOTest {
 @Rule public final transient TestPipeline p = TestPipeline.create();



[1/2] beam git commit: [BEAM-1554] Add verification of Coder structural equality after enc/dec

2017-02-24 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 415546eda -> 437ba2505


[BEAM-1554] Add verification of Coder structural equality after enc/dec

This also improves the class documentation.


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

Branch: refs/heads/master
Commit: 2736c4ae901ad428f8e05fb7a80ad72f0318735e
Parents: 415546e
Author: Ismaël Mejía 
Authored: Fri Feb 24 16:46:17 2017 +0100
Committer: Ismaël Mejía 
Committed: Sat Feb 25 04:16:28 2017 +0100

--
 .../beam/sdk/testing/CoderProperties.java   | 66 ++--
 1 file changed, 61 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/2736c4ae/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
index 910b939..8065505 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
@@ -87,7 +87,6 @@ public class CoderProperties {
   public static  void coderDeterministicInContext(
   Coder coder, Coder.Context context, T value1, T value2)
   throws Exception {
-
 try {
   coder.verifyDeterministic();
 } catch (NonDeterministicException e) {
@@ -189,22 +188,34 @@ public class CoderProperties {
 }
   }
 
+  /**
+   * Verifies that the given {@code Coder} can be correctly serialized and
+   * deserialized.
+   */
   public static  void coderSerializable(Coder coder) {
 SerializableUtils.ensureSerializable(coder);
   }
 
+  /**
+   * Verifies that for the given {@code Coder} and values of
+   * type {@code T}, the values are equal if and only if the
+   * encoded bytes are equal.
+   */
   public static  void coderConsistentWithEquals(
   Coder coder, T value1, T value2)
   throws Exception {
-
 for (Coder.Context context : ALL_CONTEXTS) {
   CoderProperties.coderConsistentWithEqualsInContext(coder, context, 
value1, value2);
 }
   }
 
+  /**
+   * Verifies that for the given {@code Coder}, {@code Coder.Context}, and
+   * values of type {@code T}, the values are equal if and only if the
+   * encoded bytes are equal, in any {@code Coder.Context}.
+   */
   public static  void coderConsistentWithEqualsInContext(
   Coder coder, Coder.Context context, T value1, T value2) throws 
Exception {
-
 assertEquals(
 value1.equals(value2),
 Arrays.equals(
@@ -212,12 +223,20 @@ public class CoderProperties {
 encode(coder, context, value2)));
   }
 
+  /**
+   * Verifies if a {@code Coder}'s encodingId is equal to a given
+   * encodingId.
+   */
   public static  void coderHasEncodingId(Coder coder, String encodingId) 
throws Exception {
 assertThat(coder.getEncodingId(), equalTo(encodingId));
 assertThat(Structs.getString(coder.asCloudObject(), 
PropertyNames.ENCODING_ID, ""),
 equalTo(encodingId));
   }
 
+  /**
+   * Verifies if a {@code Coder} is allowed to encode using the given
+   * encodingId.
+   */
   public static  void coderAllowsEncoding(Coder coder, String 
encodingId) throws Exception {
 assertThat(coder.getAllowedEncodings(), hasItem(encodingId));
 assertThat(
@@ -230,19 +249,27 @@ public class CoderProperties {
 hasItem(encodingId));
   }
 
+  /**
+   * Verifies that for the given {@code Coder} and values of
+   * type {@code T}, the structural values are equal if and only if the
+   * encoded bytes are equal.
+   */
   public static  void structuralValueConsistentWithEquals(
   Coder coder, T value1, T value2)
   throws Exception {
-
 for (Coder.Context context : ALL_CONTEXTS) {
   CoderProperties.structuralValueConsistentWithEqualsInContext(
   coder, context, value1, value2);
 }
   }
 
+  /**
+   * Verifies that for the given {@code Coder}, {@code Coder.Context}, and
+   * values of type {@code T}, the structural values are equal if and only if 
the
+   * encoded bytes are equal, in any {@code Coder.Context}.
+   */
   public static  void structuralValueConsistentWithEqualsInContext(
   Coder coder, Coder.Context context, T value1, T value2) throws 
Exception {
-
 assertEquals(
 coder.structuralValue(value1).equals(coder.structuralValue(value2)),
 Arrays.equals(
@@ -250,6 +277,35 @@ public class CoderProperties {
 encode(coder, context, value2)));
   }
 
+  /**
+ 

[2/4] beam git commit: Make SDK-specific serialized blob really a blob

2017-02-21 Thread kenn
Make SDK-specific serialized blob really a blob


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

Branch: refs/heads/master
Commit: 380b75e2d0b55eb00cec0c030e5dddc796ec8af5
Parents: 5d6dafa
Author: Kenneth Knowles 
Authored: Thu Feb 16 14:45:05 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 21 11:50:52 2017 -0800

--
 sdks/common/runner-api/src/main/proto/beam_runner_api.proto | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/380b75e2/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index f5dc81d..2919580 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -682,7 +682,7 @@ message SdkFunctionSpec {
   // (Required) The raw data of the function that the SDK knows how to
   // deserialize, but need not be comprehensible to any other runner, SDK, or
   // other entity.
-  google.protobuf.Any data = 4;
+  bytes data = 4;
 }
 
 // TODO: transfer javadoc here



[3/4] beam git commit: Add conversion to/from Runner API proto for WindowingStrategy

2017-02-21 Thread kenn
Add conversion to/from Runner API proto for WindowingStrategy


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

Branch: refs/heads/master
Commit: 7dd571059de5eccce19c237ff7b4fe03eceb4806
Parents: aac38d6
Author: Kenneth Knowles 
Authored: Thu Feb 16 20:26:45 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 21 11:50:53 2017 -0800

--
 .../sdk/transforms/windowing/GlobalWindows.java |  10 +
 .../sdk/transforms/windowing/OutputTimeFns.java |  45 
 .../beam/sdk/transforms/windowing/Window.java   |   3 +-
 .../beam/sdk/util/WindowingStrategies.java  | 241 +++
 .../apache/beam/sdk/util/WindowingStrategy.java |  53 +++-
 .../transforms/windowing/OutputTimeFnsTest.java |  51 
 .../beam/sdk/util/WindowingStrategiesTest.java  |  91 +++
 7 files changed, 490 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/7dd57105/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
index aba00a3..6606a5a 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
@@ -54,4 +54,14 @@ public class GlobalWindows extends 
NonMergingWindowFn {
   public Instant getOutputTime(Instant inputTimestamp, GlobalWindow window) {
 return inputTimestamp;
   }
+
+  @Override
+  public boolean equals(Object other) {
+return other instanceof GlobalWindows;
+  }
+
+  @Override
+  public String toString() {
+return getClass().getCanonicalName();
+  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/7dd57105/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
index 2bcd319..b5d67fa 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.common.runner.v1.RunnerApi;
 import org.joda.time.Instant;
 
 /**
@@ -163,5 +164,49 @@ public class OutputTimeFns {
 protected Instant assignOutputTime(BoundedWindow window) {
   return window.maxTimestamp();
 }
+
+@Override
+public String toString() {
+  return getClass().getCanonicalName();
+}
+  }
+
+  public static RunnerApi.OutputTime toProto(OutputTimeFn outputTimeFn) {
+if (outputTimeFn instanceof OutputAtEarliestInputTimestamp) {
+  return RunnerApi.OutputTime.EARLIEST_IN_PANE;
+} else if (outputTimeFn instanceof OutputAtLatestInputTimestamp) {
+  return RunnerApi.OutputTime.LATEST_IN_PANE;
+} else if (outputTimeFn instanceof OutputAtEndOfWindow) {
+  return RunnerApi.OutputTime.END_OF_WINDOW;
+} else {
+  throw new IllegalArgumentException(
+  String.format(
+  "Cannot convert %s to %s: %s",
+  OutputTimeFn.class.getCanonicalName(),
+  RunnerApi.OutputTime.class.getCanonicalName(),
+  outputTimeFn));
+}
+  }
+
+  public static OutputTimeFn fromProto(RunnerApi.OutputTime proto) {
+switch (proto) {
+  case EARLIEST_IN_PANE:
+return OutputTimeFns.outputAtEarliestInputTimestamp();
+  case LATEST_IN_PANE:
+return OutputTimeFns.outputAtLatestInputTimestamp();
+  case END_OF_WINDOW:
+return OutputTimeFns.outputAtEndOfWindow();
+  case UNRECOGNIZED:
+  default:
+// Whether or not it is proto that cannot recognize it (due to the 
version of the
+// generated code we link to) or the switch hasn't been updated to 
handle it,
+// the situation is the same: we don't know what this OutputTime means
+throw new IllegalArgumentException(
+String.format(
+"Cannot convert unknown %s to %s: %s",
+

[4/4] beam git commit: This closes #2030: Convert to/from WindowingStrategy proto in Java SDK

2017-02-21 Thread kenn
This closes #2030: Convert to/from WindowingStrategy proto in Java SDK

  Add conversion to/from Runner API proto for WindowingStrategy
  Add closing behavior to Runner API proto
  Make SDK-specific serialized blob really a blob


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

Branch: refs/heads/master
Commit: 154c543057b54ca46975cb9e9d360a52784518d7
Parents: 3fcc820 7dd5710
Author: Kenneth Knowles 
Authored: Tue Feb 21 13:09:17 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 21 13:09:17 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto|  19 +-
 .../sdk/transforms/windowing/GlobalWindows.java |  10 +
 .../sdk/transforms/windowing/OutputTimeFns.java |  45 
 .../beam/sdk/transforms/windowing/Window.java   |   3 +-
 .../beam/sdk/util/WindowingStrategies.java  | 241 +++
 .../apache/beam/sdk/util/WindowingStrategy.java |  53 +++-
 .../transforms/windowing/OutputTimeFnsTest.java |  51 
 .../beam/sdk/util/WindowingStrategiesTest.java  |  91 +++
 8 files changed, 507 insertions(+), 6 deletions(-)
--




[1/4] beam git commit: Add closing behavior to Runner API proto

2017-02-21 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 3fcc82081 -> 154c54305


Add closing behavior to Runner API proto


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

Branch: refs/heads/master
Commit: aac38d60c0f76db67095522ea013403642179d9d
Parents: 380b75e
Author: Kenneth Knowles 
Authored: Thu Feb 16 20:26:39 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 21 11:50:52 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto   | 17 -
 1 file changed, 16 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/aac38d60/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index 2919580..ce089f5 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -392,9 +392,12 @@ message WindowingStrategy {
   // windowing strategy.
   OutputTime output_time = 6;
 
+  // (Required) Indicate when output should be omitted upon window expiration.
+  ClosingBehavior closing_behavior = 7;
+
   // (Required) The duration, in milliseconds, beyond the end of a window at
   // which the window becomes droppable.
-  int64 allowed_lateness = 7;
+  int64 allowed_lateness = 8;
 }
 
 // Whether or not a PCollection's WindowFn is non-merging, merging, or
@@ -428,6 +431,18 @@ enum AccumulationMode {
   ACCUMULATING = 1;
 }
 
+// Controls whether or not an aggregating transform should output data
+// when a window expires.
+enum ClosingBehavior {
+
+  // Emit output when a window expires, whether or not there has been
+  // any new data since the last output.
+  EMIT_ALWAYS = 0;
+
+  // Only emit output when new data has arrives since the last output
+  EMIT_IF_NONEMPTY = 1;
+}
+
 // When a number of windowed, timestamped inputs are aggregated, the timestamp
 // for the resulting output.
 enum OutputTime {



[2/2] beam git commit: This closes #1975: Changed name of ToString.of() to ToString.elements().

2017-02-13 Thread kenn
This closes #1975: Changed name of ToString.of() to ToString.elements().


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

Branch: refs/heads/master
Commit: 98d8834afba705470a43ebc0349f76334d49f00a
Parents: f32cb3e a49acda
Author: Kenneth Knowles 
Authored: Mon Feb 13 20:29:32 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 13 20:29:32 2017 -0800

--
 .../src/main/java/org/apache/beam/sdk/transforms/ToString.java   | 4 ++--
 .../core/src/test/java/org/apache/beam/sdk/io/WriteTest.java | 2 +-
 .../test/java/org/apache/beam/sdk/transforms/ToStringTest.java   | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
--




[1/2] beam git commit: Changed name of ToString.of() to ToString.elements().

2017-02-13 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master f32cb3e7a -> 98d8834af


Changed name of ToString.of() to ToString.elements().


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

Branch: refs/heads/master
Commit: a49acdadac37fd52cc1184e5d29cc30b99742a35
Parents: effca63
Author: Jesse Anderson 
Authored: Fri Feb 10 12:17:21 2017 -0800
Committer: Jesse Anderson 
Committed: Fri Feb 10 12:17:21 2017 -0800

--
 .../src/main/java/org/apache/beam/sdk/transforms/ToString.java   | 4 ++--
 .../core/src/test/java/org/apache/beam/sdk/io/WriteTest.java | 2 +-
 .../test/java/org/apache/beam/sdk/transforms/ToStringTest.java   | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/a49acdad/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
index d5c9784..5069a3c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
@@ -42,7 +42,7 @@ public final class ToString {
* element of the input {@link PCollection} to a {@link String} using the
* {@link Object#toString} method.
*/
-  public static PTransform of() {
+  public static PTransform elements() {
 return new SimpleToString();
   }
 
@@ -97,7 +97,7 @@ public final class ToString {
* Example of use:
* {@code
* PCollection longs = ...;
-   * PCollection strings = longs.apply(ToString.of());
+   * PCollection strings = longs.apply(ToString.elements());
* }
*
*

http://git-wip-us.apache.org/repos/asf/beam/blob/a49acdad/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
--
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
index f81cc0c..846d445 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
@@ -297,7 +297,7 @@ public class WriteTest {
   @Test
   public void testWriteUnbounded() {
 PCollection unbounded = p.apply(CountingInput.unbounded())
-.apply(ToString.of());
+.apply(ToString.elements());
 
 TestSink sink = new TestSink();
 thrown.expect(IllegalArgumentException.class);

http://git-wip-us.apache.org/repos/asf/beam/blob/a49acdad/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToStringTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToStringTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToStringTest.java
index ab984f1..d2116da 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToStringTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToStringTest.java
@@ -48,7 +48,7 @@ public class ToStringTest {
 Integer[] ints = {1, 2, 3, 4, 5};
 String[] strings = {"1", "2", "3", "4", "5"};
 PCollection input = p.apply(Create.of(Arrays.asList(ints)));
-PCollection output = input.apply(ToString.of());
+PCollection output = input.apply(ToString.elements());
 PAssert.that(output).containsInAnyOrder(strings);
 p.run();
   }



[2/2] beam git commit: This closes #1891: Correct Javadoc on accessing windows in DoFn

2017-02-13 Thread kenn
This closes #1891: Correct Javadoc on accessing windows in DoFn


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

Branch: refs/heads/master
Commit: 49809d1d43c409b23771238af085f9ebcd0d3cb3
Parents: 98d8834 c1c8d83
Author: Kenneth Knowles 
Authored: Mon Feb 13 20:44:21 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 13 20:44:21 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/49809d1d/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
--



[1/2] beam git commit: Correct Javadoc on accessing windows in DoFn

2017-02-13 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 98d8834af -> 49809d1d4


Correct Javadoc on accessing windows in DoFn


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

Branch: refs/heads/master
Commit: c1c8d8386fc035c65362b5ce36c20b38fe00f9a4
Parents: 0e6b379
Author: Ben Chambers 
Authored: Wed Feb 1 15:08:13 2017 -0800
Committer: Kenneth Knowles 
Committed: Mon Feb 13 20:44:04 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/beam/blob/c1c8d838/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index a161919..1ad05bb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -532,8 +532,10 @@ public abstract class DoFn implements 
Serializable, HasDisplayD
* href="https://s.apache.org/splittable-do-fn;>splittable {@link 
DoFn} subject to the
* separate requirements described below. Items below are assuming this 
is not a splittable
* {@link DoFn}.
-   * If one of its arguments is {@link BoundedWindow}, this argument 
corresponds to the window
-   * of the current element. If absent, a runner may perform additional 
optimizations.
+   * If one of its arguments is a subtype of {@link BoundedWindow} then it 
will
+   * be passed the window of the current element. When applied by {@link 
ParDo} the subtype
+   * of {@link BoundedWindow} must match the type of windows on the input 
{@link PCollection}.
+   * If the window is not accessed a runner may perform additional 
optimizations.
* It must return {@code void}.
* 
*



[1/2] beam git commit: Add proto definition for Runner API

2017-02-09 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master cd6802bec -> 490ef8f09


Add proto definition for Runner API


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

Branch: refs/heads/master
Commit: a5ce3b4380377ebbcc75933f8f0f8c3faeefdcf4
Parents: 9ec22f1
Author: Kenneth Knowles 
Authored: Tue Feb 7 15:25:32 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 9 14:45:48 2017 -0800

--
 sdks/common/pom.xml |   1 +
 sdks/common/runner-api/pom.xml  |  91 +++
 .../src/main/proto/beam_runner_api.proto| 638 +++
 3 files changed, 730 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/a5ce3b43/sdks/common/pom.xml
--
diff --git a/sdks/common/pom.xml b/sdks/common/pom.xml
index 8364d9a..55db181 100644
--- a/sdks/common/pom.xml
+++ b/sdks/common/pom.xml
@@ -34,5 +34,6 @@
 
   
 fn-api
+runner-api
   
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a5ce3b43/sdks/common/runner-api/pom.xml
--
diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml
new file mode 100644
index 000..8eaeb8e
--- /dev/null
+++ b/sdks/common/runner-api/pom.xml
@@ -0,0 +1,91 @@
+
+
+http://maven.apache.org/POM/4.0.0;
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+  4.0.0
+
+  jar
+  
+org.apache.beam
+beam-sdks-common-parent
+0.6.0-SNAPSHOT
+../pom.xml
+  
+
+  beam-sdks-common-runner-api
+  Apache Beam :: SDKs :: Common :: Runner API
+  This artifact generates the stub bindings.
+
+  
+
+  
+src/main/resources
+true
+  
+  
+
${project.build.directory}/original_sources_to_package
+  
+
+
+
+  
+  
+org.apache.maven.plugins
+maven-checkstyle-plugin
+
+  true
+
+  
+
+  
+  
+org.codehaus.mojo
+findbugs-maven-plugin
+
+  true
+
+  
+
+  
+org.xolstice.maven.plugins
+protobuf-maven-plugin
+
+  
com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}
+  grpc-java
+  
io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier}
+
+
+  
+
+  compile
+  compile-custom
+
+  
+
+  
+
+  
+
+  
+
+  com.google.protobuf
+  protobuf-java
+
+  
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a5ce3b43/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
new file mode 100644
index 000..195ce01
--- /dev/null
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -0,0 +1,638 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*
+ * Protocol Buffers describing the Runner API, which is the runner-independent,
+ * SDK-independent definition of the Beam model.
+ */
+
+syntax = "proto3";
+
+package org.apache.beam.runner_api.v1;
+
+option java_package = "org.apache.beam.sdks.common.runner_api.v1";
+option java_outer_classname = "RunnerApi";
+
+import "google/protobuf/any.proto";
+import "google/protobuf/timestamp.proto";
+
+// A Pipeline is a hierarchical graph of PTransforms, linked
+// by PCollections.
+//
+// This is represented by a number of by-reference maps to nodes,
+// PCollections, SDK environments, UDF, etc., for
+// supporting compact 

[2/2] beam git commit: This closes #1946: Add proto definition for Runner API

2017-02-09 Thread kenn
This closes #1946: Add proto definition for Runner API


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

Branch: refs/heads/master
Commit: 490ef8f09214609bd6afa74a937a4f4bc8e79bee
Parents: cd6802b a5ce3b4
Author: Kenneth Knowles 
Authored: Thu Feb 9 15:24:19 2017 -0800
Committer: Kenneth Knowles 
Committed: Thu Feb 9 15:24:19 2017 -0800

--
 sdks/common/pom.xml |   1 +
 sdks/common/runner-api/pom.xml  |  91 +++
 .../src/main/proto/beam_runner_api.proto| 638 +++
 3 files changed, 730 insertions(+)
--




[4/5] beam git commit: Add SetState and MapState

2017-02-14 Thread kenn
Add SetState and MapState


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

Branch: refs/heads/master
Commit: a0702f5bed3c7269e90b4702266945aa34dd1aea
Parents: 0f48321
Author: JingsongLi 
Authored: Tue Feb 14 14:52:05 2017 +0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 11:06:29 2017 -0800

--
 .../translation/utils/ApexStateInternals.java   |  18 ++
 .../runners/core/InMemoryStateInternals.java| 205 ++
 .../apache/beam/runners/core/StateMerging.java  |  44 +++
 .../org/apache/beam/runners/core/StateTag.java  |   8 +
 .../org/apache/beam/runners/core/StateTags.java |  30 ++
 .../core/InMemoryStateInternalsTest.java| 280 +--
 .../apache/beam/runners/core/StateTagTest.java  |  33 +++
 .../CopyOnAccessInMemoryStateInternals.java |  46 +++
 .../CopyOnAccessInMemoryStateInternalsTest.java |  58 
 .../wrappers/streaming/FlinkStateInternals.java |  18 ++
 .../apache/beam/sdk/util/state/MapState.java|  93 ++
 .../apache/beam/sdk/util/state/SetState.java|  71 +
 .../apache/beam/sdk/util/state/StateBinder.java |   6 +
 .../apache/beam/sdk/util/state/StateSpecs.java  |  89 ++
 .../apache/beam/sdk/transforms/ParDoTest.java   |  94 +++
 15 files changed, 1063 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/a0702f5b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
--
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
index 34d993f..7634366 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
@@ -45,7 +45,9 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
 import org.apache.beam.sdk.util.CombineFnUtil;
 import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
 import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.MapState;
 import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.SetState;
 import org.apache.beam.sdk.util.state.State;
 import org.apache.beam.sdk.util.state.StateContext;
 import org.apache.beam.sdk.util.state.StateContexts;
@@ -121,6 +123,22 @@ public class ApexStateInternals implements 
StateInternals, Serializable {
 }
 
 @Override
+public  SetState bindSet(
+StateTag> address,
+Coder elemCoder) {
+  throw new UnsupportedOperationException(
+  String.format("%s is not supported", 
SetState.class.getSimpleName()));
+}
+
+@Override
+public  MapState bindMap(
+StateTag> spec,
+Coder mapKeyCoder, Coder mapValueCoder) {
+  throw new UnsupportedOperationException(
+  String.format("%s is not supported", 
MapState.class.getSimpleName()));
+}
+
+@Override
 public  AccumulatorCombiningState
 bindCombiningValue(
 StateTag> address,

http://git-wip-us.apache.org/repos/asf/beam/blob/a0702f5b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
index 6a181f3..b4b2b38 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
@@ -17,10 +17,16 @@
  */
 package org.apache.beam.runners.core;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.core.StateTag.StateBinder;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -34,7 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
 import org.apache.beam.sdk.util.CombineFnUtil;
 import 

[2/5] beam git commit: Exclude UsesSetState and UsesMapState categories from Dataflow runner

2017-02-14 Thread kenn
Exclude UsesSetState and UsesMapState categories from Dataflow runner


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

Branch: refs/heads/master
Commit: 36604a01086e0a2bf25569f06c2dae343e1ba225
Parents: 674bead
Author: JingsongLi 
Authored: Tue Feb 14 10:16:07 2017 +0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 11:06:20 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/36604a01/runners/google-cloud-dataflow-java/pom.xml
--
diff --git a/runners/google-cloud-dataflow-java/pom.xml 
b/runners/google-cloud-dataflow-java/pom.xml
index d366ddc..0f75b74 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -80,6 +80,8 @@
   
 org.apache.beam.sdk.testing.UsesAttemptedMetrics,
 org.apache.beam.sdk.testing.UsesCommittedMetrics,
+org.apache.beam.sdk.testing.UsesSetState,
+org.apache.beam.sdk.testing.UsesMapState,
 org.apache.beam.sdk.testing.UsesTimersInParDo,
 org.apache.beam.sdk.testing.UsesSplittableParDo,
 org.apache.beam.sdk.testing.UsesUnboundedPCollections,



[1/5] beam git commit: Add UsesMapState and UsesSetState JUnit categories

2017-02-14 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master ed3ef11fa -> 2596d46a1


Add UsesMapState and UsesSetState JUnit categories


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

Branch: refs/heads/master
Commit: 674bead8c346421ebfadc3e0b2e13eb79e0280aa
Parents: bea101a
Author: JingsongLi 
Authored: Tue Feb 14 14:52:05 2017 +0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 11:02:48 2017 -0800

--
 .../apache/beam/sdk/testing/UsesMapState.java   | 25 
 .../apache/beam/sdk/testing/UsesSetState.java   | 25 
 2 files changed, 50 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/674bead8/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java
new file mode 100644
index 000..9bced41
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import org.apache.beam.sdk.util.state.MapState;
+
+/**
+ * Category tag for validation tests which utilize {@link MapState}.
+ */
+public interface UsesMapState {}

http://git-wip-us.apache.org/repos/asf/beam/blob/674bead8/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java
new file mode 100644
index 000..6fd74bd
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import org.apache.beam.sdk.util.state.SetState;
+
+/**
+ * Category tag for validation tests which utilize {@link SetState}.
+ */
+public interface UsesSetState {}



[3/5] beam git commit: Exclude UsesSetState and UsesMapState tests for Flink runner

2017-02-14 Thread kenn
Exclude UsesSetState and UsesMapState tests for Flink runner


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

Branch: refs/heads/master
Commit: 0f48321e3b933904fbeb3fbe7c2a06741ff18ff0
Parents: 36604a0
Author: JingsongLi 
Authored: Tue Feb 14 10:16:07 2017 +0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 11:06:28 2017 -0800

--
 runners/flink/runner/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/0f48321e/runners/flink/runner/pom.xml
--
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index c800ed3..f254d9a 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -88,6 +88,8 @@
   
org.apache.beam.sdk.testing.RunnableOnService
   
 org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders,
+org.apache.beam.sdk.testing.UsesSetState,
+org.apache.beam.sdk.testing.UsesMapState,
 org.apache.beam.sdk.testing.UsesTimersInParDo,
 org.apache.beam.sdk.testing.UsesSplittableParDo,
 org.apache.beam.sdk.testing.UsesAttemptedMetrics,



[5/5] beam git commit: This closes #1950: State for "Set" data and "Map" data

2017-02-14 Thread kenn
This closes #1950: State for "Set" data and "Map" data

  Add SetState and MapState
  Exclude UsesSetState and UsesMapState tests for Flink runner
  Exclude UsesSetState and UsesMapState categories from Dataflow runner
  Add UsesMapState and UsesSetState JUnit categories


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

Branch: refs/heads/master
Commit: 2596d46a1ebbcfb72caa5b3a943ffe7d8d7f4ba2
Parents: ed3ef11 a0702f5
Author: Kenneth Knowles 
Authored: Tue Feb 14 13:57:38 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 13:57:38 2017 -0800

--
 .../translation/utils/ApexStateInternals.java   |  18 ++
 .../runners/core/InMemoryStateInternals.java| 205 ++
 .../apache/beam/runners/core/StateMerging.java  |  44 +++
 .../org/apache/beam/runners/core/StateTag.java  |   8 +
 .../org/apache/beam/runners/core/StateTags.java |  30 ++
 .../core/InMemoryStateInternalsTest.java| 280 +--
 .../apache/beam/runners/core/StateTagTest.java  |  33 +++
 .../CopyOnAccessInMemoryStateInternals.java |  46 +++
 .../CopyOnAccessInMemoryStateInternalsTest.java |  58 
 runners/flink/runner/pom.xml|   2 +
 .../wrappers/streaming/FlinkStateInternals.java |  18 ++
 runners/google-cloud-dataflow-java/pom.xml  |   2 +
 .../apache/beam/sdk/testing/UsesMapState.java   |  25 ++
 .../apache/beam/sdk/testing/UsesSetState.java   |  25 ++
 .../apache/beam/sdk/util/state/MapState.java|  93 ++
 .../apache/beam/sdk/util/state/SetState.java|  71 +
 .../apache/beam/sdk/util/state/StateBinder.java |   6 +
 .../apache/beam/sdk/util/state/StateSpecs.java  |  89 ++
 .../apache/beam/sdk/transforms/ParDoTest.java   |  94 +++
 19 files changed, 1117 insertions(+), 30 deletions(-)
--




[4/9] beam git commit: Upgrade Java triggers to support runner API deserialization

2017-02-14 Thread kenn
Upgrade Java triggers to support runner API deserialization


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

Branch: refs/heads/master
Commit: 5d5602dbff41ef48add2ea763527f8c0901f0bc0
Parents: 40c4a5c
Author: Kenneth Knowles 
Authored: Sun Feb 12 15:53:17 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:49 2017 -0800

--
 .../AfterSynchronizedProcessingTimeStateMachine.java  |  6 +-
 .../beam/runners/core/triggers/TriggerStateMachines.java  |  2 +-
 .../AfterSynchronizedProcessingTimeStateMachineTest.java  |  3 ++-
 .../apache/beam/sdk/transforms/windowing/AfterAll.java|  7 +++
 .../apache/beam/sdk/transforms/windowing/AfterEach.java   |  7 +++
 .../apache/beam/sdk/transforms/windowing/AfterFirst.java  |  7 +++
 .../windowing/AfterSynchronizedProcessingTime.java| 10 +++---
 .../windowing/AfterSynchronizedProcessingTimeTest.java|  2 +-
 8 files changed, 37 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/5d5602db/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
index 1319a13..07fab22 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
@@ -28,13 +28,17 @@ import org.joda.time.Instant;
 // This should not really have the superclass 
https://issues.apache.org/jira/browse/BEAM-1486
 class AfterSynchronizedProcessingTimeStateMachine extends 
AfterDelayFromFirstElementStateMachine {
 
+  public static AfterSynchronizedProcessingTimeStateMachine ofFirstElement() {
+return new AfterSynchronizedProcessingTimeStateMachine();
+  }
+
   @Override
   @Nullable
   public Instant getCurrentTime(TriggerStateMachine.TriggerContext context) {
 return context.currentSynchronizedProcessingTime();
   }
 
-  public AfterSynchronizedProcessingTimeStateMachine() {
+  private AfterSynchronizedProcessingTimeStateMachine() {
 super(TimeDomain.SYNCHRONIZED_PROCESSING_TIME,
 Collections.>emptyList());
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/5d5602db/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java
index f0e9d21..b13ac40 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java
@@ -116,7 +116,7 @@ public class TriggerStateMachines {
 }
 
 private OnceTriggerStateMachine 
evaluateSpecific(AfterSynchronizedProcessingTime v) {
-  return new AfterSynchronizedProcessingTimeStateMachine();
+  return AfterSynchronizedProcessingTimeStateMachine.ofFirstElement();
 }
 
 private OnceTriggerStateMachine evaluateSpecific(AfterFirst v) {

http://git-wip-us.apache.org/repos/asf/beam/blob/5d5602db/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
--
diff --git 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
index 140bd62..7bfd48d 100644
--- 
a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
+++ 
b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java
@@ -36,7 +36,8 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class AfterSynchronizedProcessingTimeStateMachineTest {
 
-  private TriggerStateMachine underTest = new 

[3/9] beam git commit: Fix typo in runner API generated Java class

2017-02-14 Thread kenn
Fix typo in runner API generated Java class


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

Branch: refs/heads/master
Commit: 67854e66e97889b3f3e03de297e5af5b73c3fab1
Parents: 5d5602d
Author: Kenneth Knowles 
Authored: Sat Feb 11 16:48:05 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:49 2017 -0800

--
 sdks/common/runner-api/src/main/proto/beam_runner_api.proto | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/67854e66/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index 195ce01..370b57c 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -25,7 +25,7 @@ syntax = "proto3";
 
 package org.apache.beam.runner_api.v1;
 
-option java_package = "org.apache.beam.sdks.common.runner_api.v1";
+option java_package = "org.apache.beam.sdk.common.runner_api.v1";
 option java_outer_classname = "RunnerApi";
 
 import "google/protobuf/any.proto";



[2/9] beam git commit: Flesh out triggers in Runner API proto

2017-02-14 Thread kenn
Flesh out triggers in Runner API proto


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

Branch: refs/heads/master
Commit: 661cd8d7407d5f414d5d94badacdeadb519107b7
Parents: 67854e6
Author: Kenneth Knowles 
Authored: Sat Feb 11 17:32:21 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:49 2017 -0800

--
 .../src/main/proto/beam_runner_api.proto| 109 ++-
 1 file changed, 83 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/661cd8d7/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index 370b57c..91f1558 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -29,7 +29,6 @@ option java_package = 
"org.apache.beam.sdk.common.runner_api.v1";
 option java_outer_classname = "RunnerApi";
 
 import "google/protobuf/any.proto";
-import "google/protobuf/timestamp.proto";
 
 // A Pipeline is a hierarchical graph of PTransforms, linked
 // by PCollections.
@@ -402,6 +401,24 @@ enum OutputTime {
   EARLIEST_IN_PANE = 2;
 }
 
+// The different time domains in the Beam model.
+enum TimeDomain {
+
+  // Event time is time from the perspective of the data
+  EVENT_TIME = 0;
+
+  // Processing time is time from the perspective of the
+  // execution of your pipeline
+  PROCESSING_TIME = 1;
+
+  // Synchronized processing time is the minimum of the
+  // processing time of all pending elements.
+  //
+  // The "processing time" of an element refers to
+  // the local processing time at which it was emitted
+  SYNCHRONIZED_PROCESSING_TIME = 2;
+}
+
 // A small DSL for expressing when to emit new aggregations
 // from a GroupByKey or CombinePerKey
 //
@@ -439,27 +456,31 @@ message Trigger {
   }
 
   // After input arrives, ready when the specified delay has passed.
-  message AfterProcessingTimeDelay {
-// (Required) The delay, in milliseconds.
-int64 delay_millis = 1;
+  message AfterProcessingTime {
+
+// (Required) The transforms to apply to an arriving element's timestamp,
+// in order
+repeated TimestampTransform timestamp_transforms = 1;
   }
 
-  // After input arrives, ready when the synchronized processing time
-  // progresses as far as the given delay.
-  message AfterSynchronizedProcessingTimeDelay {
-// (Required) The delay, in milliseconds.
-int64 delay_millis = 1;
+  // Ready whenever upstream processing time has all caught up with
+  // the arrival time of an input element
+  message AfterSynchronizedProcessingTime {
+  }
+
+  // The default trigger. Equivalent to Repeat { AfterEndOfWindow } but
+  // specially denoted to indicate the user did not alter the triggering.
+  message Default {
+  }
+
+  // Ready whenever the requisite number of input elements have arrived
+  message ElementCount {
+int32 element_count = 1;
   }
 
   // Never ready. There will only be an ON_TIME output and a final
   // output at window expiration.
-  message Never { }
-
-  // Ready whenever the subtrigger is ready; resets state when the subtrigger
-  // completes.
-  message Repeat {
-// (Require) Trigger that is run repeatedly.
-Trigger subtrigger = 1;
+  message Never {
   }
 
   // Ready whenever either of its subtriggers are ready, but finishes output
@@ -473,9 +494,12 @@ message Trigger {
 Trigger finally = 2;
   }
 
-  // The default trigger. Equivalent to Repeat { AfterEndOfWindow } but
-  // specially denoted to indicate the user did not alter the triggering.
-  message Default { }
+  // Ready whenever the subtrigger is ready; resets state when the subtrigger
+  // completes.
+  message Repeat {
+// (Require) Trigger that is run repeatedly.
+Trigger subtrigger = 1;
+  }
 
   // The full disjoint union of possible triggers.
   oneof trigger {
@@ -483,12 +507,39 @@ message Trigger {
 AfterAny after_any = 2;
 AfterEach after_each = 3;
 AfterEndOfWindow after_end_of_widow = 4;
-AfterProcessingTimeDelay after_processing_time_delay = 5;
-AfterSynchronizedProcessingTimeDelay 
after_synchronized_processing_time_delay = 6;
-Never never = 7;
-Repeat repeat = 8;
-OrFinally or_finally = 9;
-Default default = 10;
+AfterProcessingTime after_processing_time = 5;
+AfterSynchronizedProcessingTime after_synchronized_processing_time = 6;
+Default default = 7;
+ElementCount element_count = 8;
+

[8/9] beam git commit: Upgrade Dataflow container version to beam-master-20170214

2017-02-14 Thread kenn
Upgrade Dataflow container version to beam-master-20170214


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

Branch: refs/heads/master
Commit: e097b7575c0be830cf966bd179b3268eb463659e
Parents: b6e9f73
Author: Kenneth Knowles 
Authored: Tue Feb 14 19:30:56 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 19:30:56 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/e097b757/runners/google-cloud-dataflow-java/pom.xml
--
diff --git a/runners/google-cloud-dataflow-java/pom.xml 
b/runners/google-cloud-dataflow-java/pom.xml
index d366ddc..ec1a927 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -33,7 +33,7 @@
   jar
 
   
-
beam-master-20170208
+
beam-master-20170214
 6
   
 



[6/9] beam git commit: Add runner API config to poms

2017-02-14 Thread kenn
Add runner API config to poms


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

Branch: refs/heads/master
Commit: 2803864ba689df92c993b0c4afc392df4558b6bf
Parents: 661cd8d
Author: Kenneth Knowles 
Authored: Sat Feb 11 16:47:15 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:49 2017 -0800

--
 pom.xml| 8 
 sdks/java/core/pom.xml | 5 +
 2 files changed, 13 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/2803864b/pom.xml
--
diff --git a/pom.xml b/pom.xml
index be75659..d53502e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -320,6 +320,7 @@
 beam-sdks-common-fn-api
 ${project.version}
   
+
   
 org.apache.beam
 beam-sdks-common-fn-api
@@ -329,6 +330,13 @@
 
   
 org.apache.beam
+beam-sdks-common-runner-api
+${project.version}
+  
+
+
+  
+org.apache.beam
 beam-runners-flink_2.10-examples
 ${project.version}
   

http://git-wip-us.apache.org/repos/asf/beam/blob/2803864b/sdks/java/core/pom.xml
--
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index 266e144..4f89550 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -183,6 +183,11 @@
 
   
 
+  org.apache.beam
+  beam-sdks-common-runner-api
+
+
+
   io.grpc
   grpc-auth
 



[1/3] beam-site git commit: Fix typo in stateful processing post

2017-02-14 Thread kenn
Repository: beam-site
Updated Branches:
  refs/heads/asf-site 00e375003 -> f3c189568


Fix typo in stateful processing post


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

Branch: refs/heads/asf-site
Commit: 47b1c5cdaef819bc82fb6ec069a3a5d1c3e9165c
Parents: 00e3750
Author: Kenneth Knowles 
Authored: Tue Feb 14 21:13:48 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 21:13:48 2017 -0800

--
 src/_posts/2017-02-13-stateful-processing.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam-site/blob/47b1c5cd/src/_posts/2017-02-13-stateful-processing.md
--
diff --git a/src/_posts/2017-02-13-stateful-processing.md 
b/src/_posts/2017-02-13-stateful-processing.md
index 28aee12..b00361a 100644
--- a/src/_posts/2017-02-13-stateful-processing.md
+++ b/src/_posts/2017-02-13-stateful-processing.md
@@ -25,7 +25,7 @@ what you might use it for, and what it looks like in code.
 
 First, a quick recap: In Beam, a big data processing _pipeline_ is a directed,
 acyclic graph of parallel operations called _`PTransforms`_ processing data
-from _`PCollections`_ I'll expand on that by walking through this illustration:
+from _`PCollections`_. I'll expand on that by walking through this 
illustration:
 
 

[3/3] beam-site git commit: This closes #155: Fix typo in stateful processing post

2017-02-14 Thread kenn
This closes #155: Fix typo in stateful processing post

  Regenerate site
  Fix typo in stateful processing post


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

Branch: refs/heads/asf-site
Commit: f3c189568ed5a1a9e0fb84132cf508e271e4cb63
Parents: 00e3750 7602168
Author: Kenneth Knowles 
Authored: Tue Feb 14 21:29:34 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 21:29:34 2017 -0800

--
 content/blog/2017/02/13/stateful-processing.html | 2 +-
 content/feed.xml | 2 +-
 src/_posts/2017-02-13-stateful-processing.md | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
--




[2/3] beam-site git commit: Regenerate site

2017-02-14 Thread kenn
Regenerate site


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

Branch: refs/heads/asf-site
Commit: 7602168530feee8aa6eedfd20f5549617b4545e6
Parents: 47b1c5c
Author: Kenneth Knowles 
Authored: Tue Feb 14 21:29:15 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 21:29:15 2017 -0800

--
 content/blog/2017/02/13/stateful-processing.html | 2 +-
 content/feed.xml | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam-site/blob/76021685/content/blog/2017/02/13/stateful-processing.html
--
diff --git a/content/blog/2017/02/13/stateful-processing.html 
b/content/blog/2017/02/13/stateful-processing.html
index dcf077a..833b2fd 100644
--- a/content/blog/2017/02/13/stateful-processing.html
+++ b/content/blog/2017/02/13/stateful-processing.html
@@ -175,7 +175,7 @@ the current status in each runner.
 
 First, a quick recap: In Beam, a big data processing pipeline is a 
directed,
 acyclic graph of parallel operations called PTransforms processing data
-from PCollections I’ll 
expand on that by walking through this illustration:
+from PCollections. I’ll 
expand on that by walking through this illustration:
 
 
 

http://git-wip-us.apache.org/repos/asf/beam-site/blob/76021685/content/feed.xml
--
diff --git a/content/feed.xml b/content/feed.xml
index 10bccbd..726cdd0 100644
--- a/content/feed.xml
+++ b/content/feed.xml
@@ -29,7 +29,7 @@ the current status in each runner./p
 
 pFirst, a quick recap: In Beam, a big data processing 
empipeline/em is a directed,
 acyclic graph of parallel operations called emcode 
class=highlighter-rougePTransforms/code/em 
processing data
-from emcode 
class=highlighter-rougePCollections/code/em 
I’ll expand on that by walking through this illustration:/p
+from emcode 
class=highlighter-rougePCollections/code/em. 
I’ll expand on that by walking through this illustration:/p
 
 pimg class=center-block 
src=/images/blog/stateful-processing/pipeline.png alt=A Beam 
Pipeline - PTransforms are boxes - PCollections are arrows 
width=300 //p
 



[7/9] beam git commit: Remove underscore from Runner API proto Java package

2017-02-14 Thread kenn
Remove underscore from Runner API proto Java package


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

Branch: refs/heads/master
Commit: b6e9f73ae00813167c34b55459b9832e20d9aa41
Parents: f4ceaee
Author: Kenneth Knowles 
Authored: Tue Feb 14 12:33:43 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:50 2017 -0800

--
 sdks/common/runner-api/src/main/proto/beam_runner_api.proto | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/b6e9f73a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
--
diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto 
b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
index 91f1558..a9133ab 100644
--- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
+++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto
@@ -25,7 +25,7 @@ syntax = "proto3";
 
 package org.apache.beam.runner_api.v1;
 
-option java_package = "org.apache.beam.sdk.common.runner_api.v1";
+option java_package = "org.apache.beam.sdk.common.runner.v1";
 option java_outer_classname = "RunnerApi";
 
 import "google/protobuf/any.proto";



[5/9] beam git commit: Add conversion to/from Java SDK trigger to runner API proto

2017-02-14 Thread kenn
Add conversion to/from Java SDK trigger to runner API proto


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

Branch: refs/heads/master
Commit: f4ceaeefe9e8e9d069b760e166c7057a00465360
Parents: 2803864
Author: Kenneth Knowles 
Authored: Sat Feb 11 17:50:27 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:49 2017 -0800

--
 .../beam/sdk/transforms/windowing/Triggers.java | 313 +++
 .../sdk/transforms/windowing/TriggersTest.java  | 100 ++
 2 files changed, 413 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/f4ceaeef/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
new file mode 100644
index 000..8ac904c
--- /dev/null
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Triggers.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms.windowing;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.common.runner.v1.RunnerApi;
+import 
org.apache.beam.sdk.transforms.windowing.AfterWatermark.AfterWatermarkEarlyAndLate;
+import org.apache.beam.sdk.transforms.windowing.Never.NeverTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/** Utilities for working with {@link Triggers Triggers}. */
+@Experimental(Experimental.Kind.TRIGGER)
+public class Triggers implements Serializable {
+
+  @VisibleForTesting static final ProtoConverter CONVERTER = new 
ProtoConverter();
+
+  public static RunnerApi.Trigger toProto(Trigger trigger) {
+return CONVERTER.convertTrigger(trigger);
+  }
+
+  @VisibleForTesting
+  static class ProtoConverter {
+
+public RunnerApi.Trigger convertTrigger(Trigger trigger) {
+  Method evaluationMethod = getEvaluationMethod(trigger.getClass());
+  return tryConvert(evaluationMethod, trigger);
+}
+
+private RunnerApi.Trigger tryConvert(Method evaluationMethod, Trigger 
trigger) {
+  try {
+return (RunnerApi.Trigger) evaluationMethod.invoke(this, trigger);
+  } catch (InvocationTargetException exc) {
+if (exc.getCause() instanceof RuntimeException) {
+  throw (RuntimeException) exc.getCause();
+} else {
+  throw new RuntimeException(exc.getCause());
+}
+  } catch (IllegalAccessException exc) {
+throw new IllegalStateException(
+String.format("Internal error: could not invoke %s", 
evaluationMethod));
+  }
+}
+
+private Method getEvaluationMethod(Class clazz) {
+  try {
+return getClass().getDeclaredMethod("convertSpecific", clazz);
+  } catch (NoSuchMethodException exc) {
+throw new IllegalArgumentException(
+String.format(
+"Cannot translate trigger class %s to a runner-API proto.",
+clazz.getCanonicalName()),
+exc);
+  }
+}
+
+private RunnerApi.Trigger convertSpecific(DefaultTrigger v) {
+  return RunnerApi.Trigger.newBuilder()
+  .setDefault(RunnerApi.Trigger.Default.getDefaultInstance())
+  .build();
+}
+
+private RunnerApi.Trigger convertSpecific(AfterWatermark.FromEndOfWindow 
v) {
+  return 

[1/9] beam git commit: Reify delay and alignment in AfterProcessingTime transform

2017-02-14 Thread kenn
Repository: beam
Updated Branches:
  refs/heads/master 9061c65e3 -> db19c7df5


Reify delay and alignment in AfterProcessingTime transform


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

Branch: refs/heads/master
Commit: 40c4a5cb6eaa0350a26fe1f215eb812541a7b105
Parents: bea101a
Author: Kenneth Knowles 
Authored: Sun Feb 12 15:03:48 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 14:55:48 2017 -0800

--
 .../AfterDelayFromFirstElementStateMachine.java |   4 +-
 .../AfterProcessingTimeStateMachine.java|   2 +
 ...rSynchronizedProcessingTimeStateMachine.java |   1 +
 .../core/triggers/TriggerStateMachines.java |  36 ++-
 .../core/triggers/TriggerStateMachinesTest.java |   7 +-
 .../windowing/AfterDelayFromFirstElement.java   | 240 ---
 .../windowing/AfterProcessingTime.java  | 105 ++--
 .../AfterSynchronizedProcessingTime.java|  25 +-
 .../windowing/TimestampTransform.java   |  64 +
 .../windowing/AfterProcessingTimeTest.java  |   2 +-
 10 files changed, 192 insertions(+), 294 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/40c4a5cb/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
index b720644..c22 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
@@ -47,6 +47,8 @@ import org.joda.time.format.PeriodFormatter;
  *
  * This class is for internal use only and may change at any time.
  */
+// This class should be inlined to subclasses and deleted, simplifying them too
+// https://issues.apache.org/jira/browse/BEAM-1486
 @Experimental(Experimental.Kind.TRIGGER)
 public abstract class AfterDelayFromFirstElementStateMachine extends 
OnceTriggerStateMachine {
 
@@ -250,7 +252,7 @@ public abstract class 
AfterDelayFromFirstElementStateMachine extends OnceTrigger
   /**
* A {@link SerializableFunction} to delay the timestamp at which this 
triggers fires.
*/
-  private static final class DelayFn implements SerializableFunction {
+  static final class DelayFn implements SerializableFunction 
{
 private final Duration delay;
 
 public DelayFn(Duration delay) {

http://git-wip-us.apache.org/repos/asf/beam/blob/40c4a5cb/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
index 2490463..eaf5613 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java
@@ -34,6 +34,8 @@ import org.joda.time.Instant;
  * AfterDelayFromFirstElementStateMachine#plusDelayOf} or {@link
  * AfterDelayFromFirstElementStateMachine#alignedTo}.
  */
+// The superclass should be inlined here, its only real use
+// https://issues.apache.org/jira/browse/BEAM-1486
 @Experimental(Experimental.Kind.TRIGGER)
 public class AfterProcessingTimeStateMachine extends 
AfterDelayFromFirstElementStateMachine {
 

http://git-wip-us.apache.org/repos/asf/beam/blob/40c4a5cb/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
index 000f6e7..1319a13 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java
+++ 

[9/9] beam git commit: This resolves #1988: Add full-fidelity conversion to/from Java SDK triggers and Runner API protos

2017-02-14 Thread kenn
This resolves #1988: Add full-fidelity conversion to/from Java SDK triggers and 
Runner API protos

  Upgrade Dataflow container version to beam-master-20170214
  Remove underscore from Runner API proto Java package
  Add conversion to/from Java SDK trigger to runner API proto
  Add runner API config to poms
  Flesh out triggers in Runner API proto
  Fix typo in runner API generated Java class
  Upgrade Java triggers to support runner API deserialization
  Reify delay and alignment in AfterProcessingTime transform


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

Branch: refs/heads/master
Commit: db19c7df50294dd5697822d3f4038211797383d7
Parents: 9061c65 e097b75
Author: Kenneth Knowles 
Authored: Tue Feb 14 20:51:00 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 14 20:51:00 2017 -0800

--
 pom.xml |   8 +
 .../AfterDelayFromFirstElementStateMachine.java |   4 +-
 .../AfterProcessingTimeStateMachine.java|   2 +
 ...rSynchronizedProcessingTimeStateMachine.java |   7 +-
 .../core/triggers/TriggerStateMachines.java |  38 ++-
 ...chronizedProcessingTimeStateMachineTest.java |   3 +-
 .../core/triggers/TriggerStateMachinesTest.java |   7 +-
 runners/google-cloud-dataflow-java/pom.xml  |   2 +-
 .../src/main/proto/beam_runner_api.proto| 111 +--
 sdks/java/core/pom.xml  |   5 +
 .../beam/sdk/transforms/windowing/AfterAll.java |   7 +
 .../windowing/AfterDelayFromFirstElement.java   | 240 --
 .../sdk/transforms/windowing/AfterEach.java |   7 +
 .../sdk/transforms/windowing/AfterFirst.java|   7 +
 .../windowing/AfterProcessingTime.java  | 105 +--
 .../AfterSynchronizedProcessingTime.java|  31 +-
 .../windowing/TimestampTransform.java   |  64 
 .../beam/sdk/transforms/windowing/Triggers.java | 313 +++
 .../windowing/AfterProcessingTimeTest.java  |   2 +-
 .../AfterSynchronizedProcessingTimeTest.java|   2 +-
 .../sdk/transforms/windowing/TriggersTest.java  | 100 ++
 21 files changed, 738 insertions(+), 327 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/db19c7df/runners/google-cloud-dataflow-java/pom.xml
--



[6/8] beam git commit: Move StateTag and friends to runners/core-java

2017-02-09 Thread kenn
Move StateTag and friends to runners/core-java


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

Branch: refs/heads/master
Commit: 07d93276e7862e1c238e75854a7faeb15b2d5d60
Parents: c42a19b
Author: Kenneth Knowles 
Authored: Fri Feb 3 19:51:18 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 7 11:44:34 2017 -0800

--
 .../translation/utils/ApexStateInternals.java   |   4 +-
 .../utils/ApexStateInternalsTest.java   |   4 +-
 .../runners/core/InMemoryStateInternals.java|   3 +-
 .../runners/core/MergingActiveWindowSet.java|   2 -
 .../beam/runners/core/MergingStateAccessor.java |   1 -
 .../apache/beam/runners/core/NonEmptyPanes.java |   2 -
 .../beam/runners/core/PaneInfoTracker.java  |   2 -
 .../runners/core/ReduceFnContextFactory.java|   1 -
 .../beam/runners/core/SideInputHandler.java |   2 -
 .../beam/runners/core/SimpleDoFnRunner.java |   1 -
 .../beam/runners/core/SplittableParDo.java  |   2 -
 .../apache/beam/runners/core/StateAccessor.java |   1 -
 .../beam/runners/core/StateInternals.java   |   1 -
 .../apache/beam/runners/core/StateMerging.java  |   1 -
 .../apache/beam/runners/core/StateTable.java|   3 +-
 .../org/apache/beam/runners/core/StateTag.java  | 117 ++
 .../org/apache/beam/runners/core/StateTags.java | 352 +++
 .../beam/runners/core/SystemReduceFn.java   |   2 -
 .../core/TestInMemoryStateInternals.java|   1 -
 .../apache/beam/runners/core/WatermarkHold.java |   2 -
 .../AfterDelayFromFirstElementStateMachine.java |   4 +-
 .../core/triggers/AfterPaneStateMachine.java|   4 +-
 .../TriggerStateMachineContextFactory.java  |   2 +-
 .../triggers/TriggerStateMachineRunner.java |   4 +-
 .../core/InMemoryStateInternalsTest.java|   2 -
 .../beam/runners/core/ReduceFnTester.java   |   1 -
 .../apache/beam/runners/core/StateTagTest.java  | 172 +
 .../CopyOnAccessInMemoryStateInternals.java |   4 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   4 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |   4 +-
 .../runners/direct/EvaluationContextTest.java   |   4 +-
 .../StatefulParDoEvaluatorFactoryTest.java  |   4 +-
 .../wrappers/streaming/FlinkStateInternals.java |   2 +-
 .../streaming/FlinkStateInternalsTest.java  |   4 +-
 .../apache/beam/sdk/util/state/StateTag.java| 111 --
 .../apache/beam/sdk/util/state/StateTags.java   | 344 --
 .../beam/sdk/util/state/StateTagTest.java   | 172 -
 37 files changed, 667 insertions(+), 679 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/07d93276/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
--
diff --git 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
index 544000d..34d993f 100644
--- 
a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
+++ 
b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java
@@ -31,6 +31,8 @@ import java.util.List;
 import org.apache.beam.runners.core.StateInternals;
 import org.apache.beam.runners.core.StateInternalsFactory;
 import org.apache.beam.runners.core.StateNamespace;
+import org.apache.beam.runners.core.StateTag;
+import org.apache.beam.runners.core.StateTag.StateBinder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.Context;
 import org.apache.beam.sdk.coders.InstantCoder;
@@ -47,8 +49,6 @@ import org.apache.beam.sdk.util.state.ReadableState;
 import org.apache.beam.sdk.util.state.State;
 import org.apache.beam.sdk.util.state.StateContext;
 import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTag.StateBinder;
 import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/beam/blob/07d93276/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
--
diff --git 
a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
 

[7/8] beam git commit: Upgrade Dataflow container version to beam-master-20170208

2017-02-09 Thread kenn
Upgrade Dataflow container version to beam-master-20170208


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

Branch: refs/heads/master
Commit: 255c7e28d00a41341cf05764b170462cd41e35c1
Parents: 07d9327
Author: Kenneth Knowles 
Authored: Wed Feb 8 14:04:05 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 8 14:04:05 2017 -0800

--
 runners/google-cloud-dataflow-java/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/255c7e28/runners/google-cloud-dataflow-java/pom.xml
--
diff --git a/runners/google-cloud-dataflow-java/pom.xml 
b/runners/google-cloud-dataflow-java/pom.xml
index 0125e2e..9731ffc 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -33,7 +33,7 @@
   jar
 
   
-
beam-master-20170203
+
beam-master-20170208
 6
   
 



[2/8] beam git commit: Move StateMerging to runners/core-java

2017-02-09 Thread kenn
Move StateMerging to runners/core-java


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

Branch: refs/heads/master
Commit: 9c0047f74635319e28813dd992d5aa3923dac713
Parents: 459a8f8
Author: Kenneth Knowles 
Authored: Fri Feb 3 19:40:46 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 7 11:42:54 2017 -0800

--
 .../utils/ApexStateInternalsTest.java   |   2 +-
 .../apache/beam/runners/core/NonEmptyPanes.java |   1 -
 .../apache/beam/runners/core/StateMerging.java  | 267 +++
 .../beam/runners/core/SystemReduceFn.java   |   1 -
 .../apache/beam/runners/core/WatermarkHold.java |   1 -
 .../AfterDelayFromFirstElementStateMachine.java |   2 +-
 .../core/triggers/AfterPaneStateMachine.java|   2 +-
 .../core/InMemoryStateInternalsTest.java|   1 -
 .../streaming/FlinkStateInternalsTest.java  |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml |   2 +-
 .../beam/sdk/util/state/StateMerging.java   | 259 --
 11 files changed, 272 insertions(+), 268 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/9c0047f7/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
--
diff --git 
a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
 
b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
index 75f648b..d6a4515 100644
--- 
a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
+++ 
b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertThat;
 
 import com.datatorrent.lib.util.KryoCloneUtils;
 import java.util.Arrays;
+import org.apache.beam.runners.core.StateMerging;
 import org.apache.beam.runners.core.StateNamespace;
 import org.apache.beam.runners.core.StateNamespaceForTest;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -36,7 +37,6 @@ import 
org.apache.beam.sdk.util.state.AccumulatorCombiningState;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.CombiningState;
 import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateMerging;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.ValueState;

http://git-wip-us.apache.org/repos/asf/beam/blob/9c0047f7/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
index 0a6fd93..4d67c66 100644
--- 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java
@@ -26,7 +26,6 @@ import 
org.apache.beam.sdk.util.state.AccumulatorCombiningState;
 import org.apache.beam.sdk.util.state.MergingStateAccessor;
 import org.apache.beam.sdk.util.state.ReadableState;
 import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 

http://git-wip-us.apache.org/repos/asf/beam/blob/9c0047f7/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
--
diff --git 
a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
new file mode 100644
index 000..c85458c
--- /dev/null
+++ 
b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law 

[3/8] beam git commit: Move StateTag adapter code to StateTags

2017-02-09 Thread kenn
Move StateTag adapter code to StateTags


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

Branch: refs/heads/master
Commit: c42a19b16fc70b48ecdd1cdd23148d16bb41fd7b
Parents: d39cec4
Author: Kenneth Knowles 
Authored: Fri Feb 3 19:48:38 2017 -0800
Committer: Kenneth Knowles 
Committed: Tue Feb 7 11:42:55 2017 -0800

--
 .../apache/beam/sdk/util/state/StateSpecs.java  | 59 
 .../apache/beam/sdk/util/state/StateTags.java   | 59 +++-
 2 files changed, 58 insertions(+), 60 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam/blob/c42a19b1/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java
index b9c22cf..08c3a12 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java
@@ -387,63 +387,4 @@ public class StateSpecs {
 }
   }
 
-  /**
-   * @deprecated for migration purposes only
-   */
-  @Deprecated
-  public static  StateBinder adaptTagBinder(final 
StateTag.StateBinder binder) {
-return new StateBinder() {
-  @Override
-  public  ValueState bindValue(
-  String id, StateSpec> spec, Coder coder) 
{
-return binder.bindValue(StateTags.tagForSpec(id, spec), coder);
-  }
-
-  @Override
-  public  BagState bindBag(
-  String id, StateSpec> spec, Coder 
elemCoder) {
-return binder.bindBag(StateTags.tagForSpec(id, spec), elemCoder);
-  }
-
-  @Override
-  public 
-  AccumulatorCombiningState 
bindCombiningValue(
-  String id,
-  StateSpec> spec,
-  Coder accumCoder,
-  CombineFn combineFn) {
-return binder.bindCombiningValue(StateTags.tagForSpec(id, spec), 
accumCoder, combineFn);
-  }
-
-  @Override
-  public 
-  AccumulatorCombiningState 
bindKeyedCombiningValue(
-  String id,
-  StateSpec> spec,
-  Coder accumCoder,
-  KeyedCombineFn combineFn) {
-return binder.bindKeyedCombiningValue(
-StateTags.tagForSpec(id, spec), accumCoder, combineFn);
-  }
-
-  @Override
-  public 
-  AccumulatorCombiningState 
bindKeyedCombiningValueWithContext(
-  String id,
-  StateSpec> spec,
-  Coder accumCoder,
-  KeyedCombineFnWithContext 
combineFn) {
-return binder.bindKeyedCombiningValueWithContext(
-StateTags.tagForSpec(id, spec), accumCoder, combineFn);
-  }
-
-  @Override
-  public  WatermarkHoldState bindWatermark(
-  String id,
-  StateSpec> spec,
-  OutputTimeFn outputTimeFn) {
-return binder.bindWatermark(StateTags.tagForSpec(id, spec), 
outputTimeFn);
-  }
-};
-  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/c42a19b1/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
--
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
index 4fe3a4f..acb1f08 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java
@@ -43,6 +43,63 @@ public class StateTags {
 STANDARD_REGISTRY.registerStandardCoders();
   }
 
+  /** @deprecated for migration purposes only */
+  @Deprecated
+  private static  StateBinder adaptTagBinder(final 
StateTag.StateBinder binder) {
+return new StateBinder() {
+  @Override
+  public  ValueState bindValue(
+  String id, StateSpec> spec, Coder coder) 
{
+return binder.bindValue(tagForSpec(id, spec), coder);
+  }
+
+  @Override
+  public  BagState bindBag(
+  String id, StateSpec> spec, Coder 
elemCoder) {
+return binder.bindBag(tagForSpec(id, spec), elemCoder);
+  }
+
+  @Override
+  public 
+  AccumulatorCombiningState 
bindCombiningValue(
+  

[5/8] beam git commit: Move StateTag and friends to runners/core-java

2017-02-09 Thread kenn
http://git-wip-us.apache.org/repos/asf/beam/blob/07d93276/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
--
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
deleted file mode 100644
index 7c06dbd..000
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util.state;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Max;
-import org.apache.beam.sdk.transforms.Min;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.util.CombineFnUtil;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link StateTag}.
- */
-@RunWith(JUnit4.class)
-public class StateTagTest {
-  @Test
-  public void testValueEquality() {
-StateTag fooVarInt1 = StateTags.value("foo", VarIntCoder.of());
-StateTag fooVarInt2 = StateTags.value("foo", VarIntCoder.of());
-StateTag fooBigEndian = StateTags.value("foo", 
BigEndianIntegerCoder.of());
-StateTag barVarInt = StateTags.value("bar", VarIntCoder.of());
-
-assertEquals(fooVarInt1, fooVarInt2);
-assertNotEquals(fooVarInt1, fooBigEndian);
-assertNotEquals(fooVarInt1, barVarInt);
-  }
-
-  @Test
-  public void testBagEquality() {
-StateTag fooVarInt1 = StateTags.bag("foo", VarIntCoder.of());
-StateTag fooVarInt2 = StateTags.bag("foo", VarIntCoder.of());
-StateTag fooBigEndian = StateTags.bag("foo", 
BigEndianIntegerCoder.of());
-StateTag barVarInt = StateTags.bag("bar", VarIntCoder.of());
-
-assertEquals(fooVarInt1, fooVarInt2);
-assertNotEquals(fooVarInt1, fooBigEndian);
-assertNotEquals(fooVarInt1, barVarInt);
-  }
-
-  @Test
-  public void testWatermarkBagEquality() {
-StateTag foo1 = StateTags.watermarkStateInternal(
-"foo", OutputTimeFns.outputAtEarliestInputTimestamp());
-StateTag foo2 = StateTags.watermarkStateInternal(
-"foo", OutputTimeFns.outputAtEarliestInputTimestamp());
-StateTag bar = StateTags.watermarkStateInternal(
-"bar", OutputTimeFns.outputAtEarliestInputTimestamp());
-
-StateTag bar2 = StateTags.watermarkStateInternal(
-"bar", OutputTimeFns.outputAtLatestInputTimestamp());
-
-// Same id, same fn.
-assertEquals(foo1, foo2);
-// Different id, same fn.
-assertNotEquals(foo1, bar);
-// Same id, different fn.
-assertEquals(bar, bar2);
-  }
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @Test
-  public void testCombiningValueEquality() {
-Combine.BinaryCombineIntegerFn maxFn = Max.ofIntegers();
-Coder input1 = VarIntCoder.of();
-Coder input2 = BigEndianIntegerCoder.of();
-Combine.BinaryCombineIntegerFn minFn = Min.ofIntegers();
-
-StateTag fooCoder1Max1 = 
StateTags.combiningValueFromInputInternal("foo", input1, maxFn);
-StateTag fooCoder1Max2 = 
StateTags.combiningValueFromInputInternal("foo", input1, maxFn);
-StateTag fooCoder1Min = 
StateTags.combiningValueFromInputInternal("foo", input1, minFn);
-
-StateTag fooCoder2Max = 
StateTags.combiningValueFromInputInternal("foo", input2, maxFn);
-StateTag barCoder1Max = 
StateTags.combiningValueFromInputInternal("bar", input1, maxFn);
-
-// Same name, coder and combineFn
-assertEquals(fooCoder1Max1, fooCoder1Max2);
-assertEquals(
-StateTags.convertToBagTagInternal((StateTag) fooCoder1Max1),
-StateTags.convertToBagTagInternal((StateTag) fooCoder1Max2));
-
-// Different combineFn, but we treat them as equal since we only serialize 
the bits.
-assertEquals(fooCoder1Max1, fooCoder1Min);

[2/3] beam-site git commit: Touch up some punctuation in state blog post

2017-02-15 Thread kenn
Touch up some punctuation in state blog post


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

Branch: refs/heads/asf-site
Commit: f928b50e1594cba453f07aa608ab28afb6329aff
Parents: 8518baa
Author: Kenneth Knowles 
Authored: Wed Feb 15 10:08:31 2017 -0800
Committer: Kenneth Knowles 
Committed: Wed Feb 15 10:08:31 2017 -0800

--
 src/_posts/2017-02-13-stateful-processing.md | 7 ---
 1 file changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam-site/blob/f928b50e/src/_posts/2017-02-13-stateful-processing.md
--
diff --git a/src/_posts/2017-02-13-stateful-processing.md 
b/src/_posts/2017-02-13-stateful-processing.md
index b3e0c63..fbbe76b 100644
--- a/src/_posts/2017-02-13-stateful-processing.md
+++ b/src/_posts/2017-02-13-stateful-processing.md
@@ -238,11 +238,12 @@ key+window pairs, like this:
 keys and windows are independent dimensions)
 
 You can provide the opportunity for parallelism by making sure that table has
-enough columns, either via:
+enough columns. You might have many keys and many windows, or you might have
+many of just one or the other:
 
-- Many keys in few windows for example, a globally windowed stateful 
computation
+- Many keys in few windows, for example a globally windowed stateful 
computation
   keyed by user ID.
-- Many windows over few keys for example, a fixed windowed stateful computation
+- Many windows over few keys, for example a fixed windowed stateful computation
   over a global key.
 
 Caveat: all Beam runners today parallelize only over the key.



[1/3] beam-site git commit: Fix some typos and small formatting issues.

2017-02-15 Thread kenn
Repository: beam-site
Updated Branches:
  refs/heads/asf-site f3c189568 -> f48e97f67


Fix some typos and small formatting issues.


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

Branch: refs/heads/asf-site
Commit: 8518baa70e1ec0dc609dbd36889e246752bc995e
Parents: f3c1895
Author: Ismaël Mejía 
Authored: Wed Feb 15 17:45:25 2017 +0100
Committer: Ismaël Mejía 
Committed: Wed Feb 15 17:45:25 2017 +0100

--
 src/_posts/2017-02-13-stateful-processing.md | 14 +-
 1 file changed, 9 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/beam-site/blob/8518baa7/src/_posts/2017-02-13-stateful-processing.md
--
diff --git a/src/_posts/2017-02-13-stateful-processing.md 
b/src/_posts/2017-02-13-stateful-processing.md
index b00361a..b3e0c63 100644
--- a/src/_posts/2017-02-13-stateful-processing.md
+++ b/src/_posts/2017-02-13-stateful-processing.md
@@ -196,7 +196,7 @@ want to write a transform that maps input to output like 
this:
 
+width="180">
 
 The order of the elements A, B, C, D, E is arbitrary, hence their assigned
 indices are arbitrary, but downstream transforms just need to be OK with this.
@@ -238,9 +238,13 @@ key+window pairs, like this:
 keys and windows are independent dimensions)
 
 You can provide the opportunity for parallelism by making sure that table has
-enough columns, either via many keys in few windows - for example, a globally
-windowed stateful computation keyed by user ID - or via many windows over few
-keys - for example, a fixed windowed stateful computation over a global key.
+enough columns, either via:
+
+- Many keys in few windows for example, a globally windowed stateful 
computation
+  keyed by user ID.
+- Many windows over few keys for example, a fixed windowed stateful computation
+  over a global key.
+
 Caveat: all Beam runners today parallelize only over the key.
 
 Most often your mental model of state can be focused on only a single column of
@@ -444,7 +448,7 @@ outputs from the `ParDo` that will be processed downstream. 
If the
 output, then you cannot use a `Filter` transform to reduce data volume 
downstream.
 
 Stateful processing lets you address both the latency problem of side inputs
-and the cost problem of excessive uninterseting output. Here is the code, using
+and the cost problem of excessive uninteresting output. Here is the code, using
 only features I have already introduced:
 
 ```java



  1   2   3   4   5   6   7   8   9   10   >