You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by th...@apache.org on 2016/10/28 14:47:35 UTC

[01/50] incubator-beam git commit: Encode bundle elements in the DirectRunner

Repository: incubator-beam
Updated Branches:
  refs/heads/apex-runner fa3a6aa8d -> 968eb32b8


Encode bundle elements in the DirectRunner

This ensures that any changes that are caused when an element is encoded
and decoded is caught within the pipeline.


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

Branch: refs/heads/apex-runner
Commit: 2ceaa3effa8a6d9de3753a05db9d1648e8eed576
Parents: c03e3e9
Author: Thomas Groh <tg...@google.com>
Authored: Tue Sep 20 11:43:40 2016 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Tue Oct 25 11:03:43 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/CloningBundleFactory.java    |  98 ++++++++++
 .../beam/runners/direct/DirectRunner.java       |   5 +-
 .../direct/ImmutableListBundleFactory.java      |   4 +-
 .../direct/CloningBundleFactoryTest.java        | 177 +++++++++++++++++++
 .../EncodabilityEnforcementFactoryTest.java     |   6 +-
 5 files changed, 285 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
new file mode 100644
index 0000000..33241e3
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+
+/**
+ * A {@link BundleFactory} where a created {@link UncommittedBundle} clones all elements added to it
+ * using the coder of the {@link PCollection}.
+ */
+class CloningBundleFactory implements BundleFactory {
+  private static final CloningBundleFactory INSTANCE = new CloningBundleFactory();
+
+  public static CloningBundleFactory create() {
+    return INSTANCE;
+  }
+
+  private final ImmutableListBundleFactory underlying;
+  private CloningBundleFactory() {
+    this.underlying = ImmutableListBundleFactory.create();
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createRootBundle() {
+    // The DirectRunner is responsible for these elements, but they need not be encodable.
+    return underlying.createRootBundle();
+  }
+
+  @Override
+  public <T> UncommittedBundle<T> createBundle(
+      PCollection<T> output) {
+    return new CloningBundle<>(underlying.createBundle(output));
+  }
+
+  @Override
+  public <K, T> UncommittedBundle<T> createKeyedBundle(
+      StructuralKey<K> key, PCollection<T> output) {
+    return new CloningBundle<>(underlying.createKeyedBundle(key, output));
+  }
+
+  private static class CloningBundle<T> implements UncommittedBundle<T> {
+    private final UncommittedBundle<T> underlying;
+    private final Coder<T> coder;
+
+    private CloningBundle(UncommittedBundle<T> underlying) {
+      this.underlying = underlying;
+      this.coder = underlying.getPCollection().getCoder();
+    }
+
+    @Override
+    public PCollection<T> getPCollection() {
+      return underlying.getPCollection();
+    }
+
+    @Override
+    public UncommittedBundle<T> add(WindowedValue<T> element) {
+      try {
+        // Use the cloned value to ensure that if the coder behaves poorly (e.g. a NoOpCoder that
+        // does not expect to be used) that is reflected in the values given to downstream
+        // transforms
+        WindowedValue<T> clone = element.withValue(CoderUtils.clone(coder, element.getValue()));
+        underlying.add(clone);
+      } catch (CoderException e) {
+        throw UserCodeException.wrap(e);
+      }
+      return this;
+    }
+
+    @Override
+    public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
+      return underlying.commit(synchronizedProcessingTime);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index b79a42f..e02c8a6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -319,7 +319,10 @@ public class DirectRunner
   }
 
   private BundleFactory createBundleFactory(DirectOptions pipelineOptions) {
-    BundleFactory bundleFactory = ImmutableListBundleFactory.create();
+    BundleFactory bundleFactory =
+        pipelineOptions.isEnforceEncodability()
+            ? CloningBundleFactory.create()
+            : ImmutableListBundleFactory.create();
     if (pipelineOptions.isEnforceImmutability()) {
       bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
index db92542..abc6dd8 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
@@ -32,8 +32,10 @@ import org.joda.time.Instant;
  * A factory that produces bundles that perform no additional validation.
  */
 class ImmutableListBundleFactory implements BundleFactory {
+  private static final ImmutableListBundleFactory FACTORY = new ImmutableListBundleFactory();
+
   public static ImmutableListBundleFactory create() {
-    return new ImmutableListBundleFactory();
+    return FACTORY;
   }
 
   private ImmutableListBundleFactory() {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
new file mode 100644
index 0000000..03846d9
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.direct;
+
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.isA;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.theInstance;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.Record;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.RecordNoDecodeCoder;
+import org.apache.beam.runners.direct.EncodabilityEnforcementFactoryTest.RecordNoEncodeCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link CloningBundleFactory}.
+ */
+@RunWith(JUnit4.class)
+public class CloningBundleFactoryTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private CloningBundleFactory factory = CloningBundleFactory.create();
+
+  @Test
+  public void rootBundleSucceedsIgnoresCoder() {
+    WindowedValue<Record> one = WindowedValue.valueInGlobalWindow(new Record());
+    WindowedValue<Record> two = WindowedValue.valueInGlobalWindow(new Record());
+    CommittedBundle<Record> root =
+        factory.<Record>createRootBundle().add(one).add(two).commit(Instant.now());
+
+    assertThat(root.getElements(), containsInAnyOrder(one, two));
+  }
+
+  @Test
+  public void bundleWorkingCoderSucceedsClonesOutput() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of()));
+    PCollection<KV<String, Integer>> kvs =
+        created
+            .apply(WithKeys.<String, Integer>of("foo"))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+    WindowedValue<KV<String, Integer>> fooOne = WindowedValue.valueInGlobalWindow(KV.of("foo", 1));
+    WindowedValue<KV<String, Integer>> fooThree =
+        WindowedValue.valueInGlobalWindow(KV.of("foo", 3));
+    CommittedBundle<KV<String, Integer>> bundle =
+        factory.createBundle(kvs).add(fooOne).add(fooThree).commit(Instant.now());
+
+    assertThat(bundle.getElements(), containsInAnyOrder(fooOne, fooThree));
+    assertThat(
+        bundle.getElements(), not(containsInAnyOrder(theInstance(fooOne), theInstance(fooThree))));
+    for (WindowedValue<KV<String, Integer>> foo : bundle.getElements()) {
+      assertThat(
+          foo.getValue(),
+          not(anyOf(theInstance(fooOne.getValue()), theInstance(fooThree.getValue()))));
+    }
+    assertThat(bundle.getPCollection(), equalTo(kvs));
+  }
+
+  @Test
+  public void keyedBundleWorkingCoderSucceedsClonesOutput() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Integer> created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of()));
+
+    PCollection<KV<String, Iterable<Integer>>> keyed =
+        created
+            .apply(WithKeys.<String, Integer>of("foo"))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+            .apply(GroupByKey.<String, Integer>create());
+    WindowedValue<KV<String, Iterable<Integer>>> foos =
+        WindowedValue.valueInGlobalWindow(
+            KV.<String, Iterable<Integer>>of("foo", ImmutableList.of(1, 3)));
+    CommittedBundle<KV<String, Iterable<Integer>>> keyedBundle =
+        factory
+            .createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), keyed)
+            .add(foos)
+            .commit(Instant.now());
+
+    assertThat(keyedBundle.getElements(), containsInAnyOrder(foos));
+    assertThat(
+        Iterables.getOnlyElement(keyedBundle.getElements()).getValue(),
+        not(theInstance(foos.getValue())));
+    assertThat(keyedBundle.getPCollection(), equalTo(keyed));
+    assertThat(
+        keyedBundle.getKey(),
+        Matchers.<StructuralKey<?>>equalTo(StructuralKey.of("foo", StringUtf8Coder.of())));
+  }
+
+  @Test
+  public void bundleEncodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoEncodeCoder()));
+    UncommittedBundle<Record> bundle = factory.createBundle(pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Encode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void bundleDecodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoDecodeCoder()));
+    UncommittedBundle<Record> bundle = factory.createBundle(pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Decode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void keyedBundleEncodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoEncodeCoder()));
+    UncommittedBundle<Record> bundle =
+        factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Encode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+
+  @Test
+  public void keyedBundleDecodeFailsAddFails() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Record> pc = p.apply(Create.<Record>of().withCoder(new RecordNoDecodeCoder()));
+    UncommittedBundle<Record> bundle =
+        factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(isA(CoderException.class));
+    thrown.expectMessage("Decode not allowed");
+    bundle.add(WindowedValue.valueInGlobalWindow(new Record()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ceaa3ef/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
index e62bf01..e6bdbd0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java
@@ -208,8 +208,8 @@ public class EncodabilityEnforcementFactoryTest {
         Collections.<CommittedBundle<?>>emptyList());
   }
 
-  private static class Record {}
-  private static class RecordNoEncodeCoder extends AtomicCoder<Record> {
+  static class Record {}
+  static class RecordNoEncodeCoder extends AtomicCoder<Record> {
 
     @Override
     public void encode(
@@ -228,7 +228,7 @@ public class EncodabilityEnforcementFactoryTest {
     }
   }
 
-  private static class RecordNoDecodeCoder extends AtomicCoder<Record> {
+  static class RecordNoDecodeCoder extends AtomicCoder<Record> {
     @Override
     public void encode(
         Record value,


[46/50] incubator-beam git commit: This closes #1205

Posted by th...@apache.org.
This closes #1205


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

Branch: refs/heads/apex-runner
Commit: f2ec82453d01c6ede971912689ae40fcbfa9d3ac
Parents: 3fd3951 324a36d
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 27 14:55:46 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 14:55:46 2016 -0700

----------------------------------------------------------------------
 pom.xml | 39 +++++++++++++++++++++++++++++++++++++++
 1 file changed, 39 insertions(+)
----------------------------------------------------------------------



[40/50] incubator-beam git commit: This closes #1176

Posted by th...@apache.org.
This closes #1176


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

Branch: refs/heads/apex-runner
Commit: 064f18a8f704ba4713701b7c8432a6f9b401a110
Parents: ee6ad2f 73db560
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 27 10:48:35 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 10:48:35 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/DataflowRunner.java   |  4 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    | 43 ++++++++++++++++++--
 2 files changed, 41 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[07/50] incubator-beam git commit: This closes #1178

Posted by th...@apache.org.
This closes #1178


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

Branch: refs/heads/apex-runner
Commit: 13af296dec5fd5b9b4f922dac6ce1dc9b026cf60
Parents: 7abdcbf 2872019
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 13:12:17 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../direct/WriteWithShardingFactory.java        |  6 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 32 +++----
 .../beam/runners/direct/DirectRunnerTest.java   | 40 +++++----
 .../ImmutabilityCheckingBundleFactoryTest.java  |  8 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  8 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |  8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  8 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 87 +++++++++---------
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 94 +++++++++-----------
 .../runners/direct/WatermarkManagerTest.java    |  8 +-
 .../apache/beam/sdk/util/ValueWithRecordId.java |  8 +-
 11 files changed, 146 insertions(+), 161 deletions(-)
----------------------------------------------------------------------



[49/50] incubator-beam git commit: Merge branch 'master' into apex-runner

Posted by th...@apache.org.
Merge branch 'master' into apex-runner


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

Branch: refs/heads/apex-runner
Commit: 6fc47ed101f02aacacecd5c62c4a026eaba3e12c
Parents: fa3a6aa 215980a
Author: Thomas Weise <th...@apache.org>
Authored: Thu Oct 27 19:25:06 2016 -0700
Committer: Thomas Weise <th...@apache.org>
Committed: Thu Oct 27 19:25:06 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |  22 +-
 .../beam/examples/DebuggingWordCount.java       |   2 +-
 .../apache/beam/examples/MinimalWordCount.java  |   2 +-
 .../org/apache/beam/examples/WordCount.java     |   2 +-
 .../apache/beam/examples/complete/TfIdf.java    |   2 +-
 .../examples/complete/TopWikipediaSessions.java |   2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/DeDupExample.java    |   2 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../beam/examples/cookbook/JoinExamples.java    |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../org/apache/beam/examples/WordCountTest.java |   2 +-
 .../examples/complete/AutoCompleteTest.java     |   6 +-
 .../beam/examples/complete/TfIdfTest.java       |   2 +-
 .../complete/TopWikipediaSessionsTest.java      |   2 +-
 .../examples/cookbook/DeDupExampleTest.java     |   4 +-
 .../examples/cookbook/JoinExamplesTest.java     |   2 +-
 .../examples/cookbook/TriggerExampleTest.java   |   2 +-
 .../beam/examples/MinimalWordCountJava8.java    |   2 +-
 .../beam/examples/complete/game/GameStats.java  |  52 +-
 .../examples/complete/game/HourlyTeamScore.java |  30 +-
 .../examples/complete/game/LeaderBoard.java     |  43 +-
 .../beam/examples/complete/game/UserScore.java  |  16 +-
 .../complete/game/utils/WriteToBigQuery.java    |  49 +-
 .../game/utils/WriteWindowedToBigQuery.java     |  14 +-
 .../examples/complete/game/GameStatsTest.java   |   2 +-
 .../complete/game/HourlyTeamScoreTest.java      |   2 +-
 .../examples/complete/game/LeaderBoardTest.java |  10 +-
 .../examples/complete/game/UserScoreTest.java   |   6 +-
 pom.xml                                         |  47 +-
 .../runners/core/GroupAlsoByWindowsDoFn.java    |  19 -
 .../runners/direct/CloningBundleFactory.java    |  98 ++++
 .../beam/runners/direct/DirectRunner.java       |   7 +-
 .../runners/direct/DoFnLifecycleManager.java    |  56 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |   8 +-
 .../direct/ImmutableListBundleFactory.java      |   4 +-
 .../beam/runners/direct/WatermarkManager.java   |  17 +-
 .../direct/WriteWithShardingFactory.java        |   6 +-
 .../direct/CloningBundleFactoryTest.java        | 177 +++++++
 .../ConsumerTrackingPipelineVisitorTest.java    |  32 +-
 .../beam/runners/direct/DirectRunnerTest.java   |  40 +-
 .../direct/DoFnLifecycleManagerTest.java        |  74 ++-
 .../EncodabilityEnforcementFactoryTest.java     |   6 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   8 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   8 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   8 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  87 ++--
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  94 ++--
 .../runners/direct/WatermarkManagerTest.java    |   8 +-
 .../dataflow/BlockingDataflowRunner.java        |  13 +-
 .../runners/dataflow/DataflowPipelineJob.java   |  17 +-
 .../dataflow/DataflowPipelineTranslator.java    |   4 +
 .../beam/runners/dataflow/DataflowRunner.java   |   4 +-
 .../options/DataflowPipelineOptions.java        |  12 +
 .../beam/runners/dataflow/util/DoFnInfo.java    |  43 +-
 runners/spark/pom.xml                           |  66 ++-
 .../runners/spark/SparkPipelineOptions.java     |  11 +
 .../apache/beam/runners/spark/SparkRunner.java  |  19 -
 .../metrics/AggregatorMetricSource.java         |   9 +-
 .../metrics/WithNamedAggregatorsSupport.java    |   7 +-
 .../coders/BeamSparkRunnerRegistrator.java      |  46 ++
 .../runners/spark/io/EmptyCheckpointMark.java   |  52 ++
 .../apache/beam/runners/spark/io/KafkaIO.java   | 131 -----
 .../beam/runners/spark/io/MicrobatchSource.java | 262 ++++++++++
 .../beam/runners/spark/io/SourceDStream.java    | 156 ++++++
 .../apache/beam/runners/spark/io/SourceRDD.java |  75 ++-
 .../runners/spark/io/SparkUnboundedSource.java  | 167 ++++++
 .../spark/stateful/StateSpecFunctions.java      | 167 ++++++
 .../runners/spark/stateful/package-info.java    |  22 +
 .../spark/translation/EvaluationContext.java    |   6 +-
 .../translation/GroupCombineFunctions.java      |  66 +--
 .../spark/translation/SparkContextFactory.java  |   5 +-
 .../spark/translation/SparkRuntimeContext.java  |   2 +-
 .../spark/translation/TransformTranslator.java  |  43 +-
 .../spark/translation/TranslationUtils.java     |  17 +
 .../SparkRunnerStreamingContextFactory.java     |   7 +
 .../streaming/StreamingEvaluationContext.java   |   6 +-
 .../streaming/StreamingTransformTranslator.java | 100 +---
 .../spark/util/SparkSideInputReader.java        |   2 +-
 .../runners/spark/ClearAggregatorsRule.java     |  33 --
 .../runners/spark/InMemoryMetricsSinkRule.java  |  32 --
 .../metrics/sink/ClearAggregatorsRule.java      |  33 ++
 .../metrics/sink/InMemoryMetrics.java           |  15 +-
 .../metrics/sink/InMemoryMetricsSinkRule.java   |  31 ++
 .../metrics/sink/NamedAggregatorsTest.java      |  92 ++++
 .../streaming/KafkaStreamingTest.java           | 150 ++++--
 .../ResumeFromCheckpointStreamingTest.java      |  34 +-
 .../streaming/utils/EmbeddedKafkaCluster.java   |  51 +-
 .../utils/KafkaWriteOnBatchCompleted.java       | 105 ++++
 .../org/apache/beam/sdk/PipelineResult.java     |  10 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     | 143 +++++-
 .../beam/sdk/options/ValueProviderUtils.java    |  60 +++
 .../sdk/transforms/DelegatingAggregator.java    |   7 +-
 .../sdk/transforms/display/DisplayData.java     |   6 +-
 .../beam/sdk/transforms/windowing/AfterAll.java |  49 --
 .../windowing/AfterDelayFromFirstElement.java   |  99 ----
 .../sdk/transforms/windowing/AfterEach.java     |  61 ---
 .../sdk/transforms/windowing/AfterFirst.java    |  50 --
 .../sdk/transforms/windowing/AfterPane.java     |  52 --
 .../windowing/AfterProcessingTime.java          |   7 -
 .../AfterSynchronizedProcessingTime.java        |   7 -
 .../transforms/windowing/AfterWatermark.java    | 158 ------
 .../transforms/windowing/DefaultTrigger.java    |  35 --
 .../beam/sdk/transforms/windowing/Never.java    |  17 -
 .../transforms/windowing/OrFinallyTrigger.java  |  46 --
 .../sdk/transforms/windowing/Repeatedly.java    |  30 --
 .../beam/sdk/transforms/windowing/Trigger.java  | 412 ++-------------
 .../apache/beam/sdk/util/ExecutableTrigger.java |  40 +-
 .../apache/beam/sdk/util/FinishedTriggers.java  |  44 --
 .../beam/sdk/util/FinishedTriggersBitSet.java   |  67 ---
 .../beam/sdk/util/FinishedTriggersSet.java      |  72 ---
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |  14 -
 .../beam/sdk/util/TriggerContextFactory.java    | 507 -------------------
 .../apache/beam/sdk/util/ValueWithRecordId.java |   8 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  29 ++
 .../beam/sdk/options/ValueProviderTest.java     |  17 +-
 .../sdk/options/ValueProviderUtilsTest.java     |  78 +++
 .../sdk/transforms/windowing/AfterAllTest.java  |  98 ----
 .../sdk/transforms/windowing/AfterEachTest.java |  64 ---
 .../transforms/windowing/AfterFirstTest.java    | 120 -----
 .../sdk/transforms/windowing/AfterPaneTest.java |  77 ---
 .../windowing/AfterProcessingTimeTest.java      |  94 ----
 .../AfterSynchronizedProcessingTimeTest.java    |  75 ---
 .../windowing/AfterWatermarkTest.java           | 308 -----------
 .../windowing/DefaultTriggerTest.java           | 130 -----
 .../sdk/transforms/windowing/NeverTest.java     |  34 +-
 .../windowing/OrFinallyTriggerTest.java         | 136 -----
 .../transforms/windowing/RepeatedlyTest.java    | 161 +-----
 .../sdk/transforms/windowing/StubTrigger.java   |  17 -
 .../sdk/transforms/windowing/TriggerTest.java   |  28 -
 .../beam/sdk/util/ExecutableTriggerTest.java    |  18 -
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 --
 .../sdk/util/FinishedTriggersProperties.java    | 110 ----
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ---
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  23 -
 .../org/apache/beam/sdk/util/TriggerTester.java | 410 ---------------
 138 files changed, 2738 insertions(+), 4620 deletions(-)
----------------------------------------------------------------------



[38/50] incubator-beam git commit: Update Dataflow worker image pointer

Posted by th...@apache.org.
Update Dataflow worker image pointer


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

Branch: refs/heads/apex-runner
Commit: 95bf7a835d81a7b62b734362386e0aea64a42532
Parents: ee6ad2f
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 14:45:54 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 10:48:34 2016 -0700

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


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


[08/50] incubator-beam git commit: Remove OldDoFn to DoFn in DirectRunner

Posted by th...@apache.org.
Remove OldDoFn to DoFn in DirectRunner


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

Branch: refs/heads/apex-runner
Commit: 287201916776cb51b98b9ddd27c169f87bb89e1a
Parents: 3d08685
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 11:18:10 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/direct/WriteWithShardingFactory.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28720191/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
index d74cd56..8727cb5 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
@@ -25,9 +25,9 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.apache.beam.sdk.io.Write;
 import org.apache.beam.sdk.io.Write.Bound;
 import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Values;
@@ -102,7 +102,7 @@ class WriteWithShardingFactory implements PTransformOverrideFactory {
   }
 
   @VisibleForTesting
-  static class KeyBasedOnCountFn<T> extends OldDoFn<T, KV<Integer, T>> {
+  static class KeyBasedOnCountFn<T> extends DoFn<T, KV<Integer, T>> {
     @VisibleForTesting
     static final int MIN_SHARDS_FOR_LOG = 3;
 
@@ -116,7 +116,7 @@ class WriteWithShardingFactory implements PTransformOverrideFactory {
       this.randomExtraShards = extraShards;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
       if (maxShards == 0) {
         maxShards = calculateShards(c.sideInput(numRecords));


[19/50] incubator-beam git commit: Revert "Fix invalid snapshot repo in Travis CI's maven settings.xml"

Posted by th...@apache.org.
Revert "Fix invalid snapshot repo in Travis CI's maven settings.xml"

This reverts commit fbc23ed9234088c9d88bf88b19a3a4d6d5e52446.


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

Branch: refs/heads/apex-runner
Commit: 3c71e4da3490c599436d71ac344354c5fa8b693f
Parents: 78e2c03
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 19:31:26 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:13:59 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3c71e4da/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 277f883..b11f61b 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -44,10 +44,6 @@ matrix:
       env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
 
 before_install:
-  # Workaround for https://github.com/travis-ci/travis-ci/issues/4629
-  - cat ~/.m2/settings.xml
-  - sed -i.bak -e 's|https://nexus.codehaus.org/snapshots/|https://oss.sonatype.org/content/repositories/codehaus-snapshots/|g' ~/.m2/settings.xml
-  - cat ~/.m2/settings.xml
   - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
   - echo $'MAVEN_OPTS="$MAVEN_OPTS -Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS"' >> ~/.mavenrc
   - cat ~/.mavenrc


[47/50] incubator-beam git commit: Use a int Sequence instead of a Random UUID for Aggregator IDs

Posted by th...@apache.org.
Use a int Sequence instead of a Random UUID for Aggregator IDs

Aggregator IDs are used to ensure that an Aggregator's identity is
consistent across synchronization barriers. This is only relevant when
constructing the map of Step -> Aggregator to enable querying, as the
DoFns represented within the graph may be serialized. The identity has
no impact on the interaction between the runner and aggregator, which is
the responsibility of the ProcessContext object and
setupDelegateAggregators.

UUID#randomUUID uses a shared SecureRandom to create the bytes of the
UUID; SecureRandom#nextBytes is a synchronized method, so regardless of
the underlying source of randomness, only one random UUID can be
generated at a time. Instead, use an atomically increasing int to
identify aggregators. This should be sufficient for user-created
aggregators, and system aggregators should not care about the id.


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

Branch: refs/heads/apex-runner
Commit: dd854b1a71770b9b452361e0d92e018b65f1b3e8
Parents: f2ec824
Author: Thomas Groh <tg...@google.com>
Authored: Thu Oct 27 10:19:03 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Oct 27 16:28:32 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd854b1a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
index d92bb71..e03d3b1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
@@ -22,7 +22,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import com.google.common.base.MoreObjects;
 import java.io.Serializable;
 import java.util.Objects;
-import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 
 /**
@@ -37,7 +37,8 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn;
  */
 class DelegatingAggregator<AggInputT, AggOutputT>
     implements Aggregator<AggInputT, AggOutputT>, Serializable {
-  private final UUID id;
+  private static final AtomicInteger ID_GEN = new AtomicInteger();
+  private final int id;
 
   private final String name;
 
@@ -47,7 +48,7 @@ class DelegatingAggregator<AggInputT, AggOutputT>
 
   public DelegatingAggregator(String name,
       CombineFn<? super AggInputT, ?, AggOutputT> combiner) {
-    this.id = UUID.randomUUID();
+    this.id = ID_GEN.getAndIncrement();
     this.name = checkNotNull(name, "name cannot be null");
     // Safe contravariant cast
     @SuppressWarnings("unchecked")


[50/50] incubator-beam git commit: Adjust for merge from master.

Posted by th...@apache.org.
Adjust for merge from master.


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

Branch: refs/heads/apex-runner
Commit: 968eb32b8a7bbbb7d5613f7645c0f04ce194588e
Parents: 6fc47ed
Author: Thomas Weise <th...@apache.org>
Authored: Thu Oct 27 20:21:52 2016 -0700
Committer: Thomas Weise <th...@apache.org>
Committed: Thu Oct 27 20:21:52 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/968eb32b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java
index d5613fe..4e3a8d2 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java
@@ -62,12 +62,12 @@ public class ApexRunnerResult implements PipelineResult {
   }
 
   @Override
-  public State waitUntilFinish(Duration duration) throws IOException, InterruptedException {
+  public State waitUntilFinish(Duration duration) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public State waitUntilFinish() throws IOException, InterruptedException {
+  public State waitUntilFinish() {
     throw new UnsupportedOperationException();
   }
 


[36/50] incubator-beam git commit: [BEAM-443] Update Beam batch examples to call waitUntilFinish()

Posted by th...@apache.org.
[BEAM-443] Update Beam batch examples to call waitUntilFinish()


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

Branch: refs/heads/apex-runner
Commit: e5afbe560b604ae7081e420af5b0d855508d53ad
Parents: eba099f
Author: Pei He <pe...@google.com>
Authored: Thu Oct 13 14:44:13 2016 -0700
Committer: Pei He <pe...@google.com>
Committed: Wed Oct 26 16:02:17 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/examples/DebuggingWordCount.java |  2 +-
 .../java/org/apache/beam/examples/MinimalWordCount.java   |  2 +-
 .../src/main/java/org/apache/beam/examples/WordCount.java |  2 +-
 .../java/org/apache/beam/examples/complete/TfIdf.java     |  2 +-
 .../beam/examples/complete/TopWikipediaSessions.java      |  2 +-
 .../apache/beam/examples/cookbook/BigQueryTornadoes.java  |  2 +-
 .../beam/examples/cookbook/CombinePerKeyExamples.java     |  2 +-
 .../org/apache/beam/examples/cookbook/DeDupExample.java   |  2 +-
 .../org/apache/beam/examples/cookbook/FilterExamples.java |  2 +-
 .../org/apache/beam/examples/cookbook/JoinExamples.java   |  2 +-
 .../apache/beam/examples/cookbook/MaxPerKeyExamples.java  |  2 +-
 .../test/java/org/apache/beam/examples/WordCountTest.java |  2 +-
 .../apache/beam/examples/complete/AutoCompleteTest.java   |  6 +++---
 .../java/org/apache/beam/examples/complete/TfIdfTest.java |  2 +-
 .../beam/examples/complete/TopWikipediaSessionsTest.java  |  2 +-
 .../apache/beam/examples/cookbook/DeDupExampleTest.java   |  4 ++--
 .../apache/beam/examples/cookbook/JoinExamplesTest.java   |  2 +-
 .../apache/beam/examples/cookbook/TriggerExampleTest.java |  2 +-
 .../org/apache/beam/examples/MinimalWordCountJava8.java   |  2 +-
 .../beam/examples/complete/game/HourlyTeamScore.java      |  2 +-
 .../org/apache/beam/examples/complete/game/UserScore.java |  2 +-
 .../apache/beam/examples/complete/game/GameStatsTest.java |  2 +-
 .../beam/examples/complete/game/HourlyTeamScoreTest.java  |  2 +-
 .../beam/examples/complete/game/LeaderBoardTest.java      | 10 +++++-----
 .../apache/beam/examples/complete/game/UserScoreTest.java |  6 +++---
 25 files changed, 34 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
index 90d77b3..1d2c83a 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
@@ -194,6 +194,6 @@ public class DebuggingWordCount {
         KV.of("stomach", 1L));
     PAssert.that(filteredWords).containsInAnyOrder(expectedResults);
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
index 14ffa18..6fc873e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -119,6 +119,6 @@ public class MinimalWordCount {
      .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
 
     // Run the pipeline.
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
index 1e03b34..e7eab6e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
@@ -197,6 +197,6 @@ public class WordCount {
      .apply(MapElements.via(new FormatAsTextFn()))
      .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
index d4107c9..c0ba1e9 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -417,6 +417,6 @@ public class TfIdf {
         .apply(new ComputeTfIdf())
         .apply(new WriteTfIdf(options.getOutput()));
 
-    pipeline.run();
+    pipeline.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
index 15923eb..d57cc3a 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -208,6 +208,6 @@ public class TopWikipediaSessions {
      .apply(new ComputeTopSessions(samplingThreshold))
      .apply("Write", TextIO.Write.withoutSharding().to(options.getOutput()));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
index 391ea90..a4c1a6b 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java
@@ -164,6 +164,6 @@ public class BigQueryTornadoes {
         .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
         .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
index 1f0abce..93eee15 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java
@@ -208,6 +208,6 @@ public class CombinePerKeyExamples {
         .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
         .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
index 92f5b93..0883815 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
@@ -91,6 +91,6 @@ public class DeDupExample {
      .apply(RemoveDuplicates.<String>create())
      .apply("DedupedShakespeare", TextIO.Write.to(options.getOutput()));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
index 0b2ae73..6e6452c 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
@@ -247,6 +247,6 @@ public class FilterExamples {
         .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
         .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
index d66e070..7cf0942 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java
@@ -170,7 +170,7 @@ public class JoinExamples {
     PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
     PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
     formattedResults.apply(TextIO.Write.to(options.getOutput()));
-    p.run();
+    p.run().waitUntilFinish();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
index eed4bbd..abc10f3 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java
@@ -157,6 +157,6 @@ public class MaxPerKeyExamples {
         .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
         .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
index 98c5b17..c8809de 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java
@@ -80,6 +80,6 @@ public class WordCountTest {
       .apply(MapElements.via(new FormatAsTextFn()));
 
     PAssert.that(output).containsInAnyOrder(COUNTS_ARRAY);
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
index b6751c5..5dbfa70 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java
@@ -99,7 +99,7 @@ public class AutoCompleteTest implements Serializable {
         KV.of("bl", parseList("blackberry:3", "blueberry:2")),
         KV.of("c", parseList("cherry:1")),
         KV.of("ch", parseList("cherry:1")));
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   @Test
@@ -117,7 +117,7 @@ public class AutoCompleteTest implements Serializable {
         KV.of("x", parseList("x:3", "xy:2")),
         KV.of("xy", parseList("xy:2", "xyz:1")),
         KV.of("xyz", parseList("xyz:1")));
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   @Test
@@ -153,7 +153,7 @@ public class AutoCompleteTest implements Serializable {
         // Window [2, 3)
         KV.of("x", parseList("xB:2")),
         KV.of("xB", parseList("xB:2")));
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   private static List<CompletionCandidate> parseList(String... entries) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
index c2d654e..1aee8f9 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java
@@ -61,6 +61,6 @@ public class TfIdfTest {
 
     PAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d"));
 
-    pipeline.run();
+    pipeline.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
index 42fb06a..154ea73 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java
@@ -56,6 +56,6 @@ public class TopWikipediaSessionsTest {
         "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)"
         + " : 1 : 1970-02-01T00:00:00.000Z"));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
index c725e4f..d29fc42 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java
@@ -59,7 +59,7 @@ public class DeDupExampleTest {
 
     PAssert.that(output)
         .containsInAnyOrder("k1", "k5", "k2", "k3");
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   @Test
@@ -77,6 +77,6 @@ public class DeDupExampleTest {
         input.apply(RemoveDuplicates.<String>create());
 
     PAssert.that(output).empty();
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
index 60f71a2..6c54aff 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java
@@ -108,6 +108,6 @@ public class JoinExamplesTest {
 
     PCollection<String> output = JoinExamples.joinEvents(input1, input2);
     PAssert.that(output).containsInAnyOrder(JOINED_EVENTS);
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
index 3848ca1..bdda22c 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java
@@ -123,7 +123,7 @@ public class TriggerExampleTest {
 
     PAssert.that(results)
         .containsInAnyOrder(canonicalFormat(OUT_ROW_1), canonicalFormat(OUT_ROW_2));
-    pipeline.run();
+    pipeline.run().waitUntilFinish();
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
index 24dd6f9..738b64d 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -67,6 +67,6 @@ public class MinimalWordCountJava8 {
      // CHANGE 3/3: The Google Cloud Storage path is required for outputting the results to.
      .apply(TextIO.Write.to("gs://YOUR_OUTPUT_BUCKET/AND_OUTPUT_PREFIX"));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index aefa3bc..3a8d2ad 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -191,7 +191,7 @@ public class HourlyTeamScore extends UserScore {
             configureWindowedTableWrite()));
 
 
-    pipeline.run();
+    pipeline.run().waitUntilFinish();
   }
   // [END DocInclude_HTSMain]
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index f70b79c..32c939f 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -236,7 +236,7 @@ public class UserScore {
                                                    configureBigQueryWrite()));
 
     // Run the batch pipeline.
-    pipeline.run();
+    pipeline.run().waitUntilFinish();
   }
   // [END DocInclude_USMain]
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
index 7cd03f3..51ca719 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java
@@ -69,7 +69,7 @@ public class GameStatsTest implements Serializable {
     // Check the set of spammers.
     PAssert.that(output).containsInAnyOrder(SPAMMERS);
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
index f9fefb6..645f123 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java
@@ -105,7 +105,7 @@ public class HourlyTeamScoreTest implements Serializable {
 
       PAssert.that(output).containsInAnyOrder(FILTERED_EVENTS);
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
index 9cba704..676dedb 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java
@@ -110,7 +110,7 @@ public class LeaderBoardTest implements Serializable {
         .inOnTimePane(new IntervalWindow(baseTime, TEAM_WINDOW_DURATION))
         .containsInAnyOrder(KV.of(blueTeam, 12), KV.of(redTeam, 4));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /**
@@ -160,7 +160,7 @@ public class LeaderBoardTest implements Serializable {
         .inOnTimePane(window)
         .containsInAnyOrder(KV.of(blueTeam, 10), KV.of(redTeam, 9));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /**
@@ -197,7 +197,7 @@ public class LeaderBoardTest implements Serializable {
         .inOnTimePane(window)
         .containsInAnyOrder(KV.of(redTeam, 14), KV.of(blueTeam, 13));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /**
@@ -258,7 +258,7 @@ public class LeaderBoardTest implements Serializable {
     // account in earlier panes
     PAssert.that(teamScores).inFinalPane(window).containsInAnyOrder(KV.of(redTeam, 27));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /**
@@ -346,7 +346,7 @@ public class LeaderBoardTest implements Serializable {
             KV.of(TestUser.BLUE_TWO.getUser(), 3),
             KV.of(TestUser.BLUE_TWO.getUser(), 8));
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   private TimestampedValue<GameActionInfo> event(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e5afbe56/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
index 7c86adf..39de333 100644
--- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
+++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java
@@ -110,7 +110,7 @@ public class UserScoreTest implements Serializable {
     // Check the user score sums.
     PAssert.that(output).containsInAnyOrder(USER_SUMS);
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /** Tests ExtractAndSumScore("team"). */
@@ -129,7 +129,7 @@ public class UserScoreTest implements Serializable {
     // Check the team score sums.
     PAssert.that(output).containsInAnyOrder(TEAM_SUMS);
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 
   /** Test that bad input data is dropped appropriately. */
@@ -149,6 +149,6 @@ public class UserScoreTest implements Serializable {
 
     PAssert.that(extract).empty();
 
-    p.run();
+    p.run().waitUntilFinish();
   }
 }


[35/50] incubator-beam git commit: [BEAM-813] support metadata in Avro sink

Posted by th...@apache.org.
[BEAM-813] support metadata in Avro sink


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

Branch: refs/heads/apex-runner
Commit: eba099f564dba3dfbba30ae3533496b9e14f57a7
Parents: 25102f7
Author: Neville Li <ne...@spotify.com>
Authored: Mon Oct 24 18:56:36 2016 -0400
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Oct 26 15:30:50 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/AvroIO.java     | 143 ++++++++++++++++---
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  29 ++++
 2 files changed, 154 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eba099f5/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index d912ff7..6deca7f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -21,11 +21,16 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.io.BaseEncoding;
 import java.io.IOException;
 import java.nio.channels.Channels;
 import java.nio.channels.WritableByteChannel;
+import java.util.Map;
 import java.util.regex.Pattern;
 import javax.annotation.Nullable;
+
 import org.apache.avro.Schema;
 import org.apache.avro.file.CodecFactory;
 import org.apache.avro.file.DataFileWriter;
@@ -39,6 +44,7 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
 import org.apache.beam.sdk.values.PBegin;
@@ -455,6 +461,15 @@ public class AvroIO {
     }
 
     /**
+     * Returns a {@link PTransform} that writes Avro file(s) with the specified metadata.
+     *
+     * <p>Supported value types are String, Long, and byte[].
+     */
+    public static Bound<GenericRecord> withMetadata(Map<String, Object> metadata) {
+      return new Bound<>(GenericRecord.class).withMetadata(metadata);
+    }
+
+    /**
      * A {@link PTransform} that writes a bounded {@link PCollection} to an Avro file (or
      * multiple Avro files matching a sharding pattern).
      *
@@ -464,6 +479,8 @@ public class AvroIO {
       private static final String DEFAULT_SHARD_TEMPLATE = ShardNameTemplate.INDEX_OF_MAX;
       private static final SerializableAvroCodecFactory DEFAULT_CODEC =
           new SerializableAvroCodecFactory(CodecFactory.deflateCodec(6));
+      // This should be a multiple of 4 to not get a partial encoded byte.
+      private static final int METADATA_BYTES_MAX_LENGTH = 40;
 
       /** The filename to write to. */
       @Nullable
@@ -486,6 +503,8 @@ public class AvroIO {
        * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html
        */
       final SerializableAvroCodecFactory codec;
+      /** Avro file metadata. */
+      final ImmutableMap<String, Object> metadata;
 
       Bound(Class<T> type) {
         this(
@@ -497,7 +516,8 @@ public class AvroIO {
             type,
             null,
             true,
-            DEFAULT_CODEC);
+            DEFAULT_CODEC,
+            ImmutableMap.<String, Object>of());
       }
 
       Bound(
@@ -509,7 +529,8 @@ public class AvroIO {
           Class<T> type,
           Schema schema,
           boolean validate,
-          SerializableAvroCodecFactory codec) {
+          SerializableAvroCodecFactory codec,
+          Map<String, Object> metadata) {
         super(name);
         this.filenamePrefix = filenamePrefix;
         this.filenameSuffix = filenameSuffix;
@@ -519,6 +540,18 @@ public class AvroIO {
         this.schema = schema;
         this.validate = validate;
         this.codec = codec;
+
+        Map<String, String> badKeys = Maps.newLinkedHashMap();
+        for (Map.Entry<String, Object> entry : metadata.entrySet()) {
+          Object v = entry.getValue();
+          if (!(v instanceof String || v instanceof Long || v instanceof byte[])) {
+            badKeys.put(entry.getKey(), v.getClass().getSimpleName());
+          }
+        }
+        checkArgument(
+            badKeys.isEmpty(),
+            "Metadata value type must be one of String, Long, or byte[]. Found {}", badKeys);
+        this.metadata = ImmutableMap.copyOf(metadata);
       }
 
       /**
@@ -541,7 +574,8 @@ public class AvroIO {
             type,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -563,7 +597,8 @@ public class AvroIO {
             type,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -591,7 +626,8 @@ public class AvroIO {
             type,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -612,7 +648,8 @@ public class AvroIO {
             type,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -634,7 +671,8 @@ public class AvroIO {
             type,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -656,7 +694,8 @@ public class AvroIO {
             type,
             ReflectData.get().getSchema(type),
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -676,7 +715,8 @@ public class AvroIO {
             GenericRecord.class,
             schema,
             validate,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -710,7 +750,8 @@ public class AvroIO {
             type,
             schema,
             false,
-            codec);
+            codec,
+            metadata);
       }
 
       /**
@@ -729,7 +770,28 @@ public class AvroIO {
             type,
             schema,
             validate,
-            new SerializableAvroCodecFactory(codec));
+            new SerializableAvroCodecFactory(codec),
+            metadata);
+      }
+
+      /**
+       * Returns a new {@link PTransform} that's like this one but
+       * that writes to Avro file(s) with the specified metadata.
+       *
+       * <p>Does not modify this object.
+       */
+      public Bound<T> withMetadata(Map<String, Object> metadata) {
+        return new Bound<>(
+            name,
+            filenamePrefix,
+            filenameSuffix,
+            numShards,
+            shardTemplate,
+            type,
+            schema,
+            validate,
+            codec,
+            metadata);
       }
 
       @Override
@@ -749,7 +811,8 @@ public class AvroIO {
                     filenameSuffix,
                     shardTemplate,
                     AvroCoder.of(type, schema),
-                    codec));
+                    codec,
+                    metadata));
         if (getNumShards() > 0) {
           write = write.withNumShards(getNumShards());
         }
@@ -779,6 +842,24 @@ public class AvroIO {
             .addIfNotDefault(DisplayData.item("codec", codec.toString())
                 .withLabel("Avro Compression Codec"),
                 DEFAULT_CODEC.toString());
+        builder.include("Metadata", new Metadata());
+      }
+
+      private class Metadata implements HasDisplayData {
+        @Override
+        public void populateDisplayData(DisplayData.Builder builder) {
+          for (Map.Entry<String, Object> entry : metadata.entrySet()) {
+            DisplayData.Type type = DisplayData.inferType(entry.getValue());
+            if (type != null) {
+              builder.add(DisplayData.item(entry.getKey(), type, entry.getValue()));
+            } else {
+              String base64 = BaseEncoding.base64().encode((byte[]) entry.getValue());
+              String repr = base64.length() <= METADATA_BYTES_MAX_LENGTH
+                  ? base64 : base64.substring(0, METADATA_BYTES_MAX_LENGTH) + "...";
+              builder.add(DisplayData.item(entry.getKey(), repr));
+            }
+          }
+        }
       }
 
       /**
@@ -824,6 +905,10 @@ public class AvroIO {
       public CodecFactory getCodec() {
         return codec.getCodec();
       }
+
+      public Map<String, Object> getMetadata() {
+        return metadata;
+      }
     }
 
     /** Disallow construction of utility class. */
@@ -853,6 +938,7 @@ public class AvroIO {
   static class AvroSink<T> extends FileBasedSink<T> {
     private final AvroCoder<T> coder;
     private final SerializableAvroCodecFactory codec;
+    private final ImmutableMap<String, Object> metadata;
 
     @VisibleForTesting
     AvroSink(
@@ -860,16 +946,17 @@ public class AvroIO {
         String extension,
         String fileNameTemplate,
         AvroCoder<T> coder,
-        SerializableAvroCodecFactory codec) {
+        SerializableAvroCodecFactory codec,
+        ImmutableMap<String, Object> metadata) {
       super(baseOutputFilename, extension, fileNameTemplate);
       this.coder = coder;
       this.codec = codec;
-
+      this.metadata = metadata;
     }
 
     @Override
     public FileBasedSink.FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options) {
-      return new AvroWriteOperation<>(this, coder, codec);
+      return new AvroWriteOperation<>(this, coder, codec, metadata);
     }
 
     /**
@@ -879,18 +966,21 @@ public class AvroIO {
     private static class AvroWriteOperation<T> extends FileBasedWriteOperation<T> {
       private final AvroCoder<T> coder;
       private final SerializableAvroCodecFactory codec;
+      private final ImmutableMap<String, Object> metadata;
 
       private AvroWriteOperation(AvroSink<T> sink,
                                  AvroCoder<T> coder,
-                                 SerializableAvroCodecFactory codec) {
+                                 SerializableAvroCodecFactory codec,
+                                 ImmutableMap<String, Object> metadata) {
         super(sink);
         this.coder = coder;
         this.codec = codec;
+        this.metadata = metadata;
       }
 
       @Override
       public FileBasedWriter<T> createWriter(PipelineOptions options) throws Exception {
-        return new AvroWriter<>(this, coder, codec);
+        return new AvroWriter<>(this, coder, codec, metadata);
       }
     }
 
@@ -902,20 +992,37 @@ public class AvroIO {
       private final AvroCoder<T> coder;
       private DataFileWriter<T> dataFileWriter;
       private SerializableAvroCodecFactory codec;
+      private final ImmutableMap<String, Object> metadata;
 
       public AvroWriter(FileBasedWriteOperation<T> writeOperation,
                         AvroCoder<T> coder,
-                        SerializableAvroCodecFactory codec) {
+                        SerializableAvroCodecFactory codec,
+                        ImmutableMap<String, Object> metadata) {
         super(writeOperation);
         this.mimeType = MimeTypes.BINARY;
         this.coder = coder;
         this.codec = codec;
+        this.metadata = metadata;
       }
 
       @SuppressWarnings("deprecation") // uses internal test functionality.
       @Override
       protected void prepareWrite(WritableByteChannel channel) throws Exception {
         dataFileWriter = new DataFileWriter<>(coder.createDatumWriter()).setCodec(codec.getCodec());
+        for (Map.Entry<String, Object> entry : metadata.entrySet()) {
+          Object v = entry.getValue();
+          if (v instanceof String) {
+            dataFileWriter.setMeta(entry.getKey(), (String) v);
+          } else if (v instanceof Long) {
+            dataFileWriter.setMeta(entry.getKey(), (Long) v);
+          } else if (v instanceof byte[]) {
+            dataFileWriter.setMeta(entry.getKey(), (byte[]) v);
+          } else {
+            throw new IllegalStateException(
+                "Metadata value type must be one of String, Long, or byte[]. Found "
+                    + v.getClass().getSimpleName());
+          }
+        }
         dataFileWriter.create(coder.getSchema(), Channels.newOutputStream(channel));
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/eba099f5/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
index 4825875..1a07177 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
@@ -21,6 +21,7 @@ import static org.apache.avro.file.DataFileConstants.SNAPPY_CODEC;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.hasItem;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -28,6 +29,7 @@ import static org.junit.Assert.assertTrue;
 
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
 import java.io.File;
 import java.io.FileInputStream;
@@ -315,6 +317,33 @@ public class AvroIOTest {
     assertEquals(CodecFactory.xzCodec(9).toString(), serdeWrite.getCodec().toString());
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  @Category(NeedsRunner.class)
+  public void testMetdata() throws Exception {
+    TestPipeline p = TestPipeline.create();
+    List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"),
+        new GenericClass(5, "bar"));
+    File outputFile = tmpFolder.newFile("output.avro");
+
+    p.apply(Create.of(values))
+        .apply(AvroIO.Write.to(outputFile.getAbsolutePath())
+            .withoutSharding()
+            .withSchema(GenericClass.class)
+            .withMetadata(ImmutableMap.<String, Object>of(
+                "stringKey", "stringValue",
+                "longKey", 100L,
+                "bytesKey", "bytesValue".getBytes())));
+    p.run();
+
+    DataFileStream dataFileStream = new DataFileStream(new FileInputStream(outputFile),
+        new GenericDatumReader());
+    assertEquals("stringValue", dataFileStream.getMetaString("stringKey"));
+    assertEquals(100L, dataFileStream.getMetaLong("longKey"));
+    assertArrayEquals("bytesValue".getBytes(), dataFileStream.getMeta("bytesKey"));
+  }
+
+
   @SuppressWarnings("deprecation") // using AvroCoder#createDatumReader for tests.
   private void runTestWrite(String[] expectedElements, int numShards) throws IOException {
     File baseOutputFile = new File(tmpFolder.getRoot(), "prefix");


[03/50] incubator-beam git commit: This closes #1158

Posted by th...@apache.org.
This closes #1158


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

Branch: refs/heads/apex-runner
Commit: 0ec956d3f91656acbf17eb09eca9c4aa2b317b82
Parents: f603d43 5ce3aac
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 13:04:17 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/core/GroupAlsoByWindowsDoFn.java     | 19 -------------------
 1 file changed, 19 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-beam git commit: Port some example utils from OldDoFn to DoFn

Posted by th...@apache.org.
Port some example utils from OldDoFn to DoFn


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

Branch: refs/heads/apex-runner
Commit: 3bec5e03fc21c33080bb31488a849aae0a7ce9ef
Parents: 48fdd06
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 13:28:06 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Oct 26 14:59:14 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/game/GameStats.java  | 52 +++++++++++---------
 .../examples/complete/game/HourlyTeamScore.java | 28 ++++++-----
 .../examples/complete/game/LeaderBoard.java     | 43 +++++++++-------
 .../beam/examples/complete/game/UserScore.java  | 14 ++++--
 .../complete/game/utils/WriteToBigQuery.java    | 49 ++++++++++--------
 .../game/utils/WriteWindowedToBigQuery.java     | 14 +++---
 6 files changed, 114 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
index ba52e12..5ebf892 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
@@ -193,21 +193,26 @@ public class GameStats extends LeaderBoard {
       configureWindowedWrite() {
     Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
         new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> {
-            IntervalWindow w = (IntervalWindow) c.window();
-            return fmt.print(w.start());
-          }));
-    tableConfigure.put("processing_time",
+    tableConfigure.put(
+        "team",
         new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
+            "STRING", (c, w) -> c.element().getKey()));
+    tableConfigure.put(
+        "total_score",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "INTEGER", (c, w) -> c.element().getValue()));
+    tableConfigure.put(
+        "window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING",
+            (c, w) -> {
+              IntervalWindow window = (IntervalWindow) w;
+              return fmt.print(window.start());
+            }));
+    tableConfigure.put(
+        "processing_time",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", (c, w) -> fmt.print(Instant.now())));
     return tableConfigure;
   }
 
@@ -220,14 +225,17 @@ public class GameStats extends LeaderBoard {
 
     Map<String, WriteWindowedToBigQuery.FieldInfo<Double>> tableConfigure =
         new HashMap<String, WriteWindowedToBigQuery.FieldInfo<Double>>();
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<Double>("STRING",
-          c -> {
-            IntervalWindow w = (IntervalWindow) c.window();
-            return fmt.print(w.start());
-          }));
-    tableConfigure.put("mean_duration",
-        new WriteWindowedToBigQuery.FieldInfo<Double>("FLOAT", c -> c.element()));
+    tableConfigure.put(
+        "window_start",
+        new WriteWindowedToBigQuery.FieldInfo<Double>(
+            "STRING",
+            (c, w) -> {
+              IntervalWindow window = (IntervalWindow) w;
+              return fmt.print(window.start());
+            }));
+    tableConfigure.put(
+        "mean_duration",
+        new WriteWindowedToBigQuery.FieldInfo<Double>("FLOAT", (c, w) -> c.element()));
     return tableConfigure;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index 1f92906..aefa3bc 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -124,18 +124,22 @@ public class HourlyTeamScore extends UserScore {
       configureWindowedTableWrite() {
     Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfig =
         new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfig.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfig.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfig.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> {
-            IntervalWindow w = (IntervalWindow) c.window();
-            return fmt.print(w.start());
-          }));
+    tableConfig.put(
+        "team",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", (c, w) -> c.element().getKey()));
+    tableConfig.put(
+        "total_score",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "INTEGER", (c, w) -> c.element().getValue()));
+    tableConfig.put(
+        "window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING",
+            (c, w) -> {
+              IntervalWindow window = (IntervalWindow) w;
+              return fmt.print(window.start());
+            }));
     return tableConfig;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index 1eac26c..d5e3345 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -137,24 +137,30 @@ public class LeaderBoard extends HourlyTeamScore {
 
     Map<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
         new HashMap<String, WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("team",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-            c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER",
-            c -> c.element().getValue()));
-    tableConfigure.put("window_start",
-        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
-          c -> {
-            IntervalWindow w = (IntervalWindow) c.window();
-            return fmt.print(w.start());
-          }));
-    tableConfigure.put("processing_time",
+    tableConfigure.put(
+        "team",
         new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
-    tableConfigure.put("timing",
+            "STRING", (c, w) -> c.element().getKey()));
+    tableConfigure.put(
+        "total_score",
         new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> c.pane().getTiming().toString()));
+            "INTEGER", (c, w) -> c.element().getValue()));
+    tableConfigure.put(
+        "window_start",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING",
+            (c, w) -> {
+              IntervalWindow window = (IntervalWindow) w;
+              return fmt.print(window.start());
+            }));
+    tableConfigure.put(
+        "processing_time",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", (c, w) -> fmt.print(Instant.now())));
+    tableConfigure.put(
+        "timing",
+        new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", (c, w) -> c.pane().getTiming().toString()));
     return tableConfigure;
   }
 
@@ -167,9 +173,10 @@ public class LeaderBoard extends HourlyTeamScore {
 
     Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
         configureBigQueryWrite();
-    tableConfigure.put("processing_time",
+    tableConfigure.put(
+        "processing_time",
         new WriteToBigQuery.FieldInfo<KV<String, Integer>>(
-            "STRING", c -> fmt.print(Instant.now())));
+            "STRING", (c, w) -> fmt.print(Instant.now())));
     return tableConfigure;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index db89702..f70b79c 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -202,13 +202,17 @@ public class UserScore {
    * is passed to the {@link WriteToBigQuery} constructor to write user score sums.
    */
   protected static Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>
-    configureBigQueryWrite() {
+      configureBigQueryWrite() {
     Map<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>> tableConfigure =
         new HashMap<String, WriteToBigQuery.FieldInfo<KV<String, Integer>>>();
-    tableConfigure.put("user",
-        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("STRING", c -> c.element().getKey()));
-    tableConfigure.put("total_score",
-        new WriteToBigQuery.FieldInfo<KV<String, Integer>>("INTEGER", c -> c.element().getValue()));
+    tableConfigure.put(
+        "user",
+        new WriteToBigQuery.FieldInfo<KV<String, Integer>>(
+            "STRING", (c, w) -> c.element().getKey()));
+    tableConfigure.put(
+        "total_score",
+        new WriteToBigQuery.FieldInfo<KV<String, Integer>>(
+            "INTEGER", (c, w) -> c.element().getValue()));
     return tableConfigure;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
index 40c4286..89fc271 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
@@ -32,10 +32,10 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
 
@@ -44,30 +44,38 @@ import org.apache.beam.sdk.values.PDone;
  * the field names and types, as well as lambda functions that describe how to generate their
  * values.
  */
-public class WriteToBigQuery<T>
-    extends PTransform<PCollection<T>, PDone> {
+public class WriteToBigQuery<InputT>
+    extends PTransform<PCollection<InputT>, PDone> {
 
   protected String tableName;
-  protected Map<String, FieldInfo<T>> fieldInfo;
+  protected Map<String, FieldInfo<InputT>> fieldInfo;
 
   public WriteToBigQuery() {
   }
 
   public WriteToBigQuery(String tableName,
-      Map<String, FieldInfo<T>> fieldInfo) {
+      Map<String, FieldInfo<InputT>> fieldInfo) {
     this.tableName = tableName;
     this.fieldInfo = fieldInfo;
   }
 
+  /**
+   * A {@link Serializable} function from a {@link DoFn.ProcessContext}
+   * and {@link BoundedWindow} to the value for that field.
+   */
+  public interface FieldFn<InputT> extends Serializable {
+    Object apply(DoFn<InputT, TableRow>.ProcessContext context, BoundedWindow window);
+  }
+
   /** Define a class to hold information about output table field definitions. */
-  public static class FieldInfo<T> implements Serializable {
+  public static class FieldInfo<InputT> implements Serializable {
     // The BigQuery 'type' of the field
     private String fieldType;
     // A lambda function to generate the field value
-    private SerializableFunction<OldDoFn<T, TableRow>.ProcessContext, Object> fieldFn;
+    private FieldFn<InputT> fieldFn;
 
     public FieldInfo(String fieldType,
-        SerializableFunction<OldDoFn<T, TableRow>.ProcessContext, Object> fieldFn) {
+        FieldFn<InputT> fieldFn) {
       this.fieldType = fieldType;
       this.fieldFn = fieldFn;
     }
@@ -76,23 +84,22 @@ public class WriteToBigQuery<T>
       return this.fieldType;
     }
 
-    SerializableFunction<OldDoFn<T, TableRow>.ProcessContext, Object> getFieldFn() {
+    FieldFn<InputT> getFieldFn() {
       return this.fieldFn;
     }
   }
   /** Convert each key/score pair into a BigQuery TableRow as specified by fieldFn. */
-  protected class BuildRowFn extends OldDoFn<T, TableRow> {
+  protected class BuildRowFn extends DoFn<InputT, TableRow> {
 
-    @Override
-    public void processElement(ProcessContext c) {
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
 
       TableRow row = new TableRow();
-      for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
+      for (Map.Entry<String, FieldInfo<InputT>> entry : fieldInfo.entrySet()) {
           String key = entry.getKey();
-          FieldInfo<T> fcnInfo = entry.getValue();
-          SerializableFunction<OldDoFn<T, TableRow>.ProcessContext, Object> fcn =
-            fcnInfo.getFieldFn();
-          row.set(key, fcn.apply(c));
+          FieldInfo<InputT> fcnInfo = entry.getValue();
+          FieldFn<InputT> fcn = fcnInfo.getFieldFn();
+          row.set(key, fcn.apply(c, window));
         }
       c.output(row);
     }
@@ -101,9 +108,9 @@ public class WriteToBigQuery<T>
   /** Build the output table schema. */
   protected TableSchema getSchema() {
     List<TableFieldSchema> fields = new ArrayList<>();
-    for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
+    for (Map.Entry<String, FieldInfo<InputT>> entry : fieldInfo.entrySet()) {
       String key = entry.getKey();
-      FieldInfo<T> fcnInfo = entry.getValue();
+      FieldInfo<InputT> fcnInfo = entry.getValue();
       String bqType = fcnInfo.getFieldType();
       fields.add(new TableFieldSchema().setName(key).setType(bqType));
     }
@@ -111,7 +118,7 @@ public class WriteToBigQuery<T>
   }
 
   @Override
-  public PDone apply(PCollection<T> teamAndScore) {
+  public PDone apply(PCollection<InputT> teamAndScore) {
     return teamAndScore
       .apply("ConvertToRow", ParDo.of(new BuildRowFn()))
       .apply(BigQueryIO.Write

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bec5e03/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
index 09f3b6c..4f2e719 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java
@@ -22,10 +22,10 @@ import java.util.Map;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
 
@@ -43,19 +43,17 @@ public class WriteWindowedToBigQuery<T>
   }
 
   /** Convert each key/score pair into a BigQuery TableRow. */
-  protected class BuildRowFn extends OldDoFn<T, TableRow>
+  protected class BuildRowFn extends DoFn<T, TableRow>
       implements RequiresWindowAccess {
 
-    @Override
-    public void processElement(ProcessContext c) {
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
 
       TableRow row = new TableRow();
       for (Map.Entry<String, FieldInfo<T>> entry : fieldInfo.entrySet()) {
           String key = entry.getKey();
           FieldInfo<T> fcnInfo = entry.getValue();
-          SerializableFunction<OldDoFn<T, TableRow>.ProcessContext, Object> fcn =
-            fcnInfo.getFieldFn();
-          row.set(key, fcn.apply(c));
+          row.set(key, fcnInfo.getFieldFn().apply(c, window));
         }
       c.output(row);
     }


[42/50] incubator-beam git commit: This closes #1196

Posted by th...@apache.org.
This closes #1196


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

Branch: refs/heads/apex-runner
Commit: 4cb1d10dfb49c7a0622bfe8ec7b5524295ef6da1
Parents: 064f18a 34a236c
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 27 12:50:08 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 12:50:08 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/GroupAlsoByWindowEvaluatorFactory.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[18/50] incubator-beam git commit: Set up Maven settings globally for Travis CI builds

Posted by th...@apache.org.
Set up Maven settings globally for Travis CI builds


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

Branch: refs/heads/apex-runner
Commit: 3af2c1e7336d4856cc80827d39f244eb4b1ceadc
Parents: 3c71e4d
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 19:34:04 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:13:59 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3af2c1e7/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index b11f61b..5b271d1 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -32,16 +32,16 @@ matrix:
   include:
     # On OSX, run with default JDK only.
     - os: osx
-      env: MAVEN_OVERRIDE=""
+      env: MAVEN_OVERRIDE="--settings=.travis/settings.xml"
     # On Linux, run with specific JDKs only.
     - os: linux
-      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
     - os: linux
-      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
     - os: linux
-      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m' -Peclipse-jdt"
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m' -Peclipse-jdt"
     - os: linux
-      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
 
 before_install:
   - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
@@ -56,7 +56,7 @@ install:
   - rm -rf "$HOME/.m2/repository/org/apache/beam"
 
 script:
-  - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify
+  - travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify
   - travis_retry .travis/test_wordcount.sh
 
 cache:


[44/50] incubator-beam git commit: Use a NavigableSet Instead of a PriorityQueue in WatermarkManager

Posted by th...@apache.org.
Use a NavigableSet Instead of a PriorityQueue in WatermarkManager

This removes an O(n) call to remove, replacing it with an O(log(n))
call. This significantly improves scaling behavior of the DirectRunner


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

Branch: refs/heads/apex-runner
Commit: aeb3b3c4bfad3e02090b1f7f62695759e17f0189
Parents: 4cb1d10
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 26 16:35:31 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 12:51:32 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/WatermarkManager.java      | 17 ++++++++---------
 1 file changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/aeb3b3c4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index f8cbc51..31b8091 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -39,7 +39,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Objects;
-import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -388,7 +387,7 @@ public class WatermarkManager {
     private final Map<StructuralKey<?>, NavigableSet<TimerData>> processingTimers;
     private final Map<StructuralKey<?>, NavigableSet<TimerData>> synchronizedProcessingTimers;
 
-    private final PriorityQueue<TimerData> pendingTimers;
+    private final NavigableSet<TimerData> pendingTimers;
 
     private AtomicReference<Instant> earliestHold;
 
@@ -397,7 +396,7 @@ public class WatermarkManager {
       this.pendingBundles = new HashSet<>();
       this.processingTimers = new HashMap<>();
       this.synchronizedProcessingTimers = new HashMap<>();
-      this.pendingTimers = new PriorityQueue<>();
+      this.pendingTimers = new TreeSet<>();
       Instant initialHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
       for (Watermark wm : inputWms) {
         initialHold = INSTANT_ORDERING.min(initialHold, wm.get());
@@ -466,7 +465,7 @@ public class WatermarkManager {
         }
       }
       if (!pendingTimers.isEmpty()) {
-        earliest = INSTANT_ORDERING.min(pendingTimers.peek().getTimestamp(), earliest);
+        earliest = INSTANT_ORDERING.min(pendingTimers.first().getTimestamp(), earliest);
       }
       return earliest;
     }
@@ -630,7 +629,7 @@ public class WatermarkManager {
   private static final Ordering<Instant> INSTANT_ORDERING = Ordering.natural();
 
   /**
-   * For each (Object, PriorityQueue) pair in the provided map, remove each Timer that is before the
+   * For each (Object, NavigableSet) pair in the provided map, remove each Timer that is before the
    * latestTime argument and put in in the result with the same key, then remove all of the keys
    * which have no more pending timers.
    *
@@ -1003,11 +1002,11 @@ public class WatermarkManager {
 
   private static class PerKeyHolds {
     private final Map<Object, KeyedHold> keyedHolds;
-    private final PriorityQueue<KeyedHold> allHolds;
+    private final NavigableSet<KeyedHold> allHolds;
 
     private PerKeyHolds() {
       this.keyedHolds = new HashMap<>();
-      this.allHolds = new PriorityQueue<>();
+      this.allHolds = new TreeSet<>();
     }
 
     /**
@@ -1015,7 +1014,7 @@ public class WatermarkManager {
      * there are no holds within this {@link PerKeyHolds}.
      */
     public Instant getMinHold() {
-      return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.peek().getTimestamp();
+      return allHolds.isEmpty() ? THE_END_OF_TIME.get() : allHolds.first().getTimestamp();
     }
 
     /**
@@ -1026,7 +1025,7 @@ public class WatermarkManager {
       removeHold(key);
       KeyedHold newKeyedHold = KeyedHold.of(key, newHold);
       keyedHolds.put(key, newKeyedHold);
-      allHolds.offer(newKeyedHold);
+      allHolds.add(newKeyedHold);
     }
 
     /**


[13/50] incubator-beam git commit: [BEAM-799] Support GroupByKey directly.

Posted by th...@apache.org.
[BEAM-799] Support GroupByKey directly.

Remove runner override for GroupByKey.

Avoid NPE if no sideInputs are available in reader.

Handle CombineFn with or without context.


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

Branch: refs/heads/apex-runner
Commit: a54ded373fa7f6508fb46eea1a1d6f9bc405114b
Parents: f2fe1ae
Author: Sela <an...@paypal.com>
Authored: Sat Oct 22 14:51:50 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 10:00:45 2016 +0300

----------------------------------------------------------------------
 .../apache/beam/runners/spark/SparkRunner.java  | 19 ------
 .../translation/GroupCombineFunctions.java      | 66 +++++++++-----------
 .../spark/translation/TransformTranslator.java  | 43 +++----------
 .../streaming/StreamingTransformTranslator.java | 65 +++++--------------
 .../spark/util/SparkSideInputReader.java        |  2 +-
 5 files changed, 55 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a54ded37/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
index b17c38c..45c7f55 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
@@ -20,7 +20,6 @@ package org.apache.beam.runners.spark;
 
 import java.util.Collection;
 import java.util.List;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
 import org.apache.beam.runners.spark.translation.EvaluationContext;
 import org.apache.beam.runners.spark.translation.SparkContextFactory;
 import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
@@ -36,7 +35,6 @@ import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.values.PBegin;
@@ -115,23 +113,6 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
   }
 
   /**
-   * Overrides for this runner.
-   */
-  @SuppressWarnings("rawtypes")
-  @Override
-  public <OutputT extends POutput, InputT extends PInput> OutputT apply(
-      PTransform<InputT, OutputT> transform, InputT input) {
-
-    if (transform instanceof GroupByKey) {
-      return (OutputT) ((PCollection) input).apply(
-          new GroupByKeyViaGroupByKeyOnly((GroupByKey) transform));
-    } else {
-      return super.apply(transform, input);
-    }
-  }
-
-
-  /**
    * No parameter constructor defaults to running this pipeline in Spark's local mode, in a single
    * thread.
    */

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a54ded37/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java
index de02b26..e2a0f87 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java
@@ -20,11 +20,9 @@ package org.apache.beam.runners.spark.translation;
 
 
 import com.google.common.collect.Lists;
-
 import java.util.Collections;
 import java.util.Map;
 import org.apache.beam.runners.core.GroupAlsoByWindowsViaOutputBufferDoFn;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
 import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.spark.aggregators.NamedAggregators;
 import org.apache.beam.runners.spark.coders.CoderHelpers;
@@ -38,6 +36,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.ReifyTimestampAndWindowsDoFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
@@ -60,54 +59,45 @@ import scala.Tuple2;
 public class GroupCombineFunctions {
 
   /**
-   * Apply {@link GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly} to a Spark RDD.
+   * Apply {@link org.apache.beam.sdk.transforms.GroupByKey} to a Spark RDD.
    */
-  public static <K, V> JavaRDD<WindowedValue<KV<K, Iterable<V>>>> groupByKeyOnly(
-      JavaRDD<WindowedValue<KV<K, V>>> rdd, KvCoder<K, V> coder) {
+  public static <K, V,  W extends BoundedWindow> JavaRDD<WindowedValue<KV<K,
+      Iterable<V>>>> groupByKey(JavaRDD<WindowedValue<KV<K, V>>> rdd,
+                                Accumulator<NamedAggregators> accum,
+                                KvCoder<K, V> coder,
+                                SparkRuntimeContext runtimeContext,
+                                WindowingStrategy<?, W> windowingStrategy) {
+    //--- coders.
     final Coder<K> keyCoder = coder.getKeyCoder();
     final Coder<V> valueCoder = coder.getValueCoder();
+    final WindowedValue.WindowedValueCoder<V> wvCoder = WindowedValue.FullWindowedValueCoder.of(
+        valueCoder, windowingStrategy.getWindowFn().windowCoder());
+
+    //--- groupByKey.
     // Use coders to convert objects in the PCollection to byte arrays, so they
     // can be transferred over the network for the shuffle.
-    return rdd.map(WindowingHelpers.<KV<K, V>>unwindowFunction())
-        .mapToPair(TranslationUtils.<K, V>toPairFunction())
-        .mapToPair(CoderHelpers.toByteFunction(keyCoder, valueCoder))
-        .groupByKey()
-        .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, valueCoder))
-        // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
-        .map(TranslationUtils.<K, Iterable<V>>fromPairFunction())
-        .map(WindowingHelpers.<KV<K, Iterable<V>>>windowFunction());
-  }
-
-  /**
-   * Apply {@link GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow} to a Spark RDD.
-   */
-  public static <K, V, W extends BoundedWindow> JavaRDD<WindowedValue<KV<K, Iterable<V>>>>
-  groupAlsoByWindow(JavaRDD<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>> rdd,
-                    GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow<K, V> transform,
-                    SparkRuntimeContext runtimeContext,
-                    Accumulator<NamedAggregators> accum,
-                    KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder) {
-    //--- coders.
-    Coder<Iterable<WindowedValue<V>>> inputValueCoder = inputKvCoder.getValueCoder();
-    IterableCoder<WindowedValue<V>> inputIterableValueCoder =
-        (IterableCoder<WindowedValue<V>>) inputValueCoder;
-    Coder<WindowedValue<V>> inputIterableElementCoder = inputIterableValueCoder.getElemCoder();
-    WindowedValue.WindowedValueCoder<V> inputIterableWindowedValueCoder =
-        (WindowedValue.WindowedValueCoder<V>) inputIterableElementCoder;
-    Coder<V> inputIterableElementValueCoder = inputIterableWindowedValueCoder.getValueCoder();
+    JavaRDD<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>> groupedByKey =
+        rdd.mapPartitions(new DoFnFunction<KV<K, V>, KV<K, WindowedValue<V>>>(null,
+                new ReifyTimestampAndWindowsDoFn<K, V>(), runtimeContext, null, null))
+            .map(WindowingHelpers.<KV<K, WindowedValue<V>>>unwindowFunction())
+            .mapToPair(TranslationUtils.<K, WindowedValue<V>>toPairFunction())
+            .mapToPair(CoderHelpers.toByteFunction(keyCoder, wvCoder))
+            .groupByKey()
+            .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, wvCoder))
+            // empty windows are OK here, see GroupByKey#evaluateHelper in the SDK
+            .map(TranslationUtils.<K, Iterable<WindowedValue<V>>>fromPairFunction())
+            .map(WindowingHelpers.<KV<K, Iterable<WindowedValue<V>>>>windowFunction());
 
-    @SuppressWarnings("unchecked")
-    WindowingStrategy<?, W> windowingStrategy =
-        (WindowingStrategy<?, W>) transform.getWindowingStrategy();
+    //--- now group also by window.
     @SuppressWarnings("unchecked")
     WindowFn<Object, W> windowFn = (WindowFn<Object, W>) windowingStrategy.getWindowFn();
-
     // GroupAlsoByWindow current uses a dummy in-memory StateInternals
     OldDoFn<KV<K, Iterable<WindowedValue<V>>>, KV<K, Iterable<V>>> gabwDoFn =
         new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
             windowingStrategy, new TranslationUtils.InMemoryStateInternalsFactory<K>(),
-                SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
-    return rdd.mapPartitions(new DoFnFunction<>(accum, gabwDoFn, runtimeContext, null, windowFn));
+                SystemReduceFn.<K, V, W>buffering(valueCoder));
+    return groupedByKey.mapPartitions(new DoFnFunction<>(accum, gabwDoFn, runtimeContext, null,
+        windowFn));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a54ded37/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
index b55e3b2..2e682c4 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
@@ -32,8 +32,6 @@ import org.apache.avro.mapred.AvroKey;
 import org.apache.avro.mapreduce.AvroJob;
 import org.apache.avro.mapreduce.AvroKeyInputFormat;
 import org.apache.beam.runners.core.AssignWindowsDoFn;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
 import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
 import org.apache.beam.runners.spark.aggregators.NamedAggregators;
 import org.apache.beam.runners.spark.io.SourceRDD;
@@ -51,6 +49,7 @@ import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.CombineWithContext;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -58,7 +57,6 @@ import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AppliedCombineFn;
 import org.apache.beam.sdk.util.CombineFnUtil;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
@@ -112,10 +110,10 @@ public final class TransformTranslator {
     };
   }
 
-  private static <K, V> TransformEvaluator<GroupByKeyOnly<K, V>> gbko() {
-    return new TransformEvaluator<GroupByKeyOnly<K, V>>() {
+  private static <K, V> TransformEvaluator<GroupByKey<K, V>> groupByKey() {
+    return new TransformEvaluator<GroupByKey<K, V>>() {
       @Override
-      public void evaluate(GroupByKeyOnly<K, V> transform, EvaluationContext context) {
+      public void evaluate(GroupByKey<K, V> transform, EvaluationContext context) {
         @SuppressWarnings("unchecked")
         JavaRDD<WindowedValue<KV<K, V>>> inRDD =
             (JavaRDD<WindowedValue<KV<K, V>>>) context.getInputRDD(transform);
@@ -123,30 +121,11 @@ public final class TransformTranslator {
         @SuppressWarnings("unchecked")
         final KvCoder<K, V> coder = (KvCoder<K, V>) context.getInput(transform).getCoder();
 
-        context.setOutputRDD(transform, GroupCombineFunctions.groupByKeyOnly(inRDD, coder));
-      }
-    };
-  }
-
-  private static <K, V, W extends BoundedWindow>
-      TransformEvaluator<GroupAlsoByWindow<K, V>> gabw() {
-    return new TransformEvaluator<GroupAlsoByWindow<K, V>>() {
-      @Override
-      public void evaluate(GroupAlsoByWindow<K, V> transform, EvaluationContext context) {
-        @SuppressWarnings("unchecked")
-        JavaRDD<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>> inRDD =
-            (JavaRDD<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>>)
-                context.getInputRDD(transform);
-
-        @SuppressWarnings("unchecked")
-        final KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
-            (KvCoder<K, Iterable<WindowedValue<V>>>) context.getInput(transform).getCoder();
-
         final Accumulator<NamedAggregators> accum =
-            AccumulatorSingleton.getInstance(context.getSparkContext());
+                AccumulatorSingleton.getInstance(context.getSparkContext());
 
-        context.setOutputRDD(transform, GroupCombineFunctions.groupAlsoByWindow(inRDD, transform,
-            context.getRuntimeContext(), accum, inputKvCoder));
+        context.setOutputRDD(transform, GroupCombineFunctions.groupByKey(inRDD, accum, coder,
+            context.getRuntimeContext(), context.getInput(transform).getWindowingStrategy()));
       }
     };
   }
@@ -161,13 +140,10 @@ public final class TransformTranslator {
         PCollection<? extends KV<K, ? extends Iterable<InputT>>> input =
             context.getInput(transform);
         WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-        AppliedCombineFn<? super K, ? super InputT, ?, OutputT> appliedFn = transform
-            .getAppliedFn(context.getPipeline().getCoderRegistry(), input.getCoder(),
-                windowingStrategy);
         @SuppressWarnings("unchecked")
         CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT> fn =
             (CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT>)
-                CombineFnUtil.toFnWithContext(appliedFn.getFn());
+                CombineFnUtil.toFnWithContext(transform.getFn());
 
         @SuppressWarnings("unchecked")
         JavaRDDLike<WindowedValue<KV<K, Iterable<InputT>>>, ?> inRDD =
@@ -592,8 +568,7 @@ public final class TransformTranslator {
     EVALUATORS.put(HadoopIO.Write.Bound.class, writeHadoop());
     EVALUATORS.put(ParDo.Bound.class, parDo());
     EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
-    EVALUATORS.put(GroupByKeyOnly.class, gbko());
-    EVALUATORS.put(GroupAlsoByWindow.class, gabw());
+    EVALUATORS.put(GroupByKey.class, groupByKey());
     EVALUATORS.put(Combine.GroupedValues.class, combineGrouped());
     EVALUATORS.put(Combine.Globally.class, combineGlobally());
     EVALUATORS.put(Combine.PerKey.class, combinePerKey());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a54ded37/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
index 9f2d764..1af5e07 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -26,9 +26,6 @@ import java.util.Map;
 import java.util.Set;
 import kafka.serializer.Decoder;
 import org.apache.beam.runners.core.AssignWindowsDoFn;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
 import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
 import org.apache.beam.runners.spark.aggregators.NamedAggregators;
 import org.apache.beam.runners.spark.io.ConsoleIO;
@@ -50,6 +47,7 @@ import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.CombineWithContext;
 import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -58,7 +56,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.AppliedCombineFn;
 import org.apache.beam.sdk.util.CombineFnUtil;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
@@ -84,6 +81,7 @@ import org.apache.spark.streaming.kafka.KafkaUtils;
 import scala.Tuple2;
 
 
+
 /**
  * Supports translation between a Beam transform, and Spark's operations on DStreams.
  */
@@ -231,10 +229,10 @@ public final class StreamingTransformTranslator {
     };
   }
 
-  private static <K, V> TransformEvaluator<GroupByKeyOnly<K, V>> gbko() {
-    return new TransformEvaluator<GroupByKeyOnly<K, V>>() {
+  private static <K, V> TransformEvaluator<GroupByKey<K, V>> groupByKey() {
+    return new TransformEvaluator<GroupByKey<K, V>>() {
       @Override
-      public void evaluate(GroupByKeyOnly<K, V> transform, EvaluationContext context) {
+      public void evaluate(GroupByKey<K, V> transform, EvaluationContext context) {
         StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
 
         @SuppressWarnings("unchecked")
@@ -244,13 +242,20 @@ public final class StreamingTransformTranslator {
         @SuppressWarnings("unchecked")
         final KvCoder<K, V> coder = (KvCoder<K, V>) sec.getInput(transform).getCoder();
 
+        final SparkRuntimeContext runtimeContext = sec.getRuntimeContext();
+        final WindowingStrategy<?, ?> windowingStrategy =
+            sec.getInput(transform).getWindowingStrategy();
+
         JavaDStream<WindowedValue<KV<K, Iterable<V>>>> outStream =
             dStream.transform(new Function<JavaRDD<WindowedValue<KV<K, V>>>,
                 JavaRDD<WindowedValue<KV<K, Iterable<V>>>>>() {
           @Override
           public JavaRDD<WindowedValue<KV<K, Iterable<V>>>> call(
               JavaRDD<WindowedValue<KV<K, V>>> rdd) throws Exception {
-            return GroupCombineFunctions.groupByKeyOnly(rdd, coder);
+            final Accumulator<NamedAggregators> accum =
+                AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context()));
+            return GroupCombineFunctions.groupByKey(rdd, accum, coder, runtimeContext,
+                windowingStrategy);
           }
         });
         sec.setStream(transform, outStream);
@@ -258,39 +263,6 @@ public final class StreamingTransformTranslator {
     };
   }
 
-  private static <K, V, W extends BoundedWindow>
-      TransformEvaluator<GroupAlsoByWindow<K, V>> gabw() {
-    return new TransformEvaluator<GroupAlsoByWindow<K, V>>() {
-      @Override
-      public void evaluate(final GroupAlsoByWindow<K, V> transform, EvaluationContext context) {
-        final StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        final SparkRuntimeContext runtimeContext = sec.getRuntimeContext();
-        @SuppressWarnings("unchecked")
-        JavaDStream<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>> dStream =
-            (JavaDStream<WindowedValue<KV<K, Iterable<WindowedValue<V>>>>>)
-                sec.getStream(transform);
-
-        @SuppressWarnings("unchecked")
-        final KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
-            (KvCoder<K, Iterable<WindowedValue<V>>>) sec.getInput(transform).getCoder();
-
-        JavaDStream<WindowedValue<KV<K, Iterable<V>>>> outStream =
-            dStream.transform(new Function<JavaRDD<WindowedValue<KV<K,
-                Iterable<WindowedValue<V>>>>>, JavaRDD<WindowedValue<KV<K, Iterable<V>>>>>() {
-              @Override
-              public JavaRDD<WindowedValue<KV<K, Iterable<V>>>> call(JavaRDD<WindowedValue<KV<K,
-                  Iterable<WindowedValue<V>>>>> rdd) throws Exception {
-                final Accumulator<NamedAggregators> accum =
-                    AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context()));
-                return GroupCombineFunctions.groupAlsoByWindow(rdd, transform, runtimeContext,
-                    accum, inputKvCoder);
-              }
-            });
-        sec.setStream(transform, outStream);
-      }
-    };
-  }
-
   private static <K, InputT, OutputT> TransformEvaluator<Combine.GroupedValues<K, InputT, OutputT>>
   combineGrouped() {
     return new TransformEvaluator<Combine.GroupedValues<K, InputT, OutputT>>() {
@@ -302,12 +274,10 @@ public final class StreamingTransformTranslator {
         PCollection<? extends KV<K, ? extends Iterable<InputT>>> input =
             sec.getInput(transform);
         WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-        AppliedCombineFn<? super K, ? super InputT, ?, OutputT> appliedFn = transform
-            .getAppliedFn(context.getPipeline().getCoderRegistry(), input.getCoder(),
-                windowingStrategy);
         @SuppressWarnings("unchecked")
-        CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT> fn =
-            (CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT>) appliedFn.getFn();
+        final CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT> fn =
+            (CombineWithContext.KeyedCombineFnWithContext<K, InputT, ?, OutputT>)
+                CombineFnUtil.toFnWithContext(transform.getFn());
 
         @SuppressWarnings("unchecked")
         JavaDStream<WindowedValue<KV<K, Iterable<InputT>>>> dStream =
@@ -485,8 +455,7 @@ public final class StreamingTransformTranslator {
       .newHashMap();
 
   static {
-    EVALUATORS.put(GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly.class, gbko());
-    EVALUATORS.put(GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow.class, gabw());
+    EVALUATORS.put(GroupByKey.class, groupByKey());
     EVALUATORS.put(Combine.GroupedValues.class, combineGrouped());
     EVALUATORS.put(Combine.Globally.class, combineGlobally());
     EVALUATORS.put(Combine.PerKey.class, combinePerKey());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a54ded37/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
index 96c286a..0a804ae 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
@@ -90,6 +90,6 @@ public class SparkSideInputReader implements SideInputReader {
 
   @Override
   public boolean isEmpty() {
-    return sideInputs.isEmpty();
+    return sideInputs != null && sideInputs.isEmpty();
   }
 }


[48/50] incubator-beam git commit: Closes #1210

Posted by th...@apache.org.
Closes #1210


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

Branch: refs/heads/apex-runner
Commit: 215980ad36dc4dc258ad4d6c0faabd8e7a80d188
Parents: f2ec824 dd854b1
Author: Dan Halperin <dh...@google.com>
Authored: Thu Oct 27 16:28:32 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Oct 27 16:28:32 2016 -0700

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



[21/50] incubator-beam git commit: Closes #1105

Posted by th...@apache.org.
Closes #1105


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

Branch: refs/heads/apex-runner
Commit: c94b8eacd480c3b49ae07ff117c2201079150dec
Parents: 7091a01 35244a6
Author: Dan Halperin <dh...@google.com>
Authored: Wed Oct 26 10:20:22 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:20:22 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ValueProviderUtils.java    | 60 +++++++++++++++
 .../beam/sdk/options/ValueProviderTest.java     | 17 ++---
 .../sdk/options/ValueProviderUtilsTest.java     | 78 ++++++++++++++++++++
 3 files changed, 143 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[05/50] incubator-beam git commit: Add experimental ServiceAccount option

Posted by th...@apache.org.
Add experimental ServiceAccount option


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

Branch: refs/heads/apex-runner
Commit: 60907b2128acdae28ecd5fe80d3b00a406b58892
Parents: 0ec956d
Author: sammcveety <sa...@gmail.com>
Authored: Sat Oct 22 10:27:53 2016 -0400
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Oct 25 13:05:16 2016 -0700

----------------------------------------------------------------------
 pom.xml                                                 |  2 +-
 .../runners/dataflow/DataflowPipelineTranslator.java    |  4 ++++
 .../dataflow/options/DataflowPipelineOptions.java       | 12 ++++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index df388d4..307e43e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,7 +106,7 @@
     <cloudresourcemanager.version>v1-rev6-1.22.0</cloudresourcemanager.version>
     <pubsubgrpc.version>0.1.0</pubsubgrpc.version>
     <clouddebugger.version>v2-rev8-1.22.0</clouddebugger.version>
-    <dataflow.version>v1b3-rev36-1.22.0</dataflow.version>
+    <dataflow.version>v1b3-rev42-1.22.0</dataflow.version>
     <dataflow.proto.version>0.5.160222</dataflow.proto.version>
     <datastore.client.version>1.2.0</datastore.client.version>
     <datastore.proto.version>1.2.0</datastore.proto.version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/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 ad49af7..12aa696 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
@@ -479,6 +479,10 @@ public class DataflowPipelineTranslator {
       workerPools.add(workerPool);
       environment.setWorkerPools(workerPools);
 
+      if (options.getServiceAccount() != null) {
+        environment.setServiceAccountEmail(options.getServiceAccount());
+      }
+
       pipeline.traverseTopologically(this);
       return job;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60907b21/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index ec4fec6..e853f22 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -22,6 +22,7 @@ import static com.google.common.base.Strings.isNullOrEmpty;
 
 import java.io.IOException;
 import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.BigQueryOptions;
 import org.apache.beam.sdk.options.Default;
@@ -29,6 +30,7 @@ import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.GcpOptions;
 import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.Hidden;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PubsubOptions;
 import org.apache.beam.sdk.options.StreamingOptions;
@@ -80,6 +82,16 @@ public interface DataflowPipelineOptions
   void setUpdate(boolean value);
 
   /**
+   * Run the job as a specific service account, instead of the default GCE robot.
+   */
+  @Hidden
+  @Experimental
+  @Description(
+      "Run the job as a specific service account, instead of the default GCE robot.")
+  String getServiceAccount();
+  void setServiceAccount(String value);
+
+  /**
    * Returns a default staging location under {@link GcpOptions#getGcpTempLocation}.
    */
   class StagingLocationFactory implements DefaultValueFactory<String> {


[11/50] incubator-beam git commit: Display data: minor cleanups

Posted by th...@apache.org.
Display data: minor cleanups

* wrong @Nullable
* Remove unused function


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

Branch: refs/heads/apex-runner
Commit: d2553500b7ebff4025b2c1588d9b73bab8213490
Parents: 13af296
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 19:39:27 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Oct 25 13:49:55 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/display/DisplayData.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d2553500/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
index 8e4cf91..46cf94a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
@@ -36,7 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import org.apache.avro.reflect.Nullable;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -147,10 +147,6 @@ public class DisplayData implements Serializable {
     return builder.toString();
   }
 
-  private static String namespaceOf(Class<?> clazz) {
-    return clazz.getName();
-  }
-
   /**
    * Utility to build up display data from a component and its included
    * subcomponents.


[30/50] incubator-beam git commit: Closes #1181

Posted by th...@apache.org.
Closes #1181


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

Branch: refs/heads/apex-runner
Commit: 09a80ecb4a464f63b6f65fbdba7783920280efb3
Parents: f9e07cf 5c558e6
Author: Dan Halperin <dh...@google.com>
Authored: Wed Oct 26 14:44:22 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 14:44:22 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[09/50] incubator-beam git commit: Remove use of OldDoFn from some DirectRunner tests

Posted by th...@apache.org.
Remove use of OldDoFn from some DirectRunner tests


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

Branch: refs/heads/apex-runner
Commit: 3d086857de87734b087076dad3eca92f625bb417
Parents: 4051357
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 16:09:13 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../ConsumerTrackingPipelineVisitorTest.java    | 32 +++----
 .../beam/runners/direct/DirectRunnerTest.java   | 40 +++++----
 .../ImmutabilityCheckingBundleFactoryTest.java  |  8 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  8 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  |  8 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |  8 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 87 +++++++++---------
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 94 +++++++++-----------
 .../runners/direct/WatermarkManagerTest.java    |  8 +-
 9 files changed, 139 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
index 1c9b5a6..e8f2a7e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
@@ -26,8 +26,8 @@ import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.View;
@@ -60,9 +60,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
         p.apply("listCreate", Create.of("foo", "bar"))
             .apply(
                 ParDo.of(
-                    new OldDoFn<String, String>() {
-                      @Override
-                      public void processElement(OldDoFn<String, String>.ProcessContext c)
+                    new DoFn<String, String>() {
+                      @ProcessElement
+                      public void processElement(DoFn<String, String>.ProcessContext c)
                           throws Exception {
                         c.output(Integer.toString(c.element().length()));
                       }
@@ -107,9 +107,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -138,9 +138,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -155,9 +155,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     p.apply(Create.of("1", "2", "3"))
         .apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }
@@ -180,9 +180,9 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<String> transformed =
         created.apply(
             ParDo.of(
-                new OldDoFn<String, String>() {
-                  @Override
-                  public void processElement(OldDoFn<String, String>.ProcessContext c)
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
                       throws Exception {
                     c.output(Integer.toString(c.element().length()));
                   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index 4027d25..34a5469 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -59,7 +59,6 @@ import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.SimpleFunction;
@@ -221,8 +220,8 @@ public class DirectRunnerTest implements Serializable {
 
   @Test
   public void transformDisplayDataExceptionShouldFail() {
-    OldDoFn<Integer, Integer> brokenDoFn = new OldDoFn<Integer, Integer>() {
-      @Override
+    DoFn<Integer, Integer> brokenDoFn = new DoFn<Integer, Integer>() {
+      @ProcessElement
       public void processElement(ProcessContext c) throws Exception {}
 
       @Override
@@ -242,7 +241,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -251,8 +250,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
             c.output(outputList);
             outputList.set(0, 37);
@@ -267,7 +267,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -276,8 +276,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, List<Integer>>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
             c.output(outputList);
             outputList.set(0, 37);
@@ -291,7 +292,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an output with a bad equals() still fails
+   * Tests that a {@link DoFn} that mutates an output with a bad equals() still fails
    * in the {@link DirectRunner}.
    */
   @Test
@@ -300,8 +301,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(42))
-        .apply(ParDo.of(new OldDoFn<Integer, byte[]>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<Integer, byte[]>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             byte[] outputArray = new byte[]{0x1, 0x2, 0x3};
             c.output(outputArray);
             outputArray[0] = 0xa;
@@ -316,7 +318,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates its input with a good equals() fails in the
+   * Tests that a {@link DoFn} that mutates its input with a good equals() fails in the
    * {@link DirectRunner}.
    */
   @Test
@@ -326,8 +328,9 @@ public class DirectRunnerTest implements Serializable {
     pipeline
         .apply(Create.of(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))
             .withCoder(ListCoder.of(VarIntCoder.of())))
-        .apply(ParDo.of(new OldDoFn<List<Integer>, Integer>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<List<Integer>, Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             List<Integer> inputList = c.element();
             inputList.set(0, 37);
             c.output(12);
@@ -341,7 +344,7 @@ public class DirectRunnerTest implements Serializable {
   }
 
   /**
-   * Tests that a {@link OldDoFn} that mutates an input with a bad equals() still fails
+   * Tests that a {@link DoFn} that mutates an input with a bad equals() still fails
    * in the {@link DirectRunner}.
    */
   @Test
@@ -350,8 +353,9 @@ public class DirectRunnerTest implements Serializable {
 
     pipeline
         .apply(Create.of(new byte[]{0x1, 0x2, 0x3}, new byte[]{0x4, 0x5, 0x6}))
-        .apply(ParDo.of(new OldDoFn<byte[], Integer>() {
-          @Override public void processElement(ProcessContext c) {
+        .apply(ParDo.of(new DoFn<byte[], Integer>() {
+          @ProcessElement
+          public void processElement(ProcessContext c) {
             byte[] inputArray = c.element();
             inputArray[0] = 0xa;
             c.output(13);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index d445944..ea44125 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -26,7 +26,7 @@ import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
@@ -179,9 +179,9 @@ public class ImmutabilityCheckingBundleFactoryTest {
     intermediate.commit(Instant.now());
   }
 
-  private static class IdentityDoFn<T> extends OldDoFn<T, T> {
-    @Override
-    public void processElement(OldDoFn<T, T>.ProcessContext c) throws Exception {
+  private static class IdentityDoFn<T> extends DoFn<T, T> {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       c.output(c.element());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
index 812d7d5..a7277fe 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -24,7 +24,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.util.IllegalMutationException;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -57,9 +57,9 @@ public class ImmutabilityEnforcementFactoryTest implements Serializable {
         p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
             .apply(
                 ParDo.of(
-                    new OldDoFn<byte[], byte[]>() {
-                      @Override
-                      public void processElement(OldDoFn<byte[], byte[]>.ProcessContext c)
+                    new DoFn<byte[], byte[]>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext c)
                           throws Exception {
                         c.element()[0] = 'b';
                       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
index ee6b2b4..cf65936 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
@@ -31,9 +31,9 @@ import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.Keys;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.KV;
@@ -177,9 +177,9 @@ public class KeyedPValueTrackingVisitorTest {
     }
   }
 
-  private static class IdentityFn<K> extends OldDoFn<K, K> {
-    @Override
-    public void processElement(OldDoFn<K, K>.ProcessContext c) throws Exception {
+  private static class IdentityFn<K> extends DoFn<K, K> {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       c.output(c.element());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
index 1a742f0..6d00aa1 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
@@ -37,7 +37,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -168,7 +168,7 @@ public class ParDoEvaluatorTest {
         ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, output));
   }
 
-  private static class RecorderFn extends OldDoFn<Integer, Integer> {
+  private static class RecorderFn extends DoFn<Integer, Integer> {
     private Collection<Integer> processed;
     private final PCollectionView<Integer> view;
 
@@ -177,8 +177,8 @@ public class ParDoEvaluatorTest {
       this.view = view;
     }
 
-    @Override
-    public void processElement(OldDoFn<Integer, Integer>.ProcessContext c) throws Exception {
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
       processed.add(c.element());
       c.output(c.element() + c.sideInput(view));
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index 8b0070b..cc83323 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -32,7 +32,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -41,11 +41,16 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
@@ -81,8 +86,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
+                new DoFn<String, KV<String, Integer>>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(KV.<String, Integer>of(c.element(), c.element().length()));
                     c.sideOutput(elementTag, c.element());
@@ -170,8 +175,8 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
+                new DoFn<String, KV<String, Integer>>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(KV.<String, Integer>of(c.element(), c.element().length()));
                     c.sideOutput(elementTag, c.element());
@@ -258,20 +263,17 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
         StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(StateNamespaces.global(), watermarkTag)
-                        .add(new Instant(20202L + c.element().length()));
-                    c.windowingInternals()
-                        .stateInternals()
-                        .state(
-                            StateNamespaces.window(
-                                GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                            bagTag)
-                        .add(c.element());
+                new DoFn<String, KV<String, Integer>>() {
+                  private static final String STATE_ID = "my-state-id";
+
+                  @StateId(STATE_ID)
+                  private final StateSpec<Object, BagState<String>> bagSpec =
+                      StateSpecs.bag(StringUtf8Coder.of());
+
+                  @ProcessElement
+                  public void processElement(
+                      ProcessContext c, @StateId(STATE_ID) BagState<String> bagState) {
+                    bagState.add(c.element());
                   }
                 })
             .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));
@@ -362,34 +364,25 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
 
     BoundMulti<String, KV<String, Integer>> pardo =
         ParDo.of(
-                new OldDoFn<String, KV<String, Integer>>() {
-                  @Override
-                  public void processElement(ProcessContext c) {
-                    c.windowingInternals().stateInternals();
-                    c.windowingInternals()
-                        .timerInternals()
-                        .setTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0).plus(Duration.standardMinutes(5)),
-                                        new Instant(1)
-                                            .plus(Duration.standardMinutes(5))
-                                            .plus(Duration.standardHours(1)))),
-                                new Instant(54541L),
-                                TimeDomain.EVENT_TIME));
-                    c.windowingInternals()
-                        .timerInternals()
-                        .deleteTimer(
-                            TimerData.of(
-                                StateNamespaces.window(
-                                    IntervalWindow.getCoder(),
-                                    new IntervalWindow(
-                                        new Instant(0),
-                                        new Instant(0).plus(Duration.standardHours(1)))),
-                                new Instant(3400000),
-                                TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+                new DoFn<String, KV<String, Integer>>() {
+                  private static final String EVENT_TIME_TIMER = "event-time-timer";
+                  private static final String SYNC_PROC_TIME_TIMER = "sync-proc-time-timer";
+
+                  @TimerId(EVENT_TIME_TIMER)
+                  TimerSpec myTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+                  @TimerId(SYNC_PROC_TIME_TIMER)
+                  TimerSpec syncProcTimerSpec =
+                      TimerSpecs.timer(TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+                  @ProcessElement
+                  public void processElement(
+                      ProcessContext c,
+                      @TimerId(EVENT_TIME_TIMER) Timer eventTimeTimer,
+                      @TimerId(SYNC_PROC_TIME_TIMER) Timer syncProcTimeTimer) {
+
+                    eventTimeTimer.setForNowPlus(Duration.standardMinutes(5));
+                    syncProcTimeTimer.cancel();
                   }
                 })
             .withOutputTags(mainOutputTag, TupleTagList.of(elementTag));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index e562b28..d22643a 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -32,22 +32,25 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.BagState;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.StateTag;
 import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
@@ -74,8 +77,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     PCollection<Integer> collection =
         input.apply(
             ParDo.of(
-                new OldDoFn<String, Integer>() {
-                  @Override
+                new DoFn<String, Integer>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.output(c.element().length());
                   }
@@ -128,8 +131,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     PCollection<Integer> collection =
         input.apply(
             ParDo.of(
-                new OldDoFn<String, Integer>() {
-                  @Override
+                new DoFn<String, Integer>() {
+                  @ProcessElement
                   public void processElement(ProcessContext c) {
                     c.sideOutput(sideOutputTag, c.element().length());
                   }
@@ -178,26 +181,22 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
 
-    final StateTag<Object, WatermarkHoldState<BoundedWindow>> watermarkTag =
-        StateTags.watermarkStateInternal("myId", OutputTimeFns.outputAtEarliestInputTimestamp());
     final StateTag<Object, BagState<String>> bagTag = StateTags.bag("myBag", StringUtf8Coder.of());
     final StateNamespace windowNs =
         StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE);
     ParDo.Bound<String, KV<String, Integer>> pardo =
         ParDo.of(
-            new OldDoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(StateNamespaces.global(), watermarkTag)
-                    .add(new Instant(124443L - c.element().length()));
-                c.windowingInternals()
-                    .stateInternals()
-                    .state(
-                        StateNamespaces.window(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE),
-                        bagTag)
-                    .add(c.element());
+            new DoFn<String, KV<String, Integer>>() {
+              private static final String STATE_ID = "my-state-id";
+
+              @StateId(STATE_ID)
+              private final StateSpec<Object, BagState<String>> bagSpec =
+                  StateSpecs.bag(StringUtf8Coder.of());
+
+              @ProcessElement
+              public void processElement(
+                  ProcessContext c, @StateId(STATE_ID) BagState<String> bagState) {
+                bagState.add(c.element());
               }
             });
     PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);
@@ -237,9 +236,6 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     assertThat(result.getWatermarkHold(), equalTo(new Instant(124438L)));
     assertThat(result.getState(), not(nullValue()));
     assertThat(
-        result.getState().state(StateNamespaces.global(), watermarkTag).read(),
-        equalTo(new Instant(124438L)));
-    assertThat(
         result.getState().state(windowNs, bagTag).read(),
         containsInAnyOrder("foo", "bara", "bazam"));
   }
@@ -255,6 +251,8 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     PCollection<String> input = p.apply(Create.of("foo", "bara", "bazam"));
 
+    // TODO: this timer data is absolute, but the new API only support relative settings.
+    // It will require adjustments when @Ignore is removed
     final TimerData addedTimer =
         TimerData.of(
             StateNamespaces.window(
@@ -276,34 +274,24 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
 
     ParDo.Bound<String, KV<String, Integer>> pardo =
         ParDo.of(
-            new OldDoFn<String, KV<String, Integer>>() {
-              @Override
-              public void processElement(ProcessContext c) {
-                c.windowingInternals().stateInternals();
-                c.windowingInternals()
-                    .timerInternals()
-                    .setTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0).plus(Duration.standardMinutes(5)),
-                                    new Instant(1)
-                                        .plus(Duration.standardMinutes(5))
-                                        .plus(Duration.standardHours(1)))),
-                            new Instant(54541L),
-                            TimeDomain.EVENT_TIME));
-                c.windowingInternals()
-                    .timerInternals()
-                    .deleteTimer(
-                        TimerData.of(
-                            StateNamespaces.window(
-                                IntervalWindow.getCoder(),
-                                new IntervalWindow(
-                                    new Instant(0),
-                                    new Instant(0).plus(Duration.standardHours(1)))),
-                            new Instant(3400000),
-                            TimeDomain.SYNCHRONIZED_PROCESSING_TIME));
+            new DoFn<String, KV<String, Integer>>() {
+              private static final String EVENT_TIME_TIMER = "event-time-timer";
+              private static final String SYNC_PROC_TIME_TIMER = "sync-proc-time-timer";
+
+              @TimerId(EVENT_TIME_TIMER)
+              TimerSpec myTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+              @TimerId(SYNC_PROC_TIME_TIMER)
+              TimerSpec syncProcTimerSpec =
+                  TimerSpecs.timer(TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+
+              @ProcessElement
+              public void processElement(
+                  ProcessContext c,
+                  @TimerId(EVENT_TIME_TIMER) Timer eventTimeTimer,
+                  @TimerId(SYNC_PROC_TIME_TIMER) Timer syncProcTimeTimer) {
+                eventTimeTimer.setForNowPlus(Duration.standardMinutes(5));
+                syncProcTimeTimer.cancel();
               }
             });
     PCollection<KV<String, Integer>> mainOutput = input.apply(pardo);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3d086857/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
index 042abab..1954005 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -47,9 +47,9 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -101,9 +101,9 @@ public class WatermarkManagerTest implements Serializable {
     createdInts = p.apply("createdInts", Create.of(1, 2, 3));
 
     filtered = createdInts.apply("filtered", Filter.greaterThan(1));
-    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new OldDoFn<Integer, Integer>() {
-      @Override
-      public void processElement(OldDoFn<Integer, Integer>.ProcessContext c) throws Exception {
+    filteredTimesTwo = filtered.apply("timesTwo", ParDo.of(new DoFn<Integer, Integer>() {
+      @ProcessElement
+      public void processElement(ProcessContext c) throws Exception {
         c.output(c.element() * 2);
       }
     }));


[31/50] incubator-beam git commit: Remove checked exceptions from PipelineResult.waitUntilFinish interface.

Posted by th...@apache.org.
Remove checked exceptions from PipelineResult.waitUntilFinish interface.


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

Branch: refs/heads/apex-runner
Commit: 1559a7f40fb96d3c22de93391792a298f9a84480
Parents: 09a80ec
Author: Pei He <pe...@google.com>
Authored: Fri Oct 21 17:38:21 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 14:47:10 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/direct/DirectRunner.java   |  2 +-
 .../runners/dataflow/BlockingDataflowRunner.java   | 13 +------------
 .../beam/runners/dataflow/DataflowPipelineJob.java | 17 +++++++++++++----
 .../spark/translation/EvaluationContext.java       |  6 ++----
 .../streaming/StreamingEvaluationContext.java      |  6 ++----
 .../java/org/apache/beam/sdk/PipelineResult.java   | 10 ++--------
 6 files changed, 21 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index e02c8a6..44d1986 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -431,7 +431,7 @@ public class DirectRunner
     }
 
     @Override
-    public State waitUntilFinish(Duration duration) throws IOException {
+    public State waitUntilFinish(Duration duration) {
       throw new UnsupportedOperationException(
           "DirectPipelineResult does not support waitUntilFinish with a Duration parameter. See"
               + " BEAM-596.");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
index d265361..5285ade 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.runners.dataflow;
 
-import java.io.IOException;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.util.MonitoringUtil;
@@ -111,17 +110,7 @@ public class BlockingDataflowRunner extends
       Runtime.getRuntime().addShutdownHook(shutdownHook);
 
       @Nullable
-      State result;
-      try {
-        result = job.waitUntilFinish(Duration.standardSeconds(BUILTIN_JOB_TIMEOUT_SEC));
-      } catch (IOException | InterruptedException ex) {
-        if (ex instanceof InterruptedException) {
-          Thread.currentThread().interrupt();
-        }
-        LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
-        throw new DataflowServiceException(
-            job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
-      }
+      State result = job.waitUntilFinish(Duration.standardSeconds(BUILTIN_JOB_TIMEOUT_SEC));
 
       if (result == null) {
         throw new DataflowServiceException(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
index bbcf11f..c3be192 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
@@ -167,15 +167,24 @@ public class DataflowPipelineJob implements PipelineResult {
 
   @Override
   @Nullable
-  public State waitUntilFinish() throws IOException, InterruptedException {
+  public State waitUntilFinish() {
     return waitUntilFinish(Duration.millis(-1));
   }
 
   @Override
   @Nullable
-  public State waitUntilFinish(Duration duration)
-          throws IOException, InterruptedException {
-    return waitUntilFinish(duration, new MonitoringUtil.LoggingHandler());
+  public State waitUntilFinish(Duration duration) {
+    try {
+      return waitUntilFinish(duration, new MonitoringUtil.LoggingHandler());
+    } catch (Exception e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+      if (e instanceof RuntimeException) {
+        throw (RuntimeException) e;
+      }
+      throw new RuntimeException(e);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
index c1c65dd..6ccec85 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
@@ -301,14 +301,12 @@ public class EvaluationContext implements EvaluationResult {
   }
 
   @Override
-  public State waitUntilFinish()
-      throws IOException, InterruptedException {
+  public State waitUntilFinish() {
     return waitUntilFinish(Duration.millis(-1));
   }
 
   @Override
-  public State waitUntilFinish(Duration duration)
-      throws IOException, InterruptedException {
+  public State waitUntilFinish(Duration duration) {
     // This is no-op, since Spark runner in batch is blocking.
     // It needs to be updated once SparkRunner supports non-blocking execution:
     // https://issues.apache.org/jira/browse/BEAM-595

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
index 49afa26..bfba316 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
@@ -212,15 +212,13 @@ public class StreamingEvaluationContext extends EvaluationContext {
   }
 
   @Override
-  public State waitUntilFinish()
-      throws IOException, InterruptedException {
+  public State waitUntilFinish() {
     throw new UnsupportedOperationException(
         "Spark runner StreamingEvaluationContext does not support waitUntilFinish.");
   }
 
   @Override
-  public State waitUntilFinish(Duration duration)
-      throws IOException, InterruptedException {
+  public State waitUntilFinish(Duration duration) {
     throw new UnsupportedOperationException(
         "Spark runner StreamingEvaluationContext does not support waitUntilFinish.");
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1559a7f4/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
index cd09c4d..35f11eb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java
@@ -52,23 +52,17 @@ public interface PipelineResult {
    *     Provide a value less than 1 ms for an infinite wait.
    *
    * @return The final state of the pipeline or null on timeout.
-   * @throws IOException If there is a persistent problem getting job
-   *   information.
-   * @throws InterruptedException if the thread is interrupted.
    * @throws UnsupportedOperationException if the runner does not support cancellation.
    */
-  State waitUntilFinish(Duration duration) throws IOException, InterruptedException;
+  State waitUntilFinish(Duration duration);
 
   /**
    * Waits until the pipeline finishes and returns the final status.
    *
    * @return The final state of the pipeline.
-   * @throws IOException If there is a persistent problem getting job
-   *   information.
-   * @throws InterruptedException if the thread is interrupted.
    * @throws UnsupportedOperationException if the runner does not support cancellation.
    */
-  State waitUntilFinish() throws IOException, InterruptedException;
+  State waitUntilFinish();
 
   /**
    * Retrieves the current value of the provided {@link Aggregator}.


[14/50] incubator-beam git commit: This closes #1162

Posted by th...@apache.org.
This closes #1162


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

Branch: refs/heads/apex-runner
Commit: 53fe3ee425163a76b69d0830449d222d925eb9cd
Parents: f2fe1ae a54ded3
Author: Sela <an...@paypal.com>
Authored: Wed Oct 26 10:01:51 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 10:01:51 2016 +0300

----------------------------------------------------------------------
 .../apache/beam/runners/spark/SparkRunner.java  | 19 ------
 .../translation/GroupCombineFunctions.java      | 66 +++++++++-----------
 .../spark/translation/TransformTranslator.java  | 43 +++----------
 .../streaming/StreamingTransformTranslator.java | 65 +++++--------------
 .../spark/util/SparkSideInputReader.java        |  2 +-
 5 files changed, 55 insertions(+), 140 deletions(-)
----------------------------------------------------------------------



[43/50] incubator-beam git commit: This closes #1202

Posted by th...@apache.org.
This closes #1202


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

Branch: refs/heads/apex-runner
Commit: 3fd3951ded4566b9405f4a4246be7b7e8be86d3d
Parents: 4cb1d10 aeb3b3c
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 27 12:51:32 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 12:51:32 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/WatermarkManager.java      | 17 ++++++++---------
 1 file changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[45/50] incubator-beam git commit: Separate package-info.java compilation to avoid spurious rebuilds

Posted by th...@apache.org.
Separate package-info.java compilation to avoid spurious rebuilds


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

Branch: refs/heads/apex-runner
Commit: 324a36d53b1542199eb5fc6cfa1a78226bf9903f
Parents: 3fd3951
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 27 11:54:14 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 14:55:46 2016 -0700

----------------------------------------------------------------------
 pom.xml | 39 +++++++++++++++++++++++++++++++++++++++
 1 file changed, 39 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/324a36d5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 94d7b4b..01de58e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -831,6 +831,45 @@
             <!-- Another temp override, to be set to true in due course. -->
             <showDeprecation>false</showDeprecation>
           </configuration>
+          <executions>
+
+            <!--
+              Exclude package-info.java from main compilation to work around
+              https://jira.codehaus.org/browse/MCOMPILER-205
+            -->
+            <execution>
+              <id>default-compile</id>
+              <goals>
+                <goal>compile</goal>
+              </goals>
+              <phase>compile</phase>
+              <configuration>
+                <excludes>
+                  <exclude>**/package-info.java</exclude>
+                </excludes>
+              </configuration>
+            </execution>
+
+            <!-- 
+              Compile just package-info.java to avoid 
+              https://bugs.openjdk.java.net/browse/JDK-8022161
+            -->
+            <execution>
+              <id>compile-package-info</id>
+              <goals>
+                <goal>compile</goal>
+              </goals>
+              <phase>compile</phase>
+              <configuration>
+                <compilerArgs>
+                  <arg>-Xpkginfo:always</arg>
+                </compilerArgs>
+                <includes>
+                  <include>**/package-info.java</include>
+                </includes>
+              </configuration>
+            </execution>
+          </executions>
         </plugin>
 
         <plugin>


[27/50] incubator-beam git commit: Add default version for beam-sdks-java-io-kafka

Posted by th...@apache.org.
Add default version for beam-sdks-java-io-kafka


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

Branch: refs/heads/apex-runner
Commit: c88953388d2f53e1f67ab10fb22d8ab8808980ff
Parents: 9f30b21
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 13:03:45 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Oct 26 13:33:03 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c8895338/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 307e43e..94d7b4b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -271,6 +271,12 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-io-kafka</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-runners-core-java</artifactId>
         <version>${project.version}</version>
       </dependency>


[04/50] incubator-beam git commit: Remove unused GroupAlsoByWindowsDoFn.createDefault

Posted by th...@apache.org.
Remove unused GroupAlsoByWindowsDoFn.createDefault


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

Branch: refs/heads/apex-runner
Commit: 5ce3aac2d3b7203b0cbab52c394a99142a46a5cf
Parents: f603d43
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 15:51:08 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/core/GroupAlsoByWindowsDoFn.java     | 19 -------------------
 1 file changed, 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ce3aac2/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
index 9851449..1b32d84 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java
@@ -17,15 +17,12 @@
  */
 package org.apache.beam.runners.core;
 
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.KV;
 
 /**
@@ -47,20 +44,4 @@ public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends Bound
       createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
   protected final Aggregator<Long, Long> droppedDueToLateness =
       createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
-
-  /**
-   * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
-   * grouping.
-   *
-   * @param windowingStrategy The window function and trigger to use for grouping
-   * @param inputCoder the input coder to use
-   */
-  public static <K, V, W extends BoundedWindow>
-      GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W> createDefault(
-          WindowingStrategy<?, W> windowingStrategy,
-          StateInternalsFactory<K> stateInternalsFactory,
-          Coder<V> inputCoder) {
-    return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
-        windowingStrategy, stateInternalsFactory, SystemReduceFn.<K, V, W>buffering(inputCoder));
-  }
 }


[41/50] incubator-beam git commit: Use a Constant TupleTag in GroupAlsoByWindowEvaluator

Posted by th...@apache.org.
Use a Constant TupleTag in GroupAlsoByWindowEvaluator

This avoids the cost of generating a random ID, which is moderately
expensive.


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

Branch: refs/heads/apex-runner
Commit: 34a236cead89ec72fded2cf44e6e1b0b540838d3
Parents: 064f18a
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 26 14:31:41 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 12:50:08 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/GroupAlsoByWindowEvaluatorFactory.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34a236ce/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
index c7cf9e3..0c2aa1b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
@@ -86,6 +86,7 @@ class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
    */
   private static class GroupAlsoByWindowEvaluator<K, V>
       implements TransformEvaluator<KeyedWorkItem<K, V>> {
+    private static final TupleTag<Object> MAIN_OUTPUT_TAG = new TupleTag<Object>() {};
 
     private final TransformEvaluator<KeyedWorkItem<K, V>> gabwParDoEvaluator;
 
@@ -118,8 +119,6 @@ class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
               new ConstantStateInternalsFactory<K>(stateInternals),
               SystemReduceFn.<K, V, BoundedWindow>buffering(valueCoder));
 
-      TupleTag<KV<K, Iterable<V>>> mainOutputTag = new TupleTag<KV<K, Iterable<V>>>() {};
-
       // Not technically legit, as the application is not a ParDo
       this.gabwParDoEvaluator =
           ParDoEvaluator.create(
@@ -129,9 +128,10 @@ class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
               application,
               gabwDoFn,
               Collections.<PCollectionView<?>>emptyList(),
-              mainOutputTag,
+              MAIN_OUTPUT_TAG,
               Collections.<TupleTag<?>>emptyList(),
-              ImmutableMap.<TupleTag<?>, PCollection<?>>of(mainOutputTag, application.getOutput()));
+              ImmutableMap.<TupleTag<?>, PCollection<?>>of(
+                  MAIN_OUTPUT_TAG, application.getOutput()));
     }
 
     @Override


[26/50] incubator-beam git commit: Use project-wise version in SparkRunner dep on KafkaIO

Posted by th...@apache.org.
Use project-wise version in SparkRunner dep on KafkaIO


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

Branch: refs/heads/apex-runner
Commit: 9d6598d30e097e4a848bb0488271d196cc51d078
Parents: c889533
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 13:04:24 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Oct 26 13:33:03 2016 -0700

----------------------------------------------------------------------
 runners/spark/pom.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d6598d3/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 36ce7b5..71a3ac2 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -263,7 +263,6 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-kafka</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
     </dependency>
     <dependency>
       <groupId>org.apache.kafka</groupId>


[12/50] incubator-beam git commit: Closes #1182

Posted by th...@apache.org.
Closes #1182


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

Branch: refs/heads/apex-runner
Commit: f2fe1ae466284b58ebcbff98ea1103027fe6f33e
Parents: 13af296 d255350
Author: Dan Halperin <dh...@google.com>
Authored: Tue Oct 25 13:49:56 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Oct 25 13:49:56 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/display/DisplayData.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-beam git commit: Closes #1180

Posted by th...@apache.org.
Closes #1180


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

Branch: refs/heads/apex-runner
Commit: 7091a0120e070335c24198d5f297a9fcff25fe75
Parents: 78e2c03 3af2c1e
Author: Dan Halperin <dh...@google.com>
Authored: Wed Oct 26 10:13:59 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:13:59 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 16 ++++++----------
 1 file changed, 6 insertions(+), 10 deletions(-)
----------------------------------------------------------------------



[37/50] incubator-beam git commit: Closes #1097

Posted by th...@apache.org.
Closes #1097


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

Branch: refs/heads/apex-runner
Commit: ee6ad2fe416cd499aa1b6e2a51aa64da0805cc5c
Parents: eba099f e5afbe5
Author: Dan Halperin <dh...@google.com>
Authored: Thu Oct 27 10:35:13 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Oct 27 10:35:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/examples/DebuggingWordCount.java |  2 +-
 .../java/org/apache/beam/examples/MinimalWordCount.java   |  2 +-
 .../src/main/java/org/apache/beam/examples/WordCount.java |  2 +-
 .../java/org/apache/beam/examples/complete/TfIdf.java     |  2 +-
 .../beam/examples/complete/TopWikipediaSessions.java      |  2 +-
 .../apache/beam/examples/cookbook/BigQueryTornadoes.java  |  2 +-
 .../beam/examples/cookbook/CombinePerKeyExamples.java     |  2 +-
 .../org/apache/beam/examples/cookbook/DeDupExample.java   |  2 +-
 .../org/apache/beam/examples/cookbook/FilterExamples.java |  2 +-
 .../org/apache/beam/examples/cookbook/JoinExamples.java   |  2 +-
 .../apache/beam/examples/cookbook/MaxPerKeyExamples.java  |  2 +-
 .../test/java/org/apache/beam/examples/WordCountTest.java |  2 +-
 .../apache/beam/examples/complete/AutoCompleteTest.java   |  6 +++---
 .../java/org/apache/beam/examples/complete/TfIdfTest.java |  2 +-
 .../beam/examples/complete/TopWikipediaSessionsTest.java  |  2 +-
 .../apache/beam/examples/cookbook/DeDupExampleTest.java   |  4 ++--
 .../apache/beam/examples/cookbook/JoinExamplesTest.java   |  2 +-
 .../apache/beam/examples/cookbook/TriggerExampleTest.java |  2 +-
 .../org/apache/beam/examples/MinimalWordCountJava8.java   |  2 +-
 .../beam/examples/complete/game/HourlyTeamScore.java      |  2 +-
 .../org/apache/beam/examples/complete/game/UserScore.java |  2 +-
 .../apache/beam/examples/complete/game/GameStatsTest.java |  2 +-
 .../beam/examples/complete/game/HourlyTeamScoreTest.java  |  2 +-
 .../beam/examples/complete/game/LeaderBoardTest.java      | 10 +++++-----
 .../apache/beam/examples/complete/game/UserScoreTest.java |  6 +++---
 25 files changed, 34 insertions(+), 34 deletions(-)
----------------------------------------------------------------------



[20/50] incubator-beam git commit: Add utility to handle JSON option manipulation

Posted by th...@apache.org.
Add utility to handle JSON option manipulation


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

Branch: refs/heads/apex-runner
Commit: 35244a680d53ddca4733dbef5b2570e6f78140d6
Parents: 7091a01
Author: sammcveety <sa...@gmail.com>
Authored: Fri Oct 14 13:06:08 2016 -0400
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:20:21 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ValueProviderUtils.java    | 60 +++++++++++++++
 .../beam/sdk/options/ValueProviderTest.java     | 17 ++---
 .../sdk/options/ValueProviderUtilsTest.java     | 78 ++++++++++++++++++++
 3 files changed, 143 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
new file mode 100644
index 0000000..14a5f23
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.options;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Utilities for working with the {@link ValueProvider} interface.
+ */
+public class ValueProviderUtils {
+  private ValueProviderUtils() {}
+
+  /**
+   * Given {@code serializedOptions} as a JSON-serialized {@link PipelineOptions}, updates
+   * the values according to the provided values in {@code runtimeValues}.
+   */
+  public static String updateSerializedOptions(
+      String serializedOptions, Map<String, String> runtimeValues) {
+    ObjectMapper mapper = new ObjectMapper();
+    ObjectNode root, options;
+    try {
+      root = mapper.readValue(serializedOptions, ObjectNode.class);
+      options = (ObjectNode) root.get("options");
+      checkNotNull(options, "Unable to locate 'options' in %s", serializedOptions);
+    } catch (IOException e) {
+      throw new RuntimeException(
+        String.format("Unable to parse %s", serializedOptions), e);
+    }
+
+    for (Map.Entry<String, String> entry : runtimeValues.entrySet()) {
+      options.put(entry.getKey(), entry.getValue());
+    }
+    try {
+      return mapper.writeValueAsString(root);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse re-serialize options", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
index c9eb479..ed7a37a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
@@ -17,14 +17,13 @@
  */
 package org.apache.beam.sdk.options;
 
-import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import java.util.List;
 import org.apache.beam.sdk.options.ValueProvider.RuntimeValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
@@ -182,12 +181,8 @@ public class ValueProviderTest {
     ObjectMapper mapper = new ObjectMapper();
     String serializedOptions = mapper.writeValueAsString(submitOptions);
 
-    // This is the expected behavior of the runner: deserialize and set the
-    // the runtime options.
-    String anchor = "\"appName\":\"ValueProviderTest\"";
-    assertThat(serializedOptions, containsString("\"foo\":null"));
-    String runnerString = serializedOptions.replaceAll(
-      "\"foo\":null", "\"foo\":\"quux\"");
+    String runnerString = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("foo", "quux"));
     TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class)
       .as(TestOptions.class);
 
@@ -206,10 +201,8 @@ public class ValueProviderTest {
     ObjectMapper mapper = new ObjectMapper();
     String serializedOptions = mapper.writeValueAsString(submitOptions);
 
-    // This is the expected behavior of the runner: deserialize and set the
-    // the runtime options.
-    assertThat(serializedOptions, containsString("baz"));
-    String runnerString = serializedOptions.replaceAll("baz", "quux");
+    String runnerString = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("foo", "quux"));
     TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class)
       .as(TestOptions.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
new file mode 100644
index 0000000..0f8ed8e
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.options;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link ValueProviderUtils}. */
+@RunWith(JUnit4.class)
+public class ValueProviderUtilsTest {
+  /** A test interface. */
+  public interface TestOptions extends PipelineOptions {
+    String getString();
+    void setString(String value);
+
+    String getOtherString();
+    void setOtherString(String value);
+  }
+
+  @Test
+  public void testUpdateSerialize() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("string", "bar"));
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertEquals("bar", runtime.getString());
+  }
+
+  @Test
+  public void testUpdateSerializeExistingValue() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.fromArgs(
+      new String[]{"--string=baz", "--otherString=quux"}).as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("string", "bar"));
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertEquals("bar", runtime.getString());
+    assertEquals("quux", runtime.getOtherString());
+  }
+
+  @Test
+  public void testUpdateSerializeEmptyUpdate() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.<String, String>of());
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertNull(runtime.getString());
+  }
+}


[02/50] incubator-beam git commit: This closes #1095

Posted by th...@apache.org.
This closes #1095


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

Branch: refs/heads/apex-runner
Commit: f603d43e043d383ceb00c4d786459f01d9983586
Parents: c03e3e9 2ceaa3e
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Oct 25 11:04:13 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 11:04:13 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/CloningBundleFactory.java    |  98 ++++++++++
 .../beam/runners/direct/DirectRunner.java       |   5 +-
 .../direct/ImmutableListBundleFactory.java      |   4 +-
 .../direct/CloningBundleFactoryTest.java        | 177 +++++++++++++++++++
 .../EncodabilityEnforcementFactoryTest.java     |   6 +-
 5 files changed, 285 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[32/50] incubator-beam git commit: Closes #1160

Posted by th...@apache.org.
Closes #1160


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

Branch: refs/heads/apex-runner
Commit: 48fdd06760ded8bb2010b12fad430068a06bdec6
Parents: 09a80ec 1559a7f
Author: Dan Halperin <dh...@google.com>
Authored: Wed Oct 26 14:47:10 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 14:47:10 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/direct/DirectRunner.java   |  2 +-
 .../runners/dataflow/BlockingDataflowRunner.java   | 13 +------------
 .../beam/runners/dataflow/DataflowPipelineJob.java | 17 +++++++++++++----
 .../spark/translation/EvaluationContext.java       |  6 ++----
 .../streaming/StreamingEvaluationContext.java      |  6 ++----
 .../java/org/apache/beam/sdk/PipelineResult.java   | 10 ++--------
 6 files changed, 21 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[25/50] incubator-beam git commit: This closes #1185

Posted by th...@apache.org.
This closes #1185


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

Branch: refs/heads/apex-runner
Commit: 9f30b21a7b36f73685934746560cd3e02ab451e7
Parents: dc61a00 6db9424
Author: Sela <an...@paypal.com>
Authored: Wed Oct 26 21:17:12 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 21:17:12 2016 +0300

----------------------------------------------------------------------
 .../metrics/AggregatorMetricSource.java         |  9 +-
 .../metrics/WithNamedAggregatorsSupport.java    |  7 +-
 .../spark/translation/SparkRuntimeContext.java  |  2 +-
 .../runners/spark/ClearAggregatorsRule.java     | 33 -------
 .../runners/spark/InMemoryMetricsSinkRule.java  | 32 -------
 .../metrics/sink/ClearAggregatorsRule.java      | 33 +++++++
 .../metrics/sink/InMemoryMetrics.java           | 15 +++-
 .../metrics/sink/InMemoryMetricsSinkRule.java   | 31 +++++++
 .../metrics/sink/NamedAggregatorsTest.java      | 92 ++++++++++++++++++++
 9 files changed, 179 insertions(+), 75 deletions(-)
----------------------------------------------------------------------



[10/50] incubator-beam git commit: Remove use of OldDoFn in ValueWithRecordId

Posted by th...@apache.org.
Remove use of OldDoFn in ValueWithRecordId


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

Branch: refs/heads/apex-runner
Commit: 4051357870b88ed18d09ef502ee8b1cf5142979f
Parents: 7abdcbf
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 15:57:18 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Oct 25 13:12:17 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/util/ValueWithRecordId.java     | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/40513578/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
index d6b5fe3..abca598 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
@@ -31,7 +31,7 @@ import java.util.Objects;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StandardCoder;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 
 /**
  * Immutable struct containing a value as well as a unique id identifying the value.
@@ -136,9 +136,9 @@ public class ValueWithRecordId<ValueT> {
     ByteArrayCoder idCoder;
   }
 
-  /** {@link OldDoFn} to turn a {@code ValueWithRecordId<T>} back to the value {@code T}. */
-  public static class StripIdsDoFn<T> extends OldDoFn<ValueWithRecordId<T>, T> {
-    @Override
+  /** {@link DoFn} to turn a {@code ValueWithRecordId<T>} back to the value {@code T}. */
+  public static class StripIdsDoFn<T> extends DoFn<ValueWithRecordId<T>, T> {
+    @ProcessElement
     public void processElement(ProcessContext c) {
       c.output(c.element().getValue());
     }


[24/50] incubator-beam git commit: [BEAM-816] Aggregators are not properly named when reported to Graphite.

Posted by th...@apache.org.
[BEAM-816] Aggregators are not properly named when reported to Graphite.

Added NamedAggregatorTest.


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

Branch: refs/heads/apex-runner
Commit: 6db942498a6aa20e9fd253871320d0ee4aa9476d
Parents: dc61a00
Author: Stas Levin <st...@gmail.com>
Authored: Tue Oct 25 18:23:23 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 21:16:01 2016 +0300

----------------------------------------------------------------------
 .../metrics/AggregatorMetricSource.java         |  9 +-
 .../metrics/WithNamedAggregatorsSupport.java    |  7 +-
 .../spark/translation/SparkRuntimeContext.java  |  2 +-
 .../runners/spark/ClearAggregatorsRule.java     | 33 -------
 .../runners/spark/InMemoryMetricsSinkRule.java  | 32 -------
 .../metrics/sink/ClearAggregatorsRule.java      | 33 +++++++
 .../metrics/sink/InMemoryMetrics.java           | 15 +++-
 .../metrics/sink/InMemoryMetricsSinkRule.java   | 31 +++++++
 .../metrics/sink/NamedAggregatorsTest.java      | 92 ++++++++++++++++++++
 9 files changed, 179 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java
index 0658e04..2a00aec 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java
@@ -29,17 +29,18 @@ import org.apache.spark.metrics.source.Source;
  */
 public class AggregatorMetricSource implements Source {
 
-  private static final String SOURCE_NAME = "NamedAggregators";
+  private final String sourceName;
 
   private final MetricRegistry metricRegistry = new MetricRegistry();
 
-  public AggregatorMetricSource(final NamedAggregators aggregators) {
-    metricRegistry.register(SOURCE_NAME, AggregatorMetric.of(aggregators));
+  public AggregatorMetricSource(final String appName, final NamedAggregators aggregators) {
+    sourceName = appName;
+    metricRegistry.register("Beam", AggregatorMetric.of(aggregators));
   }
 
   @Override
   public String sourceName() {
-    return SOURCE_NAME;
+    return sourceName;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java
index 88e2211..6932ae6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java
@@ -118,8 +118,13 @@ public class WithNamedAggregatorsSupport extends MetricRegistry {
       @Override
       public Map<String, Gauge> apply(final Map.Entry<String, Metric> entry) {
         final NamedAggregators agg = ((AggregatorMetric) entry.getValue()).getNamedAggregators();
+        final String parentName = entry.getKey();
         final Map<String, Gauge> gaugeMap = Maps.transformEntries(agg.renderAll(), toGauge());
-        return Maps.filterValues(gaugeMap, Predicates.notNull());
+        final Map<String, Gauge> fullNameGaugeMap = Maps.newLinkedHashMap();
+        for (String shortName : gaugeMap.keySet()) {
+          fullNameGaugeMap.put(parentName + "." + shortName, gaugeMap.get(shortName));
+        }
+        return Maps.filterValues(fullNameGaugeMap, Predicates.notNull());
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
index 94c1648..181a111 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java
@@ -89,7 +89,7 @@ public class SparkRuntimeContext implements Serializable {
     if (opts.getEnableSparkSinks()) {
       final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem();
       final AggregatorMetricSource aggregatorMetricSource =
-          new AggregatorMetricSource(initialValue);
+          new AggregatorMetricSource(opts.getAppName(), initialValue);
       // in case the context was not cleared
       metricsSystem.removeSource(aggregatorMetricSource);
       metricsSystem.registerSource(aggregatorMetricSource);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/ClearAggregatorsRule.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ClearAggregatorsRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ClearAggregatorsRule.java
deleted file mode 100644
index beaae13..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/ClearAggregatorsRule.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
-import org.junit.rules.ExternalResource;
-
-/**
- * A rule that clears the {@link org.apache.beam.runners.spark.aggregators.AccumulatorSingleton}
- * which represents the Beam {@link org.apache.beam.sdk.transforms.Aggregator}s.
- */
-class ClearAggregatorsRule extends ExternalResource {
-  @Override
-  protected void before() throws Throwable {
-    AccumulatorSingleton.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java
deleted file mode 100644
index 506dbbd..0000000
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.runners.spark;
-
-import org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics;
-import org.junit.rules.ExternalResource;
-
-/**
- * A rule that cleans the {@link InMemoryMetrics} after the tests has finished.
- */
-class InMemoryMetricsSinkRule extends ExternalResource {
-  @Override
-  protected void before() throws Throwable {
-    InMemoryMetrics.clearAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java
new file mode 100644
index 0000000..79c58a7
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.aggregators.metrics.sink;
+
+import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
+import org.junit.rules.ExternalResource;
+
+/**
+ * A rule that clears the {@link org.apache.beam.runners.spark.aggregators.AccumulatorSingleton}
+ * which represents the Beam {@link org.apache.beam.sdk.transforms.Aggregator}s.
+ */
+class ClearAggregatorsRule extends ExternalResource {
+  @Override
+  protected void before() throws Throwable {
+    AccumulatorSingleton.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
index 35e6717..389cd03 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
@@ -20,12 +20,13 @@ package org.apache.beam.runners.spark.aggregators.metrics.sink;
 
 import com.codahale.metrics.MetricFilter;
 import com.codahale.metrics.MetricRegistry;
-
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterables;
 import java.util.Properties;
-
 import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
+
 /**
  * An in-memory {@link Sink} implementation for tests.
  */
@@ -45,9 +46,15 @@ public class InMemoryMetrics implements Sink {
   public static <T> T valueOf(final String name) {
     final T retVal;
 
+    // this might fail in case we have multiple aggregators with the same suffix after
+    // the last dot, but it should be good enough for tests.
     if (extendedMetricsRegistry != null
-        && extendedMetricsRegistry.getGauges().containsKey(name)) {
-      retVal = (T) extendedMetricsRegistry.getGauges().get(name).getValue();
+        && Iterables.any(extendedMetricsRegistry.getGauges().keySet(),
+        Predicates.containsPattern(name + "$"))) {
+      String key =
+          Iterables.find(extendedMetricsRegistry.getGauges().keySet(),
+              Predicates.containsPattern(name + "$"));
+      retVal = (T) extendedMetricsRegistry.getGauges().get(key).getValue();
     } else {
       retVal = null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetricsSinkRule.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetricsSinkRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetricsSinkRule.java
new file mode 100644
index 0000000..5a3d19d
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetricsSinkRule.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.aggregators.metrics.sink;
+
+import org.junit.rules.ExternalResource;
+
+/**
+ * A rule that cleans the {@link InMemoryMetrics} after the tests has finished.
+ */
+class InMemoryMetricsSinkRule extends ExternalResource {
+  @Override
+  protected void before() throws Throwable {
+    InMemoryMetrics.clearAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6db94249/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java
new file mode 100644
index 0000000..194d66a
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.aggregators.metrics.sink;
+
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkRunner;
+import org.apache.beam.runners.spark.examples.WordCount;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+
+
+/**
+ * A test for the NamedAggregators mechanism.
+ */
+public class NamedAggregatorsTest {
+
+  @Rule
+  public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule();
+
+  @Rule
+  public ClearAggregatorsRule clearAggregators = new ClearAggregatorsRule();
+
+  private Pipeline createSparkPipeline() {
+    final SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
+    options.setRunner(SparkRunner.class);
+    return Pipeline.create(options);
+  }
+
+  private void runPipeline() {
+
+    final List<String> words =
+        Arrays.asList("hi there", "hi", "hi sue bob", "hi sue", "", "bob hi");
+
+    final Set<String> expectedCounts =
+        ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");
+
+    final Pipeline pipeline = createSparkPipeline();
+
+    final PCollection<String> output =
+        pipeline
+        .apply(Create.of(words).withCoder(StringUtf8Coder.of()))
+        .apply(new WordCount.CountWords())
+        .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
+    PAssert.that(output).containsInAnyOrder(expectedCounts);
+
+    pipeline.run();
+  }
+
+  @Test
+  public void testNamedAggregators() throws Exception {
+
+    assertThat(InMemoryMetrics.valueOf("emptyLines"), is(nullValue()));
+
+    runPipeline();
+
+    assertThat(InMemoryMetrics.<Double>valueOf("emptyLines"), is(1d));
+
+  }
+}


[23/50] incubator-beam git commit: This closes #1143

Posted by th...@apache.org.
This closes #1143


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

Branch: refs/heads/apex-runner
Commit: dc61a001c1b9389450ade3f49b98a9a22c10f794
Parents: c94b8ea b72e7e3
Author: Sela <an...@paypal.com>
Authored: Wed Oct 26 20:52:17 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 20:52:17 2016 +0300

----------------------------------------------------------------------
 runners/spark/pom.xml                           |  44 ++--
 .../runners/spark/SparkPipelineOptions.java     |  11 +
 .../runners/spark/io/EmptyCheckpointMark.java   |  52 ++++
 .../apache/beam/runners/spark/io/KafkaIO.java   | 131 ----------
 .../beam/runners/spark/io/MicrobatchSource.java | 262 +++++++++++++++++++
 .../beam/runners/spark/io/SourceDStream.java    | 156 +++++++++++
 .../apache/beam/runners/spark/io/SourceRDD.java |  75 +++++-
 .../runners/spark/io/SparkUnboundedSource.java  | 167 ++++++++++++
 .../spark/stateful/StateSpecFunctions.java      | 167 ++++++++++++
 .../runners/spark/stateful/package-info.java    |  22 ++
 .../spark/translation/TranslationUtils.java     |  17 ++
 .../SparkRunnerStreamingContextFactory.java     |   7 +
 .../streaming/StreamingTransformTranslator.java |  37 +--
 .../streaming/KafkaStreamingTest.java           | 150 ++++++++---
 .../ResumeFromCheckpointStreamingTest.java      |  34 ++-
 .../streaming/utils/EmbeddedKafkaCluster.java   |  51 ++--
 .../utils/KafkaWriteOnBatchCompleted.java       | 105 ++++++++
 17 files changed, 1224 insertions(+), 264 deletions(-)
----------------------------------------------------------------------



[28/50] incubator-beam git commit: This closes #1195

Posted by th...@apache.org.
This closes #1195


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

Branch: refs/heads/apex-runner
Commit: f9e07cfb2970ceb7e6ace8393eb947ad5efa34c9
Parents: 9f30b21 9d6598d
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 13:33:03 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Oct 26 13:33:03 2016 -0700

----------------------------------------------------------------------
 pom.xml               | 6 ++++++
 runners/spark/pom.xml | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[16/50] incubator-beam git commit: This closes #1171

Posted by th...@apache.org.
This closes #1171


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

Branch: refs/heads/apex-runner
Commit: 78e2c038766063081a8c045607c74ad158afdf68
Parents: 53fe3ee 13b8385
Author: Sela <an...@paypal.com>
Authored: Wed Oct 26 18:54:09 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 18:54:09 2016 +0300

----------------------------------------------------------------------
 runners/spark/pom.xml                           | 23 ++++++++++
 .../coders/BeamSparkRunnerRegistrator.java      | 46 ++++++++++++++++++++
 .../spark/translation/SparkContextFactory.java  |  5 ++-
 3 files changed, 73 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[33/50] incubator-beam git commit: This closes #1173

Posted by th...@apache.org.
This closes #1173


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

Branch: refs/heads/apex-runner
Commit: 25102f798cf8ddc2a706c3932e856a7825555dc5
Parents: 48fdd06 3bec5e0
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 14:59:14 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Oct 26 14:59:14 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/game/GameStats.java  | 52 +++++++++++---------
 .../examples/complete/game/HourlyTeamScore.java | 28 ++++++-----
 .../examples/complete/game/LeaderBoard.java     | 43 +++++++++-------
 .../beam/examples/complete/game/UserScore.java  | 14 ++++--
 .../complete/game/utils/WriteToBigQuery.java    | 49 ++++++++++--------
 .../game/utils/WriteWindowedToBigQuery.java     | 14 +++---
 6 files changed, 114 insertions(+), 86 deletions(-)
----------------------------------------------------------------------



[29/50] incubator-beam git commit: Streamline Travis CI build

Posted by th...@apache.org.
Streamline Travis CI build


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

Branch: refs/heads/apex-runner
Commit: 5c558e6f6b0fe63324de7400121f33a91418fc02
Parents: f9e07cf
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 26 10:37:09 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 14:44:21 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5c558e6f/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 5b271d1..6a8b604 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -28,20 +28,25 @@ notifications:
     on_success: change
     on_failure: always
 
+env:
+  global:
+   - MAVEN_OVERRIDE="--settings=.travis/settings.xml -Djavadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true"
+   - MAVEN_CONTAINER_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
+
 matrix:
   include:
     # On OSX, run with default JDK only.
     - os: osx
-      env: MAVEN_OVERRIDE="--settings=.travis/settings.xml"
+
     # On Linux, run with specific JDKs only.
     - os: linux
-      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
     - os: linux
-      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
     - os: linux
-      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m' -Peclipse-jdt"
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE -Peclipse-jdt"
     - os: linux
-      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="--settings=.travis/settings.xml -DbeamSurefireArgline='-Xmx512m'"
+      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
 
 before_install:
   - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
@@ -56,8 +61,7 @@ install:
   - rm -rf "$HOME/.m2/repository/org/apache/beam"
 
 script:
-  - travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify
-  - travis_retry .travis/test_wordcount.sh
+  - travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE install && travis_retry bash -ex .travis/test_wordcount.sh
 
 cache:
   directories:


[39/50] incubator-beam git commit: Make DoFnInfo carry OldDoFn or DoFn

Posted by th...@apache.org.
Make DoFnInfo carry OldDoFn or DoFn

This will allow consumers to prepare to accept
DoFn while still accepting existing jobs that
use OldDoFn. It is a move towards treating
the Fn itself as just a serialized blob.


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

Branch: refs/heads/apex-runner
Commit: 73db5608a58ff64a0b452140736a150f973986b8
Parents: 95bf7a8
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 15:11:12 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 27 10:48:34 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/util/DoFnInfo.java    | 43 ++++++++++++++++++--
 1 file changed, 39 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/73db5608/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
index b211c04..bfa12e2 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java
@@ -17,29 +17,38 @@
  */
 package org.apache.beam.runners.dataflow.util;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import java.io.Serializable;
 import java.util.Map;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
 /**
- * Wrapper class holding the necessary information to serialize a {@link OldDoFn}.
+ * Wrapper class holding the necessary information to serialize a {@link OldDoFn}
+ * or {@link DoFn}.
  *
  * @param <InputT> the type of the (main) input elements of the {@link OldDoFn}
  * @param <OutputT> the type of the (main) output elements of the {@link OldDoFn}
  */
 public class DoFnInfo<InputT, OutputT> implements Serializable {
-  private final OldDoFn<InputT, OutputT> doFn;
+  private final Serializable doFn;
   private final WindowingStrategy<?, ?> windowingStrategy;
   private final Iterable<PCollectionView<?>> sideInputViews;
   private final Coder<InputT> inputCoder;
   private final long mainOutput;
   private final Map<Long, TupleTag<?>> outputMap;
 
-  public DoFnInfo(OldDoFn<InputT, OutputT> doFn,
+  /**
+   * Creates a {@link DoFnInfo} for the given {@link DoFn} or {@link OldDoFn} and auxiliary bits and
+   * pieces.
+   */
+  public DoFnInfo(
+      Serializable doFn,
       WindowingStrategy<?, ?> windowingStrategy,
       Iterable<PCollectionView<?>> sideInputViews,
       Coder<InputT> inputCoder,
@@ -53,10 +62,36 @@ public class DoFnInfo<InputT, OutputT> implements Serializable {
     this.outputMap = outputMap;
   }
 
-  public OldDoFn<InputT, OutputT> getDoFn() {
+  /**
+   * @deprecated call the constructor with a {@link Serializable}
+   */
+  @Deprecated
+  public DoFnInfo(
+      OldDoFn doFn,
+      WindowingStrategy<?, ?> windowingStrategy,
+      Iterable<PCollectionView<?>> sideInputViews,
+      Coder<InputT> inputCoder,
+      long mainOutput,
+      Map<Long, TupleTag<?>> outputMap) {
+    this((Serializable) doFn, windowingStrategy, sideInputViews, inputCoder, mainOutput, outputMap);
+  }
+
+  /** Returns the embedded serialized function. It may be a {@code DoFn} or {@code OldDoFn}. */
+  public Serializable getFn() {
     return doFn;
   }
 
+  /** @deprecated use {@link #getFn()} */
+  @Deprecated
+  public OldDoFn getDoFn() {
+    checkState(
+        doFn instanceof OldDoFn,
+        "Deprecated %s.getDoFn() called when the payload was actually a new %s",
+        DoFnInfo.class.getSimpleName(),
+        DoFn.class.getSimpleName());
+    return (OldDoFn) doFn;
+  }
+
   public WindowingStrategy<?, ?> getWindowingStrategy() {
     return windowingStrategy;
   }


[15/50] incubator-beam git commit: [BEAM-809] Create a KryoRegistrator for the SparkRunner.

Posted by th...@apache.org.
[BEAM-809] Create a KryoRegistrator for the SparkRunner.

Use Class#getName() instead of canonicalName().


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

Branch: refs/heads/apex-runner
Commit: 13b83858746356068a6d618e04da6839e837d28c
Parents: 53fe3ee
Author: Sela <an...@paypal.com>
Authored: Mon Oct 24 22:35:39 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 18:53:28 2016 +0300

----------------------------------------------------------------------
 runners/spark/pom.xml                           | 23 ++++++++++
 .../coders/BeamSparkRunnerRegistrator.java      | 46 ++++++++++++++++++++
 .../spark/translation/SparkContextFactory.java  |  5 ++-
 3 files changed, 73 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/13b83858/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index ccec3c6..458205a 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -147,6 +147,29 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>com.esotericsoftware.kryo</groupId>
+      <artifactId>kryo</artifactId>
+      <version>2.21</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>de.javakaffee</groupId>
+      <artifactId>kryo-serializers</artifactId>
+      <version>0.39</version>
+      <exclusions>
+        <!-- Use Spark's Kryo -->
+        <exclusion>
+          <groupId>com.esotericsoftware</groupId>
+          <artifactId>kryo</artifactId>
+        </exclusion>
+        <!-- We only really need the serializer implementations -->
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-java</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>com.google.code.findbugs</groupId>
       <artifactId>jsr305</artifactId>
       <version>1.3.9</version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/13b83858/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java
new file mode 100644
index 0000000..0e62781
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.coders;
+
+import com.esotericsoftware.kryo.Kryo;
+import de.javakaffee.kryoserializers.UnmodifiableCollectionsSerializer;
+import de.javakaffee.kryoserializers.guava.ImmutableListSerializer;
+import de.javakaffee.kryoserializers.guava.ImmutableMapSerializer;
+import de.javakaffee.kryoserializers.guava.ImmutableMultimapSerializer;
+import de.javakaffee.kryoserializers.guava.ImmutableSetSerializer;
+import de.javakaffee.kryoserializers.guava.ReverseListSerializer;
+import org.apache.spark.serializer.KryoRegistrator;
+
+
+/**
+ * Custom {@link com.esotericsoftware.kryo.Serializer}s for Beam's Spark runner needs.
+ */
+public class BeamSparkRunnerRegistrator implements KryoRegistrator {
+
+  @Override
+  public void registerClasses(Kryo kryo) {
+    UnmodifiableCollectionsSerializer.registerSerializers(kryo);
+    // Guava
+    ImmutableListSerializer.registerSerializers(kryo);
+    ImmutableSetSerializer.registerSerializers(kryo);
+    ImmutableMapSerializer.registerSerializers(kryo);
+    ImmutableMultimapSerializer.registerSerializers(kryo);
+    ReverseListSerializer.registerSerializers(kryo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/13b83858/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
index 4877f6e..ee2104a 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
@@ -19,6 +19,7 @@
 package org.apache.beam.runners.spark.translation;
 
 import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.coders.BeamSparkRunnerRegistrator;
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.serializer.KryoSerializer;
@@ -85,7 +86,9 @@ public final class SparkContextFactory {
         conf.setMaster(options.getSparkMaster());
       }
       conf.setAppName(options.getAppName());
-      conf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
+      // register immutable collections serializers because the SDK uses them.
+      conf.set("spark.kryo.registrator", BeamSparkRunnerRegistrator.class.getName());
+      conf.set("spark.serializer", KryoSerializer.class.getName());
       return new JavaSparkContext(conf);
     }
   }


[22/50] incubator-beam git commit: [BEAM-658] Support Read.Unbounded primitive.

Posted by th...@apache.org.
[BEAM-658] Support Read.Unbounded primitive.

Changed mapSourceFunction to use scala's native Option.

Upgrade to kryo-serializers 0.39 that provides support for ReverseList (used by Top).

Better logging.

Allow a longer read time-frame for read in tests.

Assert initial parallelism is gerater than zero.

Add OnBatchCompleted listener that writes to Kafka.

Test latest.

Give read from Kafka more time to avoid flaky tests.

Change log level in MicrobatchSource.Reader to debug.


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

Branch: refs/heads/apex-runner
Commit: b72e7e34f70527e4ee575aaf0bc8c4c31ed31b0a
Parents: c94b8ea
Author: Sela <an...@paypal.com>
Authored: Thu Sep 29 16:30:32 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Wed Oct 26 20:51:16 2016 +0300

----------------------------------------------------------------------
 runners/spark/pom.xml                           |  44 ++--
 .../runners/spark/SparkPipelineOptions.java     |  11 +
 .../runners/spark/io/EmptyCheckpointMark.java   |  52 ++++
 .../apache/beam/runners/spark/io/KafkaIO.java   | 131 ----------
 .../beam/runners/spark/io/MicrobatchSource.java | 262 +++++++++++++++++++
 .../beam/runners/spark/io/SourceDStream.java    | 156 +++++++++++
 .../apache/beam/runners/spark/io/SourceRDD.java |  75 +++++-
 .../runners/spark/io/SparkUnboundedSource.java  | 167 ++++++++++++
 .../spark/stateful/StateSpecFunctions.java      | 167 ++++++++++++
 .../runners/spark/stateful/package-info.java    |  22 ++
 .../spark/translation/TranslationUtils.java     |  17 ++
 .../SparkRunnerStreamingContextFactory.java     |   7 +
 .../streaming/StreamingTransformTranslator.java |  37 +--
 .../streaming/KafkaStreamingTest.java           | 150 ++++++++---
 .../ResumeFromCheckpointStreamingTest.java      |  34 ++-
 .../streaming/utils/EmbeddedKafkaCluster.java   |  51 ++--
 .../utils/KafkaWriteOnBatchCompleted.java       | 105 ++++++++
 17 files changed, 1224 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 458205a..36ce7b5 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -36,7 +36,7 @@
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <spark.version>1.6.2</spark.version>
     <hadoop.version>2.2.0</hadoop.version>
-    <kafka.version>0.8.2.1</kafka.version>
+    <kafka.version>0.9.0.1</kafka.version>
     <dropwizard.metrics.version>3.1.2</dropwizard.metrics.version>
   </properties>
 
@@ -112,29 +112,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming-kafka_2.10</artifactId>
-      <version>${spark.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
       <artifactId>spark-network-common_2.10</artifactId>
       <version>${spark.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka_2.10</artifactId>
-      <version>${kafka.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka-clients</artifactId>
-      <version>${kafka.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
@@ -208,6 +190,11 @@
       <artifactId>joda-time</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <version>${google-clients.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-compress</artifactId>
       <version>1.9</version>
@@ -272,6 +259,18 @@
       <version>${dropwizard.metrics.version}</version>
     </dependency>
 
+    <!-- KafkaIO -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-kafka</artifactId>
+      <version>0.3.0-incubating-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <version>0.9.0.1</version>
+    </dependency>
+
     <!-- test dependencies -->
     <dependency>
       <groupId>junit</groupId>
@@ -284,12 +283,17 @@
         </exclusion>
       </exclusions>
     </dependency>
-
     <dependency>
       <groupId>org.hamcrest</groupId>
       <artifactId>hamcrest-all</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_2.10</artifactId>
+      <version>${kafka.version}</version>
+      <scope>test</scope>
+    </dependency>
 
     <!-- Depend on test jar to scan for RunnableOnService tests -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
index 4d9d51f..08e14fe 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java
@@ -52,6 +52,17 @@ public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions,
   Long getBatchIntervalMillis();
   void setBatchIntervalMillis(Long batchInterval);
 
+  @Description("Minimum time to spend on read, for each micro-batch.")
+  @Default.Long(200)
+  Long getMinReadTimeMillis();
+  void setMinReadTimeMillis(Long minReadTimeMillis);
+
+  @Description("A value between 0-1 to describe the percentage of a micro-batch dedicated "
+      + "to reading from UnboundedSource.")
+  @Default.Double(0.1)
+  Double getReadTimePercentage();
+  void setReadTimePercentage(Double readTimePercentage);
+
   @Description("A checkpoint directory for streaming resilience, ignored in batch. "
       + "For durability, a reliable filesystem such as HDFS/S3/GS is necessary.")
   @Default.InstanceFactory(TmpCheckpointDirFactory.class)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/EmptyCheckpointMark.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/EmptyCheckpointMark.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/EmptyCheckpointMark.java
new file mode 100644
index 0000000..a4ab379
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/EmptyCheckpointMark.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.beam.sdk.io.UnboundedSource;
+
+
+/**
+ * Passing null values to Spark's Java API may cause problems because of Guava preconditions.
+ * See: {@link org.apache.spark.api.java.JavaUtils#optionToOptional}
+ */
+public class EmptyCheckpointMark implements UnboundedSource.CheckpointMark, Serializable {
+  private static final EmptyCheckpointMark INSTANCE = new EmptyCheckpointMark();
+  private static final int ID = 2654265; // some constant to serve as identifier.
+
+  private EmptyCheckpointMark() {};
+
+  public static EmptyCheckpointMark get() {
+    return INSTANCE;
+  }
+
+  @Override
+  public void finalizeCheckpoint() throws IOException { }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj instanceof EmptyCheckpointMark;
+  }
+
+  @Override
+  public int hashCode() {
+    return ID;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
deleted file mode 100644
index 8cf2083..0000000
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.spark.io;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Map;
-import java.util.Set;
-import kafka.serializer.Decoder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Read stream from Kafka.
- */
-public final class KafkaIO {
-
-  private KafkaIO() {
-  }
-
-  /**
-   * Read operation from Kafka topics.
-   */
-  public static final class Read {
-
-    private Read() {
-    }
-
-    /**
-     * Define the Kafka consumption.
-     *
-     * @param keyDecoder    {@link Decoder} to decode the Kafka message key
-     * @param valueDecoder  {@link Decoder} to decode the Kafka message value
-     * @param key           Kafka message key Class
-     * @param value         Kafka message value Class
-     * @param topics        Kafka topics to subscribe
-     * @param kafkaParams   map of Kafka parameters
-     * @param <K>           Kafka message key Class type
-     * @param <V>           Kafka message value Class type
-     * @return KafkaIO Unbound input
-     */
-    public static <K, V> Unbound<K, V> from(Class<? extends Decoder<K>> keyDecoder,
-                                            Class<? extends Decoder<V>> valueDecoder,
-                                            Class<K> key,
-                                            Class<V> value, Set<String> topics,
-                                            Map<String, String> kafkaParams) {
-      return new Unbound<>(keyDecoder, valueDecoder, key, value, topics, kafkaParams);
-    }
-
-    /**
-     * A {@link PTransform} reading from Kafka topics and providing {@link PCollection}.
-     */
-    public static class Unbound<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
-
-      private final Class<? extends Decoder<K>> keyDecoderClass;
-      private final Class<? extends Decoder<V>> valueDecoderClass;
-      private final Class<K> keyClass;
-      private final Class<V> valueClass;
-      private final Set<String> topics;
-      private final Map<String, String> kafkaParams;
-
-      Unbound(Class<? extends Decoder<K>> keyDecoder,
-              Class<? extends Decoder<V>> valueDecoder, Class<K> key,
-              Class<V> value, Set<String> topics, Map<String, String> kafkaParams) {
-        checkNotNull(keyDecoder, "need to set the key decoder class of a KafkaIO.Read transform");
-        checkNotNull(
-            valueDecoder, "need to set the value decoder class of a KafkaIO.Read transform");
-        checkNotNull(key, "need to set the key class of a KafkaIO.Read transform");
-        checkNotNull(value, "need to set the value class of a KafkaIO.Read transform");
-        checkNotNull(topics, "need to set the topics of a KafkaIO.Read transform");
-        checkNotNull(kafkaParams, "need to set the kafkaParams of a KafkaIO.Read transform");
-        this.keyDecoderClass = keyDecoder;
-        this.valueDecoderClass = valueDecoder;
-        this.keyClass = key;
-        this.valueClass = value;
-        this.topics = topics;
-        this.kafkaParams = kafkaParams;
-      }
-
-      public Class<? extends Decoder<K>> getKeyDecoderClass() {
-        return keyDecoderClass;
-      }
-
-      public Class<? extends Decoder<V>> getValueDecoderClass() {
-        return valueDecoderClass;
-      }
-
-      public Class<V> getValueClass() {
-        return valueClass;
-      }
-
-      public Class<K> getKeyClass() {
-        return keyClass;
-      }
-
-      public Set<String> getTopics() {
-        return topics;
-      }
-
-      public Map<String, String> getKafkaParams() {
-        return kafkaParams;
-      }
-
-      @Override
-      public PCollection<KV<K, V>> apply(PBegin input) {
-        // Spark streaming micro batches are bounded by default
-        return PCollection.createPrimitiveOutputInternal(input.getPipeline(),
-            WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
new file mode 100644
index 0000000..4a174aa
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import com.google.api.client.util.BackOff;
+import com.google.common.util.concurrent.Uninterruptibles;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Mostly based on {@link org.apache.beam.sdk.io.BoundedReadFromUnboundedSource},
+ * with some adjustments for this specific use-case.
+ *
+ * <p>A {@link BoundedSource} wrapping an {@link UnboundedSource} to complement Spark's micro-batch
+ * nature.
+ *
+ * <p>By design, Spark's micro-batches are bounded by their duration. Spark also provides a
+ * back-pressure mechanism that may signal a bound by max records.
+ */
+public class MicrobatchSource<T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+    extends BoundedSource<T> {
+  private static final Logger LOG = LoggerFactory.getLogger(MicrobatchSource.class);
+
+  private final UnboundedSource<T, CheckpointMarkT> source;
+  private final Duration maxReadTime;
+  private final int numInitialSplits;
+  private final long maxNumRecords;
+
+  // each split of the underlying UnboundedSource is associated with a (consistent) id
+  // to match it's corresponding CheckpointMark state.
+  private final int splitId;
+
+  MicrobatchSource(UnboundedSource<T, CheckpointMarkT> source,
+                   Duration maxReadTime,
+                   int numInitialSplits,
+                   long maxNumRecords,
+                   int splitId) {
+    this.source = source;
+    this.maxReadTime = maxReadTime;
+    this.numInitialSplits = numInitialSplits;
+    this.maxNumRecords = maxNumRecords;
+    this.splitId = splitId;
+  }
+
+  /**
+   * Divide the given number of records into {@code numSplits} approximately
+   * equal parts that sum to {@code numRecords}.
+   */
+  private static long[] splitNumRecords(long numRecords, int numSplits) {
+    long[] splitNumRecords = new long[numSplits];
+    for (int i = 0; i < numSplits; i++) {
+      splitNumRecords[i] = numRecords / numSplits;
+    }
+    for (int i = 0; i < numRecords % numSplits; i++) {
+      splitNumRecords[i] = splitNumRecords[i] + 1;
+    }
+    return splitNumRecords;
+  }
+
+  @Override
+  public List<? extends BoundedSource<T>>
+      splitIntoBundles(long desiredBundleSizeBytes,
+                       PipelineOptions options) throws Exception {
+    List<MicrobatchSource<T, CheckpointMarkT>> result = new ArrayList<>();
+    List<? extends UnboundedSource<T, CheckpointMarkT>> splits =
+        source.generateInitialSplits(numInitialSplits, options);
+    int numSplits = splits.size();
+    long[] numRecords = splitNumRecords(maxNumRecords, numSplits);
+    for (int i = 0; i < numSplits; i++) {
+      // splits must be stable, and cannot change during consecutive executions
+      // for example: Kafka should not add partitions if more then one topic is read.
+      result.add(new MicrobatchSource<>(splits.get(i), maxReadTime, 1, numRecords[i], i));
+    }
+    return result;
+  }
+
+  @Override
+  public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+    return 0;
+  }
+
+  @Override
+  public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+    return false;
+  }
+
+  @Override
+  public BoundedReader<T> createReader(PipelineOptions options) throws IOException {
+    return createReader(options, null);
+  }
+
+  public BoundedReader<T> createReader(PipelineOptions options, CheckpointMarkT checkpointMark)
+      throws IOException {
+    return new Reader(source.createReader(options, checkpointMark));
+  }
+
+  @Override
+  public void validate() {
+    source.validate();
+  }
+
+  @Override
+  public Coder<T> getDefaultOutputCoder() {
+    return source.getDefaultOutputCoder();
+  }
+
+  public Coder<CheckpointMarkT> getCheckpointMarkCoder() {
+    return source.getCheckpointMarkCoder();
+  }
+
+  public int getSplitId() {
+    return splitId;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof MicrobatchSource)) {
+      return false;
+    }
+    MicrobatchSource<?, ?> that = (MicrobatchSource<?, ?>) o;
+
+    return splitId == that.splitId;
+  }
+
+  @Override
+  public int hashCode() {
+    return splitId;
+  }
+
+  /**
+   * A {@link BoundedSource.BoundedReader}
+   * wrapping an {@link UnboundedSource.UnboundedReader}.
+   *
+   * <p>This Reader will read until it reached the bound of duration, or max records,
+   * whichever comes first.
+   */
+  public class Reader extends BoundedSource.BoundedReader<T> {
+    private long recordsRead = 0L;
+    private final Instant endTime;
+    private final FluentBackoff backoffFactory;
+    private final UnboundedSource.UnboundedReader<T> reader;
+
+    private Reader(UnboundedSource.UnboundedReader<T> reader) {
+      endTime = Instant.now().plus(maxReadTime);
+      this.reader = reader;
+      backoffFactory =
+          FluentBackoff.DEFAULT
+              .withInitialBackoff(Duration.millis(10))
+              .withMaxBackoff(maxReadTime.minus(1))
+              .withMaxCumulativeBackoff(maxReadTime.minus(1));
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      LOG.debug("MicrobatchReader-{}: Starting a microbatch read from an unbounded source with a "
+          + "max read time of {} msec, and max number of records {}.", splitId, maxReadTime,
+              maxNumRecords);
+      if (reader.start()) {
+        recordsRead++;
+        return true;
+      } else {
+        return advanceWithBackoff();
+      }
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      if (recordsRead >= maxNumRecords) {
+        finalizeCheckpoint();
+        return false;
+      }
+      return advanceWithBackoff();
+    }
+
+    private boolean advanceWithBackoff() throws IOException {
+      // Try reading from the source with exponential backoff
+      BackOff backoff = backoffFactory.backoff();
+      long nextSleep = backoff.nextBackOffMillis();
+      while (nextSleep != BackOff.STOP) {
+        if (endTime != null && Instant.now().isAfter(endTime)) {
+          finalizeCheckpoint();
+          return false;
+        }
+        if (reader.advance()) {
+          recordsRead++;
+          return true;
+        }
+        Uninterruptibles.sleepUninterruptibly(nextSleep, TimeUnit.MILLISECONDS);
+        nextSleep = backoff.nextBackOffMillis();
+      }
+      finalizeCheckpoint();
+      return false;
+    }
+
+    private void finalizeCheckpoint() throws IOException {
+      reader.getCheckpointMark().finalizeCheckpoint();
+      LOG.debug("MicrobatchReader-{}: finalized CheckpointMark successfully after "
+          + "reading {} records.", splitId, recordsRead);
+    }
+
+    @Override
+    public T getCurrent() throws NoSuchElementException {
+      return reader.getCurrent();
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() throws NoSuchElementException {
+      return reader.getCurrentTimestamp();
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public BoundedSource<T> getCurrentSource() {
+      return MicrobatchSource.this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public CheckpointMarkT getCheckpointMark() {
+      return (CheckpointMarkT) reader.getCheckpointMark();
+    }
+
+    public long getNumRecordsRead() {
+      return recordsRead;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
new file mode 100644
index 0000000..4e47757
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.spark.api.java.JavaSparkContext$;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.streaming.StreamingContext;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.dstream.InputDStream;
+import org.apache.spark.streaming.scheduler.RateController;
+import org.apache.spark.streaming.scheduler.RateController$;
+import org.apache.spark.streaming.scheduler.rate.RateEstimator;
+import org.apache.spark.streaming.scheduler.rate.RateEstimator$;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Tuple2;
+
+
+
+/**
+ * A {@link SourceDStream} is an {@link InputDStream} of {@link SourceRDD.Unbounded}s.
+ *
+ * <p>This InputDStream will create a stream of partitioned {@link UnboundedSource}s,
+ * and their respective, (optional) starting {@link UnboundedSource.CheckpointMark}.
+ *
+ * <p>The underlying Source is actually a {@link MicrobatchSource} with bounds on read duration,
+ * and max records. Both set here.
+ * Read duration bound is affected by {@link SparkPipelineOptions#getReadTimePercentage()} and
+ * {@link SparkPipelineOptions#getMinReadTimeMillis()}.
+ * Records bound is controlled by the {@link RateController} mechanism.
+ */
+public class SourceDStream<T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+      extends InputDStream<Tuple2<Source<T>, CheckpointMarkT>> {
+  private static final Logger LOG = LoggerFactory.getLogger(SourceDStream.class);
+
+  private final UnboundedSource<T, CheckpointMarkT> unboundedSource;
+  private final SparkRuntimeContext runtimeContext;
+  private final Duration boundReadDuration;
+  // the initial parallelism, set by Spark's backend, will be determined once when the job starts.
+  // in case of resuming/recovering from checkpoint, the DStream will be reconstructed and this
+  // property should not be reset.
+  private final int initialParallelism;
+
+  public SourceDStream(StreamingContext ssc,
+                       UnboundedSource<T, CheckpointMarkT> unboundedSource,
+                       SparkRuntimeContext runtimeContext) {
+    super(ssc, JavaSparkContext$.MODULE$.<scala.Tuple2<Source<T>, CheckpointMarkT>>fakeClassTag());
+    this.unboundedSource = unboundedSource;
+    this.runtimeContext = runtimeContext;
+    SparkPipelineOptions options = runtimeContext.getPipelineOptions().as(
+        SparkPipelineOptions.class);
+    this.boundReadDuration = boundReadDuration(options.getReadTimePercentage(),
+        options.getMinReadTimeMillis());
+    // set initial parallelism once.
+    this.initialParallelism = ssc().sc().defaultParallelism();
+    checkArgument(this.initialParallelism > 0, "Number of partitions must be greater than zero.");
+  }
+
+  @Override
+  public scala.Option<RDD<Tuple2<Source<T>, CheckpointMarkT>>> compute(Time validTime) {
+    MicrobatchSource<T, CheckpointMarkT> microbatchSource = new MicrobatchSource<>(
+        unboundedSource, boundReadDuration, initialParallelism, rateControlledMaxRecords(), -1);
+    RDD<scala.Tuple2<Source<T>, CheckpointMarkT>> rdd = new SourceRDD.Unbounded<>(
+        ssc().sc(), runtimeContext, microbatchSource);
+    return scala.Option.apply(rdd);
+  }
+
+  @Override
+  public void start() { }
+
+  @Override
+  public void stop() { }
+
+  @Override
+  public String name() {
+    return "Beam UnboundedSource [" + id() + "]";
+  }
+
+  //---- Bound by time.
+
+  // return the largest between the proportional read time (%batchDuration dedicated for read)
+  // and the min. read time set.
+  private Duration boundReadDuration(double readTimePercentage, long minReadTimeMillis) {
+    long batchDurationMillis = ssc().graph().batchDuration().milliseconds();
+    Duration proportionalDuration = new Duration(Math.round(
+        batchDurationMillis * readTimePercentage));
+    Duration lowerBoundDuration = new Duration(minReadTimeMillis);
+    Duration readDuration = proportionalDuration.isLongerThan(lowerBoundDuration)
+        ? proportionalDuration : lowerBoundDuration;
+    LOG.info("Read duration set to: " + readDuration);
+    return readDuration;
+  }
+
+  //---- Bound by records.
+
+  private long rateControlledMaxRecords() {
+    scala.Option<RateController> rateControllerOption = rateController();
+    if (rateControllerOption.isDefined()) {
+      long rateLimitPerSecond = rateControllerOption.get().getLatestRate();
+      if (rateLimitPerSecond > 0) {
+        long totalRateLimit =
+            rateLimitPerSecond * (ssc().graph().batchDuration().milliseconds() / 1000);
+        LOG.info("RateController set limit to {}", totalRateLimit);
+        return totalRateLimit;
+      }
+    }
+    LOG.info("RateController had nothing to report, default is Long.MAX_VALUE");
+    return Long.MAX_VALUE;
+  }
+
+  private final RateController rateController = new SourceRateController(id(),
+      RateEstimator$.MODULE$.create(ssc().conf(), ssc().graph().batchDuration()));
+
+  @Override
+  public scala.Option<RateController> rateController() {
+    if (RateController$.MODULE$.isBackPressureEnabled(ssc().conf())) {
+      return scala.Option.apply(rateController);
+    } else {
+      return scala.Option.empty();
+    }
+  }
+
+  private static class SourceRateController extends RateController {
+
+    private SourceRateController(int id, RateEstimator rateEstimator) {
+      super(id, rateEstimator);
+    }
+
+    @Override
+    public void publish(long rate) { }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
index 679b8b1..3995c89 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
 import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.spark.Dependency;
 import org.apache.spark.InterruptibleIterator;
@@ -46,7 +47,8 @@ import org.slf4j.LoggerFactory;
 public class SourceRDD {
 
   /**
-   * A SourceRDD.Bounded reads input from a {@link BoundedSource} and creates a Spark {@link RDD}.
+   * A {@link SourceRDD.Bounded} reads input from a {@link BoundedSource}
+   * and creates a Spark {@link RDD}.
    * This is the default way for the SparkRunner to read data from Beam's BoundedSources.
    */
   public static class Bounded<T> extends RDD<WindowedValue<T>> {
@@ -196,5 +198,76 @@ public class SourceRDD {
     public int hashCode() {
       return 41 * (41 + rddId) + index;
     }
+
+    public Source<T> getSource() {
+      return source;
+    }
+  }
+
+  /**
+   * A {@link SourceRDD.Unbounded} is the implementation of a micro-batch
+   * in a {@link SourceDStream}.
+   *
+   * <p>This RDD is made of P partitions, each containing a single pair-element of the partitioned
+   * {@link MicrobatchSource} and an optional starting {@link UnboundedSource.CheckpointMark}.
+   */
+  public static class Unbounded<T, CheckpointMarkT extends
+        UnboundedSource.CheckpointMark> extends RDD<scala.Tuple2<Source<T>, CheckpointMarkT>> {
+    private final MicrobatchSource<T, CheckpointMarkT> microbatchSource;
+    private final SparkRuntimeContext runtimeContext;
+
+    private static final scala.collection.immutable.List<Dependency<?>> NIL =
+        scala.collection.immutable.List.empty();
+
+    public Unbounded(SparkContext sc,
+                     SparkRuntimeContext runtimeContext,
+                     MicrobatchSource<T, CheckpointMarkT> microbatchSource) {
+      super(sc, NIL,
+          JavaSparkContext$.MODULE$.<scala.Tuple2<Source<T>, CheckpointMarkT>>fakeClassTag());
+      this.runtimeContext = runtimeContext;
+      this.microbatchSource = microbatchSource;
+    }
+
+    @Override
+    public Partition[] getPartitions() {
+      try {
+        List<? extends Source<T>> partitionedSources = microbatchSource.splitIntoBundles(
+            -1 /* ignored */, runtimeContext.getPipelineOptions());
+        Partition[] partitions = new CheckpointableSourcePartition[partitionedSources.size()];
+        for (int i = 0; i < partitionedSources.size(); i++) {
+          partitions[i] = new CheckpointableSourcePartition<>(id(), i, partitionedSources.get(i),
+              EmptyCheckpointMark.get());
+        }
+        return partitions;
+      } catch (Exception e) {
+        throw new RuntimeException("Failed to create partitions.", e);
+      }
+    }
+
+    @Override
+    public scala.collection.Iterator<scala.Tuple2<Source<T>, CheckpointMarkT>>
+    compute(Partition split, TaskContext context) {
+      @SuppressWarnings("unchecked")
+      CheckpointableSourcePartition<T, CheckpointMarkT> partition =
+          (CheckpointableSourcePartition<T, CheckpointMarkT>) split;
+      scala.Tuple2<Source<T>, CheckpointMarkT> tuple2 =
+          new scala.Tuple2<>(partition.getSource(), partition.checkpointMark);
+      return scala.collection.JavaConversions.asScalaIterator(
+          Collections.singleton(tuple2).iterator());
+    }
+  }
+
+  /** A {@link SourcePartition} with a {@link UnboundedSource.CheckpointMark}. */
+  private static class CheckpointableSourcePartition<T, CheckpointMarkT extends
+      UnboundedSource.CheckpointMark> extends SourcePartition<T> {
+    private final CheckpointMarkT checkpointMark;
+
+    CheckpointableSourcePartition(int rddId,
+                                  int index,
+                                  Source<T> source,
+                                  CheckpointMarkT checkpointMark) {
+      super(rddId, index, source);
+      this.checkpointMark = checkpointMark;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
new file mode 100644
index 0000000..b12098d
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.io;
+
+import java.util.Collections;
+import java.util.Iterator;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.stateful.StateSpecFunctions;
+import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
+import org.apache.beam.runners.spark.translation.TranslationUtils;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.spark.api.java.JavaSparkContext$;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.StateSpec;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaMapWithStateDStream;
+import org.apache.spark.streaming.api.java.JavaPairInputDStream;
+import org.apache.spark.streaming.api.java.JavaPairInputDStream$;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.dstream.DStream;
+import org.apache.spark.streaming.scheduler.StreamInputInfo;
+
+
+/**
+ * A "composite" InputDStream implementation for {@link UnboundedSource}s.
+ *
+ * <p>This read is a composite of the following steps:
+ * <ul>
+ * <li>Create a single-element (per-partition) stream, that contains the (partitioned)
+ * {@link Source} and an optional {@link UnboundedSource.CheckpointMark} to start from.</li>
+ * <li>Read from within a stateful operation {@link JavaPairInputDStream#mapWithState(StateSpec)}
+ * using the {@link StateSpecFunctions#mapSourceFunction(SparkRuntimeContext)} mapping function,
+ * which manages the state of the CheckpointMark per partition.</li>
+ * <li>Since the stateful operation is a map operation, the read iterator needs to be flattened,
+ * while reporting the properties of the read (such as number of records) to the tracker.</li>
+ * </ul>
+ */
+public class SparkUnboundedSource {
+
+  public static <T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+  JavaDStream<WindowedValue<T>> read(JavaStreamingContext jssc,
+                                     SparkRuntimeContext rc,
+                                     UnboundedSource<T, CheckpointMarkT> source) {
+    JavaPairInputDStream<Source<T>, CheckpointMarkT> inputDStream =
+        JavaPairInputDStream$.MODULE$.fromInputDStream(new SourceDStream<>(jssc.ssc(), source, rc),
+            JavaSparkContext$.MODULE$.<Source<T>>fakeClassTag(),
+                JavaSparkContext$.MODULE$.<CheckpointMarkT>fakeClassTag());
+
+    // call mapWithState to read from a checkpointable sources.
+    //TODO: consider broadcasting the rc instead of re-sending every batch.
+    JavaMapWithStateDStream<Source<T>, CheckpointMarkT, byte[],
+        Iterator<WindowedValue<T>>> mapWithStateDStream = inputDStream.mapWithState(
+            StateSpec.function(StateSpecFunctions.<T, CheckpointMarkT>mapSourceFunction(rc)));
+
+    // set checkpoint duration for read stream, if set.
+    checkpointStream(mapWithStateDStream, rc);
+    // flatmap and report read elements. Use the inputDStream's id to tie between the reported
+    // info and the inputDStream it originated from.
+    int id = inputDStream.inputDStream().id();
+    ReportingFlatMappedDStream<WindowedValue<T>> reportingFlatMappedDStream =
+        new ReportingFlatMappedDStream<>(mapWithStateDStream.dstream(), id,
+            getSourceName(source, id));
+
+    return JavaDStream.fromDStream(reportingFlatMappedDStream,
+        JavaSparkContext$.MODULE$.<WindowedValue<T>>fakeClassTag());
+  }
+
+  private static <T> String getSourceName(Source<T> source, int id) {
+    StringBuilder sb = new StringBuilder();
+    for (String s: source.getClass().getSimpleName().replace("$", "").split("(?=[A-Z])")) {
+      String trimmed = s.trim();
+      if (!trimmed.isEmpty()) {
+        sb.append(trimmed).append(" ");
+      }
+    }
+    return sb.append("[").append(id).append("]").toString();
+  }
+
+  private static void checkpointStream(JavaDStream<?> dStream,
+                                       SparkRuntimeContext rc) {
+    long checkpointDurationMillis = rc.getPipelineOptions().as(SparkPipelineOptions.class)
+        .getCheckpointDurationMillis();
+    if (checkpointDurationMillis > 0) {
+      dStream.checkpoint(new Duration(checkpointDurationMillis));
+    }
+  }
+
+  /**
+   * A flatMap DStream function that "flattens" the Iterators read by the
+   * {@link MicrobatchSource.Reader}s, while reporting the properties of the read to the
+   * {@link org.apache.spark.streaming.scheduler.InputInfoTracker} for RateControl purposes
+   * and visibility.
+   */
+  private static class ReportingFlatMappedDStream<T> extends DStream<T> {
+    private final DStream<Iterator<T>> parent;
+    private final int inputDStreamId;
+    private final String sourceName;
+
+    ReportingFlatMappedDStream(DStream<Iterator<T>> parent,
+                               int inputDStreamId,
+                               String sourceName) {
+      super(parent.ssc(), JavaSparkContext$.MODULE$.<T>fakeClassTag());
+      this.parent = parent;
+      this.inputDStreamId = inputDStreamId;
+      this.sourceName = sourceName;
+    }
+
+    @Override
+    public Duration slideDuration() {
+      return parent.slideDuration();
+    }
+
+    @Override
+    public scala.collection.immutable.List<DStream<?>> dependencies() {
+      return scala.collection.JavaConversions.asScalaBuffer(
+          Collections.<DStream<?>>singletonList(parent)).toList();
+    }
+
+    @Override
+    public scala.Option<RDD<T>> compute(Time validTime) {
+      // compute parent.
+      scala.Option<RDD<Iterator<T>>> computedParentRDD = parent.getOrCompute(validTime);
+      // compute this DStream - take single-iterator partitions an flatMap them.
+      if (computedParentRDD.isDefined()) {
+        RDD<T> computedRDD = computedParentRDD.get().toJavaRDD()
+            .flatMap(TranslationUtils.<T>flattenIter()).rdd().cache();
+        // report - for RateEstimator and visibility.
+        report(validTime, computedRDD.count());
+        return scala.Option.apply(computedRDD);
+      } else {
+        report(validTime, 0);
+        return scala.Option.empty();
+      }
+    }
+
+    private void report(Time batchTime, long count) {
+      // metadata - #records read and a description.
+      scala.collection.immutable.Map<String, Object> metadata =
+          new scala.collection.immutable.Map.Map1<String, Object>(
+              StreamInputInfo.METADATA_KEY_DESCRIPTION(),
+                  String.format("Read %d records from %s for batch time: %s", count, sourceName,
+                      batchTime));
+      StreamInputInfo streamInputInfo = new StreamInputInfo(inputDStreamId, count, metadata);
+      ssc().scheduler().inputInfoTracker().reportInfo(batchTime, streamInputInfo);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
new file mode 100644
index 0000000..48849c2
--- /dev/null
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.spark.stateful;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Iterators;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.runners.spark.io.EmptyCheckpointMark;
+import org.apache.beam.runners.spark.io.MicrobatchSource;
+import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.spark.streaming.State;
+import org.apache.spark.streaming.StateSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Option;
+import scala.runtime.AbstractFunction3;
+
+/**
+ * A class containing {@link org.apache.spark.streaming.StateSpec} mappingFunctions.
+ */
+public class StateSpecFunctions {
+  private static final Logger LOG = LoggerFactory.getLogger(StateSpecFunctions.class);
+
+  /**
+   * A helper class that is essentially a {@link Serializable} {@link AbstractFunction3}.
+   */
+  private abstract static class SerializableFunction3<T1, T2, T3, T4>
+      extends AbstractFunction3<T1, T2, T3, T4> implements Serializable {
+  }
+
+  /**
+   * A {@link org.apache.spark.streaming.StateSpec} function to support reading from
+   * an {@link UnboundedSource}.
+   *
+   * <p>This StateSpec function expects the following:
+   * <ul>
+   * <li>Key: The (partitioned) Source to read from.</li>
+   * <li>Value: An optional {@link UnboundedSource.CheckpointMark} to start from.</li>
+   * <li>State: A byte representation of the (previously) persisted CheckpointMark.</li>
+   * </ul>
+   * And returns an iterator over all read values (for the micro-batch).
+   *
+   * <p>This stateful operation could be described as a flatMap over a single-element stream, which
+   * outputs all the elements read from the {@link UnboundedSource} for this micro-batch.
+   * Since micro-batches are bounded, the provided UnboundedSource is wrapped by a
+   * {@link MicrobatchSource} that applies bounds in the form of duration and max records
+   * (per micro-batch).
+   *
+   *
+   * <p>In order to avoid using Spark Guava's classes which pollute the
+   * classpath, we use the {@link StateSpec#function(scala.Function3)} signature which employs
+   * scala's native {@link scala.Option}, instead of the
+   * {@link StateSpec#function(org.apache.spark.api.java.function.Function3)} signature,
+   * which employs Guava's {@link com.google.common.base.Optional}.
+   *
+   * <p>See also <a href="https://issues.apache.org/jira/browse/SPARK-4819">SPARK-4819</a>.</p>
+   *
+   * @param runtimeContext    A serializable {@link SparkRuntimeContext}.
+   * @param <T>               The type of the input stream elements.
+   * @param <CheckpointMarkT> The type of the {@link UnboundedSource.CheckpointMark}.
+   * @return The appropriate {@link org.apache.spark.streaming.StateSpec} function.
+   */
+  public static <T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
+  scala.Function3<Source<T>, scala.Option<CheckpointMarkT>, /* CheckpointMarkT */State<byte[]>,
+      Iterator<WindowedValue<T>>> mapSourceFunction(final SparkRuntimeContext runtimeContext) {
+
+    return new SerializableFunction3<Source<T>, Option<CheckpointMarkT>, State<byte[]>,
+        Iterator<WindowedValue<T>>>() {
+
+      @Override
+      public Iterator<WindowedValue<T>> apply(Source<T> source, scala.Option<CheckpointMarkT>
+          startCheckpointMark, State<byte[]> state) {
+        // source as MicrobatchSource
+        MicrobatchSource<T, CheckpointMarkT> microbatchSource =
+            (MicrobatchSource<T, CheckpointMarkT>) source;
+
+        // if state exists, use it, otherwise it's first time so use the startCheckpointMark.
+        // startCheckpointMark may be EmptyCheckpointMark (the Spark Java API tries to apply
+        // Optional(null)), which is handled by the UnboundedSource implementation.
+        Coder<CheckpointMarkT> checkpointCoder = microbatchSource.getCheckpointMarkCoder();
+        CheckpointMarkT checkpointMark;
+        if (state.exists()) {
+          checkpointMark = CoderHelpers.fromByteArray(state.get(), checkpointCoder);
+          LOG.info("Continue reading from an existing CheckpointMark.");
+        } else if (startCheckpointMark.isDefined()
+            && !startCheckpointMark.get().equals(EmptyCheckpointMark.get())) {
+          checkpointMark = startCheckpointMark.get();
+          LOG.info("Start reading from a provided CheckpointMark.");
+        } else {
+          checkpointMark = null;
+          LOG.info("No CheckpointMark provided, start reading from default.");
+        }
+
+        // create reader.
+        BoundedSource.BoundedReader<T> reader;
+        try {
+          reader =
+              microbatchSource.createReader(runtimeContext.getPipelineOptions(), checkpointMark);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+
+        // read microbatch.
+        final List<WindowedValue<T>> readValues = new ArrayList<>();
+        try {
+          // measure how long a read takes per-partition.
+          Stopwatch stopwatch = Stopwatch.createStarted();
+          boolean finished = !reader.start();
+          while (!finished) {
+            readValues.add(WindowedValue.of(reader.getCurrent(), reader.getCurrentTimestamp(),
+                GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+            finished = !reader.advance();
+          }
+
+          // close and checkpoint reader.
+          reader.close();
+          LOG.info("Source id {} spent {} msec on reading.", microbatchSource.getSplitId(),
+              stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
+
+          // if the Source does not supply a CheckpointMark skip updating the state.
+          @SuppressWarnings("unchecked")
+          CheckpointMarkT finishedReadCheckpointMark =
+              (CheckpointMarkT) ((MicrobatchSource.Reader) reader).getCheckpointMark();
+          if (finishedReadCheckpointMark != null) {
+            state.update(CoderHelpers.toByteArray(finishedReadCheckpointMark, checkpointCoder));
+          } else {
+            LOG.info("Skipping checkpoint marking because the reader failed to supply one.");
+          }
+        } catch (IOException e) {
+          throw new RuntimeException("Failed to read from reader.", e);
+        }
+
+        return Iterators.unmodifiableIterator(readValues.iterator());
+      }
+    };
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
index 51937ca..647f8c3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
@@ -21,6 +21,7 @@ package org.apache.beam.runners.spark.translation;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import java.io.Serializable;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.runners.spark.util.BroadcastHelper;
@@ -37,6 +38,7 @@ import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.api.java.function.FlatMapFunction;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.PairFunction;
 import org.apache.spark.streaming.api.java.JavaDStream;
@@ -143,6 +145,21 @@ public final class TranslationUtils {
     };
   }
 
+  /** A Flatmap iterator function, flattening iterators into their elements. */
+  public static <T> FlatMapFunction<Iterator<T>, T> flattenIter() {
+    return new FlatMapFunction<Iterator<T>, T>() {
+      @Override
+      public Iterable<T> call(final Iterator<T> t) throws Exception {
+        return new Iterable<T>() {
+          @Override
+          public Iterator<T> iterator() {
+            return t;
+          }
+        };
+      }
+    };
+  }
+
   /**
    * A utility class to filter {@link TupleTag}s.
    *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java
index 79c87fb..2378788 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java
@@ -18,6 +18,8 @@
 
 package org.apache.beam.runners.spark.translation.streaming;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import java.net.MalformedURLException;
@@ -61,6 +63,11 @@ public class SparkRunnerStreamingContextFactory implements JavaStreamingContextF
   @Override
   public JavaStreamingContext create() {
     LOG.info("Creating a new Spark Streaming Context");
+    // validate unbounded read properties.
+    checkArgument(options.getMinReadTimeMillis() < options.getBatchIntervalMillis(),
+        "Minimum read time has to be less than batch time.");
+    checkArgument(options.getReadTimePercentage() > 0 && options.getReadTimePercentage() < 1,
+        "Read time percentage is bound to (0, 1).");
 
     SparkPipelineTranslator translator = new StreamingTransformTranslator.Translator(
         new TransformTranslator.Translator());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
index 1af5e07..71c27df 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -23,14 +23,12 @@ import com.google.common.collect.Maps;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import kafka.serializer.Decoder;
 import org.apache.beam.runners.core.AssignWindowsDoFn;
 import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
 import org.apache.beam.runners.spark.aggregators.NamedAggregators;
 import org.apache.beam.runners.spark.io.ConsoleIO;
 import org.apache.beam.runners.spark.io.CreateStream;
-import org.apache.beam.runners.spark.io.KafkaIO;
+import org.apache.beam.runners.spark.io.SparkUnboundedSource;
 import org.apache.beam.runners.spark.translation.DoFnFunction;
 import org.apache.beam.runners.spark.translation.EvaluationContext;
 import org.apache.beam.runners.spark.translation.GroupCombineFunctions;
@@ -44,6 +42,7 @@ import org.apache.beam.runners.spark.translation.WindowingHelpers;
 import org.apache.beam.runners.spark.util.BroadcastHelper;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.CombineWithContext;
 import org.apache.beam.sdk.transforms.Flatten;
@@ -74,11 +73,6 @@ import org.apache.spark.streaming.Durations;
 import org.apache.spark.streaming.api.java.JavaDStream;
 import org.apache.spark.streaming.api.java.JavaDStreamLike;
 import org.apache.spark.streaming.api.java.JavaPairDStream;
-import org.apache.spark.streaming.api.java.JavaPairInputDStream;
-import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.kafka.KafkaUtils;
-
-import scala.Tuple2;
 
 
 
@@ -103,28 +97,13 @@ public final class StreamingTransformTranslator {
     };
   }
 
-  private static <K, V> TransformEvaluator<KafkaIO.Read.Unbound<K, V>> kafka() {
-    return new TransformEvaluator<KafkaIO.Read.Unbound<K, V>>() {
+  private static <T> TransformEvaluator<Read.Unbounded<T>> readUnbounded() {
+    return new TransformEvaluator<Read.Unbounded<T>>() {
       @Override
-      public void evaluate(KafkaIO.Read.Unbound<K, V> transform, EvaluationContext context) {
+      public void evaluate(Read.Unbounded<T> transform, EvaluationContext context) {
         StreamingEvaluationContext sec = (StreamingEvaluationContext) context;
-        JavaStreamingContext jssc = sec.getStreamingContext();
-        Class<K> keyClazz = transform.getKeyClass();
-        Class<V> valueClazz = transform.getValueClass();
-        Class<? extends Decoder<K>> keyDecoderClazz = transform.getKeyDecoderClass();
-        Class<? extends Decoder<V>> valueDecoderClazz = transform.getValueDecoderClass();
-        Map<String, String> kafkaParams = transform.getKafkaParams();
-        Set<String> topics = transform.getTopics();
-        JavaPairInputDStream<K, V> inputPairStream = KafkaUtils.createDirectStream(jssc, keyClazz,
-                valueClazz, keyDecoderClazz, valueDecoderClazz, kafkaParams, topics);
-        JavaDStream<WindowedValue<KV<K, V>>> inputStream =
-            inputPairStream.map(new Function<Tuple2<K, V>, KV<K, V>>() {
-          @Override
-          public KV<K, V> call(Tuple2<K, V> t2) throws Exception {
-            return KV.of(t2._1(), t2._2());
-          }
-        }).map(WindowingHelpers.<KV<K, V>>windowFunction());
-        sec.setStream(transform, inputStream);
+        sec.setStream(transform, SparkUnboundedSource.read(sec.getStreamingContext(),
+            sec.getRuntimeContext(), transform.getSource()));
       }
     };
   }
@@ -455,6 +434,7 @@ public final class StreamingTransformTranslator {
       .newHashMap();
 
   static {
+    EVALUATORS.put(Read.Unbounded.class, readUnbounded());
     EVALUATORS.put(GroupByKey.class, groupByKey());
     EVALUATORS.put(Combine.GroupedValues.class, combineGrouped());
     EVALUATORS.put(Combine.Globally.class, combineGlobally());
@@ -463,7 +443,6 @@ public final class StreamingTransformTranslator {
     EVALUATORS.put(ParDo.BoundMulti.class, multiDo());
     EVALUATORS.put(ConsoleIO.Write.Unbound.class, print());
     EVALUATORS.put(CreateStream.QueuedValues.class, createFromQueue());
-    EVALUATORS.put(KafkaIO.Read.Unbound.class, kafka());
     EVALUATORS.put(Window.Bound.class, window());
     EVALUATORS.put(Flatten.FlattenPCollectionList.class, flattenPColl());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
index d72e70e..8728948 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java
@@ -19,20 +19,22 @@ package org.apache.beam.runners.spark.translation.streaming;
 
 import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
-import kafka.serializer.StringDecoder;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.io.KafkaIO;
 import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster;
+import org.apache.beam.runners.spark.translation.streaming.utils.KafkaWriteOnBatchCompleted;
 import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
 import org.apache.beam.runners.spark.translation.streaming.utils.TestOptionsForStreaming;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.RemoveDuplicates;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.KV;
@@ -41,6 +43,7 @@ import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.spark.streaming.api.java.JavaStreamingListener;
 import org.joda.time.Duration;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -53,32 +56,14 @@ import org.junit.rules.TemporaryFolder;
  */
 public class KafkaStreamingTest {
   private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
-          new EmbeddedKafkaCluster.EmbeddedZookeeper();
+      new EmbeddedKafkaCluster.EmbeddedZookeeper();
   private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER =
-          new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(), new Properties());
-  private static final String TOPIC = "kafka_beam_test_topic";
-  private static final Map<String, String> KAFKA_MESSAGES = ImmutableMap.of(
-      "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
-  );
-  private static final String[] EXPECTED = {"k1,v1", "k2,v2", "k3,v3", "k4,v4"};
+      new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection());
 
   @BeforeClass
   public static void init() throws IOException {
     EMBEDDED_ZOOKEEPER.startup();
     EMBEDDED_KAFKA_CLUSTER.startup();
-
-    // write to Kafka
-    Properties producerProps = new Properties();
-    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
-    producerProps.put("request.required.acks", 1);
-    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
-    Serializer<String> stringSerializer = new StringSerializer();
-    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
-        new KafkaProducer(producerProps, stringSerializer, stringSerializer)) {
-      for (Map.Entry<String, String> en : KAFKA_MESSAGES.entrySet()) {
-        kafkaProducer.send(new ProducerRecord<>(TOPIC, en.getKey(), en.getValue()));
-      }
-    }
   }
 
   @Rule
@@ -88,25 +73,122 @@ public class KafkaStreamingTest {
   public TestOptionsForStreaming commonOptions = new TestOptionsForStreaming();
 
   @Test
-  public void testRun() throws Exception {
+  public void testEarliest2Topics() throws Exception {
+    SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
+        checkpointParentDir.newFolder(getClass().getSimpleName()));
+    // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec,
+    // so to be on the safe side we'll set to 750 msec.
+    options.setMinReadTimeMillis(750L);
+    //--- setup
+    // two topics.
+    final String topic1 = "topic1";
+    final String topic2 = "topic2";
+    // messages.
+    final Map<String, String> messages = ImmutableMap.of(
+        "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
+    );
+    // expected.
+    final String[] expected = {"k1,v1", "k2,v2", "k3,v3", "k4,v4"};
+    // batch and window duration.
+    final Duration batchAndWindowDuration = Duration.standardSeconds(1);
+
+    // write to both topics ahead.
+    produce(topic1, messages);
+    produce(topic2, messages);
+
+    //------- test: read and dedup.
+    Pipeline p = Pipeline.create(options);
+
+    Map<String, Object> consumerProps = ImmutableMap.<String, Object>of(
+        "auto.offset.reset", "earliest"
+    );
+
+    KafkaIO.Read<String, String> read = KafkaIO.read()
+        .withBootstrapServers(EMBEDDED_KAFKA_CLUSTER.getBrokerList())
+        .withTopics(Arrays.asList(topic1, topic2))
+        .withKeyCoder(StringUtf8Coder.of())
+        .withValueCoder(StringUtf8Coder.of())
+        .updateConsumerProperties(consumerProps);
+
+    PCollection<String> deduped =
+        p.apply(read.withoutMetadata()).setCoder(
+            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))
+        .apply(Window.<KV<String, String>>into(FixedWindows.of(batchAndWindowDuration)))
+        .apply(ParDo.of(new FormatKVFn()))
+        .apply(RemoveDuplicates.<String>create());
+
+    PAssertStreaming.runAndAssertContents(p, deduped, expected);
+  }
+
+  @Test
+  public void testLatest() throws Exception {
     SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
         checkpointParentDir.newFolder(getClass().getSimpleName()));
-    Map<String, String> kafkaParams = ImmutableMap.of(
-        "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
-        "auto.offset.reset", "smallest"
+    //--- setup
+    final String topic = "topic";
+    // messages.
+    final Map<String, String> messages = ImmutableMap.of(
+        "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4"
     );
+    // expected.
+    final String[] expected = {"k1,v1", "k2,v2", "k3,v3", "k4,v4"};
+    // batch and window duration.
+    final Duration batchAndWindowDuration = Duration.standardSeconds(1);
+
+    // write once first batch completes, this will guarantee latest-like behaviour.
+    options.setListeners(Collections.<JavaStreamingListener>singletonList(
+        KafkaWriteOnBatchCompleted.once(messages, Collections.singletonList(topic),
+            EMBEDDED_KAFKA_CLUSTER.getProps(), EMBEDDED_KAFKA_CLUSTER.getBrokerList())));
+    // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec,
+    // so to be on the safe side we'll set to 750 msec.
+    options.setMinReadTimeMillis(750L);
+    // run for more than 1 batch interval, so that reading of latest is attempted in the
+    // first batch with no luck, while the OnBatchCompleted injected-input afterwards will be read
+    // in the second interval.
+    options.setTimeout(Duration.standardSeconds(3).getMillis());
 
+    //------- test: read and format.
     Pipeline p = Pipeline.create(options);
-    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(StringDecoder.class,
-        StringDecoder.class, String.class, String.class, Collections.singleton(TOPIC),
-        kafkaParams))
-        .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
-    PCollection<KV<String, String>> windowedWords = kafkaInput
-        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
 
-    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(new FormatKVFn()));
+    Map<String, Object> consumerProps = ImmutableMap.<String, Object>of(
+        "auto.offset.reset", "latest"
+    );
+
+    KafkaIO.Read<String, String> read = KafkaIO.read()
+        .withBootstrapServers(EMBEDDED_KAFKA_CLUSTER.getBrokerList())
+        .withTopics(Collections.singletonList(topic))
+        .withKeyCoder(StringUtf8Coder.of())
+        .withValueCoder(StringUtf8Coder.of())
+        .updateConsumerProperties(consumerProps);
 
-    PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED);
+    PCollection<String> formatted =
+        p.apply(read.withoutMetadata()).setCoder(
+            KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))
+        .apply(Window.<KV<String, String>>into(FixedWindows.of(batchAndWindowDuration)))
+        .apply(ParDo.of(new FormatKVFn()));
+
+    PAssertStreaming.runAndAssertContents(p, formatted, expected);
+  }
+
+  private static void produce(String topic, Map<String, String> messages) {
+    Serializer<String> stringSerializer = new StringSerializer();
+    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
+        new KafkaProducer(defaultProducerProps(), stringSerializer, stringSerializer)) {
+          // feed topic.
+          for (Map.Entry<String, String> en : messages.entrySet()) {
+            kafkaProducer.send(new ProducerRecord<>(topic, en.getKey(), en.getValue()));
+          }
+          // await send completion.
+          kafkaProducer.flush();
+        }
+  }
+
+  private static Properties defaultProducerProps() {
+    Properties producerProps = new Properties();
+    producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps());
+    producerProps.put("acks", "1");
+    producerProps.put("bootstrap.servers", EMBEDDED_KAFKA_CLUSTER.getBrokerList());
+    return producerProps;
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
index fd1d11a..ca0b668 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
@@ -27,17 +27,15 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
-import kafka.serializer.StringDecoder;
 import org.apache.beam.runners.spark.EvaluationResult;
 import org.apache.beam.runners.spark.SparkPipelineOptions;
 import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton;
-import org.apache.beam.runners.spark.io.KafkaIO;
 import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster;
 import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming;
 import org.apache.beam.runners.spark.translation.streaming.utils.TestOptionsForStreaming;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -112,6 +110,9 @@ public class ResumeFromCheckpointStreamingTest {
   public void testRun() throws Exception {
     SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
         checkpointParentDir.newFolder(getClass().getSimpleName()));
+    // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec,
+    // so to be on the safe side we'll set to 750 msec.
+    options.setMinReadTimeMillis(750L);
 
     // checkpoint after first (and only) interval.
     options.setCheckpointDurationMillis(options.getBatchIntervalMillis());
@@ -141,19 +142,24 @@ public class ResumeFromCheckpointStreamingTest {
   private static EvaluationResult run(SparkPipelineOptions options) {
     // write to Kafka
     produce();
-    Map<String, String> kafkaParams = ImmutableMap.of(
-            "metadata.broker.list", EMBEDDED_KAFKA_CLUSTER.getBrokerList(),
-            "auto.offset.reset", "smallest"
+    Map<String, Object> consumerProps = ImmutableMap.<String, Object>of(
+        "auto.offset.reset", "earliest"
     );
+
+    KafkaIO.Read<String, String> read = KafkaIO.read()
+        .withBootstrapServers(EMBEDDED_KAFKA_CLUSTER.getBrokerList())
+        .withTopics(Collections.singletonList(TOPIC))
+        .withKeyCoder(StringUtf8Coder.of())
+        .withValueCoder(StringUtf8Coder.of())
+        .updateConsumerProperties(consumerProps);
+
+    Duration windowDuration = new Duration(options.getBatchIntervalMillis());
+
     Pipeline p = Pipeline.create(options);
-    PCollection<KV<String, String>> kafkaInput = p.apply(KafkaIO.Read.from(
-        StringDecoder.class, StringDecoder.class, String.class, String.class,
-            Collections.singleton(TOPIC), kafkaParams)).setCoder(KvCoder.of(StringUtf8Coder.of(),
-                StringUtf8Coder.of()));
-    PCollection<KV<String, String>> windowedWords = kafkaInput
-        .apply(Window.<KV<String, String>>into(FixedWindows.of(Duration.standardSeconds(1))));
-    PCollection<String> formattedKV = windowedWords.apply(ParDo.of(
-        new FormatAsText()));
+    PCollection<String> formattedKV =
+        p.apply(read.withoutMetadata())
+        .apply(Window.<KV<String, String>>into(FixedWindows.of(windowDuration)))
+        .apply(ParDo.of(new FormatAsText()));
 
     // requires a graceful stop so that checkpointing of the first run would finish successfully
     // before stopping and attempting to resume.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
index d1729a4..e5e3c56 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
@@ -28,8 +28,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-import kafka.utils.Time;
+import kafka.server.KafkaServerStartable;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -50,7 +49,7 @@ public class EmbeddedKafkaCluster {
 
   private final String brokerList;
 
-  private final List<KafkaServer> brokers;
+  private final List<KafkaServerStartable> brokers;
   private final List<File> logDirs;
 
   public EmbeddedKafkaCluster(String zkConnection) {
@@ -93,7 +92,7 @@ public class EmbeddedKafkaCluster {
       if (sb.length() > 0) {
         sb.append(",");
       }
-      sb.append("localhost:").append(port);
+      sb.append("127.0.0.1:").append(port);
     }
     return sb.toString();
   }
@@ -107,12 +106,16 @@ public class EmbeddedKafkaCluster {
       properties.putAll(baseProperties);
       properties.setProperty("zookeeper.connect", zkConnection);
       properties.setProperty("broker.id", String.valueOf(i + 1));
-      properties.setProperty("host.name", "localhost");
+      properties.setProperty("advertised.host.name", "127.0.0.1");
+      properties.setProperty("host.name", "127.0.0.1");
+      properties.setProperty("advertised.port", Integer.toString(port));
       properties.setProperty("port", Integer.toString(port));
-      properties.setProperty("log.dir", logDir.getAbsolutePath());
+      properties.setProperty("log.dirs", logDir.getAbsolutePath());
+      properties.setProperty("offsets.topic.num.partitions", "1");
+      properties.setProperty("offsets.topic.replication.factor", "1");
       properties.setProperty("log.flush.interval.messages", String.valueOf(1));
 
-      KafkaServer broker = startBroker(properties);
+      KafkaServerStartable broker = startBroker(properties);
 
       brokers.add(broker);
       logDirs.add(logDir);
@@ -120,8 +123,8 @@ public class EmbeddedKafkaCluster {
   }
 
 
-  private static KafkaServer startBroker(Properties props) {
-    KafkaServer server = new KafkaServer(new KafkaConfig(props), new SystemTime());
+  private static KafkaServerStartable startBroker(Properties props) {
+    KafkaServerStartable server = new KafkaServerStartable(new KafkaConfig(props));
     server.startup();
     return server;
   }
@@ -129,8 +132,7 @@ public class EmbeddedKafkaCluster {
   public Properties getProps() {
     Properties props = new Properties();
     props.putAll(baseProperties);
-    props.put("metadata.broker.list", brokerList);
-    props.put("zookeeper.connect", zkConnection);
+    props.put("bootstrap.servers", brokerList);
     return props;
   }
 
@@ -147,7 +149,7 @@ public class EmbeddedKafkaCluster {
   }
 
   public void shutdown() {
-    for (KafkaServer broker : brokers) {
+    for (KafkaServerStartable broker : brokers) {
       try {
         broker.shutdown();
       } catch (Exception e) {
@@ -203,7 +205,7 @@ public class EmbeddedKafkaCluster {
       if (this.port == -1) {
         this.port = TestUtils.getAvailablePort();
       }
-      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("localhost", port),
+      this.factory = NIOServerCnxnFactory.createFactory(new InetSocketAddress("127.0.0.1", port),
               1024);
       this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot");
       this.logDir = TestUtils.constructTempDir("embedded-zk/log");
@@ -233,7 +235,7 @@ public class EmbeddedKafkaCluster {
     }
 
     public String getConnection() {
-      return "localhost:" + port;
+      return "127.0.0.1:" + port;
     }
 
     public void setPort(int port) {
@@ -258,27 +260,6 @@ public class EmbeddedKafkaCluster {
     }
   }
 
-  static class SystemTime implements Time {
-    @Override
-    public long milliseconds() {
-      return System.currentTimeMillis();
-    }
-
-    @Override
-    public long nanoseconds() {
-      return System.nanoTime();
-    }
-
-    @Override
-    public void sleep(long ms) {
-      try {
-        Thread.sleep(ms);
-      } catch (InterruptedException e) {
-        // Ignore
-      }
-    }
-  }
-
   static final class TestUtils {
     private static final Random RANDOM = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b72e7e34/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/KafkaWriteOnBatchCompleted.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/KafkaWriteOnBatchCompleted.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/KafkaWriteOnBatchCompleted.java
new file mode 100644
index 0000000..38a5bff
--- /dev/null
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/KafkaWriteOnBatchCompleted.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.translation.streaming.utils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.spark.streaming.api.java.JavaStreamingListener;
+import org.apache.spark.streaming.api.java.JavaStreamingListenerBatchCompleted;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Write to Kafka once the OnBatchCompleted hook is activated.
+ */
+public class KafkaWriteOnBatchCompleted extends JavaStreamingListener{
+  private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteOnBatchCompleted.class);
+
+  private final Map<String, String> messages;
+  private final List<String> topics;
+  private final Properties producerProps;
+  private final String brokerList;
+  private final boolean once;
+
+  // A flag to state that no more writes should happen.
+  private boolean done = false;
+
+  private KafkaWriteOnBatchCompleted(Map<String, String> messages,
+                                     List<String> topics,
+                                     Properties producerProps,
+                                     String brokerList,
+                                     boolean once) {
+    this.messages = messages;
+    this.topics = topics;
+    this.producerProps = producerProps;
+    this.brokerList = brokerList;
+    this.once = once;
+  }
+
+  public static KafkaWriteOnBatchCompleted once(Map<String, String> messages,
+                                                List<String> topics,
+                                                Properties producerProps,
+                                                String brokerList) {
+    return new KafkaWriteOnBatchCompleted(messages, topics, producerProps, brokerList, true);
+  }
+
+  public static KafkaWriteOnBatchCompleted always(Map<String, String> messages,
+                                                  List<String> topics,
+                                                  Properties producerProps,
+                                                  String brokerList) {
+    return new KafkaWriteOnBatchCompleted(messages, topics, producerProps, brokerList, false);
+  }
+
+  @Override
+  public void onBatchCompleted(JavaStreamingListenerBatchCompleted batchCompleted) {
+    super.onBatchCompleted(batchCompleted);
+    if (!done) {
+      LOG.info("Writing to Kafka after batchTime {} has completed.",
+          batchCompleted.batchInfo().batchTime());
+      write();
+      // once runs once.
+      if (once) {
+        done = true;
+      }
+    }
+  }
+
+  private void write() {
+    Properties props = new Properties();
+    props.putAll(producerProps);
+    props.put("acks", "1");
+    props.put("bootstrap.servers", brokerList);
+    Serializer<String> stringSerializer = new StringSerializer();
+    try (@SuppressWarnings("unchecked") KafkaProducer<String, String> kafkaProducer =
+        new KafkaProducer(props, stringSerializer, stringSerializer)) {
+          for (String topic: topics) {
+            for (Map.Entry<String, String> en : messages.entrySet()) {
+              kafkaProducer.send(new ProducerRecord<>(topic, en.getKey(), en.getValue()));
+            }
+            // await send completion.
+            kafkaProducer.flush();
+          }
+        }
+  }
+}


[06/50] incubator-beam git commit: This closes #1163

Posted by th...@apache.org.
This closes #1163


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

Branch: refs/heads/apex-runner
Commit: 7abdcbfb2fe1d1308f504dd17e46f1a4fc576407
Parents: 0ec956d 60907b2
Author: Davor Bonaci <da...@google.com>
Authored: Tue Oct 25 13:05:27 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Oct 25 13:05:27 2016 -0700

----------------------------------------------------------------------
 pom.xml                                                 |  2 +-
 .../runners/dataflow/DataflowPipelineTranslator.java    |  4 ++++
 .../dataflow/options/DataflowPipelineOptions.java       | 12 ++++++++++++
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------