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

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

Repository: incubator-beam
Updated Branches:
  refs/heads/python-sdk 3b4fd5c7d -> 3454d691f


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 3bf63fd..1d8b32c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -84,12 +84,14 @@ public class ViewTest implements Serializable {
   // anonymous inner classes inside the non-static test methods.
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(RunnableOnService.class)
   public void testSingletonSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Integer> view =
         pipeline.apply("Create47", Create.of(47)).apply(View.<Integer>asSingleton());
@@ -112,7 +114,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSingletonSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Integer> view =
         pipeline.apply("Create47", Create.timestamped(
@@ -143,7 +144,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testEmptySingletonSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Integer> view =
         pipeline.apply("CreateEmptyIntegers", Create.<Integer>of().withCoder(VarIntCoder.of()))
@@ -169,7 +169,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testNonSingletonSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> oneTwoThree = pipeline.apply(Create.<Integer>of(1, 2, 3));
     final PCollectionView<Integer> view = oneTwoThree.apply(View.<Integer>asSingleton());
@@ -194,7 +193,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testListSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<List<Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 23)).apply(View.<Integer>asList());
@@ -221,7 +219,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedListSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<List<Integer>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -262,7 +259,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyListSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<List<Integer>> view =
         pipeline.apply("CreateEmptyView", Create.<Integer>of().withCoder(VarIntCoder.of()))
@@ -289,7 +285,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testListSideInputIsImmutable() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<List<Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(11)).apply(View.<Integer>asList());
@@ -335,7 +330,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testIterableSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Iterable<Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 23))
@@ -361,7 +355,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedIterableSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Iterable<Integer>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -401,7 +394,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyIterableSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Iterable<Integer>> view =
         pipeline.apply("CreateEmptyView", Create.<Integer>of().withCoder(VarIntCoder.of()))
@@ -427,7 +419,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testIterableSideInputIsImmutable() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Iterable<Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(11)).apply(View.<Integer>asIterable());
@@ -459,7 +450,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultimapSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)))
@@ -487,7 +477,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultimapAsEntrySetSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)))
@@ -539,7 +528,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultimapSideInputWithNonDeterministicKeyCoder() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput",
@@ -569,7 +557,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMultimapSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -608,7 +595,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMultimapAsEntrySetSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -651,7 +637,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMultimapSideInputWithNonDeterministicKeyCoder() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput",
@@ -691,7 +676,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyMultimapSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateEmptyView", Create.<KV<String, Integer>>of().withCoder(
@@ -720,7 +704,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyMultimapSideInputWithNonDeterministicKeyCoder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateEmptyView",
@@ -750,7 +733,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultimapSideInputIsImmutable() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Iterable<Integer>>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1)))
@@ -798,7 +780,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMapSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("b", 3)))
@@ -825,7 +806,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMapAsEntrySetSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("b", 3)))
@@ -855,7 +835,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMapSideInputWithNonDeterministicKeyCoder() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput",
@@ -884,7 +863,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMapSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -922,7 +900,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMapAsEntrySetSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.timestamped(
@@ -964,7 +941,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedMapSideInputWithNonDeterministicKeyCoder() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput",
@@ -1004,7 +980,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyMapSideInput() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateEmptyView", Create.<KV<String, Integer>>of().withCoder(
@@ -1033,7 +1008,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyMapSideInputWithNonDeterministicKeyCoder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateEmptyView", Create.<KV<String, Integer>>of().withCoder(
@@ -1062,7 +1036,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMapSideInputWithNullValuesCatchesDuplicates() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline
@@ -1098,7 +1071,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMapSideInputIsImmutable() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1)))
@@ -1145,7 +1117,6 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testCombinedMapSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     final PCollectionView<Map<String, Integer>> view =
         pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 20), KV.of("b", 3)))
@@ -1172,10 +1143,9 @@ public class ViewTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSideInputFixedToFixed() {
-    Pipeline p = TestPipeline.create();
 
     final PCollectionView<Integer> view =
-        p.apply(
+        pipeline.apply(
              "CreateSideInput",
              Create.timestamped(TimestampedValue.of(1, new Instant(1)),
                  TimestampedValue.of(2, new Instant(11)), TimestampedValue.of(3, new Instant(13))))
@@ -1184,7 +1154,7 @@ public class ViewTest implements Serializable {
             .apply(View.<Integer>asSingleton());
 
     PCollection<String> output =
-        p.apply("CreateMainInput", Create.timestamped(
+        pipeline.apply("CreateMainInput", Create.timestamped(
                                        TimestampedValue.of("A", new Instant(4)),
                                        TimestampedValue.of("B", new Instant(15)),
                                        TimestampedValue.of("C", new Instant(7))))
@@ -1199,16 +1169,15 @@ public class ViewTest implements Serializable {
 
     PAssert.that(output).containsInAnyOrder("A1", "B5", "C1");
 
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSideInputFixedToGlobal() {
-    Pipeline p = TestPipeline.create();
 
     final PCollectionView<Integer> view =
-        p.apply(
+        pipeline.apply(
              "CreateSideInput",
              Create.timestamped(TimestampedValue.of(1, new Instant(1)),
                  TimestampedValue.of(2, new Instant(11)), TimestampedValue.of(3, new Instant(13))))
@@ -1217,7 +1186,7 @@ public class ViewTest implements Serializable {
             .apply(View.<Integer>asSingleton());
 
     PCollection<String> output =
-        p.apply("CreateMainInput", Create.timestamped(
+        pipeline.apply("CreateMainInput", Create.timestamped(
                                        TimestampedValue.of("A", new Instant(4)),
                                        TimestampedValue.of("B", new Instant(15)),
                                        TimestampedValue.of("C", new Instant(7))))
@@ -1232,23 +1201,22 @@ public class ViewTest implements Serializable {
 
     PAssert.that(output).containsInAnyOrder("A6", "B6", "C6");
 
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSideInputFixedToFixedWithDefault() {
-    Pipeline p = TestPipeline.create();
 
     final PCollectionView<Integer> view =
-        p.apply("CreateSideInput", Create.timestamped(
+        pipeline.apply("CreateSideInput", Create.timestamped(
                                        TimestampedValue.of(2, new Instant(11)),
                                        TimestampedValue.of(3, new Instant(13))))
             .apply("WindowSideInput", Window.<Integer>into(FixedWindows.of(Duration.millis(10))))
             .apply(Sum.integersGlobally().asSingletonView());
 
     PCollection<String> output =
-        p.apply("CreateMainInput", Create.timestamped(
+        pipeline.apply("CreateMainInput", Create.timestamped(
                                        TimestampedValue.of("A", new Instant(4)),
                                        TimestampedValue.of("B", new Instant(15)),
                                        TimestampedValue.of("C", new Instant(7))))
@@ -1263,16 +1231,15 @@ public class ViewTest implements Serializable {
 
     PAssert.that(output).containsInAnyOrder("A0", "B5", "C0");
 
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testSideInputWithNullDefault() {
-    Pipeline p = TestPipeline.create();
 
     final PCollectionView<Void> view =
-        p.apply("CreateSideInput", Create.of((Void) null).withCoder(VoidCoder.of()))
+        pipeline.apply("CreateSideInput", Create.of((Void) null).withCoder(VoidCoder.of()))
             .apply(Combine.globally(new SerializableFunction<Iterable<Void>, Void>() {
               @Override
               public Void apply(Iterable<Void> input) {
@@ -1281,7 +1248,7 @@ public class ViewTest implements Serializable {
             }).asSingletonView());
 
     PCollection<String> output =
-        p.apply("CreateMainInput", Create.of(""))
+        pipeline.apply("CreateMainInput", Create.of(""))
             .apply(
                 "OutputMainAndSideInputs",
                 ParDo.withSideInputs(view).of(new DoFn<String, String>() {
@@ -1293,13 +1260,12 @@ public class ViewTest implements Serializable {
 
     PAssert.that(output).containsInAnyOrder("null");
 
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testSideInputWithNestedIterables() {
-    Pipeline pipeline = TestPipeline.create();
     final PCollectionView<Iterable<Integer>> view1 =
         pipeline.apply("CreateVoid1", Create.of((Void) null).withCoder(VoidCoder.of()))
             .apply("OutputOneInteger", ParDo.of(new DoFn<Void, Integer>() {
@@ -1386,51 +1352,51 @@ public class ViewTest implements Serializable {
 
   @Test
   public void testViewUnboundedAsSingletonDirect() {
-    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asSingleton());
+    testViewUnbounded(pipeline, View.<KV<String, Integer>>asSingleton());
   }
 
   @Test
   public void testViewUnboundedAsIterableDirect() {
-    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asIterable());
+    testViewUnbounded(pipeline, View.<KV<String, Integer>>asIterable());
   }
 
   @Test
   public void testViewUnboundedAsListDirect() {
-    testViewUnbounded(TestPipeline.create(), View.<KV<String, Integer>>asList());
+    testViewUnbounded(pipeline, View.<KV<String, Integer>>asList());
   }
 
   @Test
   public void testViewUnboundedAsMapDirect() {
-    testViewUnbounded(TestPipeline.create(), View.<String, Integer>asMap());
+    testViewUnbounded(pipeline, View.<String, Integer>asMap());
   }
 
   @Test
   public void testViewUnboundedAsMultimapDirect() {
-    testViewUnbounded(TestPipeline.create(), View.<String, Integer>asMultimap());
+    testViewUnbounded(pipeline, View.<String, Integer>asMultimap());
   }
 
   @Test
   public void testViewNonmergingAsSingletonDirect() {
-    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asSingleton());
+    testViewNonmerging(pipeline, View.<KV<String, Integer>>asSingleton());
   }
 
   @Test
   public void testViewNonmergingAsIterableDirect() {
-    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asIterable());
+    testViewNonmerging(pipeline, View.<KV<String, Integer>>asIterable());
   }
 
   @Test
   public void testViewNonmergingAsListDirect() {
-    testViewNonmerging(TestPipeline.create(), View.<KV<String, Integer>>asList());
+    testViewNonmerging(pipeline, View.<KV<String, Integer>>asList());
   }
 
   @Test
   public void testViewNonmergingAsMapDirect() {
-    testViewNonmerging(TestPipeline.create(), View.<String, Integer>asMap());
+    testViewNonmerging(pipeline, View.<String, Integer>asMap());
   }
 
   @Test
   public void testViewNonmergingAsMultimapDirect() {
-    testViewNonmerging(TestPipeline.create(), View.<String, Integer>asMultimap());
+    testViewNonmerging(pipeline, View.<String, Integer>asMultimap());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
index f958807..8abbf1a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -29,6 +28,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TypeDescriptor;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -63,10 +63,12 @@ public class WithKeysTest {
     KV.of(100, "bbb")
   );
 
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(NeedsRunner.class)
   public void testExtractKeys() {
-    Pipeline p = TestPipeline.create();
 
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder(
@@ -83,7 +85,6 @@ public class WithKeysTest {
   @Test
   @Category(NeedsRunner.class)
   public void testConstantKeys() {
-    Pipeline p = TestPipeline.create();
 
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder(
@@ -105,7 +106,6 @@ public class WithKeysTest {
   @Test
   @Category(NeedsRunner.class)
   public void testWithKeysWithUnneededWithKeyTypeSucceeds() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
index 923b97c..67a2658 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
@@ -41,13 +41,16 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class WithTimestampsTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(RunnableOnService.class)
   public void withTimestampsShouldApplyTimestamps() {
-    TestPipeline p = TestPipeline.create();
 
     SerializableFunction<String, Instant> timestampFn =
         new SerializableFunction<String, Instant>() {
@@ -86,7 +89,6 @@ public class WithTimestampsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void withTimestampsBackwardsInTimeShouldThrow() {
-    TestPipeline p = TestPipeline.create();
 
     SerializableFunction<String, Instant> timestampFn =
         new SerializableFunction<String, Instant>() {
@@ -120,7 +122,6 @@ public class WithTimestampsTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void withTimestampsBackwardsInTimeAndWithAllowedTimestampSkewShouldSucceed() {
-    TestPipeline p = TestPipeline.create();
 
     SerializableFunction<String, Instant> timestampFn =
         new SerializableFunction<String, Instant>() {
@@ -181,7 +182,6 @@ public class WithTimestampsTest implements Serializable {
           }
         };
 
-    TestPipeline p = TestPipeline.create();
     String yearTwoThousand = "946684800000";
     p.apply(Create.of("1234", "0", Integer.toString(Integer.MAX_VALUE), yearTwoThousand))
      .apply(WithTimestamps.of(timestampFn));
@@ -197,7 +197,6 @@ public class WithTimestampsTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void withTimestampsWithNullFnShouldThrowOnConstruction() {
-    TestPipeline p = TestPipeline.create();
 
     SerializableFunction<String, Instant> timestampFn = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
index e8c8b15..0e5c177 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
@@ -47,6 +47,7 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
 import org.joda.time.Duration;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -118,14 +119,15 @@ public class CoGroupByKeyTest implements Serializable {
     return coGbkResults;
   }
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testCoGroupByKeyGetOnly() {
     final TupleTag<String> tag1 = new TupleTag<>();
     final TupleTag<String> tag2 = new TupleTag<>();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<Integer, CoGbkResult>> coGbkResults =
         buildGetOnlyGbk(p, tag1, tag2);
 
@@ -264,7 +266,6 @@ public class CoGroupByKeyTest implements Serializable {
     final TupleTag<String> addressesTag = new TupleTag<>();
     final TupleTag<String> purchasesTag = new TupleTag<>();
 
-    Pipeline p = TestPipeline.create();
 
     PCollection<KV<Integer, CoGbkResult>> coGbkResults =
         buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag);
@@ -456,8 +457,6 @@ public class CoGroupByKeyTest implements Serializable {
     TupleTag<String> addressesTag = new TupleTag<>();
     TupleTag<String> purchasesTag = new TupleTag<>();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<Integer, CoGbkResult>> coGbkResults =
         buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag);
 
@@ -486,8 +485,6 @@ public class CoGroupByKeyTest implements Serializable {
     TupleTag<String> clicksTag = new TupleTag<>();
     TupleTag<String> purchasesTag = new TupleTag<>();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<Integer, CoGbkResult>> coGbkResults =
         buildPurchasesCoGbkWithWindowing(p, clicksTag, purchasesTag);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
index 3125ae8..e21668e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
@@ -29,7 +29,6 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.when;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -62,11 +61,15 @@ import org.mockito.Mockito;
 public class WindowTest implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create()
+                                                             .enableAbandonedNodeEnforcement(false);
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void testWindowIntoSetWindowfn() {
-    WindowingStrategy<?, ?> strategy = TestPipeline.create()
+    WindowingStrategy<?, ?> strategy = pipeline
       .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
       .apply(Window.<String>into(FixedWindows.of(Duration.standardMinutes(10))))
       .getWindowingStrategy();
@@ -79,7 +82,7 @@ public class WindowTest implements Serializable {
   public void testWindowIntoTriggersAndAccumulating() {
     FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10));
     Repeatedly trigger = Repeatedly.forever(AfterPane.elementCountAtLeast(5));
-    WindowingStrategy<?, ?> strategy = TestPipeline.create()
+    WindowingStrategy<?, ?> strategy = pipeline
       .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
       .apply(Window.<String>into(fixed10)
           .triggering(trigger)
@@ -96,7 +99,7 @@ public class WindowTest implements Serializable {
   public void testWindowPropagatesEachPart() {
     FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10));
     Repeatedly trigger = Repeatedly.forever(AfterPane.elementCountAtLeast(5));
-    WindowingStrategy<?, ?> strategy = TestPipeline.create()
+    WindowingStrategy<?, ?> strategy = pipeline
       .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
       .apply("Mode", Window.<String>accumulatingFiredPanes())
       .apply("Lateness", Window.<String>withAllowedLateness(Duration.standardDays(1)))
@@ -112,9 +115,10 @@ public class WindowTest implements Serializable {
 
   @Test
   public void testWindowIntoPropagatesLateness() {
+
     FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10));
     FixedWindows fixed25 = FixedWindows.of(Duration.standardMinutes(25));
-    WindowingStrategy<?, ?> strategy = TestPipeline.create()
+    WindowingStrategy<?, ?> strategy = pipeline
         .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
         .apply("WindowInto10", Window.<String>into(fixed10)
             .withAllowedLateness(Duration.standardDays(1))
@@ -157,7 +161,7 @@ public class WindowTest implements Serializable {
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("requires that the accumulation mode");
-    TestPipeline.create()
+    pipeline
       .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
       .apply("Window", Window.<String>into(fixed10))
       .apply("Lateness", Window.<String>withAllowedLateness(Duration.standardDays(1)))
@@ -171,7 +175,7 @@ public class WindowTest implements Serializable {
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("requires that the allowed lateness");
-    TestPipeline.create()
+    pipeline
       .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
       .apply("Mode", Window.<String>accumulatingFiredPanes())
       .apply("Window", Window.<String>into(fixed10))
@@ -185,7 +189,7 @@ public class WindowTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testOutputTimeFnDefault() {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(true);
 
     pipeline
         .apply(
@@ -219,7 +223,7 @@ public class WindowTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testOutputTimeFnEndOfWindow() {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(true);
 
     pipeline.apply(
         Create.timestamped(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
index d4fab17..f7ae5d8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.testing.NeedsRunner;
@@ -50,6 +49,10 @@ import org.junit.runners.JUnit4;
 /** Unit tests for bucketing. */
 @RunWith(JUnit4.class)
 public class WindowingTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Rule
   public transient TemporaryFolder tmpFolder = new TemporaryFolder();
 
@@ -88,7 +91,6 @@ public class WindowingTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testPartitioningWindowing() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(
             Create.timestamped(
@@ -114,7 +116,6 @@ public class WindowingTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testNonPartitioningWindowing() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(
             Create.timestamped(
@@ -140,7 +141,6 @@ public class WindowingTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMergingWindowing() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(
             Create.timestamped(
@@ -162,7 +162,6 @@ public class WindowingTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowPreservation() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input1 = p.apply("Create12",
         Create.timestamped(
             TimestampedValue.of("a", new Instant(1)),
@@ -190,7 +189,6 @@ public class WindowingTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testEmptyInput() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(Create.<String>timestamped()
             .withCoder(StringUtf8Coder.of()));
@@ -218,7 +216,6 @@ public class WindowingTest implements Serializable {
       writer.println("d 11");
     }
 
-    Pipeline p = TestPipeline.create();
     PCollection<String> output = p.begin()
         .apply("ReadLines", TextIO.Read.from(filename))
         .apply(ParDo.of(new ExtractWordsWithTimestampsFn()))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
index d990ee0..d47cddc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 
 import com.google.common.collect.ImmutableList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -36,6 +35,7 @@ import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -65,10 +65,12 @@ public class ReshuffleTest {
         KV.of("k1", (Iterable<Integer>) ImmutableList.of(3)),
         KV.of("k2", (Iterable<Integer>) ImmutableList.of(4)));
 
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testJustReshuffle() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input = pipeline
         .apply(Create.of(ARBITRARY_KVS)
@@ -89,7 +91,6 @@ public class ReshuffleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testReshuffleAfterSessionsAndGroupByKey() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Iterable<Integer>>> input = pipeline
         .apply(Create.of(GBK_TESTABLE_KVS)
@@ -113,7 +114,6 @@ public class ReshuffleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testReshuffleAfterFixedWindowsAndGroupByKey() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Iterable<Integer>>> input = pipeline
         .apply(Create.of(GBK_TESTABLE_KVS)
@@ -137,7 +137,6 @@ public class ReshuffleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testReshuffleAfterSlidingWindowsAndGroupByKey() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Iterable<Integer>>> input = pipeline
         .apply(Create.of(GBK_TESTABLE_KVS)
@@ -161,7 +160,6 @@ public class ReshuffleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testReshuffleAfterFixedWindows() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input = pipeline
         .apply(Create.of(ARBITRARY_KVS)
@@ -185,7 +183,6 @@ public class ReshuffleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testReshuffleAfterSlidingWindows() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input = pipeline
         .apply(Create.of(ARBITRARY_KVS)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
index 1467ae8..b5351da 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -33,6 +32,7 @@ import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -41,9 +41,14 @@ import org.junit.runners.JUnit4;
 /** Unit tests for {@link PCollectionTuple}. */
 @RunWith(JUnit4.class)
 public final class PCollectionTupleTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create()
+                                                             .enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testOfThenHas() {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Object> pCollection = PCollection.createPrimitiveOutputInternal(
         pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
     TupleTag<Object> tag = new TupleTag<>();
@@ -53,7 +58,6 @@ public final class PCollectionTupleTest implements Serializable {
 
   @Test
   public void testEmpty() {
-    Pipeline pipeline = TestPipeline.create();
     TupleTag<Object> tag = new TupleTag<>();
     assertFalse(PCollectionTuple.empty(pipeline).has(tag));
   }
@@ -61,7 +65,7 @@ public final class PCollectionTupleTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testComposePCollectionTuple() {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(true);
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
index e5f2019..ba7477d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.values;
 import static org.apache.beam.sdk.TestUtils.LINES;
 
 import java.io.File;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -40,6 +39,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class PDoneTest {
+
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
 
@@ -78,8 +81,6 @@ public class PDoneTest {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyTransform() {
-    Pipeline p = TestPipeline.create();
-
     p.begin().apply(new EmptyTransform());
 
     p.run();
@@ -94,8 +95,6 @@ public class PDoneTest {
     File tmpFile = tmpFolder.newFile("file.txt");
     String filename = tmpFile.getPath();
 
-    Pipeline p = TestPipeline.create();
-
     p.begin().apply(new SimpleTransform(filename));
 
     p.run();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
index f33b3a2..8381f12 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java
@@ -22,7 +22,6 @@ import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
@@ -40,6 +39,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class TypedPValueTest {
+
+  @Rule
+  public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
@@ -51,9 +54,8 @@ public class TypedPValueTest {
     }
   }
 
-  private static PCollectionTuple buildPCollectionTupleWithTags(
+  private PCollectionTuple buildPCollectionTupleWithTags(
       TupleTag<Integer> mainOutputTag, TupleTag<Integer> sideOutputTag) {
-    Pipeline p = TestPipeline.create();
     PCollection<Integer> input = p.apply(Create.of(1, 2, 3));
     PCollectionTuple tuple = input.apply(
         ParDo
@@ -138,7 +140,6 @@ public class TypedPValueTest {
 
   @Test
   public void testParDoWithNoSideOutputsErrorDoesNotMentionTupleTag() {
-    Pipeline p = TestPipeline.create();
     PCollection<EmptyClass> input =
         p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new EmptyClassDoFn()));
 
@@ -158,7 +159,6 @@ public class TypedPValueTest {
 
   @Test
   public void testFinishSpecifyingShouldFailIfNoCoderInferrable() {
-    Pipeline p = TestPipeline.create();
     PCollection<EmptyClass> unencodable =
         p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new EmptyClassDoFn()));
 



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

Posted by ke...@apache.org.
Fix windowing in direct runner Stateful ParDo


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4fb16e8f/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index 2cea999..b35df87 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -34,8 +34,13 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
@@ -92,9 +97,12 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
     @Override
     public PCollectionTuple expand(PCollection<KV<K, InputT>> input) {
 
+      WindowingStrategy<?, ?> inputWindowingStrategy = input.getWindowingStrategy();
+
       // A KvCoder is required since this goes through GBK. Further, WindowedValueCoder
       // is not registered by default, so we explicitly set the relevant coders.
-      checkState(input.getCoder() instanceof KvCoder,
+      checkState(
+          input.getCoder() instanceof KvCoder,
           "Input to a %s using state requires a %s, but the coder was %s",
           ParDo.class.getSimpleName(),
           KvCoder.class.getSimpleName(),
@@ -102,14 +110,27 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
       KvCoder<K, InputT> kvCoder = (KvCoder<K, InputT>) input.getCoder();
       Coder<K> keyCoder = kvCoder.getKeyCoder();
       Coder<? extends BoundedWindow> windowCoder =
-          input.getWindowingStrategy().getWindowFn().windowCoder();
+          inputWindowingStrategy.getWindowFn().windowCoder();
 
-      PCollectionTuple outputs =
+      PCollection<KeyedWorkItem<K, KV<K, InputT>>> adjustedInput =
           input
               // Stash the original timestamps, etc, for when it is fed to the user's DoFn
               .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn<K, InputT>()))
               .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder)))
 
+              // We are going to GBK to gather keys and windows but otherwise do not want
+              // to alter the flow of data. This entails:
+              //  - trigger as fast as possible
+              //  - maintain the full timestamps of elements
+              //  - ensure this GBK holds to the minimum of those timestamps (via OutputTimeFn)
+              //  - discard past panes as it is "just a stream" of elements
+              .apply(
+                  Window.<KV<K, WindowedValue<KV<K, InputT>>>>triggering(
+                          Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                      .discardingFiredPanes()
+                      .withAllowedLateness(inputWindowingStrategy.getAllowedLateness())
+                      .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()))
+
               // A full GBK to group by key _and_ window
               .apply("Group by key", GroupByKey.<K, WindowedValue<KV<K, InputT>>>create())
 
@@ -117,6 +138,13 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
               .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem<K, InputT>()))
               .setCoder(KeyedWorkItemCoder.of(keyCoder, kvCoder, windowCoder))
 
+              // Because of the intervening GBK, we may have abused the windowing strategy
+              // of the input, which should be transferred to the output in a straightforward manner
+              // according to what ParDo already does.
+              .setWindowingStrategyInternal(inputWindowingStrategy);
+
+      PCollectionTuple outputs =
+          adjustedInput
               // Explode the resulting iterable into elements that are exactly the ones from
               // the input
               .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input));


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

Posted by ke...@apache.org.
This closes #1670: Fixes for direct runner expansion and evaluation of stateful ParDo

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


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

Branch: refs/heads/python-sdk
Commit: ff395169993d84cd920be21f11d9a5555f8f8d8b
Parents: 7ee8c86 55176c3
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Dec 21 13:11:54 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Dec 21 13:11:54 2016 -0800

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



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

Posted by ke...@apache.org.
Migrated the beam-runners-core module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: b6710251d8bb5d1968aea2258ce5878b43368dd5
Parents: 7106e88
Author: Stas Levin <st...@gmail.com>
Authored: Sun Dec 18 18:51:31 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b6710251/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
index 7fd8807..86450f2 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
@@ -78,6 +78,9 @@ public class UnboundedReadFromBoundedSourceTest {
   @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   public void testCheckpointCoderNulls() throws Exception {
     CheckpointCoder<String> coder = new CheckpointCoder<>(StringUtf8Coder.of());
@@ -97,8 +100,6 @@ public class UnboundedReadFromBoundedSourceTest {
     UnboundedSource<Long, Checkpoint<Long>> unboundedSource =
         new BoundedToUnboundedSourceAdapter<>(boundedSource);
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<Long> output =
         p.apply(Read.from(unboundedSource).withMaxNumRecords(numElements));
 


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

Posted by ke...@apache.org.
Fixup usage of canonical name with name since canonical name != name for inner classes.


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

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

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


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


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

Posted by ke...@apache.org.
Use informative Instant formatter in WatermarkHold


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

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

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


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

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


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

Posted by ke...@apache.org.
Allow setting timer by ID in DirectTimerInternals


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

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

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f14c463/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index 7bed751..f7bafd1 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -23,11 +23,13 @@ import com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.SortedMultiset;
+import com.google.common.collect.Table;
 import com.google.common.collect.TreeMultiset;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -56,6 +58,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.joda.time.Instant;
@@ -210,6 +213,10 @@ public class WatermarkManager {
     private final SortedMultiset<CommittedBundle<?>> pendingElements;
     private final Map<StructuralKey<?>, NavigableSet<TimerData>> objectTimers;
 
+    // Entries in this table represent the authoritative timestamp for which
+    // a per-key-and-StateNamespace timer is set.
+    private final Map<StructuralKey<?>, Table<StateNamespace, String, TimerData>> existingTimers;
+
     private AtomicReference<Instant> currentWatermark;
 
     public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
@@ -222,6 +229,7 @@ public class WatermarkManager {
       this.pendingElements =
           TreeMultiset.create(pendingBundleComparator);
       this.objectTimers = new HashMap<>();
+      this.existingTimers = new HashMap<>();
       currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
     }
 
@@ -276,14 +284,31 @@ public class WatermarkManager {
         keyTimers = new TreeSet<>();
         objectTimers.put(update.key, keyTimers);
       }
+      Table<StateNamespace, String, TimerData> existingTimersForKey =
+          existingTimers.get(update.key);
+      if (existingTimersForKey == null) {
+        existingTimersForKey = HashBasedTable.create();
+        existingTimers.put(update.key, existingTimersForKey);
+      }
+
       for (TimerData timer : update.setTimers) {
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+          @Nullable
+          TimerData existingTimer =
+              existingTimersForKey.get(timer.getNamespace(), timer.getTimerId());
+
+          if (existingTimer != null) {
+            keyTimers.remove(existingTimer);
+          }
           keyTimers.add(timer);
+          existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer);
         }
       }
+
       for (TimerData timer : update.deletedTimers) {
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
           keyTimers.remove(timer);
+          existingTimersForKey.remove(timer.getNamespace(), timer.getTimerId());
         }
       }
       // We don't keep references to timers that have been fired and delivered via #getFiredTimers()


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

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


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

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

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



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

Posted by ke...@apache.org.
This closes #1668: Remove deprecated InMemoryTimerInternals from SDK


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

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

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



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

Posted by ke...@apache.org.
Move ExecutionContext and related classes to runners-core


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
index a3d3a97..c41cd45 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
@@ -41,6 +41,7 @@ import org.apache.beam.runners.core.AggregatorFactory;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import org.apache.beam.sdk.coders.Coder;
@@ -50,7 +51,6 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnAdapters;
 import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.NullSideInputReader;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.UserCodeException;

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java
index 153d30d..24a605f 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java
@@ -20,7 +20,6 @@ package org.apache.beam.runners.core;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.ExecutionContext;
 
 /**
  * A factory for creating aggregators.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java
new file mode 100644
index 0000000..7b674dc
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * Base class for implementations of {@link ExecutionContext}.
+ *
+ * <p>A concrete subclass should implement {@link #createStepContext} to create the appropriate
+ * {@link StepContext} implementation. Any {@code StepContext} created will
+ * be cached for the lifetime of this {@link ExecutionContext}.
+ *
+ * <p>BaseExecutionContext is generic to allow implementing subclasses to return a concrete subclass
+ * of {@link StepContext} from {@link #getOrCreateStepContext(String, String)} and
+ * {@link #getAllStepContexts()} without forcing each subclass to override the method, e.g.
+ * <pre>{@code
+ * {@literal @}Override
+ * StreamingModeExecutionContext.StepContext getOrCreateStepContext(...) {
+ *   return (StreamingModeExecutionContext.StepContext) super.getOrCreateStepContext(...);
+ * }
+ * }</pre>
+ *
+ * <p>When a subclass of {@code BaseExecutionContext} has been downcast, the return types of
+ * {@link #createStepContext(String, String)},
+ * {@link #getOrCreateStepContext(String, String)}, and {@link #getAllStepContexts()}
+ * will be appropriately specialized.
+ */
+public abstract class BaseExecutionContext<T extends ExecutionContext.StepContext>
+    implements ExecutionContext {
+
+  private Map<String, T> cachedStepContexts = new HashMap<>();
+
+  /**
+   * Implementations should override this to create the specific type
+   * of {@link StepContext} they need.
+   */
+  protected abstract T createStepContext(String stepName, String transformName);
+
+  /**
+   * Returns the {@link StepContext} associated with the given step.
+   */
+  @Override
+  public T getOrCreateStepContext(String stepName, String transformName) {
+    final String finalStepName = stepName;
+    final String finalTransformName = transformName;
+    return getOrCreateStepContext(
+        stepName,
+        new CreateStepContextFunction<T>() {
+          @Override
+          public T create() {
+            return createStepContext(finalStepName, finalTransformName);
+          }
+        });
+  }
+
+  /**
+   * Factory method interface to create an execution context if none exists during
+   * {@link #getOrCreateStepContext(String, CreateStepContextFunction)}.
+   */
+  protected interface CreateStepContextFunction<T extends ExecutionContext.StepContext> {
+    T create();
+  }
+
+  protected final T getOrCreateStepContext(String stepName,
+      CreateStepContextFunction<T> createContextFunc) {
+    T context = cachedStepContexts.get(stepName);
+    if (context == null) {
+      context = createContextFunc.create();
+      cachedStepContexts.put(stepName, context);
+    }
+
+    return context;
+  }
+
+  /**
+   * Returns a collection view of all of the {@link StepContext}s.
+   */
+  @Override
+  public Collection<? extends T> getAllStepContexts() {
+    return Collections.unmodifiableCollection(cachedStepContexts.values());
+  }
+
+  /**
+   * Hook for subclasses to implement that will be called whenever
+   * {@code DoFn.Context#output}
+   * is called.
+   */
+  @Override
+  public void noteOutput(WindowedValue<?> output) {}
+
+  /**
+   * Hook for subclasses to implement that will be called whenever
+   * {@code DoFn.Context#sideOutput}
+   * is called.
+   */
+  @Override
+  public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output) {}
+
+  /**
+   * Base class for implementations of {@link ExecutionContext.StepContext}.
+   *
+   * <p>To complete a concrete subclass, implement {@link #timerInternals} and
+   * {@link #stateInternals}.
+   */
+  public abstract static class StepContext implements ExecutionContext.StepContext {
+    private final ExecutionContext executionContext;
+    private final String stepName;
+    private final String transformName;
+
+    public StepContext(ExecutionContext executionContext, String stepName, String transformName) {
+      this.executionContext = executionContext;
+      this.stepName = stepName;
+      this.transformName = transformName;
+    }
+
+    @Override
+    public String getStepName() {
+      return stepName;
+    }
+
+    @Override
+    public String getTransformName() {
+      return transformName;
+    }
+
+    @Override
+    public void noteOutput(WindowedValue<?> output) {
+      executionContext.noteOutput(output);
+    }
+
+    @Override
+    public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output) {
+      executionContext.noteSideOutput(tag, output);
+    }
+
+    @Override
+    public <T, W extends BoundedWindow> void writePCollectionViewData(
+        TupleTag<?> tag,
+        Iterable<WindowedValue<T>> data, Coder<Iterable<WindowedValue<T>>> dataCoder,
+        W window, Coder<W> windowCoder) throws IOException {
+      throw new UnsupportedOperationException("Not implemented.");
+    }
+
+    @Override
+    public abstract StateInternals<?> stateInternals();
+
+    @Override
+    public abstract TimerInternals timerInternals();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
index 0e4bf75..820bfcd 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
@@ -19,13 +19,13 @@ package org.apache.beam.runners.core;
 
 import java.util.List;
 import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
 import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
new file mode 100644
index 0000000..f67aff4
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core;
+
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * Context for the current execution. This is guaranteed to exist during processing,
+ * but does not necessarily persist between different batches of work.
+ */
+public interface ExecutionContext {
+  /**
+   * Returns the {@link StepContext} associated with the given step.
+   */
+  StepContext getOrCreateStepContext(String stepName, String transformName);
+
+  /**
+   * Returns a collection view of all of the {@link StepContext}s.
+   */
+  Collection<? extends StepContext> getAllStepContexts();
+
+  /**
+   * Hook for subclasses to implement that will be called whenever
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
+   * is called.
+   */
+  void noteOutput(WindowedValue<?> output);
+
+  /**
+   * Hook for subclasses to implement that will be called whenever
+   * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
+   * is called.
+   */
+  void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);
+
+  /**
+   * Per-step, per-key context used for retrieving state.
+   */
+  public interface StepContext {
+
+    /**
+     * The name of the step.
+     */
+    String getStepName();
+
+    /**
+     * The name of the transform for the step.
+     */
+    String getTransformName();
+
+    /**
+     * Hook for subclasses to implement that will be called whenever
+     * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
+     * is called.
+     */
+    void noteOutput(WindowedValue<?> output);
+
+    /**
+     * Hook for subclasses to implement that will be called whenever
+     * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
+     * is called.
+     */
+    void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);
+
+    /**
+     * Writes the given {@code PCollectionView} data to a globally accessible location.
+     */
+    <T, W extends BoundedWindow> void writePCollectionViewData(
+        TupleTag<?> tag,
+        Iterable<WindowedValue<T>> data,
+        Coder<Iterable<WindowedValue<T>>> dataCoder,
+        W window,
+        Coder<W> windowCoder)
+            throws IOException;
+
+    StateInternals<?> stateInternals();
+
+    TimerInternals timerInternals();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index a7d82bf..b42c57d 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Set;
 import javax.annotation.Nullable;
 import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
@@ -47,7 +48,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.TimeDomain;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
index 342a4a8..1ff0212 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
@@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.TimeDomain;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
index ec5d375..8ae09cb 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
@@ -26,13 +26,13 @@ import static org.mockito.Mockito.when;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import org.apache.beam.runners.core.BaseExecutionContext.StepContext;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
 import org.apache.beam.sdk.util.NullSideInputReader;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.Timer;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
index 0e23dcb..4610069 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
@@ -22,9 +22,8 @@ import static org.mockito.Mockito.mock;
 
 import java.util.Arrays;
 import java.util.List;
-
+import org.apache.beam.runners.core.BaseExecutionContext.StepContext;
 import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java
index c7fa4df..fd17704 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java
@@ -28,9 +28,9 @@ import java.util.concurrent.ConcurrentMap;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.GuardedBy;
 import org.apache.beam.runners.core.AggregatorFactory;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.util.ExecutionContext;
 
 /**
  * AccumT container for the current values associated with {@link Aggregator Aggregators}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
index c6051f0..8250cf1 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
@@ -17,11 +17,11 @@
  */
 package org.apache.beam.runners.direct;
 
+import org.apache.beam.runners.core.BaseExecutionContext;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
 import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.util.BaseExecutionContext;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.TimerInternals;
 
 /**
@@ -54,7 +54,7 @@ class DirectExecutionContext
    * Step Context for the {@link DirectRunner}.
    */
   public class DirectStepContext
-      extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext {
+      extends BaseExecutionContext.StepContext {
     private CopyOnAccessInMemoryStateInternals<Object> stateInternals;
     private DirectTimerInternals timerInternals;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index bbcab8e..3b9367a 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -31,6 +31,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import javax.annotation.Nullable;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.direct.CommittedResult.OutputType;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
@@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java
index c8310c9..f770800 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java
@@ -24,9 +24,9 @@ import static org.mockito.Mockito.when;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Sum.SumIntegerFn;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index 8704308..057a3e7 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import org.apache.beam.runners.core.AggregatorFactory;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
@@ -46,7 +47,6 @@ import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.NullSideInputReader;
 import org.apache.beam.sdk.util.SideInputReader;
 import org.apache.beam.sdk.util.TimerInternals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index 9cea529..9855d46 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -38,6 +38,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ScheduledFuture;
 import javax.annotation.Nullable;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
@@ -48,7 +49,6 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java
index 17d5844..fa5c8d1 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java
@@ -22,11 +22,11 @@ import com.google.common.collect.ImmutableList;
 import java.util.Collection;
 import java.util.Map;
 import org.apache.beam.runners.core.AggregatorFactory;
+import org.apache.beam.runners.core.ExecutionContext;
 import org.apache.beam.runners.spark.translation.SparkRuntimeContext;
 import org.apache.beam.sdk.AggregatorValues;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.util.ExecutionContext;
 import org.apache.spark.Accumulator;
 import org.apache.spark.api.java.JavaSparkContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
index 3a31cae..9957bf3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java
@@ -24,11 +24,11 @@ import java.io.IOException;
 import java.util.Iterator;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.state.StateInternals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
deleted file mode 100644
index e26f2b0..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Base class for implementations of {@link ExecutionContext}.
- *
- * <p>A concrete subclass should implement {@link #createStepContext} to create the appropriate
- * {@link StepContext} implementation. Any {@code StepContext} created will
- * be cached for the lifetime of this {@link ExecutionContext}.
- *
- * <p>BaseExecutionContext is generic to allow implementing subclasses to return a concrete subclass
- * of {@link StepContext} from {@link #getOrCreateStepContext(String, String)} and
- * {@link #getAllStepContexts()} without forcing each subclass to override the method, e.g.
- * <pre>{@code
- * {@literal @}Override
- * StreamingModeExecutionContext.StepContext getOrCreateStepContext(...) {
- *   return (StreamingModeExecutionContext.StepContext) super.getOrCreateStepContext(...);
- * }
- * }</pre>
- *
- * <p>When a subclass of {@code BaseExecutionContext} has been downcast, the return types of
- * {@link #createStepContext(String, String)},
- * {@link #getOrCreateStepContext(String, String)}, and {@link #getAllStepContexts()}
- * will be appropriately specialized.
- */
-public abstract class BaseExecutionContext<T extends ExecutionContext.StepContext>
-    implements ExecutionContext {
-
-  private Map<String, T> cachedStepContexts = new HashMap<>();
-
-  /**
-   * Implementations should override this to create the specific type
-   * of {@link StepContext} they need.
-   */
-  protected abstract T createStepContext(String stepName, String transformName);
-
-  /**
-   * Returns the {@link StepContext} associated with the given step.
-   */
-  @Override
-  public T getOrCreateStepContext(String stepName, String transformName) {
-    final String finalStepName = stepName;
-    final String finalTransformName = transformName;
-    return getOrCreateStepContext(
-        stepName,
-        new CreateStepContextFunction<T>() {
-          @Override
-          public T create() {
-            return createStepContext(finalStepName, finalTransformName);
-          }
-        });
-  }
-
-  /**
-   * Factory method interface to create an execution context if none exists during
-   * {@link #getOrCreateStepContext(String, CreateStepContextFunction)}.
-   */
-  protected interface CreateStepContextFunction<T extends ExecutionContext.StepContext> {
-    T create();
-  }
-
-  protected final T getOrCreateStepContext(String stepName,
-      CreateStepContextFunction<T> createContextFunc) {
-    T context = cachedStepContexts.get(stepName);
-    if (context == null) {
-      context = createContextFunc.create();
-      cachedStepContexts.put(stepName, context);
-    }
-
-    return context;
-  }
-
-  /**
-   * Returns a collection view of all of the {@link StepContext}s.
-   */
-  @Override
-  public Collection<? extends T> getAllStepContexts() {
-    return Collections.unmodifiableCollection(cachedStepContexts.values());
-  }
-
-  /**
-   * Hook for subclasses to implement that will be called whenever
-   * {@code DoFn.Context#output}
-   * is called.
-   */
-  @Override
-  public void noteOutput(WindowedValue<?> output) {}
-
-  /**
-   * Hook for subclasses to implement that will be called whenever
-   * {@code DoFn.Context#sideOutput}
-   * is called.
-   */
-  @Override
-  public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output) {}
-
-  /**
-   * Base class for implementations of {@link ExecutionContext.StepContext}.
-   *
-   * <p>To complete a concrete subclass, implement {@link #timerInternals} and
-   * {@link #stateInternals}.
-   */
-  public abstract static class StepContext implements ExecutionContext.StepContext {
-    private final ExecutionContext executionContext;
-    private final String stepName;
-    private final String transformName;
-
-    public StepContext(ExecutionContext executionContext, String stepName, String transformName) {
-      this.executionContext = executionContext;
-      this.stepName = stepName;
-      this.transformName = transformName;
-    }
-
-    @Override
-    public String getStepName() {
-      return stepName;
-    }
-
-    @Override
-    public String getTransformName() {
-      return transformName;
-    }
-
-    @Override
-    public void noteOutput(WindowedValue<?> output) {
-      executionContext.noteOutput(output);
-    }
-
-    @Override
-    public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output) {
-      executionContext.noteSideOutput(tag, output);
-    }
-
-    @Override
-    public <T, W extends BoundedWindow> void writePCollectionViewData(
-        TupleTag<?> tag,
-        Iterable<WindowedValue<T>> data, Coder<Iterable<WindowedValue<T>>> dataCoder,
-        W window, Coder<W> windowCoder) throws IOException {
-      throw new UnsupportedOperationException("Not implemented.");
-    }
-
-    @Override
-    public abstract StateInternals<?> stateInternals();
-
-    @Override
-    public abstract TimerInternals timerInternals();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d2b8e09/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
deleted file mode 100644
index 4429d76..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.TupleTag;
-
-/**
- * Context for the current execution. This is guaranteed to exist during processing,
- * but does not necessarily persist between different batches of work.
- */
-public interface ExecutionContext {
-  /**
-   * Returns the {@link StepContext} associated with the given step.
-   */
-  StepContext getOrCreateStepContext(String stepName, String transformName);
-
-  /**
-   * Returns a collection view of all of the {@link StepContext}s.
-   */
-  Collection<? extends StepContext> getAllStepContexts();
-
-  /**
-   * Hook for subclasses to implement that will be called whenever
-   * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
-   * is called.
-   */
-  void noteOutput(WindowedValue<?> output);
-
-  /**
-   * Hook for subclasses to implement that will be called whenever
-   * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
-   * is called.
-   */
-  void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);
-
-  /**
-   * Per-step, per-key context used for retrieving state.
-   */
-  public interface StepContext {
-
-    /**
-     * The name of the step.
-     */
-    String getStepName();
-
-    /**
-     * The name of the transform for the step.
-     */
-    String getTransformName();
-
-    /**
-     * Hook for subclasses to implement that will be called whenever
-     * {@link org.apache.beam.sdk.transforms.DoFn.Context#output}
-     * is called.
-     */
-    void noteOutput(WindowedValue<?> output);
-
-    /**
-     * Hook for subclasses to implement that will be called whenever
-     * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput}
-     * is called.
-     */
-    void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output);
-
-    /**
-     * Writes the given {@code PCollectionView} data to a globally accessible location.
-     */
-    <T, W extends BoundedWindow> void writePCollectionViewData(
-        TupleTag<?> tag,
-        Iterable<WindowedValue<T>> data,
-        Coder<Iterable<WindowedValue<T>>> dataCoder,
-        W window,
-        Coder<W> windowCoder)
-            throws IOException;
-
-    StateInternals<?> stateInternals();
-
-    TimerInternals timerInternals();
-  }
-}



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

Posted by ke...@apache.org.
This closes #1666: Move ExecutionContext and related classes to runners-core

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


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

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

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


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


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

Posted by ke...@apache.org.
Port direct runner StatefulParDo to KeyedWorkItem


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

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

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index c5bc069..2cea999 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -17,9 +17,15 @@
  */
 package org.apache.beam.runners.direct;
 
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItemCoder;
+import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -28,6 +34,8 @@ import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
@@ -84,16 +92,41 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
     @Override
     public PCollectionTuple expand(PCollection<KV<K, InputT>> input) {
 
-      PCollectionTuple outputs = input
-          .apply("Group by key", GroupByKey.<K, InputT>create())
-          .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input));
+      // A KvCoder is required since this goes through GBK. Further, WindowedValueCoder
+      // is not registered by default, so we explicitly set the relevant coders.
+      checkState(input.getCoder() instanceof KvCoder,
+          "Input to a %s using state requires a %s, but the coder was %s",
+          ParDo.class.getSimpleName(),
+          KvCoder.class.getSimpleName(),
+          input.getCoder());
+      KvCoder<K, InputT> kvCoder = (KvCoder<K, InputT>) input.getCoder();
+      Coder<K> keyCoder = kvCoder.getKeyCoder();
+      Coder<? extends BoundedWindow> windowCoder =
+          input.getWindowingStrategy().getWindowFn().windowCoder();
+
+      PCollectionTuple outputs =
+          input
+              // Stash the original timestamps, etc, for when it is fed to the user's DoFn
+              .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn<K, InputT>()))
+              .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder)))
+
+              // A full GBK to group by key _and_ window
+              .apply("Group by key", GroupByKey.<K, WindowedValue<KV<K, InputT>>>create())
+
+              // Adapt to KeyedWorkItem; that is how this runner delivers timers
+              .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem<K, InputT>()))
+              .setCoder(KeyedWorkItemCoder.of(keyCoder, kvCoder, windowCoder))
+
+              // Explode the resulting iterable into elements that are exactly the ones from
+              // the input
+              .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input));
 
       return outputs;
     }
   }
 
   static class StatefulParDo<K, InputT, OutputT>
-      extends PTransform<PCollection<? extends KV<K, Iterable<InputT>>>, PCollectionTuple> {
+      extends PTransform<PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>>, PCollectionTuple> {
     private final transient ParDo.BoundMulti<KV<K, InputT>, OutputT> underlyingParDo;
     private final transient PCollection<KV<K, InputT>> originalInput;
 
@@ -110,21 +143,58 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
 
     @Override
     public <T> Coder<T> getDefaultOutputCoder(
-        PCollection<? extends KV<K, Iterable<InputT>>> input, TypedPValue<T> output)
+        PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>> input, TypedPValue<T> output)
         throws CannotProvideCoderException {
       return underlyingParDo.getDefaultOutputCoder(originalInput, output);
     }
 
-    public PCollectionTuple expand(PCollection<? extends KV<K, Iterable<InputT>>> input) {
+    @Override
+    public PCollectionTuple expand(PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>> input) {
 
-      PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal(
-          input.getPipeline(),
-          TupleTagList.of(underlyingParDo.getMainOutputTag())
-              .and(underlyingParDo.getSideOutputTags().getAll()),
-          input.getWindowingStrategy(),
-          input.isBounded());
+      PCollectionTuple outputs =
+          PCollectionTuple.ofPrimitiveOutputsInternal(
+              input.getPipeline(),
+              TupleTagList.of(underlyingParDo.getMainOutputTag())
+                  .and(underlyingParDo.getSideOutputTags().getAll()),
+              input.getWindowingStrategy(),
+              input.isBounded());
 
       return outputs;
     }
   }
+
+  /**
+   * A distinguished key-preserving {@link DoFn}.
+   *
+   * <p>This wraps the {@link GroupByKey} output in a {@link KeyedWorkItem} to be able to deliver
+   * timers. It also explodes them into single {@link KV KVs} since this is what the user's {@link
+   * DoFn} needs to process anyhow.
+   */
+  static class ReifyWindowedValueFn<K, V> extends DoFn<KV<K, V>, KV<K, WindowedValue<KV<K, V>>>> {
+    @ProcessElement
+    public void processElement(final ProcessContext c, final BoundedWindow window) {
+      c.output(
+          KV.of(
+              c.element().getKey(),
+              WindowedValue.of(c.element(), c.timestamp(), window, c.pane())));
+    }
+  }
+
+  /**
+   * A runner-specific primitive that is just a key-preserving {@link ParDo}, but we do not have the
+   * machinery to detect or enforce that yet.
+   *
+   * <p>This wraps the {@link GroupByKey} output in a {@link KeyedWorkItem} to be able to deliver
+   * timers. It also explodes them into single {@link KV KVs} since this is what the user's {@link
+   * DoFn} needs to process anyhow.
+   */
+  static class ToKeyedWorkItem<K, V>
+      extends DoFn<KV<K, Iterable<WindowedValue<KV<K, V>>>>, KeyedWorkItem<K, KV<K, V>>> {
+
+    @ProcessElement
+    public void processElement(final ProcessContext c, final BoundedWindow window) {
+      final K key = c.element().getKey();
+      c.output(KeyedWorkItems.elementsWorkItem(key, c.element().getValue()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
index 1f64d9a..5f9d8f4 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
@@ -23,6 +23,8 @@ import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Lists;
 import java.util.Collections;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo;
@@ -77,12 +79,12 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
-  private TransformEvaluator<KV<K, Iterable<InputT>>> createEvaluator(
+  private TransformEvaluator<KeyedWorkItem<K, KV<K, InputT>>> createEvaluator(
       AppliedPTransform<
-              PCollection<? extends KV<K, Iterable<InputT>>>, PCollectionTuple,
+              PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>>, PCollectionTuple,
               StatefulParDo<K, InputT, OutputT>>
           application,
-      CommittedBundle<KV<K, Iterable<InputT>>> inputBundle)
+      CommittedBundle<KeyedWorkItem<K, KV<K, InputT>>> inputBundle)
       throws Exception {
 
     final DoFn<KV<K, InputT>, OutputT> doFn =
@@ -185,7 +187,7 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
   @AutoValue
   abstract static class AppliedPTransformOutputKeyAndWindow<K, InputT, OutputT> {
     abstract AppliedPTransform<
-            PCollection<? extends KV<K, Iterable<InputT>>>, PCollectionTuple,
+            PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>>, PCollectionTuple,
             StatefulParDo<K, InputT, OutputT>>
         getTransform();
 
@@ -195,7 +197,7 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
 
     static <K, InputT, OutputT> AppliedPTransformOutputKeyAndWindow<K, InputT, OutputT> create(
         AppliedPTransform<
-                PCollection<? extends KV<K, Iterable<InputT>>>, PCollectionTuple,
+                PCollection<? extends KeyedWorkItem<K, KV<K, InputT>>>, PCollectionTuple,
                 StatefulParDo<K, InputT, OutputT>>
             transform,
         StructuralKey<K> key,
@@ -206,7 +208,7 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
   }
 
   private static class StatefulParDoEvaluator<K, InputT>
-      implements TransformEvaluator<KV<K, Iterable<InputT>>> {
+      implements TransformEvaluator<KeyedWorkItem<K, KV<K, InputT>>> {
 
     private final TransformEvaluator<KV<K, InputT>> delegateEvaluator;
 
@@ -215,20 +217,20 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
     }
 
     @Override
-    public void processElement(WindowedValue<KV<K, Iterable<InputT>>> gbkResult) throws Exception {
+    public void processElement(WindowedValue<KeyedWorkItem<K, KV<K, InputT>>> gbkResult)
+        throws Exception {
 
-      for (InputT value : gbkResult.getValue().getValue()) {
-        delegateEvaluator.processElement(
-            gbkResult.withValue(KV.of(gbkResult.getValue().getKey(), value)));
+      for (WindowedValue<KV<K, InputT>> windowedValue : gbkResult.getValue().elementsIterable()) {
+        delegateEvaluator.processElement(windowedValue);
       }
     }
 
     @Override
-    public TransformResult<KV<K, Iterable<InputT>>> finishBundle() throws Exception {
+    public TransformResult<KeyedWorkItem<K, KV<K, InputT>>> finishBundle() throws Exception {
       TransformResult<KV<K, InputT>> delegateResult = delegateEvaluator.finishBundle();
 
-      StepTransformResult.Builder<KV<K, Iterable<InputT>>> regroupedResult =
-          StepTransformResult.<KV<K, Iterable<InputT>>>withHold(
+      StepTransformResult.Builder<KeyedWorkItem<K, KV<K, InputT>>> regroupedResult =
+          StepTransformResult.<KeyedWorkItem<K, KV<K, InputT>>>withHold(
                   delegateResult.getTransform(), delegateResult.getWatermarkHold())
               .withTimerUpdate(delegateResult.getTimerUpdate())
               .withAggregatorChanges(delegateResult.getAggregatorChanges())
@@ -240,12 +242,10 @@ final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements Transfo
       // outputs, but just make a bunch of singletons
       for (WindowedValue<?> untypedUnprocessed : delegateResult.getUnprocessedElements()) {
         WindowedValue<KV<K, InputT>> windowedKv = (WindowedValue<KV<K, InputT>>) untypedUnprocessed;
-        WindowedValue<KV<K, Iterable<InputT>>> pushedBack =
+        WindowedValue<KeyedWorkItem<K, KV<K, InputT>>> pushedBack =
             windowedKv.withValue(
-                KV.of(
-                    windowedKv.getValue().getKey(),
-                    (Iterable<InputT>)
-                        Collections.singletonList(windowedKv.getValue().getValue())));
+                KeyedWorkItems.elementsWorkItem(
+                    windowedKv.getValue().getKey(), Collections.singleton(windowedKv)));
 
         regroupedResult.addUnprocessedElements(pushedBack);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/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 a357005..a1fb81b 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
@@ -22,8 +22,10 @@ import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
 import java.util.Collections;
+import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -32,8 +34,12 @@ 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.ParDo;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -41,9 +47,7 @@ import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * Tests for {@link KeyedPValueTrackingVisitor}.
- */
+/** Tests for {@link KeyedPValueTrackingVisitor}. */
 @RunWith(JUnit4.class)
 public class KeyedPValueTrackingVisitorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
@@ -61,8 +65,7 @@ public class KeyedPValueTrackingVisitorTest {
   @Test
   public void groupByKeyProducesKeyedOutput() {
     PCollection<KV<String, Iterable<Integer>>> keyed =
-        p.apply(Create.of(KV.of("foo", 3)))
-            .apply(GroupByKey.<String, Integer>create());
+        p.apply(Create.of(KV.of("foo", 3))).apply(GroupByKey.<String, Integer>create());
 
     p.traverseTopologically(visitor);
     assertThat(visitor.getKeyedPValues(), hasItem(keyed));
@@ -91,16 +94,66 @@ public class KeyedPValueTrackingVisitorTest {
   }
 
   @Test
+  public void unkeyedInputWithKeyPreserving() {
+
+    PCollection<KV<String, Iterable<WindowedValue<KV<String, Integer>>>>> input =
+        p.apply(
+            Create.of(
+                    KV.of(
+                        "hello",
+                        (Iterable<WindowedValue<KV<String, Integer>>>)
+                            Collections.<WindowedValue<KV<String, Integer>>>emptyList()))
+                .withCoder(
+                    KvCoder.of(
+                        StringUtf8Coder.of(),
+                        IterableCoder.of(
+                            WindowedValue.getValueOnlyCoder(
+                                KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))))));
+
+    PCollection<KeyedWorkItem<String, KV<String, Integer>>> unkeyed =
+        input.apply(ParDo.of(new ParDoMultiOverrideFactory.ToKeyedWorkItem<String, Integer>()));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
+  }
+
+  @Test
+  public void keyedInputWithKeyPreserving() {
+
+    PCollection<KV<String, WindowedValue<KV<String, Integer>>>> input =
+        p.apply(
+            Create.of(
+                    KV.of(
+                        "hello",
+                        WindowedValue.of(
+                            KV.of("hello", 3),
+                            new Instant(0),
+                            new IntervalWindow(new Instant(0), new Instant(9)),
+                            PaneInfo.NO_FIRING)))
+                .withCoder(
+                    KvCoder.of(
+                        StringUtf8Coder.of(),
+                        WindowedValue.getValueOnlyCoder(
+                            KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())))));
+
+    PCollection<KeyedWorkItem<String, KV<String, Integer>>> keyed =
+        input
+            .apply(GroupByKey.<String, WindowedValue<KV<String, Integer>>>create())
+            .apply(ParDo.of(new ParDoMultiOverrideFactory.ToKeyedWorkItem<String, Integer>()));
+
+    p.traverseTopologically(visitor);
+    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+  }
+
+  @Test
   public void traverseMultipleTimesThrows() {
     p.apply(
-            Create.<KV<Integer, Void>>of(
-                    KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
+            Create.of(KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null))
                 .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of())))
         .apply(GroupByKey.<Integer, Void>create())
         .apply(Keys.<Integer>create());
 
     p.traverseTopologically(visitor);
-
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("already been finalized");
     thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1f018ab6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
index d312aa3..b88d5e0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
@@ -27,12 +27,14 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import org.apache.beam.runners.core.KeyedWorkItem;
+import org.apache.beam.runners.core.KeyedWorkItems;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo;
@@ -136,7 +138,7 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
         new StatefulParDoEvaluatorFactory(mockEvaluationContext);
 
     AppliedPTransform<
-            PCollection<? extends KV<String, Iterable<Integer>>>, PCollectionTuple,
+            PCollection<? extends KeyedWorkItem<String, KV<String, Integer>>>, PCollectionTuple,
             StatefulParDo<String, Integer, Integer>>
         producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced);
 
@@ -245,7 +247,7 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
 
     // This will be the stateful ParDo from the expansion
     AppliedPTransform<
-            PCollection<KV<String, Iterable<Integer>>>, PCollectionTuple,
+            PCollection<KeyedWorkItem<String, KV<String, Integer>>>, PCollectionTuple,
             StatefulParDo<String, Integer, Integer>>
         producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced);
 
@@ -270,37 +272,50 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
     // A single bundle with some elements in the global window; it should register cleanup for the
     // global window state merely by having the evaluator created. The cleanup logic does not
     // depend on the window.
-    WindowedValue<KV<String, Iterable<Integer>>> gbkOutputElement =
-        WindowedValue.of(
-            KV.<String, Iterable<Integer>>of("hello", Lists.newArrayList(1, 13, 15)),
-            new Instant(3),
-            firstWindow,
-            PaneInfo.NO_FIRING);
-    CommittedBundle<KV<String, Iterable<Integer>>> inputBundle =
+    String key = "hello";
+    WindowedValue<KV<String, Integer>> firstKv = WindowedValue.of(
+        KV.of(key, 1),
+        new Instant(3),
+        firstWindow,
+        PaneInfo.NO_FIRING);
+
+    WindowedValue<KeyedWorkItem<String, KV<String, Integer>>> gbkOutputElement =
+        firstKv.withValue(
+            KeyedWorkItems.elementsWorkItem(
+                "hello",
+                ImmutableList.of(
+                    firstKv,
+                    firstKv.withValue(KV.of(key, 13)),
+                    firstKv.withValue(KV.of(key, 15)))));
+
+    CommittedBundle<KeyedWorkItem<String, KV<String, Integer>>> inputBundle =
         BUNDLE_FACTORY
             .createBundle(producingTransform.getInput())
             .add(gbkOutputElement)
             .commit(Instant.now());
-    TransformEvaluator<KV<String, Iterable<Integer>>> evaluator =
+    TransformEvaluator<KeyedWorkItem<String, KV<String, Integer>>> evaluator =
         factory.forApplication(producingTransform, inputBundle);
+
     evaluator.processElement(gbkOutputElement);
 
     // This should push back every element as a KV<String, Iterable<Integer>>
     // in the appropriate window. Since the keys are equal they are single-threaded
-    TransformResult<KV<String, Iterable<Integer>>> result = evaluator.finishBundle();
+    TransformResult<KeyedWorkItem<String, KV<String, Integer>>> result =
+        evaluator.finishBundle();
 
     List<Integer> pushedBackInts = new ArrayList<>();
 
-    for (WindowedValue<?> unprocessedElement : result.getUnprocessedElements()) {
-      WindowedValue<KV<String, Iterable<Integer>>> unprocessedKv =
-          (WindowedValue<KV<String, Iterable<Integer>>>) unprocessedElement;
+    for (WindowedValue<? extends KeyedWorkItem<String, KV<String, Integer>>> unprocessedElement :
+        result.getUnprocessedElements()) {
 
       assertThat(
           Iterables.getOnlyElement(unprocessedElement.getWindows()),
           equalTo((BoundedWindow) firstWindow));
-      assertThat(unprocessedKv.getValue().getKey(), equalTo("hello"));
-      for (Integer i : unprocessedKv.getValue().getValue()) {
-        pushedBackInts.add(i);
+
+      assertThat(unprocessedElement.getValue().key(), equalTo("hello"));
+      for (WindowedValue<KV<String, Integer>> windowedKv :
+          unprocessedElement.getValue().elementsIterable()) {
+        pushedBackInts.add(windowedKv.getValue().getValue());
       }
     }
     assertThat(pushedBackInts, containsInAnyOrder(1, 13, 15));


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

Posted by ke...@apache.org.
This closes #1160: Support set and delete of timer by ID in InMemoryTimerInternals


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

Branch: refs/heads/python-sdk
Commit: 7ee8c86d3b0553d8cb7de60b0dc1a03103dfbbc5
Parents: a9447a2 df2e540
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Dec 21 11:02:02 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Dec 21 11:02:02 2016 -0800

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


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


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

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


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/445c1205/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
index 9189191..efcd771 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
@@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.joda.time.Instant;

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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/445c1205/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
index db0cf91..890195a 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
@@ -73,7 +73,6 @@ import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
 import org.apache.beam.sdk.util.state.StateTag;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/445c1205/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
index 0f0b106..74a566b 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
@@ -28,6 +28,7 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -196,6 +197,8 @@ public class SplittableParDoTest {
         tester;
     private Instant currentProcessingTime;
 
+    private InMemoryTimerInternals timerInternals;
+
     ProcessFnTester(
         Instant currentProcessingTime,
         DoFn<InputT, OutputT> fn,
@@ -206,6 +209,7 @@ public class SplittableParDoTest {
           new SplittableParDo.ProcessFn<>(
               fn, inputCoder, restrictionCoder, IntervalWindow.getCoder());
       this.tester = DoFnTester.of(processFn);
+      this.timerInternals = new InMemoryTimerInternals();
       processFn.setStateInternalsFactory(
           new StateInternalsFactory<String>() {
             @Override
@@ -217,7 +221,7 @@ public class SplittableParDoTest {
           new TimerInternalsFactory<String>() {
             @Override
             public TimerInternals timerInternalsForKey(String key) {
-              return tester.getTimerInternals();
+              return timerInternals;
             }
           });
       processFn.setOutputWindowedValue(
@@ -253,7 +257,7 @@ public class SplittableParDoTest {
       // through the state/timer/output callbacks.
       this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE);
       this.tester.startBundle();
-      this.tester.advanceProcessingTime(currentProcessingTime);
+      timerInternals.advanceProcessingTime(currentProcessingTime);
 
       this.currentProcessingTime = currentProcessingTime;
     }
@@ -291,7 +295,13 @@ public class SplittableParDoTest {
      */
     boolean advanceProcessingTimeBy(Duration duration) throws Exception {
       currentProcessingTime = currentProcessingTime.plus(duration);
-      List<TimerInternals.TimerData> timers = tester.advanceProcessingTime(currentProcessingTime);
+      timerInternals.advanceProcessingTime(currentProcessingTime);
+
+      List<TimerInternals.TimerData> timers = new ArrayList<>();
+      TimerInternals.TimerData nextTimer;
+      while ((nextTimer = timerInternals.removeNextProcessingTimer()) != null) {
+        timers.add(nextTimer);
+      }
       if (timers.isEmpty()) {
         return false;
       }

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

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

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

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

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


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

Posted by ke...@apache.org.
Hold output watermark according to pending timers


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dfe2e62d/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index f7bafd1..248fafd 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -211,12 +211,18 @@ public class WatermarkManager {
   private static class AppliedPTransformInputWatermark implements Watermark {
     private final Collection<? extends Watermark> inputWatermarks;
     private final SortedMultiset<CommittedBundle<?>> pendingElements;
-    private final Map<StructuralKey<?>, NavigableSet<TimerData>> objectTimers;
+
+    // This tracks only the quantity of timers at each timestamp, for quickly getting the cross-key
+    // minimum
+    private final SortedMultiset<Instant> pendingTimers;
 
     // Entries in this table represent the authoritative timestamp for which
     // a per-key-and-StateNamespace timer is set.
     private final Map<StructuralKey<?>, Table<StateNamespace, String, TimerData>> existingTimers;
 
+    // This per-key sorted set allows quick retrieval of timers that should fire for a key
+    private final Map<StructuralKey<?>, NavigableSet<TimerData>> objectTimers;
+
     private AtomicReference<Instant> currentWatermark;
 
     public AppliedPTransformInputWatermark(Collection<? extends Watermark> inputWatermarks) {
@@ -224,10 +230,13 @@ public class WatermarkManager {
       // The ordering must order elements by timestamp, and must not compare two distinct elements
       // as equal. This is built on the assumption that any element added as a pending element will
       // be consumed without modifications.
+      //
+      // The same logic is applied for pending timers
       Ordering<CommittedBundle<?>> pendingBundleComparator =
           new BundleByElementTimestampComparator().compound(Ordering.arbitrary());
       this.pendingElements =
           TreeMultiset.create(pendingBundleComparator);
+      this.pendingTimers = TreeMultiset.create();
       this.objectTimers = new HashMap<>();
       this.existingTimers = new HashMap<>();
       currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
@@ -278,6 +287,14 @@ public class WatermarkManager {
       pendingElements.remove(completed);
     }
 
+    private synchronized Instant getEarliestTimerTimestamp() {
+      if (pendingTimers.isEmpty()) {
+        return BoundedWindow.TIMESTAMP_MAX_VALUE;
+      } else {
+        return pendingTimers.firstEntry().getElement();
+      }
+    }
+
     private synchronized void updateTimers(TimerUpdate update) {
       NavigableSet<TimerData> keyTimers = objectTimers.get(update.key);
       if (keyTimers == null) {
@@ -291,27 +308,43 @@ public class WatermarkManager {
         existingTimers.put(update.key, existingTimersForKey);
       }
 
-      for (TimerData timer : update.setTimers) {
+      for (TimerData timer : update.getSetTimers()) {
+        if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
+          @Nullable
+          TimerData existingTimer =
+              existingTimersForKey.get(timer.getNamespace(), timer.getTimerId());
+
+          if (existingTimer == null) {
+            pendingTimers.add(timer.getTimestamp());
+            keyTimers.add(timer);
+          } else if (!existingTimer.equals(timer)) {
+            keyTimers.remove(existingTimer);
+            keyTimers.add(timer);
+          } // else the timer is already set identically, so noop
+
+          existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer);
+        }
+      }
+
+      for (TimerData timer : update.getDeletedTimers()) {
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
           @Nullable
           TimerData existingTimer =
               existingTimersForKey.get(timer.getNamespace(), timer.getTimerId());
 
           if (existingTimer != null) {
+            pendingTimers.remove(existingTimer.getTimestamp());
             keyTimers.remove(existingTimer);
+            existingTimersForKey.remove(existingTimer.getNamespace(), existingTimer.getTimerId());
           }
-          keyTimers.add(timer);
-          existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer);
         }
       }
 
-      for (TimerData timer : update.deletedTimers) {
+      for (TimerData timer : update.getCompletedTimers()) {
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
-          keyTimers.remove(timer);
-          existingTimersForKey.remove(timer.getNamespace(), timer.getTimerId());
+          pendingTimers.remove(timer.getTimestamp());
         }
       }
-      // We don't keep references to timers that have been fired and delivered via #getFiredTimers()
     }
 
     private synchronized Map<StructuralKey<?>, List<TimerData>> extractFiredEventTimeTimers() {
@@ -336,11 +369,12 @@ public class WatermarkManager {
    * {@link #refresh()} for more information.
    */
   private static class AppliedPTransformOutputWatermark implements Watermark {
-    private final Watermark inputWatermark;
+    private final AppliedPTransformInputWatermark inputWatermark;
     private final PerKeyHolds holds;
     private AtomicReference<Instant> currentWatermark;
 
-    public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) {
+    public AppliedPTransformOutputWatermark(
+        AppliedPTransformInputWatermark inputWatermark) {
       this.inputWatermark = inputWatermark;
       holds = new PerKeyHolds();
       currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE);
@@ -377,7 +411,10 @@ public class WatermarkManager {
     @Override
     public synchronized WatermarkUpdate refresh() {
       Instant oldWatermark = currentWatermark.get();
-      Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold());
+      Instant newWatermark = INSTANT_ORDERING.min(
+          inputWatermark.get(),
+          inputWatermark.getEarliestTimerTimestamp(),
+          holds.getMinHold());
       newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark);
       currentWatermark.set(newWatermark);
       return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule.
Plus, fixed some checkstyle errors from previous modules' migration.


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
index 86450f2..0f09cd1 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
@@ -36,7 +36,6 @@ import java.util.Random;
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter;
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint;
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.CheckpointCoder;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/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
index e5299a2..505d3a2 100644
--- 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
@@ -62,7 +62,7 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class CloningBundleFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
-  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+  @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
 
   private CloningBundleFactory factory = CloningBundleFactory.create();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
index 35245f4..12ef66c 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
@@ -62,14 +62,10 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class CopyOnAccessInMemoryStateInternalsTest {
 
-  @Rule public TestPipeline pipeline = TestPipeline.create();
+  @Rule public final TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
   private String key = "foo";
 
-  public CopyOnAccessInMemoryStateInternalsTest() {
-    pipeline = TestPipeline.create();
-  }
-
   @Test
   public void testGetWithEmpty() {
     CopyOnAccessInMemoryStateInternals<String> internals =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/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 2448078..eccb3a6 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
@@ -47,7 +47,7 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class ImmutabilityCheckingBundleFactoryTest {
 
-  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+  @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
   private ImmutabilityCheckingBundleFactory factory;
   private PCollection<byte[]> created;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
index 46f02cd..3327ccd 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
@@ -57,7 +57,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ImmutableListBundleFactoryTest {
-  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+  @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   private ImmutableListBundleFactory bundleFactory = ImmutableListBundleFactory.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
index a8c4c02..7432e61 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
@@ -69,7 +69,7 @@ public class WriteWithShardingFactoryTest {
   public static final int INPUT_SIZE = 10000;
   @Rule public TemporaryFolder tmp = new TemporaryFolder();
   private WriteWithShardingFactory<Object> factory = new WriteWithShardingFactory<>();
-  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+  @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
 
   @Test
   public void dynamicallyReshardedWrite() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index fea1554..d8e4ef4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -62,6 +62,7 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class PipelineTest {
 
+  @Rule public final TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedLogs logged = ExpectedLogs.none(Pipeline.class);
   @Rule public ExpectedException thrown = ExpectedException.none();
 
@@ -128,8 +129,7 @@ public class PipelineTest {
     PTransform<PCollection<? extends String>, PCollection<String>> myTransform =
         addSuffix("+");
 
-    Pipeline p = TestPipeline.create();
-    PCollection<String> input = p.apply(Create.<String>of(ImmutableList.of("a", "b")));
+    PCollection<String> input = pipeline.apply(Create.<String>of(ImmutableList.of("a", "b")));
 
     PCollection<String> left = input.apply("Left1", myTransform).apply("Left2", myTransform);
     PCollection<String> right = input.apply("Right", myTransform);
@@ -139,7 +139,7 @@ public class PipelineTest {
 
     PAssert.that(both).containsInAnyOrder("a++", "b++", "a+", "b+");
 
-    p.run();
+    pipeline.run();
   }
 
   private static PTransform<PCollection<? extends String>, PCollection<String>> addSuffix(
@@ -162,35 +162,36 @@ public class PipelineTest {
 
   @Test
   public void testStableUniqueNameOff() {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setStableUniqueNames(CheckEnabled.OFF);
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    pipeline.getOptions().setStableUniqueNames(CheckEnabled.OFF);
 
-    p.apply(Create.of(5, 6, 7));
-    p.apply(Create.of(5, 6, 7));
+    pipeline.apply(Create.of(5, 6, 7));
+    pipeline.apply(Create.of(5, 6, 7));
 
     logged.verifyNotLogged("does not have a stable unique name.");
   }
 
   @Test
   public void testStableUniqueNameWarning() {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setStableUniqueNames(CheckEnabled.WARNING);
+    pipeline.enableAbandonedNodeEnforcement(false);
 
-    p.apply(Create.of(5, 6, 7));
-    p.apply(Create.of(5, 6, 7));
+    pipeline.getOptions().setStableUniqueNames(CheckEnabled.WARNING);
+
+    pipeline.apply(Create.of(5, 6, 7));
+    pipeline.apply(Create.of(5, 6, 7));
 
     logged.verifyWarn("does not have a stable unique name.");
   }
 
   @Test
   public void testStableUniqueNameError() {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setStableUniqueNames(CheckEnabled.ERROR);
+    pipeline.getOptions().setStableUniqueNames(CheckEnabled.ERROR);
 
-    p.apply(Create.of(5, 6, 7));
+    pipeline.apply(Create.of(5, 6, 7));
 
     thrown.expectMessage("does not have a stable unique name.");
-    p.apply(Create.of(5, 6, 7));
+    pipeline.apply(Create.of(5, 6, 7));
   }
 
   /**
@@ -199,7 +200,6 @@ public class PipelineTest {
   @Test
   @Category(RunnableOnService.class)
   public void testIdentityTransform() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> output = pipeline
         .apply(Create.<Integer>of(1, 2, 3, 4))
@@ -223,8 +223,6 @@ public class PipelineTest {
   @Test
   @Category(RunnableOnService.class)
   public void testTupleProjectionTransform() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<Integer> input = pipeline
         .apply(Create.<Integer>of(1, 2, 3, 4));
 
@@ -258,8 +256,6 @@ public class PipelineTest {
   @Test
   @Category(RunnableOnService.class)
   public void testTupleInjectionTransform() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<Integer> input = pipeline
         .apply(Create.<Integer>of(1, 2, 3, 4));
 
@@ -292,7 +288,6 @@ public class PipelineTest {
   @Test
   @Category(NeedsRunner.class)
   public void testEmptyPipeline() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     pipeline.run();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
index adfa0d2..60dc07a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
@@ -55,7 +55,6 @@ import org.apache.avro.reflect.Stringable;
 import org.apache.avro.reflect.Union;
 import org.apache.avro.specific.SpecificData;
 import org.apache.avro.util.Utf8;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder.Context;
 import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
 import org.apache.beam.sdk.testing.CoderProperties;
@@ -73,6 +72,7 @@ import org.hamcrest.Matcher;
 import org.hamcrest.Matchers;
 import org.hamcrest.TypeSafeMatcher;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -143,6 +143,9 @@ public class AvroCoderTest {
     }
   }
 
+  @Rule
+  public TestPipeline pipeline = TestPipeline.create();
+
   @Test
   public void testAvroCoderEncoding() throws Exception {
     AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class);
@@ -287,17 +290,15 @@ public class AvroCoderTest {
   @Test
   @Category(NeedsRunner.class)
   public void testDefaultCoder() throws Exception {
-    Pipeline p = TestPipeline.create();
-
     // Use MyRecord as input and output types without explicitly specifying
     // a coder (this uses the default coders, which may not be AvroCoder).
     PCollection<String> output =
-        p.apply(Create.of(new Pojo("hello", 1), new Pojo("world", 2)))
+        pipeline.apply(Create.of(new Pojo("hello", 1), new Pojo("world", 2)))
             .apply(ParDo.of(new GetTextFn()));
 
     PAssert.that(output)
         .containsInAnyOrder("hello", "world");
-    p.run();
+    pipeline.run();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
index d7badab..8c0e584 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
@@ -35,7 +35,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CoderRegistry.IncompatibleCoderException;
 import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
 import org.apache.beam.sdk.testing.NeedsRunner;
@@ -63,6 +62,9 @@ import org.junit.runners.JUnit4;
 public class CoderRegistryTest {
 
   @Rule
+  public TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   public static CoderRegistry getStandardRegistry() {
@@ -414,7 +416,6 @@ public class CoderRegistryTest {
   @Test
   @Category(NeedsRunner.class)
   public void testSpecializedButIgnoredGenericInPipeline() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline
         .apply(Create.of("hello", "goodbye"))
@@ -443,7 +444,6 @@ public class CoderRegistryTest {
   @Test
   @Category(NeedsRunner.class)
   public void testIgnoredGenericInPipeline() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline
         .apply(Create.of("hello", "goodbye"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
index 8d344de..296ddc9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
@@ -26,7 +26,6 @@ import java.io.Serializable;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -40,6 +39,7 @@ import org.apache.beam.sdk.util.Serializer;
 import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.Matchers;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -98,6 +98,9 @@ public class SerializableCoderTest implements Serializable {
       "To be,",
       "or not to be");
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testSerializableCoder() throws Exception {
     IterableCoder<MyRecord> coder = IterableCoder
@@ -136,7 +139,7 @@ public class SerializableCoderTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testDefaultCoder() throws Exception {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(true);
 
     // Use MyRecord as input and output types without explicitly specifying
     // a coder (this uses the default coders, which may not be

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/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 41a630f..b669968 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
@@ -74,6 +74,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class AvroIOTest {
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
 
@@ -135,7 +139,6 @@ public class AvroIOTest {
   @Test
   @Category(NeedsRunner.class)
   public void testAvroIOWriteAndReadASingleFile() throws Throwable {
-    TestPipeline p = TestPipeline.create();
     List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"),
         new GenericClass(5, "bar"));
     File outputFile = tmpFolder.newFile("output.avro");
@@ -146,7 +149,6 @@ public class AvroIOTest {
           .withSchema(GenericClass.class));
     p.run();
 
-    p = TestPipeline.create();
     PCollection<GenericClass> input = p
         .apply(AvroIO.Read.from(outputFile.getAbsolutePath()).withSchema(GenericClass.class));
 
@@ -158,7 +160,6 @@ public class AvroIOTest {
   @SuppressWarnings("unchecked")
   @Category(NeedsRunner.class)
   public void testAvroIOCompressedWriteAndReadASingleFile() throws Throwable {
-    TestPipeline p = TestPipeline.create();
     List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"),
         new GenericClass(5, "bar"));
     File outputFile = tmpFolder.newFile("output.avro");
@@ -170,7 +171,6 @@ public class AvroIOTest {
             .withSchema(GenericClass.class));
     p.run();
 
-    p = TestPipeline.create();
     PCollection<GenericClass> input = p
         .apply(AvroIO.Read
             .from(outputFile.getAbsolutePath())
@@ -187,7 +187,6 @@ public class AvroIOTest {
   @SuppressWarnings("unchecked")
   @Category(NeedsRunner.class)
   public void testAvroIONullCodecWriteAndReadASingleFile() throws Throwable {
-    TestPipeline p = TestPipeline.create();
     List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"),
         new GenericClass(5, "bar"));
     File outputFile = tmpFolder.newFile("output.avro");
@@ -199,7 +198,6 @@ public class AvroIOTest {
             .withCodec(CodecFactory.nullCodec()));
     p.run();
 
-    p = TestPipeline.create();
     PCollection<GenericClass> input = p
         .apply(AvroIO.Read
             .from(outputFile.getAbsolutePath())
@@ -257,7 +255,6 @@ public class AvroIOTest {
   @Test
   @Category(NeedsRunner.class)
   public void testAvroIOWriteAndReadSchemaUpgrade() throws Throwable {
-    TestPipeline p = TestPipeline.create();
     List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"),
         new GenericClass(5, "bar"));
     File outputFile = tmpFolder.newFile("output.avro");
@@ -270,7 +267,7 @@ public class AvroIOTest {
 
     List<GenericClassV2> expected = ImmutableList.of(new GenericClassV2(3, "hi", null),
         new GenericClassV2(5, "bar", null));
-    p = TestPipeline.create();
+
     PCollection<GenericClassV2> input = p
         .apply(AvroIO.Read.from(outputFile.getAbsolutePath()).withSchema(GenericClassV2.class));
 
@@ -321,7 +318,6 @@ public class AvroIOTest {
   @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");
@@ -348,7 +344,7 @@ public class AvroIOTest {
   private void runTestWrite(String[] expectedElements, int numShards) throws IOException {
     File baseOutputFile = new File(tmpFolder.getRoot(), "prefix");
     String outputFilePrefix = baseOutputFile.getAbsolutePath();
-    TestPipeline p = TestPipeline.create();
+
     Bound<String> write = AvroIO.Write.to(outputFilePrefix).withSchema(String.class);
     if (numShards > 1) {
       write = write.withNumShards(numShards);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
index 4d7814c..d49873e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
@@ -26,7 +26,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -36,6 +35,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -46,6 +46,9 @@ import org.junit.runners.JUnit4;
 public class BoundedReadFromUnboundedSourceTest implements Serializable{
   private static final int NUM_RECORDS = 100;
 
+  @Rule
+  public transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testNoDedup() throws Exception {
@@ -112,7 +115,6 @@ public class BoundedReadFromUnboundedSourceTest implements Serializable{
   }
 
   private void test(boolean dedup, boolean timeBound) throws Exception {
-    Pipeline p = TestPipeline.create();
 
     TestCountingSource source = new TestCountingSource(Integer.MAX_VALUE).withoutSplitting();
     if (dedup) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
index f8769ea..3871159 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
@@ -47,7 +47,6 @@ import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.zip.GZIPOutputStream;
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
@@ -80,6 +79,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class CompressedSourceTest {
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
 
@@ -199,8 +202,6 @@ public class CompressedSourceTest {
       os.write(totalGz);
     }
 
-    Pipeline p = TestPipeline.create();
-
     CompressedSource<Byte> source =
         CompressedSource.from(new ByteSource(tmpFile.getAbsolutePath(), 1))
             .withDecompression(CompressionMode.GZIP);
@@ -274,8 +275,6 @@ public class CompressedSourceTest {
 
     String filePattern = new File(tmpFolder.getRoot().toString(), baseName + ".*").toString();
 
-    Pipeline p = TestPipeline.create();
-
     CompressedSource<Byte> source =
         CompressedSource.from(new ByteSource(filePattern, 1));
     PCollection<Byte> output = p.apply(Read.from(source));
@@ -395,8 +394,6 @@ public class CompressedSourceTest {
       expected.addAll(Bytes.asList(generated));
     }
 
-    Pipeline p = TestPipeline.create();
-
     CompressedSource<Byte> source =
         CompressedSource.from(new ByteSource(filePattern, 1))
             .withDecompression(CompressionMode.GZIP);
@@ -476,7 +473,6 @@ public class CompressedSourceTest {
 
   private void verifyReadContents(byte[] expected, File inputFile,
       @Nullable DecompressingChannelFactory decompressionFactory) {
-    Pipeline p = TestPipeline.create();
     CompressedSource<Byte> source =
         CompressedSource.from(new ByteSource(inputFile.toPath().toString(), 1));
     if (decompressionFactory != null) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
index dfc4919..f23ee76 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java
@@ -21,7 +21,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -39,6 +38,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -66,10 +66,12 @@ public class CountingInputTest {
         .isEqualTo(end - 1);
   }
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testBoundedInput() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
     PCollection<Long> input = p.apply(CountingInput.upTo(numElements));
 
@@ -80,7 +82,6 @@ public class CountingInputTest {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyBoundedInput() {
-    Pipeline p = TestPipeline.create();
     PCollection<Long> input = p.apply(CountingInput.upTo(0));
 
     PAssert.that(input).empty();
@@ -90,7 +91,6 @@ public class CountingInputTest {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyBoundedInputSubrange() {
-    Pipeline p = TestPipeline.create();
     PCollection<Long> input = p.apply(CountingInput.forSubrange(42, 42));
 
     PAssert.that(input).empty();
@@ -101,7 +101,6 @@ public class CountingInputTest {
   @Test
   @Category(RunnableOnService.class)
   public void testBoundedInputSubrange() {
-    Pipeline p = TestPipeline.create();
     long start = 10;
     long end = 1000;
     PCollection<Long> input = p.apply(CountingInput.forSubrange(start, end));
@@ -128,7 +127,6 @@ public class CountingInputTest {
   @Test
   @Category(RunnableOnService.class)
   public void testUnboundedInput() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
 
     PCollection<Long> input = p.apply(CountingInput.unbounded().withMaxNumRecords(numElements));
@@ -140,7 +138,6 @@ public class CountingInputTest {
   @Test
   @Category(NeedsRunner.class)
   public void testUnboundedInputRate() {
-    Pipeline p = TestPipeline.create();
     long numElements = 5000;
 
     long elemsPerPeriod = 10L;
@@ -169,7 +166,6 @@ public class CountingInputTest {
   @Test
   @Category(RunnableOnService.class)
   public void testUnboundedInputTimestamps() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
 
     PCollection<Long> input =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
index 5eccde6..dfd0949 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
 import org.apache.beam.sdk.io.CountingSource.CounterMark;
 import org.apache.beam.sdk.io.CountingSource.UnboundedCountingSource;
@@ -48,6 +47,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -79,10 +79,12 @@ public class CountingSourceTest {
       .isEqualTo(numElements - 1);
   }
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testBoundedSource() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
     PCollection<Long> input = p.apply(Read.from(CountingSource.upTo(numElements)));
 
@@ -93,7 +95,6 @@ public class CountingSourceTest {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyBoundedSource() {
-    Pipeline p = TestPipeline.create();
     PCollection<Long> input = p.apply(Read.from(CountingSource.upTo(0)));
 
     PAssert.that(input).empty();
@@ -103,7 +104,6 @@ public class CountingSourceTest {
   @Test
   @Category(RunnableOnService.class)
   public void testBoundedSourceSplits() throws Exception {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
     long numSplits = 10;
     long splitSizeBytes = numElements * 8 / numSplits;  // 8 bytes per long element.
@@ -157,7 +157,6 @@ public class CountingSourceTest {
   @Test
   @Category(RunnableOnService.class)
   public void testUnboundedSource() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
 
     PCollection<Long> input = p
@@ -177,7 +176,6 @@ public class CountingSourceTest {
   @Test
   @Category(RunnableOnService.class)
   public void testUnboundedSourceTimestamps() {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
 
     PCollection<Long> input = p.apply(
@@ -197,7 +195,6 @@ public class CountingSourceTest {
   @Test
   @Category(NeedsRunner.class)
   public void testUnboundedSourceWithRate() {
-    Pipeline p = TestPipeline.create();
 
     Duration period = Duration.millis(5);
     long numElements = 1000L;
@@ -232,7 +229,6 @@ public class CountingSourceTest {
   @Test
   @Category(RunnableOnService.class)
   public void testUnboundedSourceSplits() throws Exception {
-    Pipeline p = TestPipeline.create();
     long numElements = 1000;
     int numSplits = 10;
 
@@ -257,7 +253,6 @@ public class CountingSourceTest {
   @Test
   @Category(NeedsRunner.class)
   public void testUnboundedSourceRateSplits() throws Exception {
-    Pipeline p = TestPipeline.create();
     int elementsPerPeriod = 10;
     Duration period = Duration.millis(5);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
index f4b8574..f709e22 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
@@ -42,7 +42,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Random;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.io.FileBasedSource.FileBasedReader;
@@ -73,6 +72,7 @@ public class FileBasedSourceTest {
 
   Random random = new Random(0L);
 
+  @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
   @Rule public ExpectedException thrown = ExpectedException.none();
 
@@ -719,7 +719,6 @@ public class FileBasedSourceTest {
   @Test
   @Category(NeedsRunner.class)
   public void testDataflowFile() throws IOException {
-    Pipeline p = TestPipeline.create();
     List<String> data = createStringDataset(3, 50);
 
     String fileName = "file";
@@ -735,7 +734,6 @@ public class FileBasedSourceTest {
   @Test
   @Category(NeedsRunner.class)
   public void testDataflowFilePattern() throws IOException {
-    Pipeline p = TestPipeline.create();
 
     List<String> data1 = createStringDataset(3, 50);
     File file1 = createFileWithData("file1", data1);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
index 518136f..5bc1664 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java
@@ -39,6 +39,7 @@ import org.apache.beam.sdk.util.PubsubTestClient;
 import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -67,6 +68,9 @@ public class PubsubUnboundedSinkTest {
     return Hashing.murmur3_128().hashBytes(data.getBytes()).toString();
   }
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   public void saneCoder() throws Exception {
     OutgoingMessage message = new OutgoingMessage(DATA.getBytes(), TIMESTAMP, getRecordId(DATA));
@@ -88,7 +92,7 @@ public class PubsubUnboundedSinkTest {
           new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
               TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
               Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC);
-      TestPipeline p = TestPipeline.create();
+
       p.apply(Create.of(ImmutableList.of(DATA)))
        .apply(ParDo.of(new Stamp()))
        .apply(sink);
@@ -117,7 +121,7 @@ public class PubsubUnboundedSinkTest {
           new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(),
               TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes,
               Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC);
-      TestPipeline p = TestPipeline.create();
+
       p.apply(Create.of(data))
        .apply(ParDo.of(new Stamp()))
        .apply(sink);
@@ -153,7 +157,7 @@ public class PubsubUnboundedSinkTest {
               StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL,
               NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2),
               RecordIdMethod.DETERMINISTIC);
-      TestPipeline p = TestPipeline.create();
+
       p.apply(Create.of(data))
        .apply(ParDo.of(new Stamp()))
        .apply(sink);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
index f6165c5..601e2c8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java
@@ -57,6 +57,7 @@ import org.apache.beam.sdk.util.PubsubTestClient;
 import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory;
 import org.joda.time.Instant;
 import org.junit.After;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -82,6 +83,9 @@ public class PubsubUnboundedSourceTest {
   private PubsubTestClientFactory factory;
   private PubsubSource<String> primSource;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   private void setupOneMessage(Iterable<IncomingMessage> incoming) {
     now = new AtomicLong(REQ_TIME);
     clock = new Clock() {
@@ -124,7 +128,6 @@ public class PubsubUnboundedSourceTest {
   @Test
   public void readOneMessage() throws IOException {
     setupOneMessage();
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     // Read one message.
     assertTrue(reader.start());
@@ -139,7 +142,6 @@ public class PubsubUnboundedSourceTest {
   @Test
   public void timeoutAckAndRereadOneMessage() throws IOException {
     setupOneMessage();
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
     assertTrue(reader.start());
@@ -160,7 +162,6 @@ public class PubsubUnboundedSourceTest {
   @Test
   public void extendAck() throws IOException {
     setupOneMessage();
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
     // Pull the first message but don't take a checkpoint for it.
@@ -183,7 +184,6 @@ public class PubsubUnboundedSourceTest {
   @Test
   public void timeoutAckExtensions() throws IOException {
     setupOneMessage();
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
     // Pull the first message but don't take a checkpoint for it.
@@ -220,7 +220,6 @@ public class PubsubUnboundedSourceTest {
       incoming.add(new IncomingMessage(data.getBytes(), TIMESTAMP, 0, ackid, RECORD_ID));
     }
     setupOneMessage(incoming);
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     // Consume two messages, only read one.
     assertTrue(reader.start());
@@ -281,7 +280,6 @@ public class PubsubUnboundedSourceTest {
     }
     setupOneMessage(incoming);
 
-    TestPipeline p = TestPipeline.create();
     PubsubReader<String> reader = primSource.createReader(p.getOptions(), null);
     PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient();
 
@@ -342,7 +340,6 @@ public class PubsubUnboundedSourceTest {
             null);
     assertThat(source.getSubscription(), nullValue());
 
-    TestPipeline.create().apply(source);
     assertThat(source.getSubscription(), nullValue());
 
     PipelineOptions options = PipelineOptionsFactory.create();
@@ -373,7 +370,6 @@ public class PubsubUnboundedSourceTest {
             null);
     assertThat(source.getSubscription(), nullValue());
 
-    TestPipeline.create().apply(source);
     assertThat(source.getSubscription(), nullValue());
 
     PipelineOptions options = PipelineOptionsFactory.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index 472399a..b8b28eb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -147,6 +147,9 @@ public class TextIOTest {
   private static File largeZip;
 
   @Rule
+  public TestPipeline p = TestPipeline.create();
+
+  @Rule
   public ExpectedException expectedException = ExpectedException.none();
 
   private static File writeToFile(String[] lines, String filename, CompressionType compression)
@@ -224,8 +227,6 @@ public class TextIOTest {
       }
     }
 
-    Pipeline p = TestPipeline.create();
-
     TextIO.Read.Bound<T> read;
     if (coder.equals(StringUtf8Coder.of())) {
       TextIO.Read.Bound<String> readStrings = TextIO.Read.from(filename);
@@ -273,7 +274,7 @@ public class TextIOTest {
 
   @Test
   public void testReadNamed() throws Exception {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
 
     assertEquals(
         "TextIO.Read/Read.out",
@@ -330,8 +331,6 @@ public class TextIOTest {
     Path baseDir = Files.createTempDirectory(tempFolder, "testwrite");
     String baseFilename = baseDir.resolve(outputName).toString();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<T> input = p.apply(Create.of(Arrays.asList(elems)).withCoder(coder));
 
     TextIO.Write.Bound<T> write;
@@ -511,7 +510,6 @@ public class TextIOTest {
     Coder<String> coder = StringUtf8Coder.of();
     String outputName = "file.txt";
     Path baseDir = Files.createTempDirectory(tempFolder, "testwrite");
-    Pipeline p = TestPipeline.create();
 
     PCollection<String> input = p.apply(Create.of(Arrays.asList(LINES2_ARRAY)).withCoder(coder));
 
@@ -601,11 +599,10 @@ public class TextIOTest {
 
   @Test
   public void testUnsupportedFilePattern() throws IOException {
+    p.enableAbandonedNodeEnforcement(false);
     // Windows doesn't like resolving paths with * in them.
     String filename = tempFolder.resolve("output@5").toString();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(LINES_ARRAY))
             .withCoder(StringUtf8Coder.of()));
@@ -621,13 +618,13 @@ public class TextIOTest {
    */
   @Test
   public void testBadWildcardRecursive() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
 
     // Check that applying does fail.
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage("wildcard");
 
-    pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
+    p.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
   }
 
   /** Options for testing. */
@@ -641,9 +638,11 @@ public class TextIOTest {
 
   @Test
   public void testRuntimeOptionsNotCalledInApply() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class);
-    pipeline
+
+    p
         .apply(TextIO.Read.from(options.getInput()).withoutValidation())
         .apply(TextIO.Write.to(options.getOutput()).withoutValidation());
   }
@@ -686,12 +685,12 @@ public class TextIOTest {
    * Helper method that runs TextIO.Read.from(filename).withCompressionType(compressionType)
    * and asserts that the results match the given expected output.
    */
-  private static void assertReadingCompressedFileMatchesExpected(
+  private void assertReadingCompressedFileMatchesExpected(
       File file, CompressionType compressionType, String[] expected) {
-    Pipeline p = TestPipeline.create();
+
     TextIO.Read.Bound<String> read =
         TextIO.Read.from(file.getPath()).withCompressionType(compressionType);
-    PCollection<String> output = p.apply(read);
+    PCollection<String> output = p.apply("Read_" + file + "_" + compressionType.toString(), read);
 
     PAssert.that(output).containsInAnyOrder(expected);
     p.run();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
index 5a7c994..79f4c4b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
@@ -79,6 +79,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class WriteTest {
+  @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   // Static store that can be accessed within the writer
@@ -294,7 +295,6 @@ public class WriteTest {
 
   @Test
   public void testWriteUnbounded() {
-    TestPipeline p = TestPipeline.create();
     PCollection<String> unbounded = p.apply(CountingInput.unbounded())
         .apply(MapElements.via(new ToStringFn()));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
index 1f154d5..d6898d5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
@@ -40,7 +40,6 @@ import java.util.List;
 import java.util.Random;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Source.Reader;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -64,6 +63,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class XmlSourceTest {
+
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Rule
   public TemporaryFolder tempFolder = new TemporaryFolder();
 
@@ -566,8 +569,6 @@ public class XmlSourceTest {
   @Test
   @Category(NeedsRunner.class)
   public void testReadXMLSmallPipeline() throws IOException {
-    Pipeline p = TestPipeline.create();
-
     File file = tempFolder.newFile("trainXMLSmall");
     Files.write(file.toPath(), trainXML.getBytes(StandardCharsets.UTF_8));
 
@@ -661,7 +662,6 @@ public class XmlSourceTest {
     List<Train> trains = generateRandomTrainList(100);
     File file = createRandomTrainXML(fileName, trains);
 
-    Pipeline p = TestPipeline.create();
     XmlSource<Train> source =
         XmlSource.<Train>from(file.toPath().toString())
             .withRootElement("trains")
@@ -808,8 +808,6 @@ public class XmlSourceTest {
     generateRandomTrainList(8);
     createRandomTrainXML("otherfile.xml", trains1);
 
-    Pipeline p = TestPipeline.create();
-
     XmlSource<Train> source =
         XmlSource.<Train>from(file.getParent() + "/" + "temp*.xml")
             .withRootElement("trains")

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index 5e97eed..4e257f1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -50,7 +50,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
@@ -90,6 +89,9 @@ public class ProxyInvocationHandlerTest {
     void setString(String value);
   }
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   public void testPropertySettingAndGetting() throws Exception {
     ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.<String, Object>newHashMap());
@@ -785,7 +787,6 @@ public class ProxyInvocationHandlerTest {
       }
     };
 
-    Pipeline p = TestPipeline.create();
     p.getOptions().as(ObjectPipelineOptions.class).setValue(brokenValueType);
 
     p.apply(Create.of(1, 2, 3));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
index b0c17d8..2327459 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
@@ -51,14 +51,16 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class TransformHierarchyTest {
+
+  @Rule public final TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
+
   private TransformHierarchy hierarchy;
-  private TestPipeline pipeline;
+
 
   @Before
   public void setup() {
     hierarchy = new TransformHierarchy();
-    pipeline = TestPipeline.create();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index d70aa2f..6a6e0fc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -55,6 +55,8 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class TransformTreeTest {
+
+  @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
 
   enum TransformsSeen {
@@ -112,11 +114,11 @@ public class TransformTreeTest {
   // visits the nodes and verifies that the hierarchy was captured.
   @Test
   public void testCompositeCapture() throws Exception {
+    p.enableAbandonedNodeEnforcement(false);
+
     File inputFile = tmpFolder.newFile();
     File outputFile = tmpFolder.newFile();
 
-    Pipeline p = TestPipeline.create();
-
     p.apply("ReadMyFile", TextIO.Read.from(inputFile.getPath()))
         .apply(Sample.<String>any(10))
         .apply("WriteMyFile", TextIO.Write.to(outputFile.getPath()));
@@ -170,18 +172,15 @@ public class TransformTreeTest {
 
   @Test(expected = IllegalArgumentException.class)
   public void testOutputChecking() throws Exception {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
 
     p.apply(new InvalidCompositeTransform());
-
     p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {});
   }
 
   @Test
   @Category(NeedsRunner.class)
   public void testMultiGraphSetup() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<Integer> input = p.begin()
         .apply(Create.of(1, 2, 3));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
index 417147f..a96e3f8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
@@ -37,6 +37,7 @@ import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -45,10 +46,12 @@ import org.junit.runners.JUnit4;
 /** Tests for {@link GatherAllPanes}. */
 @RunWith(JUnit4.class)
 public class GatherAllPanesTest implements Serializable {
+
+  @Rule public transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(NeedsRunner.class)
   public void singlePaneSingleReifiedPane() {
-    TestPipeline p = TestPipeline.create();
     PCollection<Iterable<ValueInSingleWindow<Iterable<Long>>>> accumulatedPanes =
         p.apply(CountingInput.upTo(20000))
             .apply(
@@ -91,8 +94,6 @@ public class GatherAllPanesTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void multiplePanesMultipleReifiedPane() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Long> someElems = p.apply("someLongs", CountingInput.upTo(20000));
     PCollection<Long> otherElems = p.apply("otherLongs", CountingInput.upTo(20000));
     PCollection<Iterable<ValueInSingleWindow<Iterable<Long>>>> accumulatedPanes =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
index be8924f..1997bbe 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
@@ -59,6 +59,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class PAssertTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
@@ -116,8 +120,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testContainsInAnyOrderNotSerializable() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<NotSerializableObject> pcollection = pipeline
         .apply(Create.of(
           new NotSerializableObject(),
@@ -139,8 +141,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testSerializablePredicate() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<NotSerializableObject> pcollection = pipeline
         .apply(Create.of(
           new NotSerializableObject(),
@@ -166,8 +166,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedSerializablePredicate() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<NotSerializableObject> pcollection = pipeline
         .apply(Create.timestamped(
             TimestampedValue.of(new NotSerializableObject(), new Instant(250L)),
@@ -207,7 +205,6 @@ public class PAssertTest implements Serializable {
     thrown.expect(UnsupportedOperationException.class);
     thrown.expectMessage("isEqualTo");
 
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
     PAssert.thatSingleton(pcollection).equals(42);
   }
@@ -222,7 +219,6 @@ public class PAssertTest implements Serializable {
     thrown.expect(UnsupportedOperationException.class);
     thrown.expectMessage("containsInAnyOrder");
 
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
     PAssert.that(pcollection).equals(42);
   }
@@ -237,7 +233,6 @@ public class PAssertTest implements Serializable {
     thrown.expect(UnsupportedOperationException.class);
     thrown.expectMessage(".hashCode() is not supported.");
 
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
     PAssert.thatSingleton(pcollection).hashCode();
   }
@@ -252,7 +247,6 @@ public class PAssertTest implements Serializable {
     thrown.expect(UnsupportedOperationException.class);
     thrown.expectMessage(".hashCode() is not supported.");
 
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(42));
     PAssert.that(pcollection).hashCode();
   }
@@ -263,7 +257,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testIsEqualTo() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(43));
     PAssert.thatSingleton(pcollection).isEqualTo(43);
     pipeline.run();
@@ -275,7 +268,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedIsEqualTo() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection =
         pipeline.apply(Create.timestamped(TimestampedValue.of(43, new Instant(250L)),
             TimestampedValue.of(22, new Instant(-250L))))
@@ -295,7 +287,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testNotEqualTo() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(43));
     PAssert.thatSingleton(pcollection).notEqualTo(42);
     pipeline.run();
@@ -307,7 +298,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testContainsInAnyOrder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(1, 2, 3, 4));
     PAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3);
     pipeline.run();
@@ -319,7 +309,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testGlobalWindowContainsInAnyOrder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection = pipeline.apply(Create.of(1, 2, 3, 4));
     PAssert.that(pcollection).inWindow(GlobalWindow.INSTANCE).containsInAnyOrder(2, 1, 4, 3);
     pipeline.run();
@@ -331,7 +320,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedContainsInAnyOrder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> pcollection =
         pipeline.apply(Create.timestamped(TimestampedValue.of(1, new Instant(100L)),
             TimestampedValue.of(2, new Instant(200L)),
@@ -361,13 +349,12 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmpty() {
-    Pipeline p = TestPipeline.create();
     PCollection<Long> vals =
-        p.apply(Create.<Long>of().withCoder(VarLongCoder.of()));
+        pipeline.apply(Create.<Long>of().withCoder(VarLongCoder.of()));
 
     PAssert.that(vals).empty();
 
-    p.run();
+    pipeline.run();
   }
 
   /**
@@ -376,8 +363,6 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testContainsInAnyOrderFalse() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<Integer> pcollection = pipeline
         .apply(Create.of(1, 2, 3, 4));
 
@@ -399,11 +384,10 @@ public class PAssertTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyFalse() throws Exception {
-    Pipeline p = TestPipeline.create();
-    PCollection<Long> vals = p.apply(CountingInput.upTo(5L));
+    PCollection<Long> vals = pipeline.apply(CountingInput.upTo(5L));
     PAssert.that(vals).empty();
 
-    Throwable thrown = runExpectingAssertionFailure(p);
+    Throwable thrown = runExpectingAssertionFailure(pipeline);
 
     assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order"));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index a1b4e4a..64aeca3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -65,6 +65,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class TestStreamTest implements Serializable {
+  @Rule public transient TestPipeline p = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
@@ -85,7 +86,6 @@ public class TestStreamTest implements Serializable {
             TimestampedValue.of(-3, instant))
         .advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Integer> windowed = p
         .apply(source)
         .apply(Window.<Integer>into(FixedWindows.of(Duration.standardMinutes(5))).triggering(
@@ -146,7 +146,6 @@ public class TestStreamTest implements Serializable {
         .advanceProcessingTime(Duration.standardMinutes(6))
         .advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> sum = p.apply(source)
         .apply(Window.<Long>triggering(AfterWatermark.pastEndOfWindow()
             .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane()
@@ -175,7 +174,6 @@ public class TestStreamTest implements Serializable {
                 TimestampedValue.of("alsoFinalLatePane", new Instant(250)))
             .advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L));
     Duration allowedLateness = Duration.millis(5000L);
     PCollection<String> values =
@@ -220,7 +218,6 @@ public class TestStreamTest implements Serializable {
             .addElements(TimestampedValue.of("onTime", new Instant(100)))
             .advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L));
     Duration allowedLateness = Duration.millis(5000L);
     PCollection<String> values = p.apply(stream)
@@ -249,7 +246,6 @@ public class TestStreamTest implements Serializable {
             TimestampedValue.of("bar", endOfGlobalWindow))
         .advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     FixedWindows windows = FixedWindows.of(Duration.standardHours(6));
     PCollection<String> windowedValues = p.apply(stream)
         .apply(Window.<String>into(windows))
@@ -274,7 +270,6 @@ public class TestStreamTest implements Serializable {
     TestStream<Integer> other =
         TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity();
 
-    TestPipeline p = TestPipeline.create();
     PCollection<String> createStrings =
         p.apply("CreateStrings", stream)
             .apply("WindowStrings",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
index ab13946..cd7898b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java
@@ -42,6 +42,7 @@ import org.hamcrest.CoreMatchers;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.TypeSafeDiagnosingMatcher;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -63,6 +64,9 @@ public class ApproximateQuantilesTest {
       KV.of("b", 100)
   );
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   public PCollection<KV<String, Integer>> createInputTable(Pipeline p) {
     return p.apply(Create.of(TABLE).withCoder(
         KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -71,8 +75,6 @@ public class ApproximateQuantilesTest {
   @Test
   @Category(NeedsRunner.class)
   public void testQuantilesGlobally() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> input = intRangeCollection(p, 101);
     PCollection<List<Integer>> quantiles =
         input.apply(ApproximateQuantiles.<Integer>globally(5));
@@ -85,8 +87,6 @@ public class ApproximateQuantilesTest {
   @Test
   @Category(NeedsRunner.class)
   public void testQuantilesGobally_comparable() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> input = intRangeCollection(p, 101);
     PCollection<List<Integer>> quantiles =
         input.apply(
@@ -100,8 +100,6 @@ public class ApproximateQuantilesTest {
   @Test
   @Category(NeedsRunner.class)
   public void testQuantilesPerKey() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input = createInputTable(p);
     PCollection<KV<String, List<Integer>>> quantiles = input.apply(
         ApproximateQuantiles.<String, Integer>perKey(2));
@@ -117,8 +115,6 @@ public class ApproximateQuantilesTest {
   @Test
   @Category(NeedsRunner.class)
   public void testQuantilesPerKey_reversed() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input = createInputTable(p);
     PCollection<KV<String, List<Integer>>> quantiles = input.apply(
         ApproximateQuantiles.<String, Integer, DescendingIntComparator>perKey(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
index b63c73d..3afc759 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
@@ -40,6 +40,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -52,6 +53,9 @@ import org.junit.runners.JUnit4;
 public class ApproximateUniqueTest implements Serializable {
   // implements Serializable just to make it easy to use anonymous inner DoFn subclasses
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   public void testEstimationErrorToSampleSize() {
     assertEquals(40000, ApproximateUnique.sampleSizeFromEstimationError(0.01));
@@ -67,8 +71,6 @@ public class ApproximateUniqueTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testApproximateUniqueWithSmallInput() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<Integer> input = p.apply(
         Create.of(Arrays.asList(1, 2, 3, 3)));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
index 8862531..cdd4707 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
@@ -28,7 +28,6 @@ import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -62,6 +61,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class  CombineFnsTest {
+  @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
   @Test
@@ -123,7 +123,6 @@ public class  CombineFnsTest {
   @Test
   @Category(RunnableOnService.class)
   public void testComposedCombine() {
-    Pipeline p = TestPipeline.create();
     p.getCoderRegistry().registerCoder(UserString.class, UserStringCoder.of());
 
     PCollection<KV<String, KV<Integer, UserString>>> perKeyInput = p.apply(
@@ -178,7 +177,6 @@ public class  CombineFnsTest {
   @Test
   @Category(RunnableOnService.class)
   public void testComposedCombineWithContext() {
-    Pipeline p = TestPipeline.create();
     p.getCoderRegistry().registerCoder(UserString.class, UserStringCoder.of());
 
     PCollectionView<String> view = p
@@ -240,7 +238,6 @@ public class  CombineFnsTest {
   @Test
   @Category(RunnableOnService.class)
   public void testComposedCombineNullValues() {
-    Pipeline p = TestPipeline.create();
     p.getCoderRegistry().registerCoder(UserString.class, NullableCoder.of(UserStringCoder.of()));
     p.getCoderRegistry().registerCoder(String.class, NullableCoder.of(StringUtf8Coder.of()));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
index 671f00e..0ac9502 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
@@ -82,6 +82,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.POutput;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -108,6 +109,9 @@ public class CombineTest implements Serializable {
 
   @Mock private DoFn<?, ?>.ProcessContext processContext;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   PCollection<KV<String, Integer>> createInput(Pipeline p,
                                                List<KV<String, Integer>> table) {
     return p.apply(Create.of(table).withCoder(
@@ -117,7 +121,6 @@ public class CombineTest implements Serializable {
   private void runTestSimpleCombine(List<KV<String, Integer>> table,
                                     int globalSum,
                                     List<KV<String, String>> perKeyCombines) {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<KV<String, Integer>> input = createInput(pipeline, table);
 
     PCollection<Integer> sum = input
@@ -138,7 +141,6 @@ public class CombineTest implements Serializable {
                                                int globalSum,
                                                List<KV<String, String>> perKeyCombines,
                                                String[] globallyCombines) {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<KV<String, Integer>> perKeyInput = createInput(pipeline, table);
     PCollection<Integer> globallyInput = perKeyInput.apply(Values.<Integer>create());
 
@@ -197,7 +199,6 @@ public class CombineTest implements Serializable {
   private void runTestBasicCombine(List<KV<String, Integer>> table,
                                    Set<Integer> globalUnique,
                                    List<KV<String, Set<Integer>>> perKeyUnique) {
-    Pipeline pipeline = TestPipeline.create();
     pipeline.getCoderRegistry().registerCoder(Set.class, SetCoder.class);
     PCollection<KV<String, Integer>> input = createInput(pipeline, table);
 
@@ -233,7 +234,6 @@ public class CombineTest implements Serializable {
   private void runTestAccumulatingCombine(List<KV<String, Integer>> table,
                                           Double globalMean,
                                           List<KV<String, Double>> perKeyMeans) {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<KV<String, Integer>> input = createInput(pipeline, table);
 
     PCollection<Double> mean = input
@@ -253,8 +253,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFixedWindowsCombine() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 1L, 6L, 7L, 8L))
                 .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
@@ -279,8 +277,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFixedWindowsCombineWithContext() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<KV<String, Integer>> perKeyInput =
         pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 1L, 6L, 7L, 8L))
                 .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
@@ -316,8 +312,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testSlidingWindowsCombineWithContext() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<KV<String, Integer>> perKeyInput =
         pipeline.apply(Create.timestamped(TABLE, Arrays.asList(2L, 3L, 8L, 9L, 10L))
                 .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
@@ -365,7 +359,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testGlobalCombineWithDefaultsAndTriggers() {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> input = pipeline.apply(Create.of(1, 1));
 
     PCollection<String> output = input
@@ -392,8 +385,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testSessionsCombine() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 4L, 7L, 10L, 16L))
                 .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
@@ -417,8 +408,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testSessionsCombineWithContext() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<KV<String, Integer>> perKeyInput =
         pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 4L, 7L, 10L, 16L))
                 .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -459,8 +448,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowedCombineEmpty() {
-    Pipeline pipeline = TestPipeline.create();
-
     PCollection<Double> mean = pipeline
         .apply(Create.<Integer>of().withCoder(BigEndianIntegerCoder.of()))
         .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(1))))
@@ -517,7 +504,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testHotKeyCombining() {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<KV<String, Integer>> input = copy(createInput(pipeline, TABLE), 10);
 
     KeyedCombineFn<String, Integer, ?, Double> mean =
@@ -552,7 +538,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testHotKeyCombiningWithAccumulationMode() {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> input = pipeline.apply(Create.of(1, 2, 3, 4, 5));
 
     PCollection<Integer> output = input
@@ -577,7 +562,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testBinaryCombineFn() {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<KV<String, Integer>> input = copy(createInput(pipeline, TABLE), 2);
     PCollection<KV<String, Integer>> intProduct = input
         .apply("IntProduct", Combine.<String, Integer, Integer>perKey(new TestProdInt()));
@@ -632,7 +616,6 @@ public class CombineTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testCombineGloballyAsSingletonView() {
-    Pipeline pipeline = TestPipeline.create();
     final PCollectionView<Integer> view = pipeline
         .apply("CreateEmptySideInput", Create.<Integer>of().withCoder(BigEndianIntegerCoder.of()))
         .apply(Sum.integersGlobally().asSingletonView());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
index 7f77ae7..eafb12d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java
@@ -22,13 +22,13 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -45,12 +45,13 @@ public class CountTest {
 
   static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
 
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   @SuppressWarnings("unchecked")
   public void testCountPerElementBasic() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input = p.apply(Create.of(WORDS));
 
     PCollection<KV<String, Long>> output =
@@ -71,8 +72,6 @@ public class CountTest {
   @Category(RunnableOnService.class)
   @SuppressWarnings("unchecked")
   public void testCountPerElementEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input = p.apply(Create.of(NO_LINES).withCoder(StringUtf8Coder.of()));
 
     PCollection<KV<String, Long>> output =
@@ -85,8 +84,6 @@ public class CountTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCountGloballyBasic() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input = p.apply(Create.of(WORDS));
 
     PCollection<Long> output =
@@ -100,8 +97,6 @@ public class CountTest {
   @Test
   @Category(RunnableOnService.class)
   public void testCountGloballyEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input = p.apply(Create.of(NO_LINES).withCoder(StringUtf8Coder.of()));
 
     PCollection<Long> output =


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 6dea0992d9976b39232cf846906831feaa25ec43
Parents: 63331aa
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 13:26:07 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6dea0992/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
index dc566d2..4ddfdea 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -643,6 +643,7 @@ public class BigQueryIOTest implements Serializable {
     }
   }
 
+  @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
   @Rule public transient ExpectedLogs logged = ExpectedLogs.none(BigQueryIO.class);
   @Rule public transient TemporaryFolder testFolder = new TemporaryFolder();
@@ -1370,7 +1371,7 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testBuildWriteWithoutTable() {
-    Pipeline p = TestPipeline.create();
+
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("must set the table reference");
     p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
@@ -1591,9 +1592,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsCreateNoSchema() {
+    p.enableAbandonedNodeEnforcement(false);
+
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("no schema was provided");
-    TestPipeline.create()
+    p
         .apply(Create.<TableRow>of())
         .apply(BigQueryIO.Write
             .to("dataset.table")
@@ -1602,9 +1605,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsTableAndTableSpec() {
+    p.enableAbandonedNodeEnforcement(false);
+
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("Cannot set both a table reference and a table function");
-    TestPipeline.create()
+    p
         .apply(Create.<TableRow>of())
         .apply(BigQueryIO.Write
             .to("dataset.table")
@@ -1618,9 +1623,11 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteValidateFailsNoTableAndNoTableSpec() {
+    p.enableAbandonedNodeEnforcement(false);
+
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("must set the table reference of a BigQueryIO.Write transform");
-    TestPipeline.create()
+    p
         .apply(Create.<TableRow>of())
         .apply("name", BigQueryIO.Write.withoutValidation());
   }
@@ -1950,7 +1957,6 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPassThroughThenCleanup() throws Exception {
-    Pipeline p = TestPipeline.create();
 
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3))
@@ -1968,7 +1974,6 @@ public class BigQueryIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPassThroughThenCleanupExecuted() throws Exception {
-    Pipeline p = TestPipeline.create();
 
     p.apply(Create.<Integer>of())
         .apply(new PassThroughThenCleanup<Integer>(new CleanupOperation() {
@@ -2025,6 +2030,8 @@ public class BigQueryIOTest implements Serializable {
 
   private void testWritePartition(long numFiles, long fileSize, long expectedNumPartitions)
       throws Exception {
+    p.enableAbandonedNodeEnforcement(false);
+
     List<Long> expectedPartitionIds = Lists.newArrayList();
     for (long i = 1; i <= expectedNumPartitions; ++i) {
       expectedPartitionIds.add(i);
@@ -2044,7 +2051,7 @@ public class BigQueryIOTest implements Serializable {
         new TupleTag<KV<Long, List<String>>>("singlePartitionTag") {};
 
     PCollectionView<Iterable<KV<String, Long>>> filesView = PCollectionViews.iterableView(
-        TestPipeline.create(),
+        p,
         WindowingStrategy.globalDefault(),
         KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()));
 
@@ -2164,6 +2171,8 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testWriteRename() throws Exception {
+    p.enableAbandonedNodeEnforcement(false);
+
     FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
         .withJobService(new FakeJobService()
             .startJobReturns("done", "done")
@@ -2179,7 +2188,7 @@ public class BigQueryIOTest implements Serializable {
     }
 
     PCollectionView<Iterable<String>> tempTablesView = PCollectionViews.iterableView(
-        TestPipeline.create(),
+        p,
         WindowingStrategy.globalDefault(),
         StringUtf8Coder.of());
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6dea0992/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index 98215df..e3d1f47 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -106,6 +106,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class BigtableIOTest {
+  @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
   @Rule public ExpectedLogs logged = ExpectedLogs.none(BigtableIO.class);
 
@@ -140,7 +141,7 @@ public class BigtableIOTest {
     service = new FakeBigtableService();
     defaultRead = defaultRead.withBigtableService(service);
     defaultWrite = defaultWrite.withBigtableService(service);
-    bigtableCoder = TestPipeline.create().getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE);
+    bigtableCoder = p.getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE);
   }
 
   @Test
@@ -261,6 +262,8 @@ public class BigtableIOTest {
   /** Tests that when reading from a non-existent table, the read fails. */
   @Test
   public void testReadingFailsTableDoesNotExist() throws Exception {
+    p.enableAbandonedNodeEnforcement(false);
+
     final String table = "TEST-TABLE";
 
     BigtableIO.Read read =
@@ -273,7 +276,7 @@ public class BigtableIOTest {
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(String.format("Table %s does not exist", table));
 
-    TestPipeline.create().apply(read);
+    p.apply(read);
   }
 
   /** Tests that when reading from an empty table, the read succeeds. */
@@ -589,7 +592,6 @@ public class BigtableIOTest {
 
     service.createTable(table);
 
-    TestPipeline p = TestPipeline.create();
     p.apply("single row", Create.of(makeWrite(key, value)).withCoder(bigtableCoder))
         .apply("write", defaultWrite.withTableId(table));
     p.run();
@@ -606,10 +608,12 @@ public class BigtableIOTest {
   /** Tests that when writing to a non-existent table, the write fails. */
   @Test
   public void testWritingFailsTableDoesNotExist() throws Exception {
+    p.enableAbandonedNodeEnforcement(false);
+
     final String table = "TEST-TABLE";
 
     PCollection<KV<ByteString, Iterable<Mutation>>> emptyInput =
-        TestPipeline.create().apply(Create.<KV<ByteString, Iterable<Mutation>>>of());
+        p.apply(Create.<KV<ByteString, Iterable<Mutation>>>of());
 
     // Exception will be thrown by write.validate() when write is applied.
     thrown.expect(IllegalArgumentException.class);
@@ -625,7 +629,6 @@ public class BigtableIOTest {
     final String key = "KEY";
     service.createTable(table);
 
-    TestPipeline p = TestPipeline.create();
     p.apply(Create.of(makeBadWrite(key)).withCoder(bigtableCoder))
         .apply(defaultWrite.withTableId(table));
 


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

Posted by ke...@apache.org.
Add UsesTestStream for use with JUnit @Category


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

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

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


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

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


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

Posted by ke...@apache.org.
Actually propagate and commit state in direct runner


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

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

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


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


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

Posted by ke...@apache.org.
This closes #1664: [BEAM-1176] Migrating tests to use TestPipeline as a JUnit rule

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


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

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

----------------------------------------------------------------------
 .../org/apache/beam/examples/WordCountTest.java |   7 +-
 .../examples/complete/AutoCompleteTest.java     |  11 +-
 .../beam/examples/complete/TfIdfTest.java       |   6 +-
 .../complete/TopWikipediaSessionsTest.java      |   7 +-
 .../examples/cookbook/DistinctExampleTest.java  |   9 +-
 .../examples/cookbook/JoinExamplesTest.java     |   6 +-
 .../examples/cookbook/TriggerExampleTest.java   |   6 +-
 .../examples/MinimalWordCountJava8Test.java     |   6 +-
 .../examples/complete/game/GameStatsTest.java   |   7 +-
 .../complete/game/HourlyTeamScoreTest.java      |   5 +-
 .../examples/complete/game/LeaderBoardTest.java |  11 +-
 .../examples/complete/game/UserScoreTest.java   |  10 +-
 .../core/PushbackSideInputDoFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  |  10 +-
 .../UnboundedReadFromBoundedSourceTest.java     |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  13 +-
 .../direct/CloningBundleFactoryTest.java        |   8 +-
 .../runners/direct/CommittedResultTest.java     |   6 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java |   7 +-
 .../runners/direct/DirectGraphVisitorTest.java  |   3 +-
 .../runners/direct/EvaluationContextTest.java   |   7 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |   6 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |   5 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |   5 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   4 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   3 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  14 ++-
 .../direct/KeyedPValueTrackingVisitorTest.java  |   6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   5 +-
 .../runners/direct/SideInputContainerTest.java  |   5 +-
 .../StatefulParDoEvaluatorFactoryTest.java      |   7 +-
 .../runners/direct/StepTransformResultTest.java |   5 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |   5 +-
 .../runners/direct/TransformExecutorTest.java   |   4 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   9 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   5 +-
 .../direct/WatermarkCallbackExecutorTest.java   |   5 +-
 .../runners/direct/WatermarkManagerTest.java    |   6 +-
 .../direct/WindowEvaluatorFactoryTest.java      |   5 +-
 .../direct/WriteWithShardingFactoryTest.java    |  14 +--
 .../java/org/apache/beam/sdk/PipelineTest.java  |  37 +++---
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  11 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |   6 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   7 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  16 +--
 .../io/BoundedReadFromUnboundedSourceTest.java  |   6 +-
 .../beam/sdk/io/CompressedSourceTest.java       |  12 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  12 +-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  13 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |   4 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  10 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  12 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  29 +++--
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   2 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  10 +-
 .../sdk/options/ProxyInvocationHandlerTest.java |   5 +-
 .../sdk/runners/TransformHierarchyTest.java     |   6 +-
 .../beam/sdk/runners/TransformTreeTest.java     |  11 +-
 .../beam/sdk/testing/GatherAllPanesTest.java    |   7 +-
 .../apache/beam/sdk/testing/PAssertTest.java    |  32 ++---
 .../apache/beam/sdk/testing/TestStreamTest.java |   7 +-
 .../transforms/ApproximateQuantilesTest.java    |  12 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |   6 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   5 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  25 +---
 .../apache/beam/sdk/transforms/CountTest.java   |  13 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |  27 +----
 .../beam/sdk/transforms/DistinctTest.java       |  12 +-
 .../apache/beam/sdk/transforms/DoFnTest.java    |   4 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |   6 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |  18 +--
 .../sdk/transforms/FlatMapElementsTest.java     |  10 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |  35 +-----
 .../beam/sdk/transforms/GroupByKeyTest.java     |  30 ++---
 .../apache/beam/sdk/transforms/KeysTest.java    |   9 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   9 +-
 .../apache/beam/sdk/transforms/LatestTest.java  |  12 +-
 .../beam/sdk/transforms/MapElementsTest.java    |  14 +--
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  17 +--
 .../apache/beam/sdk/transforms/ParDoTest.java   | 118 +++++++------------
 .../beam/sdk/transforms/PartitionTest.java      |   8 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  25 +---
 .../apache/beam/sdk/transforms/SampleTest.java  |  34 +++---
 .../beam/sdk/transforms/SplittableDoFnTest.java |  12 +-
 .../org/apache/beam/sdk/transforms/TopTest.java |  15 ++-
 .../apache/beam/sdk/transforms/ValuesTest.java  |   7 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |  84 ++++---------
 .../beam/sdk/transforms/WithKeysTest.java       |   8 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |   9 +-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  11 +-
 .../sdk/transforms/windowing/WindowTest.java    |  22 ++--
 .../sdk/transforms/windowing/WindowingTest.java |  11 +-
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  11 +-
 .../beam/sdk/values/PCollectionTupleTest.java   |  12 +-
 .../org/apache/beam/sdk/values/PDoneTest.java   |   9 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |  10 +-
 .../extensions/joinlibrary/InnerJoinTest.java   |   9 +-
 .../joinlibrary/OuterLeftJoinTest.java          |  10 +-
 .../joinlibrary/OuterRightJoinTest.java         |  10 +-
 .../sdk/extensions/sorter/SortValuesTest.java   |   7 +-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  25 ++--
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |  13 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |   8 +-
 .../org/apache/beam/sdk/io/jms/JmsIOTest.java   |   9 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  25 ++--
 .../sdk/io/kinesis/KinesisMockReadTest.java     |   7 +-
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |   6 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |   9 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |   7 +-
 .../beam/sdk/transforms/CombineJava8Test.java   |   8 +-
 .../beam/sdk/transforms/DistinctJava8Test.java  |   5 +-
 .../beam/sdk/transforms/FilterJava8Test.java    |   9 +-
 .../transforms/FlatMapElementsJava8Test.java    |   7 +-
 .../sdk/transforms/MapElementsJava8Test.java    |   9 +-
 .../beam/sdk/transforms/PartitionJava8Test.java |   7 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |   6 +-
 .../sdk/transforms/WithTimestampsJava8Test.java |  14 ++-
 117 files changed, 640 insertions(+), 769 deletions(-)
----------------------------------------------------------------------



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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 8d478c0f38c656d3533d590a65c6ed95da229f81
Parents: 5ccbe67
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 17:31:23 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


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


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

Posted by ke...@apache.org.
Merge remote-tracking branch 'origin/master' into python-sdk


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

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

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 .jenkins/common_job_properties.groovy           | 173 ++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 ++
 ...ostCommit_Java_RunnableOnService_Apex.groovy |  41 ++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 ++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 ++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 ++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 ++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 ++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 ++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 ++
 .jenkins/job_seed.groovy                        |  47 ++
 .travis.yml                                     |   2 +-
 .travis/README.md                               |   2 +-
 README.md                                       |   2 +-
 examples/java/pom.xml                           |  20 +-
 .../apache/beam/examples/WindowedWordCount.java | 177 +++---
 .../org/apache/beam/examples/WordCount.java     |   2 +-
 .../examples/common/WriteWindowedFilesDoFn.java |  77 +++
 .../beam/examples/complete/AutoComplete.java    |   6 +-
 .../org/apache/beam/examples/complete/README.md |  14 +-
 .../apache/beam/examples/complete/TfIdf.java    |   6 +-
 .../examples/complete/TopWikipediaSessions.java |  15 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   4 +-
 .../beam/examples/complete/TrafficRoutes.java   |   4 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/DeDupExample.java    |  96 ---
 .../beam/examples/cookbook/DistinctExample.java |  96 +++
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../org/apache/beam/examples/cookbook/README.md |  16 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/WindowedWordCountIT.java      | 182 +++++-
 .../org/apache/beam/examples/WordCountIT.java   |   2 +-
 .../org/apache/beam/examples/WordCountTest.java |   7 +-
 .../examples/complete/AutoCompleteTest.java     |  13 +-
 .../beam/examples/complete/TfIdfTest.java       |   6 +-
 .../complete/TopWikipediaSessionsTest.java      |   7 +-
 .../examples/cookbook/DistinctExampleTest.java  |   9 +-
 .../examples/cookbook/JoinExamplesTest.java     |   6 +-
 .../examples/cookbook/TriggerExampleTest.java   |   6 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |   2 +-
 .../examples/complete/game/LeaderBoard.java     |   4 +-
 .../beam/examples/complete/game/UserScore.java  |   2 +-
 .../complete/game/utils/WriteToBigQuery.java    |   2 +-
 .../game/utils/WriteWindowedToBigQuery.java     |   7 +-
 .../examples/MinimalWordCountJava8Test.java     |   6 +-
 .../examples/complete/game/GameStatsTest.java   |   7 +-
 .../complete/game/HourlyTeamScoreTest.java      |   5 +-
 .../examples/complete/game/LeaderBoardTest.java |  11 +-
 .../examples/complete/game/UserScoreTest.java   |  10 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         |  15 +-
 runners/apex/pom.xml                            |  69 ++-
 .../apache/beam/runners/apex/ApexRunner.java    | 111 ++--
 .../beam/runners/apex/ApexRunnerResult.java     |  50 +-
 .../beam/runners/apex/ApexYarnLauncher.java     | 395 ++++++++++++
 .../translation/ApexPipelineTranslator.java     |  14 +-
 .../translation/ParDoBoundMultiTranslator.java  |  22 +-
 .../apex/translation/ParDoBoundTranslator.java  |  22 +-
 .../apex/translation/TranslationContext.java    |  10 +-
 .../apex/translation/WindowBoundTranslator.java |  78 +++
 .../operators/ApexGroupByKeyOperator.java       |  26 +-
 .../operators/ApexParDoOperator.java            |  29 +-
 .../apex/translation/utils/NoOpStepContext.java |   3 +-
 .../beam/runners/apex/ApexYarnLauncherTest.java | 138 +++++
 .../FlattenPCollectionTranslatorTest.java       |  15 +-
 .../translation/GroupByKeyTranslatorTest.java   |  21 +-
 .../translation/ParDoBoundTranslatorTest.java   |  38 +-
 .../translation/ReadUnboundTranslatorTest.java  |  15 +-
 runners/core-java/pom.xml                       |   6 +-
 .../beam/runners/core/ActiveWindowSet.java      | 173 ++++++
 .../beam/runners/core/AggregatorFactory.java    |   1 -
 .../apache/beam/runners/core/AssignWindows.java |  46 --
 .../beam/runners/core/BaseExecutionContext.java | 176 ++++++
 .../apache/beam/runners/core/DoFnRunner.java    |  24 +-
 .../apache/beam/runners/core/DoFnRunners.java   |   4 +-
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../beam/runners/core/ExecutionContext.java     | 102 +++
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 --
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  12 +-
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  55 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  30 +-
 .../runners/core/InMemoryTimerInternals.java    | 302 +++++++++
 .../apache/beam/runners/core/KeyedWorkItem.java |  44 ++
 .../beam/runners/core/KeyedWorkItemCoder.java   | 130 ++++
 .../beam/runners/core/KeyedWorkItems.java       | 122 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  34 +-
 .../runners/core/MergingActiveWindowSet.java    | 428 +++++++++++++
 .../runners/core/NonMergingActiveWindowSet.java |  91 +++
 .../beam/runners/core/PaneInfoTracker.java      |   4 +
 .../runners/core/PerKeyCombineFnRunner.java     | 149 +++++
 .../runners/core/PerKeyCombineFnRunners.java    | 262 ++++++++
 .../core/PushbackSideInputDoFnRunner.java       |  11 +
 .../runners/core/ReduceFnContextFactory.java    |  26 +-
 .../beam/runners/core/ReduceFnRunner.java       | 496 +++++++++------
 .../beam/runners/core/SimpleDoFnRunner.java     | 328 +++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  27 +-
 .../beam/runners/core/SplittableParDo.java      | 392 +++++++++---
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../apache/beam/runners/core/WatermarkHold.java |   9 +-
 .../TriggerStateMachineContextFactory.java      |   2 +-
 .../triggers/TriggerStateMachineRunner.java     |  14 +-
 .../core/GroupAlsoByWindowsProperties.java      | 590 ++++++++++--------
 .../core/InMemoryTimerInternalsTest.java        | 197 ++++++
 .../runners/core/KeyedWorkItemCoderTest.java    |  64 ++
 .../core/LateDataDroppingDoFnRunnerTest.java    |   3 +
 .../core/MergingActiveWindowSetTest.java        | 262 ++++++++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  60 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   | 138 +++--
 .../beam/runners/core/ReduceFnTester.java       |  78 ++-
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 301 +++++++++
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   3 +-
 .../beam/runners/core/SplittableParDoTest.java  | 158 ++++-
 .../UnboundedReadFromBoundedSourceTest.java     |   6 +-
 .../beam/runners/core/WindowMatchers.java       | 204 ++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 +++
 .../triggers/TriggerStateMachineTester.java     |  27 +-
 runners/direct-java/pom.xml                     |   3 +-
 .../direct/AbstractModelEnforcement.java        |   2 +-
 .../runners/direct/AggregatorContainer.java     |   4 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   8 +-
 .../beam/runners/direct/CommittedResult.java    |   2 +-
 .../beam/runners/direct/CompletionCallback.java |   2 +-
 .../direct/ConsumerTrackingPipelineVisitor.java | 173 ------
 .../CopyOnAccessInMemoryStateInternals.java     | 467 ++++++++++++++
 .../runners/direct/DirectExecutionContext.java  |   7 +-
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  42 +-
 .../apache/beam/runners/direct/DirectGraph.java |  89 +++
 .../beam/runners/direct/DirectGraphVisitor.java | 142 +++++
 .../beam/runners/direct/DirectGroupByKey.java   |  46 +-
 .../direct/DirectGroupByKeyOverrideFactory.java |   3 +-
 .../beam/runners/direct/DirectRunner.java       |  62 +-
 .../runners/direct/DirectTimerInternals.java    |  14 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../runners/direct/EmptyTransformEvaluator.java |   4 +-
 .../beam/runners/direct/EvaluationContext.java  | 105 ++--
 .../direct/ExecutorServiceParallelExecutor.java |  29 +-
 .../runners/direct/FlattenEvaluatorFactory.java |  10 +-
 .../runners/direct/ForwardingPTransform.java    |   6 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  13 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   6 +-
 .../ImmutabilityCheckingBundleFactory.java      |  21 +-
 .../direct/ImmutabilityEnforcementFactory.java  |   2 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  64 +-
 .../beam/runners/direct/ModelEnforcement.java   |   2 +-
 .../direct/PTransformOverrideFactory.java       |  35 --
 .../beam/runners/direct/ParDoEvaluator.java     |  45 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  97 ++-
 .../direct/ParDoMultiOverrideFactory.java       | 193 +++++-
 .../ParDoSingleViaMultiOverrideFactory.java     |  21 +-
 .../direct/PassthroughTransformEvaluator.java   |   4 +-
 ...littableProcessElementsEvaluatorFactory.java | 145 +++++
 .../direct/StatefulParDoEvaluatorFactory.java   | 257 ++++++++
 .../runners/direct/StepTransformResult.java     |  39 +-
 .../direct/TestStreamEvaluatorFactory.java      |   9 +-
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |   7 +
 .../beam/runners/direct/TransformExecutor.java  |   4 +-
 .../beam/runners/direct/TransformResult.java    |  17 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   3 +-
 .../runners/direct/ViewEvaluatorFactory.java    |  10 +-
 .../direct/WatermarkCallbackExecutor.java       |  34 +
 .../beam/runners/direct/WatermarkManager.java   | 139 +++--
 .../runners/direct/WindowEvaluatorFactory.java  |   6 +-
 .../direct/WriteWithShardingFactory.java        |  10 +-
 .../runners/direct/AggregatorContainerTest.java |   2 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  47 +-
 .../direct/CloningBundleFactoryTest.java        |   8 +-
 .../runners/direct/CommittedResultTest.java     |   8 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 287 ---------
 .../CopyOnAccessInMemoryStateInternalsTest.java | 565 +++++++++++++++++
 .../runners/direct/DirectGraphVisitorTest.java  | 265 ++++++++
 .../beam/runners/direct/DirectGraphs.java       |  35 ++
 .../beam/runners/direct/DirectRunnerTest.java   |  36 --
 ...leManagerRemovingTransformEvaluatorTest.java |   4 +-
 .../runners/direct/EvaluationContextTest.java   | 132 ++--
 .../direct/FlattenEvaluatorFactoryTest.java     |  27 +-
 .../direct/ForwardingPTransformTest.java        |   4 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  11 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  12 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |  10 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  11 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  14 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  | 137 ++--
 .../beam/runners/direct/ParDoEvaluatorTest.java |  11 +-
 .../runners/direct/SideInputContainerTest.java  |   5 +-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 -------
 .../StatefulParDoEvaluatorFactoryTest.java      | 323 ++++++++++
 .../runners/direct/StepTransformResultTest.java |  32 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  29 +-
 .../runners/direct/TransformExecutorTest.java   | 228 +++----
 .../UnboundedReadEvaluatorFactoryTest.java      |  53 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   9 +-
 .../direct/WatermarkCallbackExecutorTest.java   |  11 +-
 .../runners/direct/WatermarkManagerTest.java    | 260 ++++----
 .../direct/WindowEvaluatorFactoryTest.java      |  21 +-
 .../direct/WriteWithShardingFactoryTest.java    |  18 +-
 runners/flink/examples/pom.xml                  |  14 +-
 .../beam/runners/flink/examples/TFIDF.java      |   6 +-
 .../beam/runners/flink/examples/WordCount.java  |   2 +-
 .../flink/examples/streaming/AutoComplete.java  |   6 +-
 runners/flink/pom.xml                           |   2 +-
 runners/flink/runner/pom.xml                    |  15 +-
 .../FlinkPipelineExecutionEnvironment.java      |  12 +
 .../runners/flink/FlinkPipelineOptions.java     |   5 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  36 +-
 .../FlinkBatchPipelineTranslator.java           |  18 +-
 .../FlinkBatchTransformTranslators.java         |  55 +-
 .../FlinkStreamingPipelineTranslator.java       |  23 +-
 .../FlinkStreamingTransformTranslators.java     | 140 ++---
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../functions/FlinkDoFnFunction.java            |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   4 +-
 .../FlinkMergingPartialReduceFunction.java      |   4 +-
 .../functions/FlinkMergingReduceFunction.java   |   4 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |  10 +-
 .../functions/FlinkPartialReduceFunction.java   |   4 +-
 .../functions/FlinkProcessContextBase.java      |  29 +-
 .../functions/FlinkReduceFunction.java          |   4 +-
 .../wrappers/streaming/DoFnOperator.java        |  48 +-
 .../streaming/SingletonKeyedWorkItem.java       |   2 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  25 +-
 .../wrappers/streaming/WorkItemKeySelector.java |   2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   6 +-
 .../flink/streaming/DoFnOperatorTest.java       |  13 +-
 .../flink/streaming/GroupByNullKeyTest.java     |  18 +-
 .../streaming/TopWikipediaSessionsITCase.java   |  10 +-
 runners/google-cloud-dataflow-java/pom.xml      |  14 +-
 .../beam/runners/dataflow/DataflowClient.java   | 140 +++++
 .../runners/dataflow/DataflowPipelineJob.java   |  31 +-
 .../dataflow/DataflowPipelineTranslator.java    |  36 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 210 ++++---
 .../runners/dataflow/DataflowRunnerInfo.java    |  92 +++
 .../dataflow/internal/AssignWindows.java        |  10 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   2 +-
 .../options/DataflowPipelineOptions.java        |  19 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  57 +-
 .../options/DataflowProfilingOptions.java       |   8 +-
 .../dataflow/testing/TestDataflowRunner.java    |  35 +-
 .../dataflow/util/DataflowTemplateJob.java      |   2 +-
 .../runners/dataflow/util/MonitoringUtil.java   |  22 +-
 .../beam/runners/dataflow/dataflow.properties   |  23 +
 .../dataflow/DataflowPipelineJobTest.java       |  49 +-
 .../DataflowPipelineTranslatorTest.java         |  63 +-
 .../dataflow/DataflowRunnerInfoTest.java        |  51 ++
 .../runners/dataflow/DataflowRunnerTest.java    | 333 ++++++----
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 .../options/DataflowPipelineOptionsTest.java    |  20 +-
 .../options/DataflowProfilingOptionsTest.java   |   6 +-
 .../testing/TestDataflowRunnerTest.java         |  52 +-
 .../transforms/DataflowGroupByKeyTest.java      |  18 +-
 .../dataflow/transforms/DataflowViewTest.java   |  20 +-
 .../dataflow/util/MonitoringUtilTest.java       |  21 +-
 runners/pom.xml                                 |   2 +-
 runners/spark/README.md                         |  59 +-
 runners/spark/pom.xml                           |  60 +-
 .../beam/runners/spark/EvaluationResult.java    |  67 --
 .../runners/spark/SparkPipelineOptions.java     |  15 +
 .../beam/runners/spark/SparkPipelineResult.java | 193 ++++++
 .../apache/beam/runners/spark/SparkRunner.java  | 213 +++++--
 .../beam/runners/spark/TestSparkRunner.java     |  91 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   8 +-
 .../spark/aggregators/SparkAggregators.java     | 126 ++++
 .../metrics/WithNamedAggregatorsSupport.java    |   4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |  60 +-
 .../beam/runners/spark/examples/WordCount.java  |  15 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |   2 +-
 .../beam/runners/spark/io/CreateStream.java     |   2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |  20 +-
 .../beam/runners/spark/io/SourceDStream.java    |  22 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  17 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   4 +-
 .../spark/stateful/StateSpecFunctions.java      |   2 +-
 .../spark/translation/BoundedDataset.java       |   5 +-
 .../beam/runners/spark/translation/Dataset.java |   2 +-
 .../runners/spark/translation/DoFnFunction.java | 110 ++--
 .../spark/translation/EvaluationContext.java    | 137 +---
 .../translation/GroupCombineFunctions.java      |  23 +-
 .../spark/translation/MultiDoFnFunction.java    | 135 ++--
 .../spark/translation/SparkAssignWindowFn.java  |  69 +++
 .../spark/translation/SparkContextFactory.java  |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   | 214 +++++++
 .../spark/translation/SparkProcessContext.java  | 390 +++---------
 .../spark/translation/SparkRuntimeContext.java  |  62 +-
 .../translation/StorageLevelPTransform.java     |  43 ++
 .../spark/translation/TransformTranslator.java  |  91 ++-
 .../spark/translation/TranslationUtils.java     |  33 +
 .../spark/translation/WindowingHelpers.java     |   3 +-
 .../SparkRunnerStreamingContextFactory.java     |   2 +-
 .../streaming/StreamingTransformTranslator.java |  47 +-
 .../translation/streaming/UnboundedDataset.java |  13 +-
 .../util/SinglePrimitiveOutputPTransform.java   |   2 +-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 ++++
 .../runners/spark/ProvidedSparkContextTest.java |   6 +-
 .../runners/spark/SparkPipelineStateTest.java   | 217 +++++++
 .../spark/aggregators/ClearAggregatorsRule.java |  37 ++
 .../metrics/sink/ClearAggregatorsRule.java      |  33 -
 .../metrics/sink/NamedAggregatorsTest.java      |   3 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |  57 ++
 .../beam/runners/spark/io/AvroPipelineTest.java |   2 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   2 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   2 +-
 .../spark/translation/SideEffectsTest.java      |  59 --
 .../spark/translation/StorageLevelTest.java     |  56 ++
 .../streaming/EmptyStreamAssertionTest.java     |   6 +
 .../streaming/FlattenStreamingTest.java         |   2 +
 .../streaming/KafkaStreamingTest.java           |  34 +-
 .../ResumeFromCheckpointStreamingTest.java      |  32 +-
 .../streaming/SimpleStreamingWordCountTest.java |   1 +
 .../streaming/utils/PAssertStreaming.java       |  31 +-
 .../SparkTestPipelineOptionsForStreaming.java   |   6 -
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml |  68 +-
 sdks/java/core/pom.xml                          |  31 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   9 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 151 +----
 .../beam/sdk/annotations/Experimental.java      |   5 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |   2 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   4 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  16 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |  52 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |  17 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  95 ++-
 .../org/apache/beam/sdk/io/FileBasedSource.java |  12 +-
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 +
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 +
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 ++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 180 +++++-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  27 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  48 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  32 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   2 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   4 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   8 +-
 .../apache/beam/sdk/metrics/CounterCell.java    |  10 +-
 .../beam/sdk/metrics/DistributionCell.java      |  14 +-
 .../org/apache/beam/sdk/metrics/MetricCell.java |   2 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |   3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |   5 +-
 .../beam/sdk/metrics/MetricsEnvironment.java    |   8 +-
 .../org/apache/beam/sdk/options/GcpOptions.java |  19 +-
 .../beam/sdk/options/PipelineOptions.java       |   7 +
 .../sdk/options/PipelineOptionsFactory.java     | 111 +++-
 .../sdk/options/ProxyInvocationHandler.java     |  26 +
 .../beam/sdk/options/StreamingOptions.java      |   2 +-
 .../apache/beam/sdk/options/ValueProvider.java  |  29 +-
 .../sdk/runners/PTransformOverrideFactory.java  |  41 ++
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 345 +++++++++--
 .../beam/sdk/runners/TransformTreeNode.java     | 257 --------
 .../beam/sdk/testing/BigqueryMatcher.java       |  48 +-
 .../beam/sdk/testing/FileChecksumMatcher.java   | 124 ++--
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 +++
 .../org/apache/beam/sdk/testing/PAssert.java    |  87 +--
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +-
 .../apache/beam/sdk/testing/TestPipeline.java   | 225 ++++++-
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    |  24 +
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 +
 .../apache/beam/sdk/testing/UsesTestStream.java |  24 +
 .../beam/sdk/testing/UsesTimersInParDo.java     |  25 +
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 ++++
 .../apache/beam/sdk/transforms/Aggregator.java  |  19 -
 .../sdk/transforms/AggregatorRetriever.java     |   2 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |  75 +--
 .../beam/sdk/transforms/ApproximateUnique.java  |   4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  51 +-
 .../beam/sdk/transforms/CombineWithContext.java |   2 +-
 .../org/apache/beam/sdk/transforms/Count.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Create.java  |   4 +-
 .../apache/beam/sdk/transforms/Distinct.java    |   4 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  38 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |  14 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 387 ++++++------
 .../org/apache/beam/sdk/transforms/Filter.java  |   2 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   2 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   4 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   2 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |   4 +-
 .../apache/beam/sdk/transforms/MapElements.java |   2 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   8 +-
 .../apache/beam/sdk/transforms/PTransform.java  |   4 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 201 +-----
 .../apache/beam/sdk/transforms/Partition.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  14 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Values.java  |   2 +-
 .../org/apache/beam/sdk/transforms/View.java    |  16 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../beam/sdk/transforms/WithTimestamps.java     |   2 +-
 .../sdk/transforms/display/DisplayData.java     |  22 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |   2 +-
 .../transforms/join/KeyedPCollectionTuple.java  |   9 +-
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  39 +-
 .../reflect/ByteBuddyOnTimerInvokerFactory.java |  10 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |  11 +
 .../sdk/transforms/reflect/DoFnInvokers.java    |  16 +-
 .../sdk/transforms/reflect/DoFnSignature.java   |  34 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |  96 ++-
 .../sdk/transforms/windowing/BoundedWindow.java |  31 +
 .../beam/sdk/transforms/windowing/Window.java   |  13 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   | 173 ------
 .../org/apache/beam/sdk/util/ApiSurface.java    |  52 +-
 .../beam/sdk/util/BaseExecutionContext.java     | 174 ------
 .../apache/beam/sdk/util/ExecutionContext.java  | 100 ---
 .../beam/sdk/util/ExplicitShardedFile.java      | 120 ++++
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 ---
 .../apache/beam/sdk/util/GcsPathValidator.java  |   3 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 107 +++-
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |  43 --
 .../beam/sdk/util/KeyedWorkItemCoder.java       | 128 ----
 .../apache/beam/sdk/util/KeyedWorkItems.java    | 121 ----
 .../beam/sdk/util/MergingActiveWindowSet.java   | 428 -------------
 .../sdk/util/NonMergingActiveWindowSet.java     |  91 ---
 .../beam/sdk/util/NumberedShardedFile.java      | 220 +++++++
 .../beam/sdk/util/PerKeyCombineFnRunner.java    | 148 -----
 .../beam/sdk/util/PerKeyCombineFnRunners.java   | 258 --------
 .../org/apache/beam/sdk/util/PropertyNames.java |   2 +
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   4 -
 .../org/apache/beam/sdk/util/Reshuffle.java     |   2 +-
 .../org/apache/beam/sdk/util/ShardedFile.java   |  42 ++
 .../apache/beam/sdk/util/TimerInternals.java    | 120 ++--
 .../beam/sdk/util/WindowingInternals.java       |  10 -
 .../CopyOnAccessInMemoryStateInternals.java     | 453 --------------
 .../sdk/util/state/InMemoryStateInternals.java  |  33 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 247 --------
 .../beam/sdk/util/state/StateInternals.java     |   2 +-
 .../beam/sdk/util/state/TimerCallback.java      |  35 --
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 .../java/org/apache/beam/sdk/values/PBegin.java |   4 +-
 .../org/apache/beam/sdk/values/PCollection.java |   4 +-
 .../apache/beam/sdk/values/PCollectionList.java |  65 +-
 .../beam/sdk/values/PCollectionTuple.java       |  30 +-
 .../java/org/apache/beam/sdk/values/PDone.java  |   4 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   4 +-
 .../org/apache/beam/sdk/values/POutput.java     |   4 +-
 .../java/org/apache/beam/sdk/values/PValue.java |  10 +
 .../org/apache/beam/sdk/values/PValueBase.java  |  11 +-
 .../apache/beam/sdk/values/TaggedPValue.java    |  42 ++
 .../org/apache/beam/sdk/values/TypedPValue.java |   2 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |  28 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |  43 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 ------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 ---
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  11 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |  10 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   7 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  16 +-
 .../io/BoundedReadFromUnboundedSourceTest.java  |   6 +-
 .../beam/sdk/io/CompressedSourceTest.java       |  12 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  69 ++-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  21 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  11 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  30 +-
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 ++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  43 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  30 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  26 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  44 +-
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   4 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  10 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  10 +-
 .../apache/beam/sdk/metrics/MetricMatchers.java |  47 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    |  63 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |  32 +-
 .../sdk/options/PipelineOptionsFactoryTest.java | 222 ++++++-
 .../beam/sdk/options/PipelineOptionsTest.java   |  24 +
 .../sdk/options/ProxyInvocationHandlerTest.java |   5 +-
 .../beam/sdk/options/ValueProviderTest.java     |  39 ++
 .../sdk/runners/TransformHierarchyTest.java     | 208 +++++--
 .../beam/sdk/runners/TransformTreeTest.java     |  27 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   |  82 +--
 .../sdk/testing/FileChecksumMatcherTest.java    |  66 +-
 .../beam/sdk/testing/GatherAllPanesTest.java    | 141 +++++
 .../apache/beam/sdk/testing/PAssertTest.java    |  32 +-
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++--
 .../beam/sdk/testing/TestPipelineTest.java      | 186 +++++-
 .../apache/beam/sdk/testing/TestStreamTest.java |  19 +-
 .../testing/ValueInSingleWindowCoderTest.java   |  51 ++
 .../transforms/ApproximateQuantilesTest.java    |  12 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |   6 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   5 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  25 +-
 .../apache/beam/sdk/transforms/CountTest.java   |  13 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |  27 +-
 .../beam/sdk/transforms/DistinctTest.java       |  12 +-
 .../apache/beam/sdk/transforms/DoFnTest.java    |   4 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |  46 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |  18 +-
 .../sdk/transforms/FlatMapElementsTest.java     |  12 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |  35 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |  34 +-
 .../apache/beam/sdk/transforms/KeysTest.java    |   9 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   9 +-
 .../apache/beam/sdk/transforms/LatestTest.java  |  12 +-
 .../beam/sdk/transforms/MapElementsTest.java    |  16 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |  72 ---
 .../apache/beam/sdk/transforms/OldDoFnTest.java | 125 ++--
 .../beam/sdk/transforms/PTransformTest.java     |   2 +-
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  45 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 617 ++++++++++++-------
 .../beam/sdk/transforms/PartitionTest.java      |   8 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  25 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |  34 +-
 .../beam/sdk/transforms/SplittableDoFnTest.java | 453 ++++++++++++++
 .../org/apache/beam/sdk/transforms/TopTest.java |  15 +-
 .../apache/beam/sdk/transforms/ValuesTest.java  |   7 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |  88 +--
 .../beam/sdk/transforms/WithKeysTest.java       |   8 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |   9 +-
 .../display/DisplayDataEvaluator.java           |   8 +-
 .../display/DisplayDataEvaluatorTest.java       |   2 +-
 .../display/DisplayDataMatchersTest.java        |   6 +-
 .../sdk/transforms/display/DisplayDataTest.java |  90 ++-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  11 +-
 .../DoFnSignaturesSplittableDoFnTest.java       |   3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  47 ++
 .../sdk/transforms/windowing/WindowTest.java    |  22 +-
 .../sdk/transforms/windowing/WindowingTest.java |  13 +-
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -----
 .../beam/sdk/util/GcsPathValidatorTest.java     |  15 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  85 ++-
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |  62 --
 .../sdk/util/MergingActiveWindowSetTest.java    | 262 --------
 .../beam/sdk/util/NumberedShardedFileTest.java  | 181 ++++++
 .../apache/beam/sdk/util/ReleaseInfoTest.java   |  45 ++
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  11 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |   6 +-
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 552 -----------------
 .../util/state/InMemoryTimerInternalsTest.java  | 116 ----
 .../beam/sdk/values/PCollectionListTest.java    | 117 ++++
 .../beam/sdk/values/PCollectionTupleTest.java   |  80 ++-
 .../org/apache/beam/sdk/values/PDoneTest.java   |  13 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |  10 +-
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 .../extensions/joinlibrary/InnerJoinTest.java   |   9 +-
 .../joinlibrary/OuterLeftJoinTest.java          |  10 +-
 .../joinlibrary/OuterRightJoinTest.java         |  10 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/extensions/sorter/README.md           |   2 +-
 sdks/java/extensions/sorter/pom.xml             |  10 +-
 .../sorter/BufferedExternalSorter.java          |  12 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |  21 +-
 .../sdk/extensions/sorter/InMemorySorter.java   |  26 +-
 .../beam/sdk/extensions/sorter/SortValues.java  |   2 +-
 .../sorter/BufferedExternalSorterTest.java      |  74 ++-
 .../extensions/sorter/ExternalSorterTest.java   |  69 ++-
 .../extensions/sorter/InMemorySorterTest.java   |   8 +
 .../sdk/extensions/sorter/SortValuesTest.java   |   7 +-
 sdks/java/io/google-cloud-platform/pom.xml      |   8 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 453 ++++++++++----
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   7 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 121 +++-
 .../io/gcp/bigquery/BigQueryTableInserter.java  | 217 -------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  28 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   6 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |   4 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 +
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 ++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  |   2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 568 +++++++++++++----
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 139 ++++-
 .../gcp/bigquery/BigQueryTableInserterTest.java | 245 --------
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  50 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |  13 +-
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++
 sdks/java/io/hdfs/pom.xml                       |   8 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 +
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 ++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 ++
 sdks/java/io/jdbc/pom.xml                       |   2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  56 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  38 +-
 sdks/java/io/jms/pom.xml                        |  22 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 342 ++++++----
 .../org/apache/beam/sdk/io/jms/JmsRecord.java   |   8 +-
 .../org/apache/beam/sdk/io/jms/JmsIOTest.java   |   9 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  20 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  25 +-
 sdks/java/io/kinesis/pom.xml                    |   2 +-
 .../sdk/io/kinesis/KinesisMockReadTest.java     |   7 +-
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |   6 +-
 sdks/java/io/mongodb/pom.xml                    |   2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |   4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |  22 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |  17 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  15 +-
 sdks/java/io/pom.xml                            |   2 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../beam/sdk/transforms/CombineJava8Test.java   |   8 +-
 .../beam/sdk/transforms/DistinctJava8Test.java  |   5 +-
 .../beam/sdk/transforms/FilterJava8Test.java    |   9 +-
 .../transforms/FlatMapElementsJava8Test.java    |   7 +-
 .../sdk/transforms/MapElementsJava8Test.java    |   9 +-
 .../beam/sdk/transforms/PartitionJava8Test.java |   7 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |   6 +-
 .../sdk/transforms/WithTimestampsJava8Test.java |  14 +-
 .../examples-java8/generate-sources.sh          |  82 +++
 .../maven-archetypes/examples-java8/pom.xml     | 182 ++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++
 .../main/resources/archetype-resources/pom.xml  | 295 +++++++++
 .../projects/basic/archetype.properties         |  21 +
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../examples/generate-sources.sh                |  68 ++
 sdks/java/maven-archetypes/examples/pom.xml     |  32 +-
 .../main/resources/archetype-resources/pom.xml  | 168 ++++-
 .../src/main/java/DebuggingWordCount.java       | 164 -----
 .../src/main/java/MinimalWordCount.java         | 118 ----
 .../src/main/java/WindowedWordCount.java        | 229 -------
 .../src/main/java/WordCount.java                | 186 ------
 .../common/ExampleBigQueryTableOptions.java     |  55 --
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 --
 .../java/common/ExamplePubsubTopicOptions.java  |  45 --
 .../src/main/java/common/ExampleUtils.java      | 352 -----------
 .../src/test/java/DebuggingWordCountTest.java   |  52 --
 .../src/test/java/WordCountTest.java            |  85 ---
 sdks/java/maven-archetypes/pom.xml              |  40 +-
 sdks/java/maven-archetypes/starter/pom.xml      |   9 +-
 .../main/resources/archetype-resources/pom.xml  |  20 +-
 .../resources/projects/basic/reference/pom.xml  |  20 +-
 .../update-examples-archetype.sh                |  59 --
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 641 files changed, 23199 insertions(+), 14088 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/77742979/.travis.yml
----------------------------------------------------------------------
diff --cc .travis.yml
index 470d2fc,a806477..cb6f790
--- a/.travis.yml
+++ b/.travis.yml
@@@ -28,13 -28,9 +28,13 @@@ notifications
      on_success: change
      on_failure: always
  
 +addons:
 +  apt:
 +    packages:
 +    - python2.7
  env:
    global:
-    - MAVEN_OVERRIDE="--settings=.travis/settings.xml -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true"
+    - MAVEN_OVERRIDE="--settings=.travis/settings.xml"
     - MAVEN_CONTAINER_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
  
  matrix:

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/77742979/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/77742979/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
----------------------------------------------------------------------
diff --cc runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
index 60d668e,5dd6beb..8430bbf
--- 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
@@@ -54,11 -56,8 +56,9 @@@ import org.apache.beam.sdk.transforms.G
  import org.apache.beam.sdk.transforms.PTransform;
  import org.apache.beam.sdk.transforms.ParDo;
  import org.apache.beam.sdk.transforms.View;
 +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
  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.CombineFnUtil;
  import org.apache.beam.sdk.util.WindowedValue;
  import org.apache.beam.sdk.util.WindowingStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/77742979/sdks/pom.xml
----------------------------------------------------------------------



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

Posted by ke...@apache.org.
Require TimeDomain to delete a timer


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

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

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


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 8970b4b..5ca276d 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -52,16 +52,24 @@ class DirectTimerInternals implements TimerInternals {
     throw new UnsupportedOperationException("Setting timer by ID not yet supported.");
   }
 
+  @Deprecated
   @Override
   public void setTimer(TimerData timerData) {
     timerUpdateBuilder.setTimer(timerData);
   }
 
   @Override
+  public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
+    throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
+  }
+
+  @Deprecated
+  @Override
   public void deleteTimer(StateNamespace namespace, String timerId) {
     throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
   }
 
+  @Deprecated
   @Override
   public void deleteTimer(TimerData timerKey) {
     timerUpdateBuilder.deletedTimer(timerKey);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index 9cea529..5398d7b 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -455,6 +455,7 @@ public class WindowDoFnOperator<K, InputT, OutputT>
           throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
         }
 
+        @Deprecated
         @Override
         public void setTimer(TimerData timerKey) {
           if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
@@ -468,11 +469,19 @@ public class WindowDoFnOperator<K, InputT, OutputT>
         }
 
         @Override
+        public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
+          throw new UnsupportedOperationException(
+              "Canceling of a timer by ID is not yet supported.");
+        }
+
+        @Deprecated
+        @Override
         public void deleteTimer(StateNamespace namespace, String timerId) {
           throw new UnsupportedOperationException(
               "Canceling of a timer by ID is not yet supported.");
         }
 
+        @Deprecated
         @Override
         public void deleteTimer(TimerData timerKey) {
           if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35a02740/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
index c3e498e..0bfcddc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
@@ -34,6 +34,7 @@ import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.InstantCoder;
 import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
@@ -61,18 +62,30 @@ public interface TimerInternals {
   void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain);
 
   /**
-   * Sets the timer described by {@code timerData}.
+   * @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}.
    */
+  @Deprecated
   void setTimer(TimerData timerData);
 
   /**
    * Deletes the given timer.
+   *
+   * <p>A timer's ID is enforced to be unique in validation of a {@link DoFn}, but runners
+   * often manage timers for different time domains in very different ways, thus the
+   * {@link TimeDomain} is a required parameter.
+   */
+  void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain);
+
+  /**
+   * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}.
    */
+  @Deprecated
   void deleteTimer(StateNamespace namespace, String timerId);
 
   /**
-   * Deletes the timer with the ID contained in the provided {@link TimerData}.
+   * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}.
    */
+  @Deprecated
   void deleteTimer(TimerData timerKey);
 
   /**


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 950aa7e1d9c50167933eb192a16e15700e483377
Parents: 12be8b1
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 17:44:15 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:46 2016 -0800

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


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

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


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

Posted by ke...@apache.org.
This closes #1673: Require TimeDomain to delete a timer


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

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

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



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

Posted by ke...@apache.org.
Add some key-preserving to KeyedPValueTrackingVisitor


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

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

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/81702e67/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 7f85169..e91a768 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -18,9 +18,15 @@
 package org.apache.beam.runners.direct;
 
 import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Predicates.in;
+import static com.google.common.collect.Iterables.all;
 
+import com.google.common.collect.ImmutableSet;
 import java.util.HashSet;
 import java.util.Set;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
+import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -38,19 +44,21 @@ import org.apache.beam.sdk.values.PValue;
 // TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
 // unkeyed
 class KeyedPValueTrackingVisitor implements PipelineVisitor {
-  @SuppressWarnings("rawtypes")
-  private final Set<Class<? extends PTransform>> producesKeyedOutputs;
+
+  private static final Set<Class<? extends PTransform>> PRODUCES_KEYED_OUTPUTS =
+      ImmutableSet.of(
+          SplittableParDo.GBKIntoKeyedWorkItems.class,
+          DirectGroupByKeyOnly.class,
+          DirectGroupAlsoByWindow.class);
+
   private final Set<PValue> keyedValues;
   private boolean finalized;
 
-  public static KeyedPValueTrackingVisitor create(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    return new KeyedPValueTrackingVisitor(producesKeyedOutputs);
+  public static KeyedPValueTrackingVisitor create() {
+    return new KeyedPValueTrackingVisitor();
   }
 
-  private KeyedPValueTrackingVisitor(
-      @SuppressWarnings("rawtypes") Set<Class<? extends PTransform>> producesKeyedOutputs) {
-    this.producesKeyedOutputs = producesKeyedOutputs;
+  private KeyedPValueTrackingVisitor() {
     this.keyedValues = new HashSet<>();
   }
 
@@ -73,7 +81,7 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
         node);
     if (node.isRootNode()) {
       finalized = true;
-    } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
+    } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) {
       keyedValues.addAll(node.getOutputs());
     }
   }
@@ -83,7 +91,9 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
 
   @Override
   public void visitValue(PValue value, TransformHierarchy.Node producer) {
-    if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
+    if (PRODUCES_KEYED_OUTPUTS.contains(producer.getTransform().getClass())
+        || (isKeyPreserving(producer.getTransform())
+            && all(producer.getInputs(), in(keyedValues)))) {
       keyedValues.add(value);
     }
   }
@@ -93,4 +103,9 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
         finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed");
     return keyedValues;
   }
+
+  private static boolean isKeyPreserving(PTransform<?, ?> transform) {
+    // There are currently no key-preserving transforms; this lays the infrastructure for them
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/81702e67/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 eef3375..a357005 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
@@ -21,9 +21,7 @@ import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
-import com.google.common.collect.ImmutableSet;
 import java.util.Collections;
-import java.util.Set;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -33,7 +31,6 @@ 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.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
@@ -57,54 +54,20 @@ public class KeyedPValueTrackingVisitorTest {
 
   @Before
   public void setup() {
-
-    @SuppressWarnings("rawtypes")
-    Set<Class<? extends PTransform>> producesKeyed =
-        ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);
-    visitor = KeyedPValueTrackingVisitor.create(producesKeyed);
-  }
-
-  @Test
-  public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(new PrimitiveKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-  @Test
-  public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply("firstKey", new PrimitiveKeyer<Integer>())
-            .apply("secondKey", new PrimitiveKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
-  }
-
-  @Test
-  public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(new CompositeKeyer<Integer>());
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), hasItem(keyed));
+    p = TestPipeline.create();
+    visitor = KeyedPValueTrackingVisitor.create();
   }
 
   @Test
-  public void compositeProducesKeyedOutputKeyedInputKeyedOutut() {
-    PCollection<Integer> keyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply("firstKey", new CompositeKeyer<Integer>())
-            .apply("secondKey", new CompositeKeyer<Integer>());
+  public void groupByKeyProducesKeyedOutput() {
+    PCollection<KV<String, Iterable<Integer>>> keyed =
+        p.apply(Create.of(KV.of("foo", 3)))
+            .apply(GroupByKey.<String, Integer>create());
 
     p.traverseTopologically(visitor);
     assertThat(visitor.getKeyedPValues(), hasItem(keyed));
   }
 
-
   @Test
   public void noInputUnkeyedOutput() {
     PCollection<KV<Integer, Iterable<Void>>> unkeyed =
@@ -117,26 +80,17 @@ public class KeyedPValueTrackingVisitorTest {
   }
 
   @Test
-  public void keyedInputNotProducesKeyedOutputUnkeyedOutput() {
-    PCollection<Integer> onceKeyed =
-        p.apply(Create.<Integer>of(1, 2, 3))
-            .apply(new PrimitiveKeyer<Integer>())
-            .apply(ParDo.of(new IdentityFn<Integer>()));
+  public void keyedInputWithoutKeyPreserving() {
+    PCollection<KV<String, Iterable<Integer>>> onceKeyed =
+        p.apply(Create.of(KV.of("hello", 42)))
+            .apply(GroupByKey.<String, Integer>create())
+            .apply(ParDo.of(new IdentityFn<KV<String, Iterable<Integer>>>()));
 
     p.traverseTopologically(visitor);
     assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed)));
   }
 
   @Test
-  public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() {
-    PCollection<Integer> unkeyed =
-        p.apply(Create.<Integer>of(1, 2, 3)).apply(ParDo.of(new IdentityFn<Integer>()));
-
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed)));
-  }
-
-  @Test
   public void traverseMultipleTimesThrows() {
     p.apply(
             Create.<KV<Integer, Void>>of(
@@ -161,22 +115,6 @@ public class KeyedPValueTrackingVisitorTest {
     visitor.getKeyedPValues();
   }
 
-  private static class PrimitiveKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
-    @Override
-    public PCollection<K> expand(PCollection<K> input) {
-      return PCollection.<K>createPrimitiveOutputInternal(
-              input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-          .setCoder(input.getCoder());
-    }
-  }
-
-  private static class CompositeKeyer<K> extends PTransform<PCollection<K>, PCollection<K>> {
-    @Override
-    public PCollection<K> expand(PCollection<K> input) {
-      return input.apply(new PrimitiveKeyer<K>()).apply(ParDo.of(new IdentityFn<K>()));
-    }
-  }
-
   private static class IdentityFn<K> extends DoFn<K, K> {
     @ProcessElement
     public void processElement(ProcessContext c) throws Exception {


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

Posted by ke...@apache.org.
Migrated the beam-runners-direct-java module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 09c404a6c407898fcbc2fd22797cba4da8839b93
Parents: b671025
Author: Stas Levin <st...@gmail.com>
Authored: Mon Dec 19 10:20:16 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

----------------------------------------------------------------------
 .../direct/BoundedReadEvaluatorFactoryTest.java       | 13 ++++++-------
 .../beam/runners/direct/CloningBundleFactoryTest.java |  8 ++------
 .../beam/runners/direct/CommittedResultTest.java      |  6 +++++-
 .../CopyOnAccessInMemoryStateInternalsTest.java       | 11 +++++++++--
 .../beam/runners/direct/DirectGraphVisitorTest.java   |  3 ++-
 .../beam/runners/direct/EvaluationContextTest.java    |  7 ++++---
 .../runners/direct/FlattenEvaluatorFactoryTest.java   |  6 ++++--
 .../direct/GroupByKeyEvaluatorFactoryTest.java        |  5 ++++-
 .../direct/GroupByKeyOnlyEvaluatorFactoryTest.java    |  5 ++++-
 .../direct/ImmutabilityCheckingBundleFactoryTest.java |  4 +++-
 .../direct/ImmutabilityEnforcementFactoryTest.java    |  3 ++-
 .../direct/ImmutableListBundleFactoryTest.java        | 14 +++++++++++---
 .../direct/KeyedPValueTrackingVisitorTest.java        |  6 +++---
 .../beam/runners/direct/ParDoEvaluatorTest.java       |  5 ++++-
 .../beam/runners/direct/SideInputContainerTest.java   |  5 +++--
 .../direct/StatefulParDoEvaluatorFactoryTest.java     |  7 +++++--
 .../beam/runners/direct/StepTransformResultTest.java  |  5 ++++-
 .../direct/TestStreamEvaluatorFactoryTest.java        |  5 ++++-
 .../beam/runners/direct/TransformExecutorTest.java    |  4 +++-
 .../direct/UnboundedReadEvaluatorFactoryTest.java     |  9 ++++-----
 .../beam/runners/direct/ViewEvaluatorFactoryTest.java |  5 ++++-
 .../runners/direct/WatermarkCallbackExecutorTest.java |  5 ++++-
 .../beam/runners/direct/WatermarkManagerTest.java     |  6 ++++--
 .../runners/direct/WindowEvaluatorFactoryTest.java    |  5 ++++-
 .../runners/direct/WriteWithShardingFactoryTest.java  | 14 +++++++-------
 25 files changed, 109 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
index acb1444..97eae27 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
@@ -62,6 +62,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -82,11 +83,13 @@ public class BoundedReadEvaluatorFactoryTest {
   private BundleFactory bundleFactory;
   private AppliedPTransform<?, ?, ?> longsProducer;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
     source = CountingSource.upTo(10L);
-    TestPipeline p = TestPipeline.create();
     longs = p.apply(Read.from(source));
 
     factory =
@@ -142,7 +145,7 @@ public class BoundedReadEvaluatorFactoryTest {
       elems[i] = (long) i;
     }
     PCollection<Long> read =
-        TestPipeline.create().apply(Read.from(new TestSource<>(VarLongCoder.of(), 5, elems)));
+        p.apply(Read.from(new TestSource<>(VarLongCoder.of(), 5, elems)));
     AppliedPTransform<?, ?, ?> transform = DirectGraphs.getProducer(read);
     Collection<CommittedBundle<?>> unreadInputs =
         new BoundedReadEvaluatorFactory.InputProvider(context).getInitialInputs(transform, 1);
@@ -191,8 +194,7 @@ public class BoundedReadEvaluatorFactoryTest {
     BoundedReadEvaluatorFactory factory = new BoundedReadEvaluatorFactory(context, 0L);
 
     PCollection<Long> read =
-        TestPipeline.create()
-            .apply(Read.from(SourceTestUtils.toUnsplittableSource(CountingSource.upTo(10L))));
+        p.apply(Read.from(SourceTestUtils.toUnsplittableSource(CountingSource.upTo(10L))));
     AppliedPTransform<?, ?, ?> transform = DirectGraphs.getProducer(read);
 
     when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle());
@@ -298,8 +300,6 @@ public class BoundedReadEvaluatorFactoryTest {
   @Test
   public void boundedSourceEvaluatorClosesReader() throws Exception {
     TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L);
-
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> pcollection = p.apply(Read.from(source));
     AppliedPTransform<?, ?, ?> sourceTransform = DirectGraphs.getProducer(pcollection);
 
@@ -320,7 +320,6 @@ public class BoundedReadEvaluatorFactoryTest {
   public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception {
     TestSource<Long> source = new TestSource<>(BigEndianLongCoder.of());
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> pcollection = p.apply(Read.from(source));
     AppliedPTransform<?, ?, ?> sourceTransform = DirectGraphs.getProducer(pcollection);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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
index bafab59..e5299a2 100644
--- 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
@@ -62,6 +62,8 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class CloningBundleFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
+  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   private CloningBundleFactory factory = CloningBundleFactory.create();
 
   @Test
@@ -76,7 +78,6 @@ public class CloningBundleFactoryTest {
 
   @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
@@ -101,7 +102,6 @@ public class CloningBundleFactoryTest {
 
   @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 =
@@ -130,7 +130,6 @@ public class CloningBundleFactoryTest {
 
   @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);
 
@@ -142,7 +141,6 @@ public class CloningBundleFactoryTest {
 
   @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);
 
@@ -154,7 +152,6 @@ public class CloningBundleFactoryTest {
 
   @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);
@@ -167,7 +164,6 @@ public class CloningBundleFactoryTest {
 
   @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);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
index c6986c0..736f554 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -38,6 +38,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -47,7 +48,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class CommittedResultTest implements Serializable {
-  private transient TestPipeline p = TestPipeline.create();
+
+  @Rule
+  public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   private transient PCollection<Integer> created = p.apply(Create.of(1, 2));
   private transient AppliedPTransform<?, ?, ?> transform =
       AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
index deefc68..35245f4 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
@@ -61,8 +61,15 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class CopyOnAccessInMemoryStateInternalsTest {
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
   private String key = "foo";
+
+  public CopyOnAccessInMemoryStateInternalsTest() {
+    pipeline = TestPipeline.create();
+  }
+
   @Test
   public void testGetWithEmpty() {
     CopyOnAccessInMemoryStateInternals<String> internals =
@@ -167,7 +174,7 @@ public class CopyOnAccessInMemoryStateInternalsTest {
     CombineFn<Long, long[], Long> sumLongFn = new Sum.SumLongFn();
 
     StateNamespace namespace = new StateNamespaceForTest("foo");
-    CoderRegistry reg = TestPipeline.create().getCoderRegistry();
+    CoderRegistry reg = pipeline.getCoderRegistry();
     StateTag<Object, AccumulatorCombiningState<Long, long[], Long>> stateTag =
         StateTags.combiningValue("summer",
             sumLongFn.getAccumulatorCoder(reg, reg.getDefaultCoder(Long.class)), sumLongFn);
@@ -197,7 +204,7 @@ public class CopyOnAccessInMemoryStateInternalsTest {
     KeyedCombineFn<String, Long, long[], Long> sumLongFn = new Sum.SumLongFn().asKeyedFn();
 
     StateNamespace namespace = new StateNamespaceForTest("foo");
-    CoderRegistry reg = TestPipeline.create().getCoderRegistry();
+    CoderRegistry reg = pipeline.getCoderRegistry();
     StateTag<String, AccumulatorCombiningState<Long, long[], Long>> stateTag =
         StateTags.keyedCombiningValue(
             "summer",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
index b88c9a4..c3bbe2d 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
@@ -60,8 +60,9 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class DirectGraphVisitorTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
+  @Rule public transient TestPipeline p = TestPipeline.create()
+                                                      .enableAbandonedNodeEnforcement(false);
 
-  private transient TestPipeline p = TestPipeline.create();
   private transient DirectGraphVisitor visitor = new DirectGraphVisitor();
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
index f11c370..bf36204 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -68,6 +68,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -77,7 +78,6 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class EvaluationContextTest {
-  private TestPipeline p;
   private EvaluationContext context;
 
   private PCollection<Integer> created;
@@ -92,13 +92,14 @@ public class EvaluationContextTest {
   private AppliedPTransform<?, ?, ?> viewProducer;
   private AppliedPTransform<?, ?, ?> unboundedProducer;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     DirectRunner runner =
         DirectRunner.fromOptions(PipelineOptionsFactory.create());
 
-    p = TestPipeline.create();
-
     created = p.apply(Create.of(1, 2, 3));
     downstream = created.apply(WithKeys.<String, Integer>of("foo"));
     view = created.apply(View.<Integer>asIterable());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
index 9e22c36..cda68f0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
@@ -37,6 +37,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -48,9 +49,11 @@ import org.junit.runners.JUnit4;
 public class FlattenEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testFlattenInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
     PCollection<Integer> left = p.apply("left", Create.of(1, 2, 4));
     PCollection<Integer> right = p.apply("right", Create.of(-1, 2, -4));
     PCollectionList<Integer> list = PCollectionList.of(left).and(right);
@@ -118,7 +121,6 @@ public class FlattenEvaluatorFactoryTest {
 
   @Test
   public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception {
-    TestPipeline p = TestPipeline.create();
     PCollectionList<Integer> list = PCollectionList.empty(p);
 
     PCollection<Integer> flattened = list.apply(Flatten.<Integer>pCollections());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
index f0b29f0..fefafd0 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
@@ -41,6 +41,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -52,9 +53,11 @@ import org.junit.runners.JUnit4;
 public class GroupByKeyEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
     KV<String, Integer> firstFoo = KV.of("foo", -1);
     KV<String, Integer> secondFoo = KV.of("foo", 1);
     KV<String, Integer> thirdFoo = KV.of("foo", 3);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
index 7efdb3d..94514ad 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
@@ -41,6 +41,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -52,9 +53,11 @@ import org.junit.runners.JUnit4;
 public class GroupByKeyOnlyEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
     KV<String, Integer> firstFoo = KV.of("foo", -1);
     KV<String, Integer> secondFoo = KV.of("foo", 1);
     KV<String, Integer> thirdFoo = KV.of("foo", 3);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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 6ab8aea..2448078 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
@@ -46,14 +46,16 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ImmutabilityCheckingBundleFactoryTest {
+
+  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
   private ImmutabilityCheckingBundleFactory factory;
   private PCollection<byte[]> created;
   private PCollection<byte[]> transformed;
 
+
   @Before
   public void setup() {
-    TestPipeline p = TestPipeline.create();
     created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
     transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
     DirectGraphVisitor visitor = new DirectGraphVisitor();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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 1ad6ba6..cd3e9b4 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
@@ -42,6 +42,8 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ImmutabilityEnforcementFactoryTest implements Serializable {
+  @Rule public transient TestPipeline p =
+      TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public transient ExpectedException thrown = ExpectedException.none();
   private transient ImmutabilityEnforcementFactory factory;
   private transient BundleFactory bundleFactory;
@@ -52,7 +54,6 @@ public class ImmutabilityEnforcementFactoryTest implements Serializable {
   public void setup() {
     factory = new ImmutabilityEnforcementFactory();
     bundleFactory = ImmutableListBundleFactory.create();
-    TestPipeline p = TestPipeline.create();
     pcollection =
         p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes()))
             .apply(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
index a36c408..46f02cd 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
@@ -57,6 +57,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ImmutableListBundleFactoryTest {
+  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   private ImmutableListBundleFactory bundleFactory = ImmutableListBundleFactory.create();
@@ -66,13 +67,12 @@ public class ImmutableListBundleFactoryTest {
 
   @Before
   public void setup() {
-    TestPipeline p = TestPipeline.create();
     created = p.apply(Create.of(1, 2, 3));
     downstream = created.apply(WithKeys.<String, Integer>of("foo"));
   }
 
   private <T> void createKeyedBundle(Coder<T> coder, T key) throws Exception {
-    PCollection<Integer> pcollection = TestPipeline.create().apply(Create.of(1));
+    PCollection<Integer> pcollection = p.apply("Create", Create.of(1));
     StructuralKey<?> skey = StructuralKey.of(key, coder);
 
     UncommittedBundle<Integer> inFlightBundle = bundleFactory.createKeyedBundle(skey, pcollection);
@@ -87,9 +87,17 @@ public class ImmutableListBundleFactoryTest {
   }
 
   @Test
-  public void keyedWithKeyShouldCreateKeyedBundle() throws Exception {
+  public void keyedWithStringKeyShouldCreateKeyedBundle() throws Exception {
     createKeyedBundle(StringUtf8Coder.of(), "foo");
+  }
+
+  @Test
+  public void keyedWithVarIntKeyShouldCreateKeyedBundle() throws Exception {
     createKeyedBundle(VarIntCoder.of(), 1234);
+  }
+
+  @Test
+  public void keyedWithByteArrayKeyShouldCreateKeyedBundle() throws Exception {
     createKeyedBundle(ByteArrayCoder.of(), new byte[] {0, 2, 4, 99});
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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 0852cd3..eef3375 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
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertThat;
 import com.google.common.collect.ImmutableSet;
 import java.util.Collections;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -53,11 +52,12 @@ public class KeyedPValueTrackingVisitorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   private KeyedPValueTrackingVisitor visitor;
-  private Pipeline p;
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
 
   @Before
   public void setup() {
-    p = TestPipeline.create();
+
     @SuppressWarnings("rawtypes")
     Set<Class<? extends PTransform>> producesKeyed =
         ImmutableSet.<Class<? extends PTransform>>of(PrimitiveKeyer.class, CompositeKeyer.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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 d48ac14..1a3207b 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
@@ -54,6 +54,7 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -72,10 +73,12 @@ public class ParDoEvaluatorTest {
   private List<TupleTag<?>> sideOutputTags;
   private BundleFactory bundleFactory;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
-    TestPipeline p = TestPipeline.create();
     inputPc = p.apply(Create.of(1, 2, 3));
     mainOutputTag = new TupleTag<Integer>() {};
     sideOutputTags = TupleTagList.empty().getAll();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
index cc7d88a..183decd 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
@@ -96,12 +96,14 @@ public class SideInputContainerTest {
       };
 
   @Rule
+  public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
+  @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Mock
   private EvaluationContext context;
 
-  private TestPipeline pipeline;
 
   private SideInputContainer container;
 
@@ -114,7 +116,6 @@ public class SideInputContainerTest {
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
-    pipeline = TestPipeline.create();
 
     PCollection<Integer> create =
         pipeline.apply("forBaseCollection", Create.<Integer>of(1, 2, 3, 4));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
index 326310b..d312aa3 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
@@ -67,6 +67,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -91,6 +92,10 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
 
   private static final BundleFactory BUNDLE_FACTORY = ImmutableListBundleFactory.create();
 
+  @Rule
+  public transient TestPipeline pipeline =
+      TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
@@ -106,7 +111,6 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
   public void windowCleanupScheduled() throws Exception {
     // To test the factory, first we set up a pipeline and then we use the constructed
     // pipeline to create the right parameters to pass to the factory
-    TestPipeline pipeline = TestPipeline.create();
 
     final String stateId = "my-state-id";
 
@@ -208,7 +212,6 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable {
   public void testUnprocessedElements() throws Exception {
     // To test the factory, first we set up a pipeline and then we use the constructed
     // pipeline to create the right parameters to pass to the factory
-    TestPipeline pipeline = TestPipeline.create();
 
     final String stateId = "my-state-id";
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
index d3a2cca..0d94b7a 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
@@ -31,6 +31,7 @@ import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.Matchers;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -44,9 +45,11 @@ public class StepTransformResultTest {
   private BundleFactory bundleFactory;
   private PCollection<Integer> pc;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
-    TestPipeline p = TestPipeline.create();
     pc = p.apply(Create.of(1, 2, 3));
     transform = DirectGraphs.getGraph(p).getProducer(pc);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
index 6bb8623..c5b3b3d 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
@@ -41,6 +41,7 @@ import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -52,6 +53,9 @@ public class TestStreamEvaluatorFactoryTest {
   private BundleFactory bundleFactory;
   private EvaluationContext context;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     context = mock(EvaluationContext.class);
@@ -62,7 +66,6 @@ public class TestStreamEvaluatorFactoryTest {
   /** Demonstrates that returned evaluators produce elements in sequence. */
   @Test
   public void producesElementsInSequence() throws Exception {
-    TestPipeline p = TestPipeline.create();
     PCollection<Integer> streamVals =
         p.apply(
             TestStream.create(VarIntCoder.of())

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
index 4ad22bc..e66ffcf 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
@@ -73,6 +73,9 @@ public class TransformExecutorTest {
   @Mock private EvaluationContext evaluationContext;
   @Mock private TransformEvaluatorRegistry registry;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
@@ -85,7 +88,6 @@ public class TransformExecutorTest {
     evaluatorCompleted = new CountDownLatch(1);
     completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
 
-    TestPipeline p = TestPipeline.create();
     created = p.apply(Create.of("foo", "spam", "third"));
     PCollection<KV<Integer, String>> downstream = created.apply(WithKeys.<Integer, String>of(3));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
index dd36a2f..92d668e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
@@ -72,6 +72,7 @@ import org.joda.time.DateTime;
 import org.joda.time.Instant;
 import org.joda.time.ReadableInstant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -93,10 +94,12 @@ public class UnboundedReadEvaluatorFactoryTest {
   private UnboundedSource<Long, ?> source;
   private DirectGraph graph;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     source = CountingSource.unboundedWithTimestampFn(new LongToInstantFn());
-    TestPipeline p = TestPipeline.create();
     longs = p.apply(Read.from(source));
 
     context = mock(EvaluationContext.class);
@@ -190,7 +193,6 @@ public class UnboundedReadEvaluatorFactoryTest {
         new TestUnboundedSource<>(BigEndianLongCoder.of(), outputs);
     source.dedupes = true;
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> pcollection = p.apply(Read.from(source));
     AppliedPTransform<?, ?, ?> sourceTransform = getProducer(pcollection);
 
@@ -231,7 +233,6 @@ public class UnboundedReadEvaluatorFactoryTest {
 
   @Test
   public void noElementsAvailableReaderIncludedInResidual() throws Exception {
-    TestPipeline p = TestPipeline.create();
     // Read with a very slow rate so by the second read there are no more elements
     PCollection<Long> pcollection =
         p.apply(Read.from(new TestUnboundedSource<>(VarLongCoder.of(), 1L)));
@@ -291,7 +292,6 @@ public class UnboundedReadEvaluatorFactoryTest {
     TestUnboundedSource<Long> source =
         new TestUnboundedSource<>(BigEndianLongCoder.of(), elems.toArray(new Long[0]));
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> pcollection = p.apply(Read.from(source));
     DirectGraph graph = DirectGraphs.getGraph(p);
     AppliedPTransform<?, ?, ?> sourceTransform =
@@ -337,7 +337,6 @@ public class UnboundedReadEvaluatorFactoryTest {
     TestUnboundedSource<Long> source =
         new TestUnboundedSource<>(BigEndianLongCoder.of(), elems.toArray(new Long[0]));
 
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> pcollection = p.apply(Read.from(source));
     AppliedPTransform<?, ?, ?> sourceTransform =
         DirectGraphs.getGraph(p).getProducer(pcollection);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
index 7c08009..6baf55a 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
@@ -41,6 +41,7 @@ import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -52,9 +53,11 @@ import org.junit.runners.JUnit4;
 public class ViewEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Test
   public void testInMemoryEvaluator() throws Exception {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> input = p.apply(Create.of("foo", "bar"));
     CreatePCollectionView<String, Iterable<String>> createView =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
index acdabb6..8d6e73f 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
@@ -37,6 +37,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -51,9 +52,11 @@ public class WatermarkCallbackExecutorTest {
   private AppliedPTransform<?, ?, ?> create;
   private AppliedPTransform<?, ?, ?> sum;
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
-    TestPipeline p = TestPipeline.create();
     PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
     PCollection<Integer> summed = created.apply(Sum.integersGlobally());
     DirectGraph graph = DirectGraphs.getGraph(p);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/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 eb4d0cd..abc8a28 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
@@ -70,6 +70,7 @@ import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.joda.time.ReadableInstant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -94,9 +95,11 @@ public class WatermarkManagerTest implements Serializable {
   private transient BundleFactory bundleFactory;
   private DirectGraph graph;
 
+  @Rule
+  public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
-    TestPipeline p = TestPipeline.create();
 
     createdInts = p.apply("createdInts", Create.of(1, 2, 3));
 
@@ -278,7 +281,6 @@ public class WatermarkManagerTest implements Serializable {
    */
   @Test
   public void getWatermarkMultiIdenticalInput() {
-    TestPipeline p = TestPipeline.create();
     PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
     PCollection<Integer> multiConsumer =
         PCollectionList.of(created).and(created).apply(Flatten.<Integer>pCollections());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
index 66c28ce..9d0c68d 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -52,6 +52,7 @@ import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -96,10 +97,12 @@ public class WindowEvaluatorFactoryTest {
           ImmutableList.of(GlobalWindow.INSTANCE, intervalWindow1, intervalWindow2),
           PaneInfo.NO_FIRING);
 
+  @Rule
+  public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
+
   @Before
   public void setup() {
     MockitoAnnotations.initMocks(this);
-    TestPipeline p = TestPipeline.create();
     input = p.apply(Create.of(1L, 2L, 3L));
 
     bundleFactory = ImmutableListBundleFactory.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/09c404a6/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
index aeb75ed..a8c4c02 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
@@ -69,6 +69,7 @@ public class WriteWithShardingFactoryTest {
   public static final int INPUT_SIZE = 10000;
   @Rule public TemporaryFolder tmp = new TemporaryFolder();
   private WriteWithShardingFactory<Object> factory = new WriteWithShardingFactory<>();
+  @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
 
   @Test
   public void dynamicallyReshardedWrite() throws Exception {
@@ -81,7 +82,6 @@ public class WriteWithShardingFactoryTest {
     String fileName = "resharded_write";
     String outputPath = tmp.getRoot().getAbsolutePath();
     String targetLocation = IOChannelUtils.resolve(outputPath, fileName);
-    TestPipeline p = TestPipeline.create();
     // TextIO is implemented in terms of the Write PTransform. When sharding is not specified,
     // resharding should be automatically applied
     p.apply(Create.of(strs)).apply(TextIO.Write.to(targetLocation));
@@ -134,7 +134,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnWithOneElement() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 0);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 
@@ -149,7 +149,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnWithTwoElements() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 0);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 
@@ -167,7 +167,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnFewElementsThreeShards() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 0);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 
@@ -191,7 +191,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnManyElements() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 0);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 
@@ -214,7 +214,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnFewElementsExtraShards() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 10);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 
@@ -238,7 +238,7 @@ public class WriteWithShardingFactoryTest {
   public void keyBasedOnCountFnManyElementsExtraShards() throws Exception {
     PCollectionView<Long> elementCountView =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of());
     KeyBasedOnCountFn<String> fn = new KeyBasedOnCountFn<>(elementCountView, 3);
     DoFnTester<String, KV<Integer, String>> fnTester = DoFnTester.of(fn);
 


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

Posted by ke...@apache.org.
Move responsibility for knowing about keyedness into EvaluationContext

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


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

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

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


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b26ceaa3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
index a308295..5a653b7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -31,7 +31,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Queue;
-import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -70,7 +69,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   private final ExecutorService executorService;
 
   private final DirectGraph graph;
-  private final Set<PValue> keyedPValues;
   private final RootProviderRegistry rootProviderRegistry;
   private final TransformEvaluatorRegistry registry;
   @SuppressWarnings("rawtypes")
@@ -105,7 +103,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   public static ExecutorServiceParallelExecutor create(
       int targetParallelism,
       DirectGraph graph,
-      Set<PValue> keyedPValues,
       RootProviderRegistry rootProviderRegistry,
       TransformEvaluatorRegistry registry,
       @SuppressWarnings("rawtypes")
@@ -115,7 +112,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
     return new ExecutorServiceParallelExecutor(
         targetParallelism,
         graph,
-        keyedPValues,
         rootProviderRegistry,
         registry,
         transformEnforcements,
@@ -125,7 +121,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   private ExecutorServiceParallelExecutor(
       int targetParallelism,
       DirectGraph graph,
-      Set<PValue> keyedPValues,
       RootProviderRegistry rootProviderRegistry,
       TransformEvaluatorRegistry registry,
       @SuppressWarnings("rawtypes")
@@ -134,7 +129,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
     this.targetParallelism = targetParallelism;
     this.executorService = Executors.newFixedThreadPool(targetParallelism);
     this.graph = graph;
-    this.keyedPValues = keyedPValues;
     this.rootProviderRegistry = rootProviderRegistry;
     this.registry = registry;
     this.transformEnforcements = transformEnforcements;
@@ -229,7 +223,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   }
 
   private boolean isKeyed(PValue pvalue) {
-    return keyedPValues.contains(pvalue);
+    return evaluationContext.isKeyed(pvalue);
   }
 
   private void scheduleConsumers(ExecutorUpdate update) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b26ceaa3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
index bf36204..15340da 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -105,11 +105,18 @@ public class EvaluationContextTest {
     view = created.apply(View.<Integer>asIterable());
     unbounded = p.apply(CountingInput.unbounded());
 
+    KeyedPValueTrackingVisitor keyedPValueTrackingVisitor = KeyedPValueTrackingVisitor.create();
+    p.traverseTopologically(keyedPValueTrackingVisitor);
+
     BundleFactory bundleFactory = ImmutableListBundleFactory.create();
     graph = DirectGraphs.getGraph(p);
     context =
         EvaluationContext.create(
-            runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph);
+            runner.getPipelineOptions(),
+            NanosOffsetClock.create(),
+            bundleFactory,
+            graph,
+            keyedPValueTrackingVisitor.getKeyedPValues());
 
     createdProducer = graph.getProducer(created);
     downstreamProducer = graph.getProducer(downstream);


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

Posted by ke...@apache.org.
Remove deprecated AggregatorFactory from SDK


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

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

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


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


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

Posted by ke...@apache.org.
Provide a better error message for non-existing gcpTempLocation

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

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


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

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

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index 66632ad..5ddc5d0 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -17,9 +17,6 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Strings.isNullOrEmpty;
-
 import java.io.IOException;
 import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -107,17 +104,21 @@ public interface DataflowPipelineOptions
     @Override
     public String create(PipelineOptions options) {
       GcsOptions gcsOptions = options.as(GcsOptions.class);
-      String gcpTempLocation = gcsOptions.getGcpTempLocation();
-      checkArgument(!isNullOrEmpty(gcpTempLocation),
-          "Error constructing default value for stagingLocation: gcpTempLocation is missing."
-          + "Either stagingLocation must be set explicitly or a valid value must be provided"
-          + "for gcpTempLocation.");
+      String gcpTempLocation;
+      try {
+        gcpTempLocation = gcsOptions.getGcpTempLocation();
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+        "Error constructing default value for stagingLocation: failed to retrieve gcpTempLocation. "
+            + "Either stagingLocation must be set explicitly or a valid value must be provided"
+            + "for gcpTempLocation.", e);
+      }
       try {
         gcsOptions.getPathValidator().validateOutputFilePrefixSupported(gcpTempLocation);
       } catch (Exception e) {
         throw new IllegalArgumentException(String.format(
             "Error constructing default value for stagingLocation: gcpTempLocation is not"
-            + " a valid GCS path, %s. ", gcpTempLocation));
+            + " a valid GCS path, %s. ", gcpTempLocation), e);
       }
       try {
         return IOChannelUtils.resolve(gcpTempLocation, "staging");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 4159b61..b29c4cd 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.dataflow;
 
 import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.both;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
@@ -280,6 +281,26 @@ public class DataflowRunnerTest {
   }
 
   @Test
+  public void testPathExistsValidation() {
+    String[] args = new String[] {
+        "--runner=DataflowRunner",
+        "--tempLocation=gs://does/not/exist",
+        "--project=test-project",
+        "--credentialFactoryClass=" + NoopCredentialFactory.class.getCanonicalName(),
+    };
+
+    try {
+      TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create());
+      fail();
+    } catch (RuntimeException e) {
+      assertThat(
+          Throwables.getStackTraceAsString(e),
+          both(containsString("gs://does/not/exist"))
+              .and(containsString("does not exist or is not writeable")));
+    }
+  }
+
+  @Test
   public void testPathValidatorOverride() {
     String[] args = new String[] {
         "--runner=DataflowRunner",
@@ -544,7 +565,7 @@ public class DataflowRunnerTest {
     thrown.expectCause(Matchers.allOf(
         instanceOf(IllegalArgumentException.class),
         ThrowableMessageMatcher.hasMessage(
-            containsString("expected a valid 'gs://' path but was given"))));
+            containsString("Expected a valid 'gs://' path but was given"))));
     p.run();
 
     ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
@@ -559,7 +580,7 @@ public class DataflowRunnerTest {
     PCollection<String> pc = p.apply("ReadMyGcsFile", TextIO.Read.from("gs://bucket/object"));
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+    thrown.expectMessage(containsString("Expected a valid 'gs://' path but was given"));
     pc.apply("WriteMyNonGcsFile", TextIO.Write.to("/tmp/file"));
   }
 
@@ -594,7 +615,7 @@ public class DataflowRunnerTest {
     options.setGcpTempLocation("file://temp/location");
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+    thrown.expectMessage(containsString("Expected a valid 'gs://' path but was given"));
     DataflowRunner.fromOptions(options);
 
     ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
@@ -609,7 +630,8 @@ public class DataflowRunnerTest {
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(
-        "DataflowRunner requires gcpTempLocation, and it is missing in PipelineOptions.");
+        "DataflowRunner requires gcpTempLocation, "
+            + "but failed to retrieve a value from PipelineOptions");
     DataflowRunner.fromOptions(options);
   }
 
@@ -621,14 +643,14 @@ public class DataflowRunnerTest {
       DataflowRunner.fromOptions(options);
       fail("fromOptions should have failed");
     } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+      assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given"));
     }
     options.setStagingLocation("my/staging/location");
     try {
       DataflowRunner.fromOptions(options);
       fail("fromOptions should have failed");
     } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+      assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given"));
     }
   }
 
@@ -640,14 +662,14 @@ public class DataflowRunnerTest {
       DataflowRunner.fromOptions(options);
       fail("fromOptions should have failed");
     } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+      assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given"));
     }
     options.setSaveProfilesToGcs("my/staging/location");
     try {
       DataflowRunner.fromOptions(options);
       fail("fromOptions should have failed");
     } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+      assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given"));
     }
   }
 
@@ -795,8 +817,8 @@ public class DataflowRunnerTest {
     options.setProject("foo-project");
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "DataflowRunner requires gcpTempLocation, and it is missing in PipelineOptions.");
+    thrown.expectMessage("DataflowRunner requires gcpTempLocation, "
+        + "but failed to retrieve a value from PipelineOption");
     DataflowRunner.fromOptions(options);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
index 52082e0..9dacfb2 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
@@ -17,9 +17,10 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
-import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
 
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.ResetDateTimeProvider;
@@ -127,9 +128,10 @@ public class DataflowPipelineOptionsTest {
   public void testStagingLocation() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
     IOChannelUtils.registerIOFactoriesAllowOverride(options);
-    options.setTempLocation("file://temp_location");
+    options.setPathValidatorClass(NoopPathValidator.class);
+    options.setTempLocation("gs://temp_location");
     options.setStagingLocation("gs://staging_location");
-    assertTrue(isNullOrEmpty(options.getGcpTempLocation()));
+    assertEquals("gs://temp_location", options.getGcpTempLocation());
     assertEquals("gs://staging_location", options.getStagingLocation());
   }
 
@@ -158,8 +160,10 @@ public class DataflowPipelineOptionsTest {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
     options.setTempLocation("file://temp_location");
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Error constructing default value for stagingLocation: gcpTempLocation is missing.");
+    thrown.expectMessage("Error constructing default value for stagingLocation: "
+        + "failed to retrieve gcpTempLocation.");
+    thrown.expectCause(hasMessage(containsString(
+        "Error constructing default value for gcpTempLocation")));
     options.getStagingLocation();
   }
 
@@ -171,6 +175,8 @@ public class DataflowPipelineOptionsTest {
     thrown.expectMessage(
         "Error constructing default value for stagingLocation: gcpTempLocation is not"
         + " a valid GCS path");
+    thrown.expectCause(
+        hasMessage(containsString("Expected a valid 'gs://' path")));
     options.getStagingLocation();
   }
 
@@ -178,8 +184,8 @@ public class DataflowPipelineOptionsTest {
   public void testDefaultStagingLocationUnset() {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(
-        "Error constructing default value for stagingLocation: gcpTempLocation is missing.");
+    thrown.expectMessage("Error constructing default value for stagingLocation: "
+        + "failed to retrieve gcpTempLocation.");
     options.getStagingLocation();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
index ffdab98..042f4b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.options;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.google.auth.Credentials;
 import com.google.common.annotations.VisibleForTesting;
@@ -195,14 +197,15 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {
     @Nullable
     public String create(PipelineOptions options) {
       String tempLocation = options.getTempLocation();
-      if (!Strings.isNullOrEmpty(tempLocation)) {
-        try {
-          PathValidator validator = options.as(GcsOptions.class).getPathValidator();
-          validator.validateOutputFilePrefixSupported(tempLocation);
-        } catch (Exception e) {
-          // Ignore the temp location because it is not a valid 'gs://' path.
-          return null;
-        }
+      checkArgument(!Strings.isNullOrEmpty(options.getTempLocation()),
+          "Error constructing default value for gcpTempLocation: tempLocation is not set");
+      try {
+        PathValidator validator = options.as(GcsOptions.class).getPathValidator();
+        validator.validateOutputFilePrefixSupported(tempLocation);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(String.format(
+            "Error constructing default value for gcpTempLocation: tempLocation is not"
+                + " a valid GCS path, %s. ", tempLocation), e);
       }
       return tempLocation;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
index c8da4d8..a5b951d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java
@@ -89,8 +89,7 @@ public class GcsPathValidator implements PathValidator {
       return GcsPath.fromUri(path);
     } catch (IllegalArgumentException e) {
       throw new IllegalArgumentException(String.format(
-          "%s expected a valid 'gs://' path but was given '%s'",
-          gcpOptions.getRunner().getSimpleName(), path), e);
+          "Expected a valid 'gs://' path but was given '%s'", path), e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
index 34077a2..7854d67 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java
@@ -17,10 +17,10 @@
  */
 package org.apache.beam.sdk.options;
 
-import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
@@ -35,6 +35,7 @@ import org.apache.beam.sdk.testing.RestoreSystemProperties;
 import org.apache.beam.sdk.util.NoopPathValidator;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
@@ -45,6 +46,7 @@ import org.junit.runners.JUnit4;
 public class GcpOptionsTest {
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
+  @Rule public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void testGetProjectFromCloudSdkConfigEnv() throws Exception {
@@ -107,7 +109,10 @@ public class GcpOptionsTest {
   @Test
   public void testEmptyGcpTempLocation() throws Exception {
     GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    assertTrue(isNullOrEmpty(options.getGcpTempLocation()));
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Error constructing default value for gcpTempLocation: tempLocation is not set");
+    options.getGcpTempLocation();
   }
 
   @Test
@@ -123,7 +128,26 @@ public class GcpOptionsTest {
   public void testDefaultGcpTempLocationInvalid() throws Exception {
     GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
     options.setTempLocation("file://");
-    assertTrue(isNullOrEmpty(options.getGcpTempLocation()));
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Error constructing default value for gcpTempLocation: tempLocation is not"
+            + " a valid GCS path");
+    options.getGcpTempLocation();
+  }
+
+  @Test
+  public void testDefaultGcpTempLocationDoesNotExist() {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    String tempLocation = "gs://does/not/exist";
+    options.setTempLocation(tempLocation);
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Error constructing default value for gcpTempLocation: tempLocation is not"
+            + " a valid GCS path");
+    thrown.expectCause(
+        hasMessage(containsString("Output path does not exist or is not writeable")));
+
+    options.getGcpTempLocation();
   }
 
   private static void makePropertiesFileWithProject(File path, String projectId)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef1a8583/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
index 16f01a0..286490d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java
@@ -21,11 +21,8 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.when;
 
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.junit.Before;
 import org.junit.Rule;
@@ -44,20 +41,12 @@ public class GcsPathValidatorTest {
   @Mock private GcsUtil mockGcsUtil;
   private GcsPathValidator validator;
 
-  private class FakeRunner extends PipelineRunner<PipelineResult> {
-    @Override
-    public PipelineResult run(Pipeline pipeline) {
-      throw new UnsupportedOperationException();
-    }
-  }
-
   @Before
   public void setUp() throws Exception {
     MockitoAnnotations.initMocks(this);
     when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(true);
     when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
     GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class);
-    options.setRunner(FakeRunner.class);
     options.setGcpCredential(new TestCredential());
     options.setGcsUtil(mockGcsUtil);
     validator = GcsPathValidator.fromOptions(options);
@@ -72,7 +61,7 @@ public class GcsPathValidatorTest {
   public void testInvalidFilePattern() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "FakeRunner expected a valid 'gs://' path but was given '/local/path'");
+        "Expected a valid 'gs://' path but was given '/local/path'");
     validator.validateInputFilePatternSupported("/local/path");
   }
 
@@ -94,7 +83,7 @@ public class GcsPathValidatorTest {
   public void testInvalidOutputPrefix() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "FakeRunner expected a valid 'gs://' path but was given '/local/path'");
+        "Expected a valid 'gs://' path but was given '/local/path'");
     validator.validateOutputFilePrefixSupported("/local/path");
   }
 }


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

Posted by ke...@apache.org.
Add static Window.withOutputTimeFn to match build method


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

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

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


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


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

Posted by ke...@apache.org.
This closes #1676: Merge master (57d9bbd) into python-sdk

No fixups. Commit from master:

  Hold output watermark according to pending timers
  Allow setting timer by ID in DirectTimerInternals
  Add UsesTestStream for use with JUnit @Category
  Add static Window.withOutputTimeFn to match build method
  Use informative Instant formatter in WatermarkHold
  Add informative Instant formatter to BoundedWindow
  Actually propagate and commit state in direct runner
  Fix windowing in direct runner Stateful ParDo
  Support set and delete of timer by ID in InMemoryTimerInternals
  Move ExecutionContext and related classes to runners-core
  Update Dataflow worker to beam-master-20161221
  Require TimeDomain to delete a timer
  Provide local tags in PInput, POutput expansions
  Remove deprecated InMemoryTimerInternals from SDK
  Remove deprecated AggregatorFactory from SDK
  Update Dataflow worker to beam-master-20161220
  Fixup usage of canonical name with name since canonical name != name for inner classes.
  Provide a better error message for non-existing gcpTempLocation
  Restore SDK's InMemoryTimerInternals, deprecated
  Move InMemoryTimerInternals to runners-core
  Port direct runner StatefulParDo to KeyedWorkItem
  Propagate key through ParDo if DoFn is key-preserving
  Move responsibility for knowing about keyedness into EvaluationContext
  Add some key-preserving to KeyedPValueTrackingVisitor
  Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.
  Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-kafka module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as a JUnit rule.
  Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. Plus, fixed some checkstyle errors from previous modules' migration.
  Migrated the beam-runners-direct-java module to TestPipeline as a JUnit rule.
  Migrated the beam-runners-core module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java8 module to TestPipeline as a JUnit rule.
  Migrated the beam-examples-java module to TestPipeline as a JUnit rule.
  Disable automatic archiving of Maven builds
  [BEAM-59] initial interfaces and classes of Beam FileSystem.
  Change counter name in TestDataflowRunner
  More escaping in Jenkins timestamp spec
  Add RunnableOnService test for Metrics
  Fix seed job fetch spec
  Show timestamps on log lines in Jenkins
  [BEAM-1165] Fix unexpected file creation when checking dependencies
  [BEAM-1178] Make naming of logger objects consistent
  [BEAM-716] Fix javadoc on with* methods [BEAM-959] Improve check preconditions in JmsIO
  [BEAM-716] Use AutoValue in JmsIO
  Fix grammar error (repeated for)
  Empty TestPipeline need not be run
  [BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage
  Change counter name in TestDataflowRunner
  BigQueryIO: fix streaming write, typo in API
  [BEAM-853] Force streaming execution on batch pipelines for testing. Expose the adapted source.
  Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest
  Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle
  Add timer support to DoFnRunner(s)
  Make TimerSpec and StateSpec fields accessible
  View.asMap: minor javadoc fixes
  Revert "Move InMemoryTimerInternals to runners-core"
  Revert "Moves DoFnAdapters to runners-core"
  Revert "Removes ArgumentProvider.windowingInternals"
  Revert "Removes code for wrapping DoFn as an OldDoFn"
  checkstyle: missed newline in DistributionCell
  Make {Metric,Counter,Distribution}Cell public
  Add PTransformOverrideFactory to the Core SDK
  Move ActiveWindowSet and implementations to runners-core
  Update Dataflow worker to beam-master-20161216
  [BEAM-1108] Remove outdated language about experimental autoscaling
  [BEAM-450] Shade modules to separate paths
  [BEAM-362] Port runners to runners-core AggregatoryFactory
  Move InMemoryTimerInternals to runners-core
  Delete deprecated TimerCallback
  Remove deprecated methods of InMemoryTimerInternals
  Don't incorrectly log error in MetricsEnvironment
  Renames ParDo.getNewFn to getFn
  Moves DoFnAdapters to runners-core
  Removes unused code from NoOpOldDoFn
  Removes ArgumentProvider.windowingInternals
  Removes code for wrapping DoFn as an OldDoFn
  Removes OldDoFn from ParDo
  Pushes uses of OldDoFn deeper inside Flink runner
  Remove ParDo.of(OldDoFn) from Apex runner
  Converts all easy OldDoFns to DoFn
  [BEAM-1022] Add testing coverage for BigQuery streaming writes
  Fix mvn command args in Apex postcommit Jenkins job
  [BEAM-932] Enable findbugs validation (and fix existing issues)
  Fail to split in FileBasedSource if filePattern expands to empty.
  [BEAM-1154] Get side input from proper window in ReduceFn
  [BEAM-1153] GcsUtil: use non-batch API for single file size requests.
  Fix NPE in StatefulParDoEvaluatorFactoryTest mocking
  [BEAM-1033] Retry Bigquery Verifier when Query Fails
  Implement GetDefaultOutputCoder in DirectGroupByKey
  SimpleDoFnRunner observes window if SideInputReader is nonempty
  Better comments and cleanup
  Allow empty string value for ValueProvider types.
  starter: fix typo in pom.xml
  Revert "Allow stateful DoFn in DataflowRunner"
  Re-exclude UsesStatefulParDo tests for Dataflow
  ...


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

Branch: refs/heads/python-sdk
Commit: 3454d691f033bd0403058ff7cf0c140a7cf93f71
Parents: 3b4fd5c 7774297
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Dec 21 14:47:00 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Wed Dec 21 14:48:32 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 .jenkins/common_job_properties.groovy           | 173 ++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 ++
 ...ostCommit_Java_RunnableOnService_Apex.groovy |  41 ++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 ++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 ++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 ++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 ++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 ++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 ++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 ++
 .jenkins/job_seed.groovy                        |  47 ++
 .travis.yml                                     |   2 +-
 .travis/README.md                               |   2 +-
 README.md                                       |   2 +-
 examples/java/pom.xml                           |  20 +-
 .../apache/beam/examples/WindowedWordCount.java | 177 +++---
 .../org/apache/beam/examples/WordCount.java     |   2 +-
 .../examples/common/WriteWindowedFilesDoFn.java |  77 +++
 .../beam/examples/complete/AutoComplete.java    |   6 +-
 .../org/apache/beam/examples/complete/README.md |  14 +-
 .../apache/beam/examples/complete/TfIdf.java    |   6 +-
 .../examples/complete/TopWikipediaSessions.java |  15 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   4 +-
 .../beam/examples/complete/TrafficRoutes.java   |   4 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/DeDupExample.java    |  96 ---
 .../beam/examples/cookbook/DistinctExample.java |  96 +++
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../org/apache/beam/examples/cookbook/README.md |  16 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/WindowedWordCountIT.java      | 182 +++++-
 .../org/apache/beam/examples/WordCountIT.java   |   2 +-
 .../org/apache/beam/examples/WordCountTest.java |   7 +-
 .../examples/complete/AutoCompleteTest.java     |  13 +-
 .../beam/examples/complete/TfIdfTest.java       |   6 +-
 .../complete/TopWikipediaSessionsTest.java      |   7 +-
 .../examples/cookbook/DistinctExampleTest.java  |   9 +-
 .../examples/cookbook/JoinExamplesTest.java     |   6 +-
 .../examples/cookbook/TriggerExampleTest.java   |   6 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |   2 +-
 .../examples/complete/game/LeaderBoard.java     |   4 +-
 .../beam/examples/complete/game/UserScore.java  |   2 +-
 .../complete/game/utils/WriteToBigQuery.java    |   2 +-
 .../game/utils/WriteWindowedToBigQuery.java     |   7 +-
 .../examples/MinimalWordCountJava8Test.java     |   6 +-
 .../examples/complete/game/GameStatsTest.java   |   7 +-
 .../complete/game/HourlyTeamScoreTest.java      |   5 +-
 .../examples/complete/game/LeaderBoardTest.java |  11 +-
 .../examples/complete/game/UserScoreTest.java   |  10 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         |  15 +-
 runners/apex/pom.xml                            |  69 ++-
 .../apache/beam/runners/apex/ApexRunner.java    | 111 ++--
 .../beam/runners/apex/ApexRunnerResult.java     |  50 +-
 .../beam/runners/apex/ApexYarnLauncher.java     | 395 ++++++++++++
 .../translation/ApexPipelineTranslator.java     |  14 +-
 .../translation/ParDoBoundMultiTranslator.java  |  22 +-
 .../apex/translation/ParDoBoundTranslator.java  |  22 +-
 .../apex/translation/TranslationContext.java    |  10 +-
 .../apex/translation/WindowBoundTranslator.java |  78 +++
 .../operators/ApexGroupByKeyOperator.java       |  26 +-
 .../operators/ApexParDoOperator.java            |  29 +-
 .../apex/translation/utils/NoOpStepContext.java |   3 +-
 .../beam/runners/apex/ApexYarnLauncherTest.java | 138 +++++
 .../FlattenPCollectionTranslatorTest.java       |  15 +-
 .../translation/GroupByKeyTranslatorTest.java   |  21 +-
 .../translation/ParDoBoundTranslatorTest.java   |  38 +-
 .../translation/ReadUnboundTranslatorTest.java  |  15 +-
 runners/core-java/pom.xml                       |   6 +-
 .../beam/runners/core/ActiveWindowSet.java      | 173 ++++++
 .../beam/runners/core/AggregatorFactory.java    |   1 -
 .../apache/beam/runners/core/AssignWindows.java |  46 --
 .../beam/runners/core/BaseExecutionContext.java | 176 ++++++
 .../apache/beam/runners/core/DoFnRunner.java    |  24 +-
 .../apache/beam/runners/core/DoFnRunners.java   |   4 +-
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../beam/runners/core/ExecutionContext.java     | 102 +++
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 --
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |  12 +-
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  55 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  30 +-
 .../runners/core/InMemoryTimerInternals.java    | 302 +++++++++
 .../apache/beam/runners/core/KeyedWorkItem.java |  44 ++
 .../beam/runners/core/KeyedWorkItemCoder.java   | 130 ++++
 .../beam/runners/core/KeyedWorkItems.java       | 122 ++++
 .../core/LateDataDroppingDoFnRunner.java        |  34 +-
 .../runners/core/MergingActiveWindowSet.java    | 428 +++++++++++++
 .../runners/core/NonMergingActiveWindowSet.java |  91 +++
 .../beam/runners/core/PaneInfoTracker.java      |   4 +
 .../runners/core/PerKeyCombineFnRunner.java     | 149 +++++
 .../runners/core/PerKeyCombineFnRunners.java    | 262 ++++++++
 .../core/PushbackSideInputDoFnRunner.java       |  11 +
 .../runners/core/ReduceFnContextFactory.java    |  26 +-
 .../beam/runners/core/ReduceFnRunner.java       | 496 +++++++++------
 .../beam/runners/core/SimpleDoFnRunner.java     | 328 +++++++++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  27 +-
 .../beam/runners/core/SplittableParDo.java      | 392 +++++++++---
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../apache/beam/runners/core/WatermarkHold.java |   9 +-
 .../TriggerStateMachineContextFactory.java      |   2 +-
 .../triggers/TriggerStateMachineRunner.java     |  14 +-
 .../core/GroupAlsoByWindowsProperties.java      | 590 ++++++++++--------
 .../core/InMemoryTimerInternalsTest.java        | 197 ++++++
 .../runners/core/KeyedWorkItemCoderTest.java    |  64 ++
 .../core/LateDataDroppingDoFnRunnerTest.java    |   3 +
 .../core/MergingActiveWindowSetTest.java        | 262 ++++++++
 .../core/PushbackSideInputDoFnRunnerTest.java   |  60 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   | 138 +++--
 .../beam/runners/core/ReduceFnTester.java       |  78 ++-
 .../beam/runners/core/SimpleDoFnRunnerTest.java | 301 +++++++++
 .../runners/core/SimpleOldDoFnRunnerTest.java   |   3 +-
 .../beam/runners/core/SplittableParDoTest.java  | 158 ++++-
 .../UnboundedReadFromBoundedSourceTest.java     |   6 +-
 .../beam/runners/core/WindowMatchers.java       | 204 ++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 +++
 .../triggers/TriggerStateMachineTester.java     |  27 +-
 runners/direct-java/pom.xml                     |   3 +-
 .../direct/AbstractModelEnforcement.java        |   2 +-
 .../runners/direct/AggregatorContainer.java     |   4 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   8 +-
 .../beam/runners/direct/CommittedResult.java    |   2 +-
 .../beam/runners/direct/CompletionCallback.java |   2 +-
 .../direct/ConsumerTrackingPipelineVisitor.java | 173 ------
 .../CopyOnAccessInMemoryStateInternals.java     | 467 ++++++++++++++
 .../runners/direct/DirectExecutionContext.java  |   7 +-
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  42 +-
 .../apache/beam/runners/direct/DirectGraph.java |  89 +++
 .../beam/runners/direct/DirectGraphVisitor.java | 142 +++++
 .../beam/runners/direct/DirectGroupByKey.java   |  46 +-
 .../direct/DirectGroupByKeyOverrideFactory.java |   3 +-
 .../beam/runners/direct/DirectRunner.java       |  62 +-
 .../runners/direct/DirectTimerInternals.java    |  14 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../runners/direct/EmptyTransformEvaluator.java |   4 +-
 .../beam/runners/direct/EvaluationContext.java  | 105 ++--
 .../direct/ExecutorServiceParallelExecutor.java |  29 +-
 .../runners/direct/FlattenEvaluatorFactory.java |  10 +-
 .../runners/direct/ForwardingPTransform.java    |   6 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |  13 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   6 +-
 .../ImmutabilityCheckingBundleFactory.java      |  21 +-
 .../direct/ImmutabilityEnforcementFactory.java  |   2 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  64 +-
 .../beam/runners/direct/ModelEnforcement.java   |   2 +-
 .../direct/PTransformOverrideFactory.java       |  35 --
 .../beam/runners/direct/ParDoEvaluator.java     |  45 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  97 ++-
 .../direct/ParDoMultiOverrideFactory.java       | 193 +++++-
 .../ParDoSingleViaMultiOverrideFactory.java     |  21 +-
 .../direct/PassthroughTransformEvaluator.java   |   4 +-
 ...littableProcessElementsEvaluatorFactory.java | 145 +++++
 .../direct/StatefulParDoEvaluatorFactory.java   | 257 ++++++++
 .../runners/direct/StepTransformResult.java     |  39 +-
 .../direct/TestStreamEvaluatorFactory.java      |   9 +-
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |   7 +
 .../beam/runners/direct/TransformExecutor.java  |   4 +-
 .../beam/runners/direct/TransformResult.java    |  17 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   3 +-
 .../runners/direct/ViewEvaluatorFactory.java    |  10 +-
 .../direct/WatermarkCallbackExecutor.java       |  34 +
 .../beam/runners/direct/WatermarkManager.java   | 139 +++--
 .../runners/direct/WindowEvaluatorFactory.java  |   6 +-
 .../direct/WriteWithShardingFactory.java        |  10 +-
 .../runners/direct/AggregatorContainerTest.java |   2 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  47 +-
 .../direct/CloningBundleFactoryTest.java        |   8 +-
 .../runners/direct/CommittedResultTest.java     |   8 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 287 ---------
 .../CopyOnAccessInMemoryStateInternalsTest.java | 565 +++++++++++++++++
 .../runners/direct/DirectGraphVisitorTest.java  | 265 ++++++++
 .../beam/runners/direct/DirectGraphs.java       |  35 ++
 .../beam/runners/direct/DirectRunnerTest.java   |  36 --
 ...leManagerRemovingTransformEvaluatorTest.java |   4 +-
 .../runners/direct/EvaluationContextTest.java   | 132 ++--
 .../direct/FlattenEvaluatorFactoryTest.java     |  27 +-
 .../direct/ForwardingPTransformTest.java        |   4 +-
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |  11 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |  12 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |  10 +-
 .../ImmutabilityEnforcementFactoryTest.java     |  11 +-
 .../direct/ImmutableListBundleFactoryTest.java  |  14 +-
 .../direct/KeyedPValueTrackingVisitorTest.java  | 137 ++--
 .../beam/runners/direct/ParDoEvaluatorTest.java |  11 +-
 .../runners/direct/SideInputContainerTest.java  |   5 +-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 -------
 .../StatefulParDoEvaluatorFactoryTest.java      | 323 ++++++++++
 .../runners/direct/StepTransformResultTest.java |  32 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  29 +-
 .../runners/direct/TransformExecutorTest.java   | 228 +++----
 .../UnboundedReadEvaluatorFactoryTest.java      |  53 +-
 .../direct/ViewEvaluatorFactoryTest.java        |   9 +-
 .../direct/WatermarkCallbackExecutorTest.java   |  11 +-
 .../runners/direct/WatermarkManagerTest.java    | 260 ++++----
 .../direct/WindowEvaluatorFactoryTest.java      |  21 +-
 .../direct/WriteWithShardingFactoryTest.java    |  18 +-
 runners/flink/examples/pom.xml                  |  14 +-
 .../beam/runners/flink/examples/TFIDF.java      |   6 +-
 .../beam/runners/flink/examples/WordCount.java  |   2 +-
 .../flink/examples/streaming/AutoComplete.java  |   6 +-
 runners/flink/pom.xml                           |   2 +-
 runners/flink/runner/pom.xml                    |  15 +-
 .../FlinkPipelineExecutionEnvironment.java      |  12 +
 .../runners/flink/FlinkPipelineOptions.java     |   5 +
 .../apache/beam/runners/flink/FlinkRunner.java  |  36 +-
 .../FlinkBatchPipelineTranslator.java           |  18 +-
 .../FlinkBatchTransformTranslators.java         |  55 +-
 .../FlinkStreamingPipelineTranslator.java       |  23 +-
 .../FlinkStreamingTransformTranslators.java     | 140 ++---
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../functions/FlinkDoFnFunction.java            |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   4 +-
 .../FlinkMergingPartialReduceFunction.java      |   4 +-
 .../functions/FlinkMergingReduceFunction.java   |   4 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |  10 +-
 .../functions/FlinkPartialReduceFunction.java   |   4 +-
 .../functions/FlinkProcessContextBase.java      |  29 +-
 .../functions/FlinkReduceFunction.java          |   4 +-
 .../wrappers/streaming/DoFnOperator.java        |  48 +-
 .../streaming/SingletonKeyedWorkItem.java       |   2 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |  25 +-
 .../wrappers/streaming/WorkItemKeySelector.java |   2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   6 +-
 .../flink/streaming/DoFnOperatorTest.java       |  13 +-
 .../flink/streaming/GroupByNullKeyTest.java     |  18 +-
 .../streaming/TopWikipediaSessionsITCase.java   |  10 +-
 runners/google-cloud-dataflow-java/pom.xml      |  14 +-
 .../beam/runners/dataflow/DataflowClient.java   | 140 +++++
 .../runners/dataflow/DataflowPipelineJob.java   |  31 +-
 .../dataflow/DataflowPipelineTranslator.java    |  36 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 210 ++++---
 .../runners/dataflow/DataflowRunnerInfo.java    |  92 +++
 .../dataflow/internal/AssignWindows.java        |  10 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   2 +-
 .../options/DataflowPipelineOptions.java        |  19 +-
 .../DataflowPipelineWorkerPoolOptions.java      |  57 +-
 .../options/DataflowProfilingOptions.java       |   8 +-
 .../dataflow/testing/TestDataflowRunner.java    |  35 +-
 .../dataflow/util/DataflowTemplateJob.java      |   2 +-
 .../runners/dataflow/util/MonitoringUtil.java   |  22 +-
 .../beam/runners/dataflow/dataflow.properties   |  23 +
 .../dataflow/DataflowPipelineJobTest.java       |  49 +-
 .../DataflowPipelineTranslatorTest.java         |  63 +-
 .../dataflow/DataflowRunnerInfoTest.java        |  51 ++
 .../runners/dataflow/DataflowRunnerTest.java    | 333 ++++++----
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 .../options/DataflowPipelineOptionsTest.java    |  20 +-
 .../options/DataflowProfilingOptionsTest.java   |   6 +-
 .../testing/TestDataflowRunnerTest.java         |  52 +-
 .../transforms/DataflowGroupByKeyTest.java      |  18 +-
 .../dataflow/transforms/DataflowViewTest.java   |  20 +-
 .../dataflow/util/MonitoringUtilTest.java       |  21 +-
 runners/pom.xml                                 |   2 +-
 runners/spark/README.md                         |  59 +-
 runners/spark/pom.xml                           |  60 +-
 .../beam/runners/spark/EvaluationResult.java    |  67 --
 .../runners/spark/SparkPipelineOptions.java     |  15 +
 .../beam/runners/spark/SparkPipelineResult.java | 193 ++++++
 .../apache/beam/runners/spark/SparkRunner.java  | 213 +++++--
 .../beam/runners/spark/TestSparkRunner.java     |  91 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   8 +-
 .../spark/aggregators/SparkAggregators.java     | 126 ++++
 .../metrics/WithNamedAggregatorsSupport.java    |   4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |  60 +-
 .../beam/runners/spark/examples/WordCount.java  |  15 +-
 .../apache/beam/runners/spark/io/ConsoleIO.java |   2 +-
 .../beam/runners/spark/io/CreateStream.java     |   2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |  20 +-
 .../beam/runners/spark/io/SourceDStream.java    |  22 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  17 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   4 +-
 .../spark/stateful/StateSpecFunctions.java      |   2 +-
 .../spark/translation/BoundedDataset.java       |   5 +-
 .../beam/runners/spark/translation/Dataset.java |   2 +-
 .../runners/spark/translation/DoFnFunction.java | 110 ++--
 .../spark/translation/EvaluationContext.java    | 137 +---
 .../translation/GroupCombineFunctions.java      |  23 +-
 .../spark/translation/MultiDoFnFunction.java    | 135 ++--
 .../spark/translation/SparkAssignWindowFn.java  |  69 +++
 .../spark/translation/SparkContextFactory.java  |   2 +-
 .../translation/SparkGroupAlsoByWindowFn.java   | 214 +++++++
 .../spark/translation/SparkProcessContext.java  | 390 +++---------
 .../spark/translation/SparkRuntimeContext.java  |  62 +-
 .../translation/StorageLevelPTransform.java     |  43 ++
 .../spark/translation/TransformTranslator.java  |  91 ++-
 .../spark/translation/TranslationUtils.java     |  33 +
 .../spark/translation/WindowingHelpers.java     |   3 +-
 .../SparkRunnerStreamingContextFactory.java     |   2 +-
 .../streaming/StreamingTransformTranslator.java |  47 +-
 .../translation/streaming/UnboundedDataset.java |  13 +-
 .../util/SinglePrimitiveOutputPTransform.java   |   2 +-
 .../beam/runners/spark/ForceStreamingTest.java  | 123 ++++
 .../runners/spark/ProvidedSparkContextTest.java |   6 +-
 .../runners/spark/SparkPipelineStateTest.java   | 217 +++++++
 .../spark/aggregators/ClearAggregatorsRule.java |  37 ++
 .../metrics/sink/ClearAggregatorsRule.java      |  33 -
 .../metrics/sink/NamedAggregatorsTest.java      |   3 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |  57 ++
 .../beam/runners/spark/io/AvroPipelineTest.java |   2 +-
 .../beam/runners/spark/io/NumShardsTest.java    |   2 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |   2 +-
 .../spark/translation/SideEffectsTest.java      |  59 --
 .../spark/translation/StorageLevelTest.java     |  56 ++
 .../streaming/EmptyStreamAssertionTest.java     |   6 +
 .../streaming/FlattenStreamingTest.java         |   2 +
 .../streaming/KafkaStreamingTest.java           |  34 +-
 .../ResumeFromCheckpointStreamingTest.java      |  32 +-
 .../streaming/SimpleStreamingWordCountTest.java |   1 +
 .../streaming/utils/PAssertStreaming.java       |  31 +-
 .../SparkTestPipelineOptionsForStreaming.java   |   6 -
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml |  68 +-
 sdks/java/core/pom.xml                          |  31 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   9 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 151 +----
 .../beam/sdk/annotations/Experimental.java      |   5 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |   2 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   4 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |  16 +-
 .../org/apache/beam/sdk/io/CountingInput.java   |  52 +-
 .../org/apache/beam/sdk/io/CountingSource.java  |  17 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  95 ++-
 .../org/apache/beam/sdk/io/FileBasedSource.java |  12 +-
 .../java/org/apache/beam/sdk/io/FileSystem.java |  29 +
 .../apache/beam/sdk/io/FileSystemRegistrar.java |  49 ++
 .../org/apache/beam/sdk/io/FileSystems.java     | 155 +++++
 .../org/apache/beam/sdk/io/LocalFileSystem.java |  27 +
 .../beam/sdk/io/LocalFileSystemRegistrar.java   |  41 ++
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 180 +++++-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  27 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  48 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  32 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   2 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   4 +-
 .../apache/beam/sdk/io/range/ByteKeyRange.java  |   4 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   8 +-
 .../apache/beam/sdk/metrics/CounterCell.java    |  10 +-
 .../beam/sdk/metrics/DistributionCell.java      |  14 +-
 .../org/apache/beam/sdk/metrics/MetricCell.java |   2 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |   3 +-
 .../org/apache/beam/sdk/metrics/Metrics.java    |   5 +-
 .../beam/sdk/metrics/MetricsEnvironment.java    |   8 +-
 .../org/apache/beam/sdk/options/GcpOptions.java |  19 +-
 .../beam/sdk/options/PipelineOptions.java       |   7 +
 .../sdk/options/PipelineOptionsFactory.java     | 111 +++-
 .../sdk/options/ProxyInvocationHandler.java     |  26 +
 .../beam/sdk/options/StreamingOptions.java      |   2 +-
 .../apache/beam/sdk/options/ValueProvider.java  |  29 +-
 .../sdk/runners/PTransformOverrideFactory.java  |  41 ++
 .../apache/beam/sdk/runners/PipelineRunner.java |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 345 +++++++++--
 .../beam/sdk/runners/TransformTreeNode.java     | 257 --------
 .../beam/sdk/testing/BigqueryMatcher.java       |  48 +-
 .../beam/sdk/testing/FileChecksumMatcher.java   | 124 ++--
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 +++
 .../org/apache/beam/sdk/testing/PAssert.java    |  87 +--
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +-
 .../apache/beam/sdk/testing/TestPipeline.java   | 225 ++++++-
 .../org/apache/beam/sdk/testing/TestStream.java |   2 +-
 .../apache/beam/sdk/testing/UsesMetrics.java    |  24 +
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 +
 .../apache/beam/sdk/testing/UsesTestStream.java |  24 +
 .../beam/sdk/testing/UsesTimersInParDo.java     |  25 +
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 ++++
 .../apache/beam/sdk/transforms/Aggregator.java  |  19 -
 .../sdk/transforms/AggregatorRetriever.java     |   2 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |  75 +--
 .../beam/sdk/transforms/ApproximateUnique.java  |   4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  51 +-
 .../beam/sdk/transforms/CombineWithContext.java |   2 +-
 .../org/apache/beam/sdk/transforms/Count.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Create.java  |   4 +-
 .../apache/beam/sdk/transforms/Distinct.java    |   4 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  38 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |  14 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 387 ++++++------
 .../org/apache/beam/sdk/transforms/Filter.java  |   2 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   2 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |   4 +-
 .../apache/beam/sdk/transforms/GroupByKey.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   2 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |   4 +-
 .../apache/beam/sdk/transforms/MapElements.java |   2 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   8 +-
 .../apache/beam/sdk/transforms/PTransform.java  |   4 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 201 +-----
 .../apache/beam/sdk/transforms/Partition.java   |   2 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  14 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |   2 +-
 .../org/apache/beam/sdk/transforms/Values.java  |   2 +-
 .../org/apache/beam/sdk/transforms/View.java    |  16 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   2 +-
 .../beam/sdk/transforms/WithTimestamps.java     |   2 +-
 .../sdk/transforms/display/DisplayData.java     |  22 +
 .../beam/sdk/transforms/join/CoGroupByKey.java  |   2 +-
 .../transforms/join/KeyedPCollectionTuple.java  |   9 +-
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  39 +-
 .../reflect/ByteBuddyOnTimerInvokerFactory.java |  10 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |  11 +
 .../sdk/transforms/reflect/DoFnInvokers.java    |  16 +-
 .../sdk/transforms/reflect/DoFnSignature.java   |  34 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |  96 ++-
 .../sdk/transforms/windowing/BoundedWindow.java |  31 +
 .../beam/sdk/transforms/windowing/Window.java   |  13 +-
 .../apache/beam/sdk/util/ActiveWindowSet.java   | 173 ------
 .../org/apache/beam/sdk/util/ApiSurface.java    |  52 +-
 .../beam/sdk/util/BaseExecutionContext.java     | 174 ------
 .../apache/beam/sdk/util/ExecutionContext.java  | 100 ---
 .../beam/sdk/util/ExplicitShardedFile.java      | 120 ++++
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 ---
 .../apache/beam/sdk/util/GcsPathValidator.java  |   3 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 107 +++-
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../org/apache/beam/sdk/util/KeyedWorkItem.java |  43 --
 .../beam/sdk/util/KeyedWorkItemCoder.java       | 128 ----
 .../apache/beam/sdk/util/KeyedWorkItems.java    | 121 ----
 .../beam/sdk/util/MergingActiveWindowSet.java   | 428 -------------
 .../sdk/util/NonMergingActiveWindowSet.java     |  91 ---
 .../beam/sdk/util/NumberedShardedFile.java      | 220 +++++++
 .../beam/sdk/util/PerKeyCombineFnRunner.java    | 148 -----
 .../beam/sdk/util/PerKeyCombineFnRunners.java   | 258 --------
 .../org/apache/beam/sdk/util/PropertyNames.java |   2 +
 .../org/apache/beam/sdk/util/ReleaseInfo.java   |   4 -
 .../org/apache/beam/sdk/util/Reshuffle.java     |   2 +-
 .../org/apache/beam/sdk/util/ShardedFile.java   |  42 ++
 .../apache/beam/sdk/util/TimerInternals.java    | 120 ++--
 .../beam/sdk/util/WindowingInternals.java       |  10 -
 .../CopyOnAccessInMemoryStateInternals.java     | 453 --------------
 .../sdk/util/state/InMemoryStateInternals.java  |  33 +-
 .../sdk/util/state/InMemoryTimerInternals.java  | 247 --------
 .../beam/sdk/util/state/StateInternals.java     |   2 +-
 .../beam/sdk/util/state/TimerCallback.java      |  35 --
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 .../java/org/apache/beam/sdk/values/PBegin.java |   4 +-
 .../org/apache/beam/sdk/values/PCollection.java |   4 +-
 .../apache/beam/sdk/values/PCollectionList.java |  65 +-
 .../beam/sdk/values/PCollectionTuple.java       |  30 +-
 .../java/org/apache/beam/sdk/values/PDone.java  |   4 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   4 +-
 .../org/apache/beam/sdk/values/POutput.java     |   4 +-
 .../java/org/apache/beam/sdk/values/PValue.java |  10 +
 .../org/apache/beam/sdk/values/PValueBase.java  |  11 +-
 .../apache/beam/sdk/values/TaggedPValue.java    |  42 ++
 .../org/apache/beam/sdk/values/TypedPValue.java |   2 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |  28 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |  43 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 ------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 ---
 .../apache/beam/sdk/coders/AvroCoderTest.java   |  11 +-
 .../beam/sdk/coders/CoderRegistryTest.java      |  10 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |   7 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  16 +-
 .../io/BoundedReadFromUnboundedSourceTest.java  |   6 +-
 .../beam/sdk/io/CompressedSourceTest.java       |  12 +-
 .../apache/beam/sdk/io/CountingInputTest.java   |  69 ++-
 .../apache/beam/sdk/io/CountingSourceTest.java  |  21 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |  11 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |  30 +-
 .../org/apache/beam/sdk/io/FileSystemsTest.java | 104 ++++
 .../sdk/io/LocalFileSystemRegistrarTest.java    |  44 ++
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  43 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  30 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  26 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  44 +-
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   4 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  10 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  10 +-
 .../apache/beam/sdk/metrics/MetricMatchers.java |  47 +-
 .../apache/beam/sdk/metrics/MetricsTest.java    |  63 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |  32 +-
 .../sdk/options/PipelineOptionsFactoryTest.java | 222 ++++++-
 .../beam/sdk/options/PipelineOptionsTest.java   |  24 +
 .../sdk/options/ProxyInvocationHandlerTest.java |   5 +-
 .../beam/sdk/options/ValueProviderTest.java     |  39 ++
 .../sdk/runners/TransformHierarchyTest.java     | 208 +++++--
 .../beam/sdk/runners/TransformTreeTest.java     |  27 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   |  82 +--
 .../sdk/testing/FileChecksumMatcherTest.java    |  66 +-
 .../beam/sdk/testing/GatherAllPanesTest.java    | 141 +++++
 .../apache/beam/sdk/testing/PAssertTest.java    |  32 +-
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++--
 .../beam/sdk/testing/TestPipelineTest.java      | 186 +++++-
 .../apache/beam/sdk/testing/TestStreamTest.java |  19 +-
 .../testing/ValueInSingleWindowCoderTest.java   |  51 ++
 .../transforms/ApproximateQuantilesTest.java    |  12 +-
 .../sdk/transforms/ApproximateUniqueTest.java   |   6 +-
 .../beam/sdk/transforms/CombineFnsTest.java     |   5 +-
 .../apache/beam/sdk/transforms/CombineTest.java |  25 +-
 .../apache/beam/sdk/transforms/CountTest.java   |  13 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |  27 +-
 .../beam/sdk/transforms/DistinctTest.java       |  12 +-
 .../apache/beam/sdk/transforms/DoFnTest.java    |   4 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |  46 +-
 .../apache/beam/sdk/transforms/FilterTest.java  |  18 +-
 .../sdk/transforms/FlatMapElementsTest.java     |  12 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |  35 +-
 .../beam/sdk/transforms/GroupByKeyTest.java     |  34 +-
 .../apache/beam/sdk/transforms/KeysTest.java    |   9 +-
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   9 +-
 .../apache/beam/sdk/transforms/LatestTest.java  |  12 +-
 .../beam/sdk/transforms/MapElementsTest.java    |  16 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |  72 ---
 .../apache/beam/sdk/transforms/OldDoFnTest.java | 125 ++--
 .../beam/sdk/transforms/PTransformTest.java     |   2 +-
 .../beam/sdk/transforms/ParDoLifecycleTest.java |  45 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 617 ++++++++++++-------
 .../beam/sdk/transforms/PartitionTest.java      |   8 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  25 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |  34 +-
 .../beam/sdk/transforms/SplittableDoFnTest.java | 453 ++++++++++++++
 .../org/apache/beam/sdk/transforms/TopTest.java |  15 +-
 .../apache/beam/sdk/transforms/ValuesTest.java  |   7 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |  88 +--
 .../beam/sdk/transforms/WithKeysTest.java       |   8 +-
 .../beam/sdk/transforms/WithTimestampsTest.java |   9 +-
 .../display/DisplayDataEvaluator.java           |   8 +-
 .../display/DisplayDataEvaluatorTest.java       |   2 +-
 .../display/DisplayDataMatchersTest.java        |   6 +-
 .../sdk/transforms/display/DisplayDataTest.java |  90 ++-
 .../sdk/transforms/join/CoGroupByKeyTest.java   |  11 +-
 .../DoFnSignaturesSplittableDoFnTest.java       |   3 +-
 .../transforms/reflect/DoFnSignaturesTest.java  |  47 ++
 .../sdk/transforms/windowing/WindowTest.java    |  22 +-
 .../sdk/transforms/windowing/WindowingTest.java |  13 +-
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -----
 .../beam/sdk/util/GcsPathValidatorTest.java     |  15 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |  85 ++-
 .../beam/sdk/util/KeyedWorkItemCoderTest.java   |  62 --
 .../sdk/util/MergingActiveWindowSetTest.java    | 262 --------
 .../beam/sdk/util/NumberedShardedFileTest.java  | 181 ++++++
 .../apache/beam/sdk/util/ReleaseInfoTest.java   |  45 ++
 .../org/apache/beam/sdk/util/ReshuffleTest.java |  11 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |   6 +-
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 .../CopyOnAccessInMemoryStateInternalsTest.java | 552 -----------------
 .../util/state/InMemoryTimerInternalsTest.java  | 116 ----
 .../beam/sdk/values/PCollectionListTest.java    | 117 ++++
 .../beam/sdk/values/PCollectionTupleTest.java   |  80 ++-
 .../org/apache/beam/sdk/values/PDoneTest.java   |  13 +-
 .../apache/beam/sdk/values/TypedPValueTest.java |  10 +-
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 .../extensions/joinlibrary/InnerJoinTest.java   |   9 +-
 .../joinlibrary/OuterLeftJoinTest.java          |  10 +-
 .../joinlibrary/OuterRightJoinTest.java         |  10 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/extensions/sorter/README.md           |   2 +-
 sdks/java/extensions/sorter/pom.xml             |  10 +-
 .../sorter/BufferedExternalSorter.java          |  12 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |  21 +-
 .../sdk/extensions/sorter/InMemorySorter.java   |  26 +-
 .../beam/sdk/extensions/sorter/SortValues.java  |   2 +-
 .../sorter/BufferedExternalSorterTest.java      |  74 ++-
 .../extensions/sorter/ExternalSorterTest.java   |  69 ++-
 .../extensions/sorter/InMemorySorterTest.java   |   8 +
 .../sdk/extensions/sorter/SortValuesTest.java   |   7 +-
 sdks/java/io/google-cloud-platform/pom.xml      |   8 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    | 453 ++++++++++----
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   7 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   | 121 +++-
 .../io/gcp/bigquery/BigQueryTableInserter.java  | 217 -------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  28 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   6 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |   4 +-
 .../beam/sdk/io/gcp/storage/GcsFileSystem.java  |  34 +
 .../io/gcp/storage/GcsFileSystemRegistrar.java  |  42 ++
 .../beam/sdk/io/gcp/storage/package-info.java   |  21 +
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  |   2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 568 +++++++++++++----
 .../gcp/bigquery/BigQueryServicesImplTest.java  | 139 ++++-
 .../gcp/bigquery/BigQueryTableInserterTest.java | 245 --------
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  50 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |  13 +-
 .../gcp/storage/GcsFileSystemRegistrarTest.java |  51 ++
 sdks/java/io/hdfs/pom.xml                       |   8 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |  29 +
 .../sdk/io/hdfs/HadoopFileSystemRegistrar.java  |  42 ++
 .../io/hdfs/HadoopFileSystemRegistrarTest.java  |  52 ++
 sdks/java/io/jdbc/pom.xml                       |   2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  56 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  38 +-
 sdks/java/io/jms/pom.xml                        |  22 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  | 342 ++++++----
 .../org/apache/beam/sdk/io/jms/JmsRecord.java   |   8 +-
 .../org/apache/beam/sdk/io/jms/JmsIOTest.java   |   9 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  20 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  25 +-
 sdks/java/io/kinesis/pom.xml                    |   2 +-
 .../sdk/io/kinesis/KinesisMockReadTest.java     |   7 +-
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |   6 +-
 sdks/java/io/mongodb/pom.xml                    |   2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |   4 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |  22 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |  17 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |  15 +-
 sdks/java/io/pom.xml                            |   2 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../beam/sdk/transforms/CombineJava8Test.java   |   8 +-
 .../beam/sdk/transforms/DistinctJava8Test.java  |   5 +-
 .../beam/sdk/transforms/FilterJava8Test.java    |   9 +-
 .../transforms/FlatMapElementsJava8Test.java    |   7 +-
 .../sdk/transforms/MapElementsJava8Test.java    |   9 +-
 .../beam/sdk/transforms/PartitionJava8Test.java |   7 +-
 .../beam/sdk/transforms/WithKeysJava8Test.java  |   6 +-
 .../sdk/transforms/WithTimestampsJava8Test.java |  14 +-
 .../examples-java8/generate-sources.sh          |  82 +++
 .../maven-archetypes/examples-java8/pom.xml     | 182 ++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++
 .../main/resources/archetype-resources/pom.xml  | 295 +++++++++
 .../projects/basic/archetype.properties         |  21 +
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../examples/generate-sources.sh                |  68 ++
 sdks/java/maven-archetypes/examples/pom.xml     |  32 +-
 .../main/resources/archetype-resources/pom.xml  | 168 ++++-
 .../src/main/java/DebuggingWordCount.java       | 164 -----
 .../src/main/java/MinimalWordCount.java         | 118 ----
 .../src/main/java/WindowedWordCount.java        | 229 -------
 .../src/main/java/WordCount.java                | 186 ------
 .../common/ExampleBigQueryTableOptions.java     |  55 --
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 --
 .../java/common/ExamplePubsubTopicOptions.java  |  45 --
 .../src/main/java/common/ExampleUtils.java      | 352 -----------
 .../src/test/java/DebuggingWordCountTest.java   |  52 --
 .../src/test/java/WordCountTest.java            |  85 ---
 sdks/java/maven-archetypes/pom.xml              |  40 +-
 sdks/java/maven-archetypes/starter/pom.xml      |   9 +-
 .../main/resources/archetype-resources/pom.xml  |  20 +-
 .../resources/projects/basic/reference/pom.xml  |  20 +-
 .../update-examples-archetype.sh                |  59 --
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 641 files changed, 23199 insertions(+), 14088 deletions(-)
----------------------------------------------------------------------




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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 24ad18319248a128a1c5db4f2bef8861f7361d9f
Parents: 75a4c91
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 00:01:31 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/24ad1831/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java
index 5a45f73..249cea3 100644
--- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java
+++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java
@@ -19,13 +19,13 @@ package org.apache.beam.sdk.extensions.joinlibrary;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 
@@ -34,15 +34,16 @@ import org.junit.Test;
  */
 public class OuterRightJoinTest {
 
-  Pipeline p;
   List<KV<String, Long>> leftListOfKv;
   List<KV<String, String>> listRightOfKv;
   List<KV<String, KV<Long, String>>> expectedResult;
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Before
   public void setup() {
 
-    p = TestPipeline.create();
     leftListOfKv = new ArrayList<>();
     listRightOfKv = new ArrayList<>();
 
@@ -133,16 +134,19 @@ public class OuterRightJoinTest {
 
   @Test(expected = NullPointerException.class)
   public void testJoinLeftCollectionNull() {
+    p.enableAbandonedNodeEnforcement(false);
     Join.rightOuterJoin(null, p.apply(Create.of(listRightOfKv)), "");
   }
 
   @Test(expected = NullPointerException.class)
   public void testJoinRightCollectionNull() {
+    p.enableAbandonedNodeEnforcement(false);
     Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), null, -1L);
   }
 
   @Test(expected = NullPointerException.class)
   public void testJoinNullValueIsNull() {
+    p.enableAbandonedNodeEnforcement(false);
     Join.rightOuterJoin(
         p.apply("CreateLeft", Create.of(leftListOfKv)),
         p.apply("CreateRight", Create.of(listRightOfKv)),


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
index 257b364..b3b3925 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
@@ -24,13 +24,13 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -41,6 +41,10 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class DistinctTest {
+
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testDistinct() {
@@ -53,8 +57,6 @@ public class DistinctTest {
         "k2",
         "k3");
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input =
         p.apply(Create.of(strings)
             .withCoder(StringUtf8Coder.of()));
@@ -72,8 +74,6 @@ public class DistinctTest {
   public void testDistinctEmpty() {
     List<String> strings = Arrays.asList();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input =
         p.apply(Create.of(strings)
             .withCoder(StringUtf8Coder.of()));
@@ -115,8 +115,6 @@ public class DistinctTest {
         KV.of("k1", "v2"),
         KV.of("k2", "v1"));
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, String>> input = p.apply(Create.of(strings));
 
     PCollection<KV<String, String>> output =

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
index e5f5cb6..19b7c51 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java
@@ -40,6 +40,9 @@ import org.junit.runners.JUnit4;
 /** Tests for {@link DoFn}. */
 @RunWith(JUnit4.class)
 public class DoFnTest implements Serializable {
+
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
   @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
@@ -200,7 +203,6 @@ public class DoFnTest implements Serializable {
    * Initialize a test pipeline with the specified {@link DoFn}.
    */
   private <InputT, OutputT> TestPipeline createTestPipeline(DoFn<InputT, OutputT> fn) {
-    TestPipeline pipeline = TestPipeline.create();
     pipeline.apply(Create.of((InputT) null))
      .apply(ParDo.of(fn));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
index 2dafa27..3859c9f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
@@ -52,6 +52,8 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class DoFnTesterTest {
+
+  @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   @Test
@@ -324,7 +326,7 @@ public class DoFnTesterTest {
   public void fnWithSideInputDefault() throws Exception {
     final PCollectionView<Integer> value =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of());
 
     try (DoFnTester<Integer, Integer> tester = DoFnTester.of(new SideInputDoFn(value))) {
       tester.processElement(1);
@@ -339,7 +341,7 @@ public class DoFnTesterTest {
   public void fnWithSideInputExplicit() throws Exception {
     final PCollectionView<Integer> value =
         PCollectionViews.singletonView(
-            TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of());
+            p, WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of());
 
     try (DoFnTester<Integer, Integer> tester = DoFnTester.of(new SideInputDoFn(value))) {
       tester.setSideInput(value, GlobalWindow.INSTANCE, -2);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
index 5221f75..81e1d02 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
@@ -26,6 +26,7 @@ import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -57,11 +58,12 @@ public class FilterTest implements Serializable {
     }
   }
 
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testIdentityFilterByPredicate() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(591, 11789, 1257, 24578, 24799, 307))
         .apply(Filter.by(new TrivialFn(true)));
@@ -73,8 +75,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testNoFilterByPredicate() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 4, 5))
         .apply(Filter.by(new TrivialFn(false)));
@@ -86,8 +86,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterByPredicate() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
         .apply(Filter.by(new EvenFn()));
@@ -99,8 +97,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterLessThan() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
         .apply(Filter.lessThan(4));
@@ -112,8 +108,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterGreaterThan() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
         .apply(Filter.greaterThan(4));
@@ -125,8 +119,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterLessThanEq() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
         .apply(Filter.lessThanEq(4));
@@ -138,8 +130,6 @@ public class FilterTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterGreaterThanEq() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<Integer> output = p
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
         .apply(Filter.greaterThanEq(4));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
index bb2877e..b24071e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
@@ -27,7 +27,6 @@ import java.io.Serializable;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -49,6 +48,9 @@ import org.junit.runners.JUnit4;
 public class FlatMapElementsTest implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   /**
@@ -57,7 +59,6 @@ public class FlatMapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFlatMapBasic() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
 
@@ -82,7 +83,6 @@ public class FlatMapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFlatMapFnOutputTypeDescriptor() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<String> output = pipeline
         .apply(Create.of("hello"))
         .apply(FlatMapElements.via(new SimpleFunction<String, Set<String>>() {
@@ -117,7 +117,8 @@ public class FlatMapElementsTest implements Serializable {
    */
   @Test
   public void testPolymorphicSimpleFunction() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
+
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
 
@@ -168,7 +169,6 @@ public class FlatMapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testVoidValues() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     pipeline
         .apply(Create.of("hello"))
         .apply(WithKeys.<String, String>of("k"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
index d4686a4..48251bc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
@@ -65,6 +65,9 @@ import org.junit.runners.JUnit4;
 public class FlattenTest implements Serializable {
 
   @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
 
@@ -74,8 +77,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenPCollectionList() {
-    Pipeline p = TestPipeline.create();
-
     List<List<String>> inputs = Arrays.asList(
       LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES);
 
@@ -90,8 +91,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenPCollectionListThenParDo() {
-    Pipeline p = TestPipeline.create();
-
     List<List<String>> inputs = Arrays.asList(
       LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES);
 
@@ -107,8 +106,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenPCollectionListEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> output =
         PCollectionList.<String>empty(p)
         .apply(Flatten.<String>pCollections()).setCoder(StringUtf8Coder.of());
@@ -120,8 +117,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenInputMultipleCopies() {
-    Pipeline p = TestPipeline.create();
-
     int count = 5;
     PCollection<Long> longs = p.apply("mkLines", CountingInput.upTo(count));
     PCollection<Long> biggerLongs =
@@ -154,8 +149,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testEmptyFlattenAsSideInput() {
-    Pipeline p = TestPipeline.create();
-
     final PCollectionView<Iterable<String>> view =
         PCollectionList.<String>empty(p)
         .apply(Flatten.<String>pCollections()).setCoder(StringUtf8Coder.of())
@@ -179,9 +172,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenPCollectionListEmptyThenParDo() {
-
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> output =
         PCollectionList.<String>empty(p)
         .apply(Flatten.<String>pCollections()).setCoder(StringUtf8Coder.of())
@@ -198,8 +188,6 @@ public class FlattenTest implements Serializable {
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("cannot provide a Coder for empty");
 
-    Pipeline p = TestPipeline.create();
-
     PCollectionList.<ClassWithoutCoder>empty(p)
         .apply(Flatten.<ClassWithoutCoder>pCollections());
 
@@ -211,8 +199,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenIterables() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<Iterable<String>> input = p
         .apply(Create.<Iterable<String>>of(LINES)
             .withCoder(IterableCoder.of(StringUtf8Coder.of())));
@@ -229,8 +215,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenIterablesLists() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<List<String>> input =
         p.apply(Create.<List<String>>of(LINES).withCoder(ListCoder.of(StringUtf8Coder.of())));
 
@@ -244,8 +228,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenIterablesSets() {
-    Pipeline p = TestPipeline.create();
-
     Set<String> linesSet = ImmutableSet.copyOf(LINES);
 
     PCollection<Set<String>> input =
@@ -261,9 +243,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenIterablesCollections() {
-
-    Pipeline p = TestPipeline.create();
-
     Set<String> linesSet = ImmutableSet.copyOf(LINES);
 
     PCollection<Collection<String>> input =
@@ -280,8 +259,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFlattenIterablesEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<Iterable<String>> input = p
         .apply(Create.<Iterable<String>>of(NO_LINES)
             .withCoder(IterableCoder.of(StringUtf8Coder.of())));
@@ -300,8 +277,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testEqualWindowFnPropagation() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input1 =
         p.apply("CreateInput1", Create.of("Input1"))
         .apply("Window1", Window.<String>into(FixedWindows.of(Duration.standardMinutes(1))));
@@ -322,8 +297,6 @@ public class FlattenTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testCompatibleWindowFnPropagation() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<String> input1 =
         p.apply("CreateInput1", Create.of("Input1"))
         .apply("Window1",
@@ -345,7 +318,7 @@ public class FlattenTest implements Serializable {
 
   @Test
   public void testIncompatibleWindowFnPropagationFailure() {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
 
     PCollection<String> input1 =
         p.apply("CreateInput1", Create.of("Input1"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
index ebde110..f4bec3a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java
@@ -39,7 +39,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
@@ -81,6 +80,9 @@ import org.junit.runners.JUnit4;
 public class GroupByKeyTest {
 
   @Rule
+  public final TestPipeline p = TestPipeline.create();
+
+  @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Test
@@ -95,8 +97,6 @@ public class GroupByKeyTest {
         KV.of("k2", -33),
         KV.of("k3", 0));
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(ungroupedPairs)
             .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -137,8 +137,6 @@ public class GroupByKeyTest {
         KV.of("k2", -33),  // window [5, 10)
         KV.of("k3", 0));  // window [5, 10)
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.timestamped(ungroupedPairs, Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L))
             .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -174,8 +172,6 @@ public class GroupByKeyTest {
   public void testGroupByKeyEmpty() {
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(ungroupedPairs)
             .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -193,8 +189,6 @@ public class GroupByKeyTest {
 
     List<KV<Map<String, String>, Integer>> ungroupedPairs = Arrays.asList();
 
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<Map<String, String>, Integer>> input =
         p.apply(Create.of(ungroupedPairs)
             .withCoder(
@@ -209,7 +203,6 @@ public class GroupByKeyTest {
   @Test
   @Category(NeedsRunner.class)
   public void testIdentityWindowFnPropagation() {
-    Pipeline p = TestPipeline.create();
 
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
@@ -230,7 +223,6 @@ public class GroupByKeyTest {
   @Test
   @Category(NeedsRunner.class)
   public void testWindowFnInvalidation() {
-    Pipeline p = TestPipeline.create();
 
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
@@ -255,7 +247,6 @@ public class GroupByKeyTest {
 
   @Test
   public void testInvalidWindowsDirect() {
-    Pipeline p = TestPipeline.create();
 
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
@@ -275,7 +266,6 @@ public class GroupByKeyTest {
   @Test
   @Category(NeedsRunner.class)
   public void testRemerge() {
-    Pipeline p = TestPipeline.create();
 
     List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
 
@@ -300,7 +290,6 @@ public class GroupByKeyTest {
 
   @Test
   public void testGroupByKeyDirectUnbounded() {
-    Pipeline p = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input =
         p.apply(
@@ -331,9 +320,8 @@ public class GroupByKeyTest {
   @Test
   @Category(RunnableOnService.class)
   public void testOutputTimeFnEarliest() {
-    Pipeline pipeline = TestPipeline.create();
 
-    pipeline.apply(
+    p.apply(
         Create.timestamped(
             TimestampedValue.of(KV.of(0, "hello"), new Instant(0)),
             TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10))))
@@ -342,7 +330,7 @@ public class GroupByKeyTest {
         .apply(GroupByKey.<Integer, String>create())
         .apply(ParDo.of(new AssertTimestamp(new Instant(0))));
 
-    pipeline.run();
+    p.run();
   }
 
 
@@ -353,9 +341,7 @@ public class GroupByKeyTest {
   @Test
   @Category(RunnableOnService.class)
   public void testOutputTimeFnLatest() {
-    Pipeline pipeline = TestPipeline.create();
-
-    pipeline.apply(
+    p.apply(
         Create.timestamped(
             TimestampedValue.of(KV.of(0, "hello"), new Instant(0)),
             TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10))))
@@ -364,7 +350,7 @@ public class GroupByKeyTest {
         .apply(GroupByKey.<Integer, String>create())
         .apply(ParDo.of(new AssertTimestamp(new Instant(10))));
 
-    pipeline.run();
+    p.run();
   }
 
   private static class AssertTimestamp<K, V> extends DoFn<KV<K, V>, Void> {
@@ -408,8 +394,6 @@ public class GroupByKeyTest {
     final int numValues = 10;
     final int numKeys = 5;
 
-    Pipeline p = TestPipeline.create();
-
     p.getCoderRegistry().registerCoder(BadEqualityKey.class, DeterministicKeyCoder.class);
 
     // construct input data

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
index fce5b2f..2a19802 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.transforms;
 
 import java.util.Arrays;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -27,6 +26,7 @@ import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -50,11 +50,12 @@ public class KeysTest {
   static final KV<String, Integer>[] EMPTY_TABLE = new KV[] {
   };
 
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testKeys() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(TABLE)).withCoder(
             KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -69,8 +70,6 @@ public class KeysTest {
   @Test
   @Category(RunnableOnService.class)
   public void testKeysEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder(
             KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
index 3598198..24186ed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.transforms;
 
 import java.util.Arrays;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -27,6 +26,7 @@ import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -50,11 +50,12 @@ public class KvSwapTest {
   static final KV<String, Integer>[] EMPTY_TABLE = new KV[] {
   };
 
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testKvSwap() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(TABLE)).withCoder(
             KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
@@ -75,8 +76,6 @@ public class KvSwapTest {
   @Test
   @Category(RunnableOnService.class)
   public void testKvSwapEmpty() {
-    Pipeline p = TestPipeline.create();
-
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder(
             KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
index ce9ae37..f71b813 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
@@ -51,12 +51,13 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class LatestTest implements Serializable {
+
+  @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(NeedsRunner.class)
   public void testGloballyEventTimestamp() {
-    TestPipeline p = TestPipeline.create();
     PCollection<String> output =
         p.apply(Create.timestamped(
             TimestampedValue.of("foo", new Instant(100)),
@@ -71,7 +72,8 @@ public class LatestTest implements Serializable {
 
   @Test
   public void testGloballyOutputCoder() {
-    TestPipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     BigEndianLongCoder inputCoder = BigEndianLongCoder.of();
 
     PCollection<Long> output =
@@ -86,7 +88,6 @@ public class LatestTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testGloballyEmptyCollection() {
-    TestPipeline p = TestPipeline.create();
     PCollection<Long> emptyInput = p.apply(Create.<Long>of()
         // Explicitly set coder such that then runner enforces encodability.
         .withCoder(VarLongCoder.of()));
@@ -99,7 +100,6 @@ public class LatestTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPerKeyEventTimestamp() {
-    TestPipeline p = TestPipeline.create();
     PCollection<KV<String, String>> output =
         p.apply(Create.timestamped(
             TimestampedValue.of(KV.of("A", "foo"), new Instant(100)),
@@ -114,7 +114,8 @@ public class LatestTest implements Serializable {
 
   @Test
   public void testPerKeyOutputCoder() {
-    TestPipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     KvCoder<String, Long> inputCoder = KvCoder.of(
         AvroCoder.of(String.class), AvroCoder.of(Long.class));
 
@@ -128,7 +129,6 @@ public class LatestTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testPerKeyEmptyCollection() {
-    TestPipeline p = TestPipeline.create();
     PCollection<KV<String, String>> output =
         p.apply(Create.<KV<String, String>>of().withCoder(KvCoder.of(
             StringUtf8Coder.of(), StringUtf8Coder.of())))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
index ac3444b..47d0b87 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertThat;
 
 import java.io.Serializable;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -48,6 +47,9 @@ import org.junit.runners.JUnit4;
 public class MapElementsTest implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   /**
@@ -79,7 +81,6 @@ public class MapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMapBasic() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(MapElements.via(new SimpleFunction<Integer, Integer>() {
@@ -98,7 +99,8 @@ public class MapElementsTest implements Serializable {
    */
   @Test
   public void testPolymorphicSimpleFunction() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
+
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
 
@@ -120,7 +122,8 @@ public class MapElementsTest implements Serializable {
    */
   @Test
   public void testNestedPolymorphicSimpleFunction() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
+
     PCollection<Integer> output =
         pipeline
             .apply(Create.of(1, 2, 3))
@@ -149,7 +152,6 @@ public class MapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMapBasicSerializableFunction() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(MapElements.via(new SerializableFunction<Integer, Integer>() {
@@ -170,7 +172,6 @@ public class MapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testSimpleFunctionOutputTypeDescriptor() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<String> output = pipeline
         .apply(Create.of("hello"))
         .apply(MapElements.via(new SimpleFunction<String, String>() {
@@ -191,7 +192,6 @@ public class MapElementsTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testVoidValues() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     pipeline
         .apply(Create.of("hello"))
         .apply(WithKeys.<String, String>of("k"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
index 9bc8a64..2c3a735 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
@@ -33,6 +33,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -43,10 +44,13 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class ParDoLifecycleTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testOldFnCallSequence() {
-    TestPipeline p = TestPipeline.create();
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
         .apply(Flatten.<Integer>pCollections())
@@ -58,7 +62,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testOldFnCallSequenceMulti() {
-    TestPipeline p = TestPipeline.create();
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
         .apply(Flatten.<Integer>pCollections())
@@ -127,7 +130,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFnCallSequence() {
-    TestPipeline p = TestPipeline.create();
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
         .apply(Flatten.<Integer>pCollections())
@@ -139,7 +141,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFnCallSequenceMulti() {
-    TestPipeline p = TestPipeline.create();
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
         .apply(Flatten.<Integer>pCollections())
@@ -206,7 +207,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testTeardownCalledAfterExceptionInSetup() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP);
     p
         .apply(Create.of(1, 2, 3))
@@ -227,7 +227,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testTeardownCalledAfterExceptionInStartBundle() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE);
     p
         .apply(Create.of(1, 2, 3))
@@ -246,7 +245,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testTeardownCalledAfterExceptionInProcessElement() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT);
     p
         .apply(Create.of(1, 2, 3))
@@ -265,7 +263,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testTeardownCalledAfterExceptionInFinishBundle() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE);
     p
         .apply(Create.of(1, 2, 3))
@@ -284,7 +281,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWithContextTeardownCalledAfterExceptionInSetup() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
     try {
@@ -300,7 +296,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWithContextTeardownCalledAfterExceptionInStartBundle() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
     try {
@@ -316,7 +311,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWithContextTeardownCalledAfterExceptionInProcessElement() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
     try {
@@ -332,7 +326,6 @@ public class ParDoLifecycleTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWithContextTeardownCalledAfterExceptionInFinishBundle() {
-    TestPipeline p = TestPipeline.create();
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
     try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 4a3e2dd..3a47fc7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -47,7 +47,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -107,6 +106,9 @@ public class ParDoTest implements Serializable {
   // anonymous inner classes inside the non-static test methods.
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   private static class PrintingDoFn extends DoFn<String, String> {
@@ -302,7 +304,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDo() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -319,7 +320,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDo2() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -336,7 +336,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoEmpty() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList();
 
@@ -354,8 +353,6 @@ public class ParDoTest implements Serializable {
   @Category(RunnableOnService.class)
   public void testParDoEmptyOutputs() {
 
-    Pipeline pipeline = TestPipeline.create();
-
     List<Integer> inputs = Arrays.asList();
 
     PCollection<String> output = pipeline
@@ -370,7 +367,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoWithSideOutputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -413,7 +409,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoEmptyWithSideOutputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList();
 
@@ -454,7 +449,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoWithEmptySideOutputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList();
 
@@ -482,7 +476,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoWithOnlySideOutputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -507,7 +500,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoWritingToUndeclaredSideOutput() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -529,7 +521,7 @@ public class ParDoTest implements Serializable {
   // TODO: The exception thrown is runner-specific, even if the behavior is general
   @Category(NeedsRunner.class)
   public void testParDoUndeclaredSideOutputLimit() {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> input = pipeline.apply(Create.of(Arrays.asList(3)));
 
     // Success for a total of 1000 outputs.
@@ -566,7 +558,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoWithSideInputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -598,7 +589,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoWithSideInputsIsCumulative() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -632,7 +622,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultiOutputParDoWithSideInputs() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -670,7 +659,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testMultiOutputParDoWithSideInputsIsCumulative() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -708,7 +696,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoReadingFromUnknownSideInput() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -746,7 +733,6 @@ public class ParDoTest implements Serializable {
     // on an input where the element is in multiple windows. The complication is
     // that side inputs are per-window, so the runner has to make sure
     // to process each window individually.
-    Pipeline p = TestPipeline.create();
 
     MutableDateTime mutableNow = Instant.now().toMutableDateTime();
     mutableNow.setMillisOfSecond(0);
@@ -754,9 +740,9 @@ public class ParDoTest implements Serializable {
 
     SlidingWindows windowFn =
         SlidingWindows.of(Duration.standardSeconds(5)).every(Duration.standardSeconds(1));
-    PCollectionView<Integer> view = p.apply(Create.of(1)).apply(View.<Integer>asSingleton());
+    PCollectionView<Integer> view = pipeline.apply(Create.of(1)).apply(View.<Integer>asSingleton());
     PCollection<String> res =
-        p.apply(Create.timestamped(TimestampedValue.of("a", now)))
+        pipeline.apply(Create.timestamped(TimestampedValue.of("a", now)))
             .apply(Window.<String>into(windowFn))
             .apply(ParDo.of(new FnWithSideInputs(view)).withSideInputs(view));
 
@@ -766,14 +752,12 @@ public class ParDoTest implements Serializable {
       PAssert.that(res).inWindow(window).containsInAnyOrder("a:1");
     }
 
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(NeedsRunner.class)
   public void testParDoWithErrorInStartBatch() {
-    Pipeline pipeline = TestPipeline.create();
-
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
     pipeline.apply(Create.of(inputs))
@@ -787,7 +771,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoWithErrorInProcessElement() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -802,7 +785,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoWithErrorInFinishBatch() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -816,23 +798,27 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testParDoOutputNameBasedOnDoFnWithTrimmedSuffix() {
-    Pipeline p = TestPipeline.create();
-    PCollection<String> output = p.apply(Create.of(1)).apply(ParDo.of(new TestDoFn()));
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    PCollection<String> output = pipeline.apply(Create.of(1)).apply(ParDo.of(new TestDoFn()));
     assertThat(output.getName(), containsString("ParDo(Test)"));
   }
 
   @Test
   public void testParDoOutputNameBasedOnLabel() {
-    Pipeline p = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
+
     PCollection<String> output =
-        p.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestDoFn()));
+        pipeline.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestDoFn()));
     assertThat(output.getName(), containsString("MyParDo"));
   }
 
   @Test
   public void testParDoOutputNameBasedDoFnWithoutMatchingSuffix() {
-    Pipeline p = TestPipeline.create();
-    PCollection<String> output = p.apply(Create.of(1)).apply(ParDo.of(new StrangelyNamedDoer()));
+    pipeline.enableAbandonedNodeEnforcement(false);
+
+    PCollection<String> output =
+        pipeline.apply(Create.of(1)).apply(ParDo.of(new StrangelyNamedDoer()));
     assertThat(output.getName(), containsString("ParDo(StrangelyNamedDoer)"));
   }
 
@@ -850,7 +836,7 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testParDoWithSideOutputsName() {
-    Pipeline p = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
 
     TupleTag<String> mainOutputTag = new TupleTag<String>("main"){};
     TupleTag<String> sideOutputTag1 = new TupleTag<String>("side1"){};
@@ -858,7 +844,7 @@ public class ParDoTest implements Serializable {
     TupleTag<String> sideOutputTag3 = new TupleTag<String>("side3"){};
     TupleTag<String> sideOutputTagUnwritten = new TupleTag<String>("sideUnwritten"){};
 
-    PCollectionTuple outputs = p
+    PCollectionTuple outputs = pipeline
         .apply(Create.of(Arrays.asList(3, -42, 666))).setName("MyInput")
         .apply("MyParDo", ParDo
                .of(new TestDoFn(
@@ -880,7 +866,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testParDoInCustomTransform() {
-    Pipeline pipeline = TestPipeline.create();
 
     List<Integer> inputs = Arrays.asList(3, -42, 666);
 
@@ -904,7 +889,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMultiOutputChaining() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollectionTuple filters = pipeline
         .apply(Create.of(Arrays.asList(3, 4, 5, 6)))
@@ -1106,7 +1090,7 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testSideOutputUnknownCoder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> input = pipeline
         .apply(Create.of(Arrays.asList(1, 2, 3)));
 
@@ -1122,7 +1106,8 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testSideOutputUnregisteredExplicitCoder() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
+
     PCollection<Integer> input = pipeline
         .apply(Create.of(Arrays.asList(1, 2, 3)));
 
@@ -1144,7 +1129,7 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMainOutputUnregisteredExplicitCoder() {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> input = pipeline
         .apply(Create.of(Arrays.asList(1, 2, 3)));
 
@@ -1165,7 +1150,6 @@ public class ParDoTest implements Serializable {
     // should not cause a crash based on lack of a coder for the
     // side output.
 
-    Pipeline pipeline = TestPipeline.create();
     final TupleTag<TestDummy> mainOutputTag = new TupleTag<TestDummy>("main");
     final TupleTag<TestDummy> sideOutputTag = new TupleTag<TestDummy>("side");
     PCollectionTuple tuple = pipeline
@@ -1204,7 +1188,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoOutputWithTimestamp() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> input =
         pipeline.apply(Create.of(Arrays.asList(3, 42, 6)));
@@ -1226,7 +1209,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoSideOutputWithTimestamp() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> input =
         pipeline.apply(Create.of(Arrays.asList(3, 42, 6)));
@@ -1258,7 +1240,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoShiftTimestamp() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> input =
         pipeline.apply(Create.of(Arrays.asList(3, 42, 6)));
@@ -1281,7 +1262,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoShiftTimestampInvalid() {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline.apply(Create.of(Arrays.asList(3, 42, 6)))
         .apply(ParDo.of(new TestOutputTimestampDoFn()))
@@ -1300,7 +1280,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testParDoShiftTimestampInvalidZeroAllowed() {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline.apply(Create.of(Arrays.asList(3, 42, 6)))
         .apply(ParDo.of(new TestOutputTimestampDoFn()))
@@ -1353,7 +1332,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testWindowingInStartAndFinishBundle() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<String> output =
         pipeline
@@ -1391,7 +1369,6 @@ public class ParDoTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWindowingInStartBundleException() {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline
         .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1))))
@@ -1477,13 +1454,12 @@ public class ParDoTest implements Serializable {
           }
         };
 
-    Pipeline p = TestPipeline.create();
     PCollection<Integer> output =
-        p.apply(Create.of(KV.of("hello", 42), KV.of("hello", 97), KV.of("hello", 84)))
+        pipeline.apply(Create.of(KV.of("hello", 42), KV.of("hello", 97), KV.of("hello", 84)))
             .apply(ParDo.of(fn));
 
     PAssert.that(output).containsInAnyOrder(0, 1, 2);
-    p.run();
+    pipeline.run();
   }
 
   @Test
@@ -1514,9 +1490,8 @@ public class ParDoTest implements Serializable {
           }
         };
 
-    Pipeline p = TestPipeline.create();
     PCollectionTuple output =
-        p.apply(
+        pipeline.apply(
                 Create.of(
                     KV.of("hello", 42),
                     KV.of("hello", 97),
@@ -1534,7 +1509,7 @@ public class ParDoTest implements Serializable {
 
     // There are 1 and 3 from "hello" and just "1" from "goodbye"
     PAssert.that(odds).containsInAnyOrder(1, 3, 1);
-    p.run();
+    pipeline.run();
   }
 
   @Test
@@ -1562,24 +1537,23 @@ public class ParDoTest implements Serializable {
           }
         };
 
-    Pipeline p = TestPipeline.create();
     PCollection<List<Integer>> output =
-        p.apply(
+        pipeline.apply(
                 Create.of(
                     KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12)))
             .apply(ParDo.of(fn));
 
     PAssert.that(output).containsInAnyOrder(Lists.newArrayList(12, 42, 84, 97));
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category({RunnableOnService.class, UsesStatefulParDo.class})
   public void testBagStateSideInput() {
-    Pipeline p = TestPipeline.create();
 
     final PCollectionView<List<Integer>> listView =
-        p.apply("Create list for side input", Create.of(2, 1, 0)).apply(View.<Integer>asList());
+        pipeline
+            .apply("Create list for side input", Create.of(2, 1, 0)).apply(View.<Integer>asList());
 
     final String stateId = "foo";
     DoFn<KV<String, Integer>, List<Integer>> fn =
@@ -1607,7 +1581,7 @@ public class ParDoTest implements Serializable {
         };
 
     PCollection<List<Integer>> output =
-        p.apply(
+        pipeline.apply(
                 "Create main input",
                 Create.of(
                     KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12)))
@@ -1616,7 +1590,7 @@ public class ParDoTest implements Serializable {
     PAssert.that(output).containsInAnyOrder(
         Lists.newArrayList(12, 42, 84, 97),
         Lists.newArrayList(0, 1, 2));
-    p.run();
+    pipeline.run();
   }
 
   /**
@@ -1658,11 +1632,9 @@ public class ParDoTest implements Serializable {
           }
         };
 
-    Pipeline p = TestPipeline.create();
-
-    PCollection<Integer> output = p.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn));
+    PCollection<Integer> output = pipeline.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn));
     PAssert.that(output).containsInAnyOrder(3, 42);
-    p.run();
+    pipeline.run();
   }
 
   @Test
@@ -1704,7 +1676,6 @@ public class ParDoTest implements Serializable {
 
   @Test
   public void testRejectsWrongWindowType() {
-    Pipeline p = TestPipeline.create();
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(GlobalWindow.class.getSimpleName());
@@ -1712,7 +1683,8 @@ public class ParDoTest implements Serializable {
     thrown.expectMessage("window type");
     thrown.expectMessage("not a supertype");
 
-    p.apply(Create.of(1, 2, 3))
+    pipeline
+        .apply(Create.of(1, 2, 3))
         .apply(
             ParDo.of(
                 new DoFn<Integer, Integer>() {
@@ -1735,9 +1707,8 @@ public class ParDoTest implements Serializable {
   public void testMultipleWindowSubtypesOK() {
     final String timerId = "gobbledegook";
 
-    Pipeline p = TestPipeline.create();
-
-    p.apply(Create.of(1, 2, 3))
+    pipeline
+        .apply(Create.of(1, 2, 3))
         .apply(Window.<Integer>into(FixedWindows.of(Duration.standardSeconds(10))))
         .apply(
             ParDo.of(
@@ -1759,26 +1730,25 @@ public class ParDoTest implements Serializable {
   public void testRejectsSplittableDoFnByDefault() {
     // ParDo with a splittable DoFn must be overridden by the runner.
     // Without an override, applying it directly must fail.
-    Pipeline p = TestPipeline.create();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(p.getRunner().getClass().getName());
+    thrown.expectMessage(pipeline.getRunner().getClass().getName());
     thrown.expectMessage("does not support Splittable DoFn");
 
-    p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new TestSplittableDoFn()));
+    pipeline.apply(Create.of(1, 2, 3)).apply(ParDo.of(new TestSplittableDoFn()));
   }
 
   @Test
   public void testMultiRejectsSplittableDoFnByDefault() {
     // ParDo with a splittable DoFn must be overridden by the runner.
     // Without an override, applying it directly must fail.
-    Pipeline p = TestPipeline.create();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage(p.getRunner().getClass().getName());
+    thrown.expectMessage(pipeline.getRunner().getClass().getName());
     thrown.expectMessage("does not support Splittable DoFn");
 
-    p.apply(Create.of(1, 2, 3))
+    pipeline
+        .apply(Create.of(1, 2, 3))
         .apply(
             ParDo.of(new TestSplittableDoFn())
                 .withOutputTags(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
index 1cbe344..87d7460 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -47,6 +46,7 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class PartitionTest implements Serializable {
 
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   static class ModFn implements PartitionFn<Integer> {
@@ -63,10 +63,10 @@ public class PartitionTest implements Serializable {
     }
   }
 
+
   @Test
   @Category(RunnableOnService.class)
   public void testEvenOddPartition() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollectionList<Integer> outputs = pipeline
         .apply(Create.of(591, 11789, 1257, 24578, 24799, 307))
@@ -81,7 +81,6 @@ public class PartitionTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testModPartition() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollectionList<Integer> outputs = pipeline
         .apply(Create.of(1, 2, 4, 5))
@@ -96,7 +95,6 @@ public class PartitionTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testOutOfBoundsPartitions() {
-    Pipeline pipeline = TestPipeline.create();
 
     pipeline
     .apply(Create.of(-1))
@@ -110,7 +108,6 @@ public class PartitionTest implements Serializable {
 
   @Test
   public void testZeroNumPartitions() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> input = pipeline.apply(Create.of(591));
 
@@ -122,7 +119,6 @@ public class PartitionTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testDroppedPartition() {
-    Pipeline pipeline = TestPipeline.create();
 
     // Compute the set of integers either 1 or 2 mod 3, the hard way.
     PCollectionList<Integer> outputs = pipeline

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java
index 6e196b4..cd707da 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java
@@ -25,6 +25,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -33,11 +34,13 @@ import org.junit.runners.JUnit4;
 /** Tests for {@link Regex}. */
 @RunWith(JUnit4.class)
 public class RegexTest implements Serializable {
+
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(NeedsRunner.class)
   public void testFind() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<String> output =
         p.apply(Create.of("aj", "xj", "yj", "zj")).apply(Regex.find("[xyz]"));
 
@@ -48,8 +51,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFindGroup() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<String> output =
         p.apply(Create.of("aj", "xj", "yj", "zj")).apply(Regex.find("([xyz])", 1));
 
@@ -60,8 +61,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFindNone() {
-    TestPipeline p = TestPipeline.create();
-
     PCollection<String> output = p.apply(Create.of("a", "b", "c", "d")).apply(Regex.find("[xyz]"));
 
     PAssert.that(output).empty();
@@ -71,7 +70,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testKVFind() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<KV<String, String>> output =
         p.apply(Create.of("a b c")).apply(Regex.findKV("a (b) (c)", 1, 2));
@@ -83,7 +81,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testKVFindNone() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<KV<String, String>> output =
         p.apply(Create.of("x y z")).apply(Regex.findKV("a (b) (c)", 1, 2));
@@ -95,7 +92,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMatches() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("a", "x", "y", "z")).apply(Regex.matches("[xyz]"));
@@ -107,7 +103,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMatchesNone() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("a", "b", "c", "d")).apply(Regex.matches("[xyz]"));
@@ -119,7 +114,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testMatchesGroup() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("a", "x xxx", "x yyy", "x zzz")).apply(Regex.matches("x ([xyz]*)", 1));
@@ -131,7 +125,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testKVMatches() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<KV<String, String>> output =
         p.apply(Create.of("a b c")).apply(Regex.matchesKV("a (b) (c)", 1, 2));
@@ -143,7 +136,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testKVMatchesNone() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<KV<String, String>> output =
         p.apply(Create.of("x y z")).apply(Regex.matchesKV("a (b) (c)", 1, 2));
@@ -154,7 +146,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReplaceAll() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("xj", "yj", "zj")).apply(Regex.replaceAll("[xyz]", "new"));
@@ -166,7 +157,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReplaceAllMixed() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("abc", "xj", "yj", "zj", "def")).apply(Regex.replaceAll("[xyz]", "new"));
@@ -178,7 +168,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReplaceFirst() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("xjx", "yjy", "zjz")).apply(Regex.replaceFirst("[xyz]", "new"));
@@ -190,7 +179,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReplaceFirstMixed() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("abc", "xjx", "yjy", "zjz", "def"))
@@ -203,7 +191,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testSplits() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("The  quick   brown fox jumps over    the lazy dog"))
@@ -217,7 +204,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testSplitsWithEmpty() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("The  quick   brown fox jumps over    the lazy dog"))
@@ -235,7 +221,6 @@ public class RegexTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testSplitsWithoutEmpty() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> output =
         p.apply(Create.of("The  quick   brown fox jumps over    the lazy dog"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
index a0555fa..9cc12d4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
@@ -32,6 +32,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -41,6 +42,7 @@ import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -102,71 +104,70 @@ public class SampleTest {
     }
   }
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testSample() {
-    Pipeline p = TestPipeline.create();
 
-    PCollection<Integer> input = p.apply(Create.of(DATA)
+    PCollection<Integer> input = pipeline.apply(Create.of(DATA)
         .withCoder(BigEndianIntegerCoder.of()));
     PCollection<Iterable<Integer>> output = input.apply(
         Sample.<Integer>fixedSizeGlobally(3));
 
     PAssert.thatSingletonIterable(output)
         .satisfies(new VerifyCorrectSample<>(3, DATA));
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testSampleEmpty() {
-    Pipeline p = TestPipeline.create();
 
-    PCollection<Integer> input = p.apply(Create.of(EMPTY)
+    PCollection<Integer> input = pipeline.apply(Create.of(EMPTY)
         .withCoder(BigEndianIntegerCoder.of()));
     PCollection<Iterable<Integer>> output = input.apply(
         Sample.<Integer>fixedSizeGlobally(3));
 
     PAssert.thatSingletonIterable(output)
         .satisfies(new VerifyCorrectSample<>(0, EMPTY));
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testSampleZero() {
-    Pipeline p = TestPipeline.create();
 
-    PCollection<Integer> input = p.apply(Create.of(DATA)
+    PCollection<Integer> input = pipeline.apply(Create.of(DATA)
         .withCoder(BigEndianIntegerCoder.of()));
     PCollection<Iterable<Integer>> output = input.apply(
         Sample.<Integer>fixedSizeGlobally(0));
 
     PAssert.thatSingletonIterable(output)
         .satisfies(new VerifyCorrectSample<>(0, DATA));
-    p.run();
+    pipeline.run();
   }
 
   @Test
   @Category(RunnableOnService.class)
   public void testSampleInsufficientElements() {
-    Pipeline p = TestPipeline.create();
 
-    PCollection<Integer> input = p.apply(Create.of(DATA)
+    PCollection<Integer> input = pipeline.apply(Create.of(DATA)
         .withCoder(BigEndianIntegerCoder.of()));
     PCollection<Iterable<Integer>> output = input.apply(
         Sample.<Integer>fixedSizeGlobally(10));
 
     PAssert.thatSingletonIterable(output)
         .satisfies(new VerifyCorrectSample<>(5, DATA));
-    p.run();
+    pipeline.run();
   }
 
   @Test(expected = IllegalArgumentException.class)
   public void testSampleNegative() {
-    Pipeline p = TestPipeline.create();
+    pipeline.enableAbandonedNodeEnforcement(false);
 
-    PCollection<Integer> input = p.apply(Create.of(DATA)
+    PCollection<Integer> input = pipeline.apply(Create.of(DATA)
         .withCoder(BigEndianIntegerCoder.of()));
     input.apply(Sample.<Integer>fixedSizeGlobally(-1));
   }
@@ -174,9 +175,8 @@ public class SampleTest {
   @Test
   @Category(RunnableOnService.class)
   public void testSampleMultiplicity() {
-    Pipeline p = TestPipeline.create();
 
-    PCollection<Integer> input = p.apply(Create.of(REPEATED_DATA)
+    PCollection<Integer> input = pipeline.apply(Create.of(REPEATED_DATA)
         .withCoder(BigEndianIntegerCoder.of()));
     // At least one value must be selected with multiplicity.
     PCollection<Iterable<Integer>> output = input.apply(
@@ -184,7 +184,7 @@ public class SampleTest {
 
     PAssert.thatSingletonIterable(output)
         .satisfies(new VerifyCorrectSample<>(6, REPEATED_DATA));
-    p.run();
+    pipeline.run();
   }
 
   private static class VerifyAnySample implements SerializableFunction<Iterable<String>, Void> {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
index 022c2e5..e3b58b7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
@@ -29,7 +29,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -54,6 +53,7 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.MutableDateTime;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -151,10 +151,13 @@ public class SplittableDoFnTest {
     }
   }
 
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category({RunnableOnService.class, UsesSplittableParDo.class})
   public void testPairWithIndexBasic() {
-    Pipeline p = TestPipeline.create();
+
     PCollection<KV<String, Integer>> res =
         p.apply(Create.of("a", "bb", "ccccc"))
             .apply(ParDo.of(new PairStringWithIndexToLength()))
@@ -180,7 +183,6 @@ public class SplittableDoFnTest {
   public void testPairWithIndexWindowedTimestamped() {
     // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps
     // of elements in the input collection.
-    Pipeline p = TestPipeline.create();
 
     MutableDateTime mutableNow = Instant.now().toMutableDateTime();
     mutableNow.setMillisOfSecond(0);
@@ -277,7 +279,6 @@ public class SplittableDoFnTest {
   @Test
   @Category({RunnableOnService.class, UsesSplittableParDo.class})
   public void testOutputAfterCheckpoint() throws Exception {
-    Pipeline p = TestPipeline.create();
     PCollection<Integer> outputs = p.apply(Create.of("foo"))
         .apply(ParDo.of(new SDFWithMultipleOutputsPerBlock()));
     PAssert.thatSingleton(outputs.apply(Count.<Integer>globally()))
@@ -317,7 +318,6 @@ public class SplittableDoFnTest {
   @Test
   @Category({RunnableOnService.class, UsesSplittableParDo.class})
   public void testSideInputsAndOutputs() throws Exception {
-    Pipeline p = TestPipeline.create();
 
     PCollectionView<String> sideInput =
         p.apply("side input", Create.of("foo")).apply(View.<String>asSingleton());
@@ -344,7 +344,6 @@ public class SplittableDoFnTest {
   @Test
   @Category({RunnableOnService.class, UsesSplittableParDo.class})
   public void testLateData() throws Exception {
-    Pipeline p = TestPipeline.create();
 
     Instant base = Instant.now();
 
@@ -439,7 +438,6 @@ public class SplittableDoFnTest {
   @Test
   @Category({RunnableOnService.class, UsesSplittableParDo.class})
   public void testLifecycleMethods() throws Exception {
-    Pipeline p = TestPipeline.create();
 
     PCollection<String> res =
         p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle()));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
index b624252..d011197 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java
@@ -53,6 +53,9 @@ import org.junit.runners.JUnit4;
 public class TopTest {
 
   @Rule
+  public final TestPipeline p = TestPipeline.create();
+
+  @Rule
   public ExpectedException expectedEx = ExpectedException.none();
 
   @SuppressWarnings("unchecked")
@@ -93,7 +96,6 @@ public class TopTest {
   @Category(NeedsRunner.class)
   @SuppressWarnings("unchecked")
   public void testTop() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION))
                  .withCoder(StringUtf8Coder.of()));
@@ -125,7 +127,6 @@ public class TopTest {
   @Category(NeedsRunner.class)
   @SuppressWarnings("unchecked")
   public void testTopEmpty() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(EMPTY_COLLECTION))
                  .withCoder(StringUtf8Coder.of()));
@@ -151,7 +152,8 @@ public class TopTest {
 
   @Test
   public void testTopEmptyWithIncompatibleWindows() {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     Bound<String> windowingFn = Window.<String>into(FixedWindows.of(Duration.standardDays(10L)));
     PCollection<String> input =
         p.apply(Create.timestamped(Collections.<String>emptyList(), Collections.<Long>emptyList()))
@@ -170,7 +172,6 @@ public class TopTest {
   @Category(NeedsRunner.class)
   @SuppressWarnings("unchecked")
   public void testTopZero() {
-    Pipeline p = TestPipeline.create();
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION))
                  .withCoder(StringUtf8Coder.of()));
@@ -202,7 +203,8 @@ public class TopTest {
   // This is a purely compile-time test.  If the code compiles, then it worked.
   @Test
   public void testPerKeySerializabilityRequirement() {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     p.apply("CreateCollection", Create.of(Arrays.asList(COLLECTION))
         .withCoder(StringUtf8Coder.of()));
 
@@ -218,7 +220,8 @@ public class TopTest {
 
   @Test
   public void testCountConstraint() {
-    Pipeline p = TestPipeline.create();
+    p.enableAbandonedNodeEnforcement(false);
+
     PCollection<String> input =
         p.apply(Create.of(Arrays.asList(COLLECTION))
             .withCoder(StringUtf8Coder.of()));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75a4c918/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
index 0bf2e2e..5e27552 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -29,6 +28,7 @@ import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -52,10 +52,12 @@ public class ValuesTest {
   static final KV<String, Integer>[] EMPTY_TABLE = new KV[] {
   };
 
+  @Rule
+  public final TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void testValues() {
-    Pipeline p = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(TABLE)).withCoder(
@@ -72,7 +74,6 @@ public class ValuesTest {
   @Test
   @Category(RunnableOnService.class)
   public void testValuesEmpty() {
-    Pipeline p = TestPipeline.create();
 
     PCollection<KV<String, Integer>> input =
         p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder(


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

Posted by ke...@apache.org.
Add informative Instant formatter to BoundedWindow


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

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

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


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


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

Posted by ke...@apache.org.
Update Dataflow worker to beam-master-20161220


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

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

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


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


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 63331aa8aa6314e8469c23a4f4a89fbf287cbc5a
Parents: 24ad183
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 09:54:57 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


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


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

Posted by ke...@apache.org.
This closes #1669: Preliminaries for timers in the direct runner

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


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

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

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



[29/51] [abbrv] incubator-beam git commit: Remove deprecated InMemoryTimerInternals from SDK

Posted by ke...@apache.org.
Remove deprecated InMemoryTimerInternals from SDK


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

Branch: refs/heads/python-sdk
Commit: 9f1d3d155303bd3d1069541be704d5f3e74926eb
Parents: 6a05d7f
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Dec 20 14:07:00 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 15:16:32 2016 -0800

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


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


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: fce4f6584ca2fd3c2c258405b9f3014be9da3514
Parents: 950aa7e
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 18:09:30 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:46 2016 -0800

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fce4f658/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
index 5faa618..e7ff712 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
@@ -57,6 +57,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
@@ -80,6 +81,9 @@ public class MongoDbIOTest implements Serializable {
 
   private static int port;
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   /**
    * Looking for an available network port.
    */
@@ -143,7 +147,6 @@ public class MongoDbIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testFullRead() throws Exception {
-    TestPipeline pipeline = TestPipeline.create();
 
     PCollection<Document> output = pipeline.apply(
         MongoDbIO.read()
@@ -177,7 +180,6 @@ public class MongoDbIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testReadWithFilter() throws Exception {
-    TestPipeline pipeline = TestPipeline.create();
 
     PCollection<Document> output = pipeline.apply(
         MongoDbIO.read()
@@ -195,7 +197,6 @@ public class MongoDbIOTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void testWrite() throws Exception {
-    TestPipeline pipeline = TestPipeline.create();
 
     ArrayList<Document> data = new ArrayList<>();
     for (int i = 0; i < 10000; i++) {


[16/51] [abbrv] incubator-beam git commit: Propagate key through ParDo if DoFn is key-preserving

Posted by ke...@apache.org.
Propagate key through ParDo if DoFn is key-preserving


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

Branch: refs/heads/python-sdk
Commit: d040b7f6a3cdefde829321015c75a800901cd88f
Parents: b26ceaa
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Dec 8 11:44:48 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 11:18:04 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/runners/direct/ParDoEvaluator.java | 13 +++++++++++--
 .../beam/runners/direct/ParDoEvaluatorFactory.java     |  3 +++
 .../SplittableProcessElementsEvaluatorFactory.java     |  1 +
 .../apache/beam/runners/direct/ParDoEvaluatorTest.java |  1 +
 4 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d040b7f6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
index a915cf0..a5de4c6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -47,6 +47,7 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
       AppliedPTransform<?, ?, ?> application,
       WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy,
       Serializable fn, // may be OldDoFn or DoFn
+      StructuralKey<?> key,
       List<PCollectionView<?>> sideInputs,
       TupleTag<OutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,
@@ -55,8 +56,16 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
 
     Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
     for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
-      outputBundles.put(
-          outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue()));
+      // Just trust the context's decision as to whether the output should be keyed.
+      // The logic for whether this ParDo is key-preserving and whether the input
+      // is keyed lives elsewhere.
+      if (evaluationContext.isKeyed(outputEntry.getValue())) {
+        outputBundles.put(
+            outputEntry.getKey(), evaluationContext.createKeyedBundle(key, outputEntry.getValue()));
+      } else {
+        outputBundles.put(
+            outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue()));
+      }
     }
     BundleOutputManager outputManager = BundleOutputManager.create(outputBundles);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d040b7f6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
index b4684e3..835e6ce 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
@@ -112,6 +112,7 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
     return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(
         createParDoEvaluator(
             application,
+            inputBundleKey,
             sideInputs,
             mainOutputTag,
             sideOutputTags,
@@ -123,6 +124,7 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
 
   ParDoEvaluator<InputT, OutputT> createParDoEvaluator(
       AppliedPTransform<PCollection<InputT>, PCollectionTuple, ?> application,
+      StructuralKey<?> key,
       List<PCollectionView<?>> sideInputs,
       TupleTag<OutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,
@@ -137,6 +139,7 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
           application,
           application.getInput().getWindowingStrategy(),
           fn,
+          key,
           sideInputs,
           mainOutputTag,
           sideOutputTags,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d040b7f6/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
index aae1149..18f3909 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
@@ -91,6 +91,7 @@ class SplittableProcessElementsEvaluatorFactory<InputT, OutputT, RestrictionT>
         parDoEvaluator =
             delegateFactory.createParDoEvaluator(
                 application,
+                inputBundle.getKey(),
                 transform.getSideInputs(),
                 transform.getMainOutputTag(),
                 transform.getSideOutputTags().getAll(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d040b7f6/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 1a3207b..b3aceeb 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
@@ -164,6 +164,7 @@ public class ParDoEvaluatorTest {
         transform,
         transform.getInput().getWindowingStrategy(),
         fn,
+        null /* key */,
         ImmutableList.<PCollectionView<?>>of(singletonView),
         mainOutputTag,
         sideOutputTags,


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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule.


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

Branch: refs/heads/python-sdk
Commit: 5ccbe6791af7e75c166ed877391e8c86bba5fe56
Parents: 6dea099
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 17:26:51 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:45 2016 -0800

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


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


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

Posted by ke...@apache.org.
This closes #1665: Remove deprecated AggregatorFactory from SDK

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


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

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

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



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

Posted by ke...@apache.org.
[BEAM-1097] Provide a better error message for non-existing gcpTempLocation

This closes #1522


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

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

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



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

Posted by ke...@apache.org.
This closes #1581: [BEAM-1117] Port direct runner StatefulParDo to KeyedWorkItem

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


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

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

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



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

Posted by ke...@apache.org.
Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps.


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

Branch: refs/heads/python-sdk
Commit: 4b23d42c31c95bed0d64bfc393fa193311e93498
Parents: fce4f65
Author: Stas Levin <st...@gmail.com>
Authored: Tue Dec 20 18:57:57 2016 +0200
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 20 09:55:46 2016 -0800

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


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

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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
index afd1c8b..f91371e 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.transforms;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -39,12 +38,14 @@ import org.junit.runners.JUnit4;
 public class FilterJava8Test implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(RunnableOnService.class)
   public void testIdentityFilterByPredicate() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(591, 11789, 1257, 24578, 24799, 307))
@@ -56,7 +57,6 @@ public class FilterJava8Test implements Serializable {
 
   @Test
   public void testNoFilterByPredicate() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 4, 5))
@@ -69,7 +69,6 @@ public class FilterJava8Test implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterByPredicate() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))
@@ -85,7 +84,6 @@ public class FilterJava8Test implements Serializable {
    */
   @Test
   public void testFilterParDoOutputTypeDescriptorRaw() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     @SuppressWarnings({"unchecked", "rawtypes"})
     PCollection<String> output = pipeline
@@ -99,7 +97,6 @@ public class FilterJava8Test implements Serializable {
   @Test
   @Category(RunnableOnService.class)
   public void testFilterByMethodReference() {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3, 4, 5, 6, 7))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
index 70cc04d..471724d 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms;
 import com.google.common.collect.ImmutableList;
 import java.io.Serializable;
 import java.util.List;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
@@ -38,6 +37,9 @@ import org.junit.runners.JUnit4;
 public class FlatMapElementsJava8Test implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   /**
@@ -46,7 +48,6 @@ public class FlatMapElementsJava8Test implements Serializable {
    */
   @Test
   public void testFlatMapBasic() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(FlatMapElements
@@ -63,7 +64,7 @@ public class FlatMapElementsJava8Test implements Serializable {
    */
   @Test
   public void testFlatMapMethodReference() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(FlatMapElements

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
index 9b556b9..ce0f111 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java
@@ -18,11 +18,11 @@
 package org.apache.beam.sdk.transforms;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TypeDescriptor;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -33,13 +33,16 @@ import org.junit.runners.JUnit4;
 @RunWith(JUnit4.class)
 public class MapElementsJava8Test implements Serializable {
 
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
   /**
    * Basic test of {@link MapElements} with a lambda (which is instantiated as a
    * {@link SerializableFunction}).
    */
   @Test
   public void testMapBasic() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(MapElements
@@ -56,7 +59,7 @@ public class MapElementsJava8Test implements Serializable {
    */
   @Test
   public void testMapMethodReference() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
+
     PCollection<Integer> output = pipeline
         .apply(Create.of(1, 2, 3))
         .apply(MapElements

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
index 0aeb41f..7d97740 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.transforms;
 import static org.junit.Assert.assertEquals;
 
 import java.io.Serializable;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -39,11 +38,14 @@ import org.junit.runners.JUnit4;
 public class PartitionJava8Test implements Serializable {
 
   @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void testModPartition() {
-    Pipeline pipeline = TestPipeline.create();
+
 
     PCollectionList<Integer> outputs = pipeline
         .apply(Create.of(1, 2, 4, 5))
@@ -61,7 +63,6 @@ public class PartitionJava8Test implements Serializable {
    */
   @Test
   public void testPartitionFnOutputTypeDescriptorRaw() throws Exception {
-    Pipeline pipeline = TestPipeline.create();
 
     PCollectionList<String> output = pipeline
         .apply(Create.of("hello"))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
index a5b9cb1..6ba41fa 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java
@@ -39,12 +39,15 @@ import org.junit.runners.JUnit4;
 public class WithKeysJava8Test {
 
   @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
+  @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Test
   @Category(RunnableOnService.class)
   public void withLambdaAndTypeDescriptorShouldSucceed() {
-    TestPipeline p = TestPipeline.create();
+
 
     PCollection<String> values = p.apply(Create.of("1234", "3210", "0", "-12"));
     PCollection<KV<Integer, String>> kvs = values.apply(
@@ -59,7 +62,6 @@ public class WithKeysJava8Test {
 
   @Test
   public void withLambdaAndNoTypeDescriptorShouldThrow() {
-    TestPipeline p = TestPipeline.create();
 
     PCollection<String> values = p.apply(Create.of("1234", "3210", "0", "-12"));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4b23d42c/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
index 5f1e74b..a0c6370 100644
--- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
+++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java
@@ -24,6 +24,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -34,15 +35,18 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class WithTimestampsJava8Test implements Serializable {
+
+  @Rule
+  public final transient TestPipeline p = TestPipeline.create();
+
   @Test
   @Category(RunnableOnService.class)
   public void withTimestampsLambdaShouldApplyTimestamps() {
-    TestPipeline p = TestPipeline.create();
 
-    String yearTwoThousand = "946684800000";
+    final String yearTwoThousand = "946684800000";
     PCollection<String> timestamped =
         p.apply(Create.of("1234", "0", Integer.toString(Integer.MAX_VALUE), yearTwoThousand))
-         .apply(WithTimestamps.of((String input) -> new Instant(Long.valueOf(yearTwoThousand))));
+         .apply(WithTimestamps.of((String input) -> new Instant(Long.valueOf(input))));
 
     PCollection<KV<String, Instant>> timestampedVals =
         timestamped.apply(ParDo.of(new DoFn<String, KV<String, Instant>>() {
@@ -58,8 +62,10 @@ public class WithTimestampsJava8Test implements Serializable {
     PAssert.that(timestampedVals)
         .containsInAnyOrder(
             KV.of("0", new Instant(0)),
-            KV.of("1234", new Instant("1234")),
+            KV.of("1234", new Instant(Long.valueOf("1234"))),
             KV.of(Integer.toString(Integer.MAX_VALUE), new Instant(Integer.MAX_VALUE)),
             KV.of(yearTwoThousand, new Instant(Long.valueOf(yearTwoThousand))));
+
+    p.run();
   }
 }


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

Posted by ke...@apache.org.
This closes #1652: Move InMemoryTimerInternals to runners-core

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


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

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

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



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

Posted by ke...@apache.org.
Support set and delete of timer by ID in InMemoryTimerInternals


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

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

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/df2e540d/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
index 5fcd088..292ac23 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
@@ -17,13 +17,15 @@
  */
 package org.apache.beam.runners.core;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.base.MoreObjects;
-import java.util.HashSet;
-import java.util.PriorityQueue;
-import java.util.Set;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Table;
+import java.util.NavigableSet;
+import java.util.TreeSet;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.TimeDomain;
@@ -35,17 +37,17 @@ import org.joda.time.Instant;
 /** {@link TimerInternals} with all watermarks and processing clock simulated in-memory. */
 public class InMemoryTimerInternals implements TimerInternals {
 
-  /** At most one timer per timestamp is kept. */
-  private Set<TimerData> existingTimers = new HashSet<>();
+  /** The current set timers by namespace and ID. */
+  Table<StateNamespace, String, TimerData> existingTimers = HashBasedTable.create();
 
   /** Pending input watermark timers, in timestamp order. */
-  private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
+  private NavigableSet<TimerData> watermarkTimers = new TreeSet<>();
 
   /** Pending processing time timers, in timestamp order. */
-  private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
+  private NavigableSet<TimerData> processingTimers = new TreeSet<>();
 
   /** Pending synchronized processing time timers, in timestamp order. */
-  private PriorityQueue<TimerData> synchronizedProcessingTimers = new PriorityQueue<>(11);
+  private NavigableSet<TimerData> synchronizedProcessingTimers = new TreeSet<>();
 
   /** Current input watermark. */
   private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
@@ -74,13 +76,13 @@ public class InMemoryTimerInternals implements TimerInternals {
     final TimerData data;
     switch (domain) {
       case EVENT_TIME:
-        data = watermarkTimers.peek();
+        data = watermarkTimers.first();
         break;
       case PROCESSING_TIME:
-        data = processingTimers.peek();
+        data = processingTimers.first();
         break;
       case SYNCHRONIZED_PROCESSING_TIME:
-        data = synchronizedProcessingTimers.peek();
+        data = synchronizedProcessingTimers.first();
         break;
       default:
         throw new IllegalArgumentException("Unexpected time domain: " + domain);
@@ -88,7 +90,7 @@ public class InMemoryTimerInternals implements TimerInternals {
     return (data == null) ? null : data.getTimestamp();
   }
 
-  private PriorityQueue<TimerData> queue(TimeDomain domain) {
+  private NavigableSet<TimerData> timersForDomain(TimeDomain domain) {
     switch (domain) {
       case EVENT_TIME:
         return watermarkTimers;
@@ -104,27 +106,45 @@ public class InMemoryTimerInternals implements TimerInternals {
   @Override
   public void setTimer(StateNamespace namespace, String timerId, Instant target,
       TimeDomain timeDomain) {
-    throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
+    setTimer(TimerData.of(timerId, namespace, target, timeDomain));
   }
 
   @Override
   public void setTimer(TimerData timerData) {
     WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData);
-    if (existingTimers.add(timerData)) {
-      queue(timerData.getDomain()).add(timerData);
+
+    @Nullable
+    TimerData existing = existingTimers.get(timerData.getNamespace(), timerData.getTimerId());
+    if (existing == null) {
+      existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData);
+      timersForDomain(timerData.getDomain()).add(timerData);
+    } else {
+      checkArgument(timerData.getDomain().equals(existing.getDomain()),
+          "Attempt to set %s for time domain %s, but it is already set for time domain %s",
+          timerData.getTimerId(), timerData.getDomain(), existing.getDomain());
+
+      if (!timerData.getTimestamp().equals(existing.getTimestamp())) {
+        NavigableSet<TimerData> timers = timersForDomain(timerData.getDomain());
+        timers.remove(existing);
+        timers.add(timerData);
+        existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData);
+      }
     }
   }
 
   @Override
   public void deleteTimer(StateNamespace namespace, String timerId) {
-    throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported.");
+    TimerData existing = existingTimers.get(namespace, timerId);
+    if (existing != null) {
+      deleteTimer(existing);
+    }
   }
 
   @Override
   public void deleteTimer(TimerData timer) {
     WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer);
-    existingTimers.remove(timer);
-    queue(timer.getDomain()).remove(timer);
+    existingTimers.remove(timer.getNamespace(), timer.getTimerId());
+    timersForDomain(timer.getDomain()).remove(timer);
   }
 
   @Override
@@ -261,10 +281,11 @@ public class InMemoryTimerInternals implements TimerInternals {
 
   @Nullable
   private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) {
-    PriorityQueue<TimerData> queue = queue(domain);
-    if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) {
-      TimerData timer = queue.remove();
-      existingTimers.remove(timer);
+    NavigableSet<TimerData> timers = timersForDomain(domain);
+
+    if (!timers.isEmpty() && currentTime.isAfter(timers.first().getTimestamp())) {
+      TimerData timer = timers.pollFirst();
+      existingTimers.remove(timer.getNamespace(), timer.getTimerId());
       return timer;
     } else {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/df2e540d/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
index 2caa874..e711285 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
@@ -19,8 +19,6 @@ package org.apache.beam.runners.core;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 
 import org.apache.beam.sdk.util.TimeDomain;
@@ -39,37 +37,79 @@ import org.junit.runners.JUnit4;
 public class InMemoryTimerInternalsTest {
 
   private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
+  private static final String ID1 = "id1";
+  private static final String ID2 = "id2";
 
   @Test
-  public void testFiringTimers() throws Exception {
+  public void testFiringEventTimers() throws Exception {
     InMemoryTimerInternals underTest = new InMemoryTimerInternals();
-    TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
-    TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+    TimerData eventTimer1 = TimerData.of(ID1, NS1, new Instant(19), TimeDomain.EVENT_TIME);
+    TimerData eventTimer2 = TimerData.of(ID2, NS1, new Instant(29), TimeDomain.EVENT_TIME);
 
-    underTest.setTimer(processingTime1);
-    underTest.setTimer(processingTime2);
+    underTest.setTimer(eventTimer1);
+    underTest.setTimer(eventTimer2);
 
-    underTest.advanceProcessingTime(new Instant(20));
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
+    underTest.advanceInputWatermark(new Instant(20));
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer1));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
 
     // Advancing just a little shouldn't refire
-    underTest.advanceProcessingTime(new Instant(21));
-    assertNull(underTest.removeNextProcessingTimer());
+    underTest.advanceInputWatermark(new Instant(21));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
 
     // Adding the timer and advancing a little should refire
-    underTest.setTimer(processingTime1);
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
+    underTest.setTimer(eventTimer1);
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer1));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
 
     // And advancing the rest of the way should still have the other timer
-    underTest.advanceProcessingTime(new Instant(30));
-    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
+    underTest.advanceInputWatermark(new Instant(30));
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer2));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
+  }
+
+  @Test
+  public void testResetById() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    Instant earlyTimestamp = new Instant(13);
+    Instant laterTimestamp = new Instant(42);
+
+    underTest.advanceInputWatermark(new Instant(0));
+    underTest.setTimer(NS1, ID1, earlyTimestamp, TimeDomain.EVENT_TIME);
+    underTest.setTimer(NS1, ID1, laterTimestamp, TimeDomain.EVENT_TIME);
+    underTest.advanceInputWatermark(earlyTimestamp.plus(1L));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
+
+    underTest.advanceInputWatermark(laterTimestamp.plus(1L));
+    assertThat(
+        underTest.removeNextEventTimer(),
+        equalTo(TimerData.of(ID1, NS1, laterTimestamp, TimeDomain.EVENT_TIME)));
+  }
+
+  @Test
+  public void testDeletionIdempotent() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    Instant timestamp = new Instant(42);
+    underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME);
+    underTest.deleteTimer(NS1, ID1);
+    underTest.deleteTimer(NS1, ID1);
   }
 
   @Test
-  public void testFiringTimersWithCallback() throws Exception {
+  public void testDeletionById() throws Exception {
+    InMemoryTimerInternals underTest = new InMemoryTimerInternals();
+    Instant timestamp = new Instant(42);
+
+    underTest.advanceInputWatermark(new Instant(0));
+    underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME);
+    underTest.deleteTimer(NS1, ID1);
+    underTest.advanceInputWatermark(new Instant(43));
+
+    assertThat(underTest.removeNextEventTimer(), nullValue());
+  }
+
+  @Test
+  public void testFiringProcessingTimeTimers() throws Exception {
     InMemoryTimerInternals underTest = new InMemoryTimerInternals();
     TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
     TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
@@ -116,23 +156,25 @@ public class InMemoryTimerInternalsTest {
     underTest.setTimer(eventTime2);
     underTest.setTimer(synchronizedProcessingTime2);
 
-    assertNull(underTest.removeNextEventTimer());
+    assertThat(underTest.removeNextEventTimer(), nullValue());
     underTest.advanceInputWatermark(new Instant(30));
-    assertEquals(eventTime1, underTest.removeNextEventTimer());
-    assertEquals(eventTime2, underTest.removeNextEventTimer());
-    assertNull(underTest.removeNextEventTimer());
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTime1));
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTime2));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
 
-    assertNull(underTest.removeNextProcessingTimer());
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
     underTest.advanceProcessingTime(new Instant(30));
-    assertEquals(processingTime1, underTest.removeNextProcessingTimer());
-    assertEquals(processingTime2, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1));
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
 
-    assertNull(underTest.removeNextSynchronizedProcessingTimer());
+    assertThat(underTest.removeNextSynchronizedProcessingTimer(), nullValue());
     underTest.advanceSynchronizedProcessingTime(new Instant(30));
-    assertEquals(synchronizedProcessingTime1, underTest.removeNextSynchronizedProcessingTimer());
-    assertEquals(synchronizedProcessingTime2, underTest.removeNextSynchronizedProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
+    assertThat(
+        underTest.removeNextSynchronizedProcessingTimer(), equalTo(synchronizedProcessingTime1));
+    assertThat(
+        underTest.removeNextSynchronizedProcessingTimer(), equalTo(synchronizedProcessingTime2));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
   }
 
   @Test
@@ -147,9 +189,9 @@ public class InMemoryTimerInternalsTest {
     underTest.advanceProcessingTime(new Instant(20));
     underTest.advanceInputWatermark(new Instant(20));
 
-    assertEquals(processingTime, underTest.removeNextProcessingTimer());
-    assertNull(underTest.removeNextProcessingTimer());
-    assertEquals(eventTime, underTest.removeNextEventTimer());
-    assertNull(underTest.removeNextEventTimer());
+    assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime));
+    assertThat(underTest.removeNextProcessingTimer(), nullValue());
+    assertThat(underTest.removeNextEventTimer(), equalTo(eventTime));
+    assertThat(underTest.removeNextEventTimer(), nullValue());
   }
 }


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

Posted by ke...@apache.org.
Restore SDK's InMemoryTimerInternals, deprecated


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

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

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


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


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

Posted by ke...@apache.org.
Update Dataflow worker to beam-master-20161221


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

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

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


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


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

Posted by ke...@apache.org.
Provide local tags in PInput, POutput expansions

Output an ordered colleciton in PInput and POutput expansions.

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

Implement PCollectionList.equals, PCollectionTuple.equals


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

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

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


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index cb9ddd8..bbcab8e 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -53,6 +53,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 import org.joda.time.Instant;
 
 /**
@@ -419,9 +420,9 @@ class EvaluationContext {
     }
     // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down,
     // the PTransform may produce additional output. It is not done.
-    for (PValue output : transform.getOutput().expand()) {
-      if (output instanceof PCollection) {
-        IsBounded bounded = ((PCollection<?>) output).isBounded();
+    for (TaggedPValue output : transform.getOutput().expand()) {
+      if (output.getValue() instanceof PCollection) {
+        IsBounded bounded = ((PCollection<?>) output.getValue()).isBounded();
         if (bounded.equals(IsBounded.UNBOUNDED)
             && !options.isShutdownUnboundedProducersWithMaxWatermark()) {
           return false;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 65c41e0..32eb692 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -18,11 +18,10 @@
 package org.apache.beam.runners.direct;
 
 import static com.google.common.base.Preconditions.checkState;
-import static com.google.common.base.Predicates.in;
-import static com.google.common.collect.Iterables.all;
 
 import com.google.common.collect.ImmutableSet;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
@@ -33,6 +32,7 @@ import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 
 /**
  * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it
@@ -83,7 +83,10 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
     if (node.isRootNode()) {
       finalized = true;
     } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) {
-      keyedValues.addAll(node.getOutputs());
+      List<TaggedPValue> outputs = node.getOutputs();
+      for (TaggedPValue output : outputs) {
+        keyedValues.add(output.getValue());
+      }
     }
   }
 
@@ -92,9 +95,12 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
 
   @Override
   public void visitValue(PValue value, TransformHierarchy.Node producer) {
+    boolean inputsAreKeyed = true;
+    for (TaggedPValue input : producer.getInputs()) {
+      inputsAreKeyed = inputsAreKeyed && keyedValues.contains(input.getValue());
+    }
     if (PRODUCES_KEYED_OUTPUTS.contains(producer.getTransform().getClass())
-        || (isKeyPreserving(producer.getTransform())
-            && all(producer.getInputs(), in(keyedValues)))) {
+        || (isKeyPreserving(producer.getTransform()) && inputsAreKeyed)) {
       keyedValues.add(value);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/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 247b1cc..7bed751 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
@@ -57,7 +57,7 @@ import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 import org.joda.time.Instant;
 
 /**
@@ -755,13 +755,14 @@ public class WatermarkManager {
 
   private Collection<Watermark> getInputProcessingWatermarks(AppliedPTransform<?, ?, ?> transform) {
     ImmutableList.Builder<Watermark> inputWmsBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
+    List<TaggedPValue> inputs = transform.getInput().expand();
     if (inputs.isEmpty()) {
       inputWmsBuilder.add(THE_END_OF_TIME);
     }
-    for (PValue pvalue : inputs) {
+    for (TaggedPValue pvalue : inputs) {
       Watermark producerOutputWatermark =
-          getTransformWatermark(graph.getProducer(pvalue)).synchronizedProcessingOutputWatermark;
+          getTransformWatermark(graph.getProducer(pvalue.getValue()))
+              .synchronizedProcessingOutputWatermark;
       inputWmsBuilder.add(producerOutputWatermark);
     }
     return inputWmsBuilder.build();
@@ -769,13 +770,13 @@ public class WatermarkManager {
 
   private List<Watermark> getInputWatermarks(AppliedPTransform<?, ?, ?> transform) {
     ImmutableList.Builder<Watermark> inputWatermarksBuilder = ImmutableList.builder();
-    Collection<? extends PValue> inputs = transform.getInput().expand();
+    List<TaggedPValue> inputs = transform.getInput().expand();
     if (inputs.isEmpty()) {
       inputWatermarksBuilder.add(THE_END_OF_TIME);
     }
-    for (PValue pvalue : inputs) {
+    for (TaggedPValue pvalue : inputs) {
       Watermark producerOutputWatermark =
-          getTransformWatermark(graph.getProducer(pvalue)).outputWatermark;
+          getTransformWatermark(graph.getProducer(pvalue.getValue())).outputWatermark;
       inputWatermarksBuilder.add(producerOutputWatermark);
     }
     List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
@@ -959,8 +960,8 @@ public class WatermarkManager {
     WatermarkUpdate updateResult = myWatermarks.refresh();
     if (updateResult.isAdvanced()) {
       Set<AppliedPTransform<?, ?, ?>> additionalRefreshes = new HashSet<>();
-      for (PValue outputPValue : toRefresh.getOutput().expand()) {
-        additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue));
+      for (TaggedPValue outputPValue : toRefresh.getOutput().expand()) {
+        additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue.getValue()));
       }
       return additionalRefreshes;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/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 3d98b87..92c07bb 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
@@ -48,6 +48,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 import org.apache.spark.Accumulator;
 import org.apache.spark.SparkEnv$;
 import org.apache.spark.api.java.JavaSparkContext;
@@ -282,7 +283,7 @@ public final class SparkRunner extends PipelineRunner<SparkPipelineResult> {
       if (node.getInputs().size() != 1) {
         return false;
       }
-      PValue input = Iterables.getOnlyElement(node.getInputs());
+      PValue input = Iterables.getOnlyElement(node.getInputs()).getValue();
       if (!(input instanceof PCollection)
           || ((PCollection) input).getWindowingStrategy().getWindowFn().isNonMerging()) {
         return false;
@@ -338,7 +339,7 @@ public final class SparkRunner extends PipelineRunner<SparkPipelineResult> {
       //--- determine if node is bounded/unbounded.
       // usually, the input determines if the PCollection to apply the next transformation to
       // is BOUNDED or UNBOUNDED, meaning RDD/DStream.
-      Collection<? extends PValue> pValues;
+      Collection<TaggedPValue> pValues;
       if (node.getInputs().isEmpty()) {
         // in case of a PBegin, it's the output.
         pValues = node.getOutputs();
@@ -353,15 +354,15 @@ public final class SparkRunner extends PipelineRunner<SparkPipelineResult> {
               : translator.translateUnbounded(transformClass);
     }
 
-    private PCollection.IsBounded isBoundedCollection(Collection<? extends PValue> pValues) {
+    private PCollection.IsBounded isBoundedCollection(Collection<TaggedPValue> pValues) {
       // anything that is not a PCollection, is BOUNDED.
       // For PCollections:
       // BOUNDED behaves as the Identity Element, BOUNDED + BOUNDED = BOUNDED
       // while BOUNDED + UNBOUNDED = UNBOUNDED.
       PCollection.IsBounded isBounded = PCollection.IsBounded.BOUNDED;
-      for (PValue pValue: pValues) {
-        if (pValue instanceof PCollection) {
-          isBounded = isBounded.and(((PCollection) pValue).isBounded());
+      for (TaggedPValue pValue: pValues) {
+        if (pValue.getValue() instanceof PCollection) {
+          isBounded = isBounded.and(((PCollection) pValue.getValue()).isBounded());
         } else {
           isBounded = isBounded.and(PCollection.IsBounded.BOUNDED);
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index 33d5231..29e7fcb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -37,6 +37,7 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 
 /**
  * Captures information about a collection of transformations and their
@@ -84,10 +85,12 @@ public class TransformHierarchy {
    */
   public void finishSpecifyingInput() {
     // Inputs must be completely specified before they are consumed by a transform.
-    for (PValue inputValue : current.getInputs()) {
-      inputValue.finishSpecifying();
-      checkState(producers.get(inputValue) != null, "Producer unknown for input %s", inputValue);
-      inputValue.finishSpecifying();
+    for (TaggedPValue inputValue : current.getInputs()) {
+      inputValue.getValue().finishSpecifying();
+      checkState(
+          producers.get(inputValue.getValue()) != null,
+          "Producer unknown for input %s",
+          inputValue.getValue());
     }
   }
 
@@ -103,9 +106,9 @@ public class TransformHierarchy {
    */
   public void setOutput(POutput output) {
     output.finishSpecifyingOutput();
-    for (PValue value : output.expand()) {
-      if (!producers.containsKey(value)) {
-        producers.put(value, current);
+    for (TaggedPValue value : output.expand()) {
+      if (!producers.containsKey(value.getValue())) {
+        producers.put(value.getValue(), current);
       }
     }
     current.setOutput(output);
@@ -133,8 +136,8 @@ public class TransformHierarchy {
    */
   List<Node> getProducingTransforms(POutput output) {
     List<Node> producingTransforms = new ArrayList<>();
-    for (PValue value : output.expand()) {
-      Node producer = getProducer(value);
+    for (TaggedPValue value : output.expand()) {
+      Node producer = getProducer(value.getValue());
       if (producer != null) {
         producingTransforms.add(producer);
       }
@@ -238,8 +241,8 @@ public class TransformHierarchy {
     private boolean returnsOthersOutput() {
       PTransform<?, ?> transform = getTransform();
       if (output != null) {
-        for (PValue outputValue : output.expand()) {
-          if (!getProducer(outputValue).getTransform().equals(transform)) {
+        for (TaggedPValue outputValue : output.expand()) {
+          if (!getProducer(outputValue.getValue()).getTransform().equals(transform)) {
             return true;
           }
         }
@@ -256,8 +259,8 @@ public class TransformHierarchy {
     }
 
     /** Returns the transform input, in unexpanded form. */
-    public Collection<? extends PValue> getInputs() {
-      return input == null ? Collections.<PValue>emptyList() : input.expand();
+    public List<TaggedPValue> getInputs() {
+      return input == null ? Collections.<TaggedPValue>emptyList() : input.expand();
     }
 
     /**
@@ -273,8 +276,8 @@ public class TransformHierarchy {
       // Validate that a primitive transform produces only primitive output, and a composite
       // transform does not produce primitive output.
       Set<Node> outputProducers = new HashSet<>();
-      for (PValue outputValue : output.expand()) {
-        outputProducers.add(getProducer(outputValue));
+      for (TaggedPValue outputValue : output.expand()) {
+        outputProducers.add(getProducer(outputValue.getValue()));
       }
       if (outputProducers.contains(this) && outputProducers.size() != 1) {
         Set<String> otherProducerNames = new HashSet<>();
@@ -296,8 +299,8 @@ public class TransformHierarchy {
     }
 
     /** Returns the transform output, in unexpanded form. */
-    public Collection<? extends PValue> getOutputs() {
-      return output == null ? Collections.<PValue>emptyList() : output.expand();
+    public List<TaggedPValue> getOutputs() {
+      return output == null ? Collections.<TaggedPValue>emptyList() : output.expand();
     }
 
     /**
@@ -320,9 +323,9 @@ public class TransformHierarchy {
 
       if (!isRootNode()) {
         // Visit inputs.
-        for (PValue inputValue : input.expand()) {
-          if (visitedValues.add(inputValue)) {
-            visitor.visitValue(inputValue, getProducer(inputValue));
+        for (TaggedPValue inputValue : input.expand()) {
+          if (visitedValues.add(inputValue.getValue())) {
+            visitor.visitValue(inputValue.getValue(), getProducer(inputValue.getValue()));
           }
         }
       }
@@ -342,9 +345,9 @@ public class TransformHierarchy {
 
       if (!isRootNode()) {
         // Visit outputs.
-        for (PValue pValue : output.expand()) {
-          if (visitedValues.add(pValue)) {
-            visitor.visitValue(pValue, this);
+        for (TaggedPValue pValue : output.expand()) {
+          if (visitedValues.add(pValue.getValue())) {
+            visitor.visitValue(pValue.getValue(), this);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
index 67b819f..13d4ee1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.transforms.join;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
@@ -28,7 +27,7 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
@@ -121,10 +120,10 @@ public class KeyedPCollectionTuple<K> implements PInput {
    * any tag-specific information.
    */
   @Override
-  public Collection<? extends PValue> expand() {
-    List<PCollection<?>> retval = new ArrayList<>();
+  public List<TaggedPValue> expand() {
+    List<TaggedPValue> retval = new ArrayList<>();
     for (TaggedKeyedPCollection<K, ?> taggedPCollection : keyedCollections) {
-      retval.add(taggedPCollection.pCollection);
+      retval.add(TaggedPValue.of(taggedPCollection.tupleTag, taggedPCollection.pCollection));
     }
     return retval;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
index f1dbb37..9aa4615 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO.Read;
 import org.apache.beam.sdk.transforms.Create;
@@ -64,7 +64,7 @@ public class PBegin implements PInput {
   }
 
   @Override
-  public Collection<? extends PValue> expand() {
+  public List<TaggedPValue> expand() {
     // A PBegin contains no PValues.
     return Collections.emptyList();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
index 4c9e220..e4bb7c5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java
@@ -18,11 +18,10 @@
 package org.apache.beam.sdk.values;
 
 import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
+import com.google.common.collect.Iterables;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Objects;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Flatten;
@@ -115,9 +114,9 @@ public class PCollectionList<T> implements PInput, POutput {
           "PCollections come from different Pipelines");
     }
     return new PCollectionList<>(pipeline,
-        new ImmutableList.Builder<PCollection<T>>()
+        ImmutableList.<TaggedPValue>builder()
             .addAll(pcollections)
-            .add(pc)
+            .add(Iterables.getOnlyElement(pc.expand()))
             .build());
   }
 
@@ -130,15 +129,16 @@ public class PCollectionList<T> implements PInput, POutput {
    * part of the same {@link Pipeline}.
    */
   public PCollectionList<T> and(Iterable<PCollection<T>> pcs) {
-    List<PCollection<T>> copy = new ArrayList<>(pcollections);
+    ImmutableList.Builder<TaggedPValue> builder = ImmutableList.builder();
+    builder.addAll(pcollections);
     for (PCollection<T> pc : pcs) {
       if (pc.getPipeline() != pipeline) {
         throw new IllegalArgumentException(
             "PCollections come from different Pipelines");
       }
-      copy.add(pc);
+      builder.add(Iterables.getOnlyElement(pc.expand()));
     }
-    return new PCollectionList<>(pipeline, copy);
+    return new PCollectionList<>(pipeline, builder.build());
   }
 
   /**
@@ -155,7 +155,9 @@ public class PCollectionList<T> implements PInput, POutput {
    * {@code [0..size()-1]}.
    */
   public PCollection<T> get(int index) {
-    return pcollections.get(index);
+    @SuppressWarnings("unchecked") // Type-safe by construction
+    PCollection<T> value = (PCollection<T>) pcollections.get(index).getValue();
+    return value;
   }
 
   /**
@@ -163,7 +165,13 @@ public class PCollectionList<T> implements PInput, POutput {
    * {@link PCollectionList}.
    */
   public List<PCollection<T>> getAll() {
-    return pcollections;
+    ImmutableList.Builder<PCollection<T>> res = ImmutableList.builder();
+    for (TaggedPValue value : pcollections) {
+      @SuppressWarnings("unchecked") // Type-safe by construction
+      PCollection<T> typedValue = (PCollection<T>) value.getValue();
+      res.add(typedValue);
+    }
+    return res.build();
   }
 
   /**
@@ -192,15 +200,16 @@ public class PCollectionList<T> implements PInput, POutput {
   // Internal details below here.
 
   final Pipeline pipeline;
-  final List<PCollection<T>> pcollections;
+  // ImmutableMap has a defined iteration order.
+  final List<TaggedPValue> pcollections;
 
   PCollectionList(Pipeline pipeline) {
-    this(pipeline, new ArrayList<PCollection<T>>());
+    this(pipeline, ImmutableList.<TaggedPValue>of());
   }
 
-  PCollectionList(Pipeline pipeline, List<PCollection<T>> pcollections) {
+  PCollectionList(Pipeline pipeline, List<TaggedPValue> values) {
     this.pipeline = pipeline;
-    this.pcollections = Collections.unmodifiableList(pcollections);
+    this.pcollections = ImmutableList.copyOf(values);
   }
 
   @Override
@@ -209,14 +218,16 @@ public class PCollectionList<T> implements PInput, POutput {
   }
 
   @Override
-  public Collection<? extends PValue> expand() {
+  public List<TaggedPValue> expand() {
     return pcollections;
   }
 
   @Override
   public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) {
     int i = 0;
-    for (PCollection<T> pc : pcollections) {
+    for (TaggedPValue tpv : pcollections) {
+      @SuppressWarnings("unchecked")
+      PCollection<T> pc = (PCollection<T>) tpv.getValue();
       pc.recordAsOutput(transform, "out" + i);
       i++;
     }
@@ -224,15 +235,29 @@ public class PCollectionList<T> implements PInput, POutput {
 
   @Override
   public void finishSpecifying() {
-    for (PCollection<T> pc : pcollections) {
-      pc.finishSpecifying();
+    for (TaggedPValue pc : pcollections) {
+      pc.getValue().finishSpecifying();
     }
   }
 
   @Override
   public void finishSpecifyingOutput() {
-    for (PCollection<T> pc : pcollections) {
-      pc.finishSpecifyingOutput();
+    for (TaggedPValue pc : pcollections) {
+      pc.getValue().finishSpecifyingOutput();
     }
   }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof PCollectionList)) {
+      return false;
+    }
+    PCollectionList that = (PCollectionList) other;
+    return this.pipeline.equals(that.pipeline) && this.pcollections.equals(that.pcollections);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.pipeline, this.pcollections);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
index 727d882..6afe59e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java
@@ -17,11 +17,13 @@
  */
 package org.apache.beam.sdk.values;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -178,7 +180,7 @@ public class PCollectionTuple implements PInput, POutput {
   /////////////////////////////////////////////////////////////////////////////
   // Internal details below here.
 
-  Pipeline pipeline;
+  final Pipeline pipeline;
   final Map<TupleTag<?>, PCollection<?>> pcollectionMap;
 
   PCollectionTuple(Pipeline pipeline) {
@@ -232,8 +234,12 @@ public class PCollectionTuple implements PInput, POutput {
   }
 
   @Override
-  public Collection<? extends PValue> expand() {
-    return pcollectionMap.values();
+  public List<TaggedPValue> expand() {
+    ImmutableList.Builder<TaggedPValue> values = ImmutableList.builder();
+    for (Map.Entry<TupleTag<?>, PCollection<?>> entry : pcollectionMap.entrySet()) {
+      values.add(TaggedPValue.of(entry.getKey(), entry.getValue()));
+    }
+    return values.build();
   }
 
   @Override
@@ -261,4 +267,18 @@ public class PCollectionTuple implements PInput, POutput {
       pc.finishSpecifyingOutput();
     }
   }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof PCollectionTuple)) {
+      return false;
+    }
+    PCollectionTuple that = (PCollectionTuple) other;
+    return this.pipeline.equals(that.pipeline) && this.pcollectionMap.equals(that.pcollectionMap);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(this.pipeline, this.pcollectionMap);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
index 9e8cae4..b4a3025 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.PTransform;
 
@@ -36,7 +36,7 @@ public class PDone extends POutputValueBase {
   }
 
   @Override
-  public Collection<? extends PValue> expand() {
+  public List<TaggedPValue> expand() {
     // A PDone contains no PValues.
     return Collections.emptyList();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
index f938aeb..a27b939 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Collection;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 
 /**
@@ -43,7 +43,7 @@ public interface PInput {
    *
    * <p>Not intended to be invoked directly by user code.
    */
-  Collection<? extends PValue> expand();
+  List<TaggedPValue> expand();
 
   /**
    * After building, finalizes this {@code PInput} to make it ready for

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
index 27a280f..e5d4504 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Collection;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -45,7 +45,7 @@ public interface POutput {
    *
    * <p>Not intended to be invoked directly by user code.
    */
-  Collection<? extends PValue> expand();
+  List<TaggedPValue> expand();
 
   /**
    * Records that this {@code POutput} is an output of the given

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
index 0cee2ca..e6dbaf7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.values;
 
+import java.util.List;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 
@@ -36,4 +37,13 @@ public interface PValue extends POutput, PInput {
    * <p>For internal use only.
    */
   AppliedPTransform<?, ?, ?> getProducingTransformInternal();
+
+  /**
+   * {@inheritDoc}.
+   *
+   * <p>A {@link PValue} always expands into itself. Calling {@link #expand()} on a PValue is almost
+   * never appropriate.
+   */
+  @Deprecated
+  List<TaggedPValue> expand();
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
index 685e32f..3a10d5d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -91,6 +91,11 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   private String name;
 
   /**
+   * A local {@link TupleTag} used in the expansion of this {@link PValueBase}.
+   */
+  private TupleTag<?> tag = new TupleTag<>();
+
+  /**
    * Whether this {@link PValueBase} has been finalized, and its core
    * properties, e.g., name, can no longer be changed.
    */
@@ -128,8 +133,8 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   }
 
   @Override
-  public Collection<? extends PValue> expand() {
-    return Collections.singletonList(this);
+  public final List<TaggedPValue> expand() {
+    return Collections.singletonList(TaggedPValue.of(tag, this));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
new file mode 100644
index 0000000..458d16f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.beam.sdk.values;
+
+import com.google.auto.value.AutoValue;
+
+/**
+ * A (TupleTag, PValue) pair used in the expansion of a {@link PInput} or {@link POutput}.
+ */
+@AutoValue
+public abstract class TaggedPValue {
+  public static TaggedPValue of(TupleTag<?> tag, PValue value) {
+    return new AutoValue_TaggedPValue(tag, value);
+  }
+
+  /**
+   * Returns the local tag associated with the {@link PValue}.
+   */
+  public abstract TupleTag<?> getTag();
+
+  /**
+   * Returns the {@link PValue}.
+   */
+  public abstract PValue getValue();
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
index 2327459..d790d39 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
@@ -22,7 +22,10 @@ import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.io.CountingSource;
@@ -38,6 +41,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TaggedPValue;
 import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Rule;
@@ -181,14 +185,16 @@ public class TransformHierarchyTest {
     assertThat(hierarchy.getCurrent(), equalTo(primitiveNode));
     hierarchy.setOutput(created);
     hierarchy.popNode();
-    assertThat(primitiveNode.getOutputs(), Matchers.<PValue>containsInAnyOrder(created));
-    assertThat(primitiveNode.getInputs(), Matchers.<PValue>emptyIterable());
+    assertThat(
+        fromTaggedValues(primitiveNode.getOutputs()), Matchers.<PValue>containsInAnyOrder(created));
+    assertThat(primitiveNode.getInputs(), Matchers.<TaggedPValue>emptyIterable());
     assertThat(primitiveNode.getTransform(), Matchers.<PTransform<?, ?>>equalTo(read));
     assertThat(primitiveNode.getEnclosingNode(), equalTo(compositeNode));
 
     hierarchy.setOutput(created);
     // The composite is listed as outputting a PValue created by the contained primitive
-    assertThat(compositeNode.getOutputs(), Matchers.<PValue>containsInAnyOrder(created));
+    assertThat(
+        fromTaggedValues(compositeNode.getOutputs()), Matchers.<PValue>containsInAnyOrder(created));
     // The producer of that PValue is still the primitive in which it is first output
     assertThat(hierarchy.getProducer(created), equalTo(primitiveNode));
     hierarchy.popNode();
@@ -226,4 +232,15 @@ public class TransformHierarchyTest {
     assertThat(visitedValuesInVisitor, Matchers.<PValue>containsInAnyOrder(created, mapped));
     assertThat(visitedValuesInVisitor, equalTo(visitedValues));
   }
+
+  private static List<PValue> fromTaggedValues(List<TaggedPValue> taggedValues) {
+    return Lists.transform(
+        taggedValues,
+        new Function<TaggedPValue, PValue>() {
+          @Override
+          public PValue apply(TaggedPValue input) {
+            return input.getValue();
+          }
+        });
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 3a47fc7..fa8874c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -29,6 +29,7 @@ import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 import static org.junit.Assert.assertArrayEquals;
@@ -50,6 +51,7 @@ import java.util.List;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
@@ -86,6 +88,7 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
+import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.MutableDateTime;
@@ -864,6 +867,37 @@ public class ParDoTest implements Serializable {
   }
 
   @Test
+  public void testMultiOutputAppliedMultipleTimesDifferentOutputs() {
+    pipeline.enableAbandonedNodeEnforcement(false);
+    PCollection<Long> longs = pipeline.apply(CountingInput.unbounded());
+
+    TupleTag<Long> mainOut = new TupleTag<>();
+    final TupleTag<String> sideOutOne = new TupleTag<>();
+    final TupleTag<Integer> sideOutTwo = new TupleTag<>();
+    DoFn<Long, Long> fn =
+        new DoFn<Long, Long>() {
+          @ProcessElement
+          public void processElement(ProcessContext cxt) {
+            cxt.output(cxt.element());
+            cxt.sideOutput(sideOutOne, Long.toString(cxt.element()));
+            cxt.sideOutput(sideOutTwo, Long.valueOf(cxt.element()).intValue());
+          }
+        };
+
+    ParDo.BoundMulti<Long, Long> parDo =
+        ParDo.of(fn).withOutputTags(mainOut, TupleTagList.of(sideOutOne).and(sideOutTwo));
+    PCollectionTuple firstApplication = longs.apply("first", parDo);
+    PCollectionTuple secondApplication = longs.apply("second", parDo);
+    assertThat(firstApplication, not(equalTo(secondApplication)));
+    assertThat(
+        firstApplication.getAll().keySet(),
+        Matchers.<TupleTag<?>>containsInAnyOrder(mainOut, sideOutOne, sideOutTwo));
+    assertThat(
+        secondApplication.getAll().keySet(),
+        Matchers.<TupleTag<?>>containsInAnyOrder(mainOut, sideOutOne, sideOutTwo));
+  }
+
+  @Test
   @Category(RunnableOnService.class)
   public void testParDoInCustomTransform() {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
index f76bf7e..2482f32 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java
@@ -18,10 +18,22 @@
 package org.apache.beam.sdk.values;
 
 import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.testing.EqualsTester;
 import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.io.CountingInput.BoundedCountingInput;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.hamcrest.Matchers;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -44,4 +56,109 @@ public class PCollectionListTest {
               + "or must first call empty(Pipeline)"));
     }
   }
+
+  @Test
+  public void testIterationOrder() {
+    Pipeline p = TestPipeline.create();
+    PCollection<Long> createOne = p.apply("CreateOne", Create.of(1L, 2L, 3L));
+    PCollection<Long> boundedCount = p.apply("CountBounded", CountingInput.upTo(23L));
+    PCollection<Long> unboundedCount = p.apply("CountUnbounded", CountingInput.unbounded());
+    PCollection<Long> createTwo = p.apply("CreateTwo", Create.of(-1L, -2L));
+    PCollection<Long> maxRecordsCount =
+        p.apply("CountLimited", CountingInput.unbounded().withMaxNumRecords(22L));
+
+    ImmutableList<PCollection<Long>> counts =
+        ImmutableList.of(boundedCount, maxRecordsCount, unboundedCount);
+    // Build a PCollectionList from a list. This should have the same order as the input list.
+    PCollectionList<Long> pcList = PCollectionList.of(counts);
+    // Contains is the order-dependent matcher
+    assertThat(
+        pcList.getAll(),
+        contains(boundedCount, maxRecordsCount, unboundedCount));
+
+    // A list that is expanded with builder methods has the added value at the end
+    PCollectionList<Long> withOneCreate = pcList.and(createTwo);
+    assertThat(
+        withOneCreate.getAll(), contains(boundedCount, maxRecordsCount, unboundedCount, createTwo));
+
+    // Lists that are built entirely from the builder return outputs in the order they were added
+    PCollectionList<Long> fromEmpty =
+        PCollectionList.<Long>empty(p)
+            .and(unboundedCount)
+            .and(createOne)
+            .and(ImmutableList.of(boundedCount, maxRecordsCount));
+    assertThat(
+        fromEmpty.getAll(), contains(unboundedCount, createOne, boundedCount, maxRecordsCount));
+
+    List<TaggedPValue> expansion = fromEmpty.expand();
+    // TaggedPValues are stable between expansions
+    assertThat(expansion, equalTo(fromEmpty.expand()));
+    // TaggedPValues are equivalent between equivalent lists
+    assertThat(
+        expansion,
+        equalTo(
+            PCollectionList.of(unboundedCount)
+                .and(createOne)
+                .and(boundedCount)
+                .and(maxRecordsCount)
+                .expand()));
+
+    List<PCollection<Long>> expectedList =
+        ImmutableList.of(unboundedCount, createOne, boundedCount, maxRecordsCount);
+    for (int i = 0; i < expansion.size(); i++) {
+      assertThat(
+          "Index " + i + " should have equal PValue",
+          expansion.get(i).getValue(),
+          Matchers.<PValue>equalTo(expectedList.get(i)));
+    }
+  }
+
+  @Test
+  public void testEquals() {
+    Pipeline p = TestPipeline.create();
+    PCollection<String> first = p.apply("Meta", Create.of("foo", "bar"));
+    PCollection<String> second = p.apply("Pythonic", Create.of("spam, ham"));
+    PCollection<String> third = p.apply("Syntactic", Create.of("eggs", "baz"));
+
+    EqualsTester tester = new EqualsTester();
+    tester.addEqualityGroup(PCollectionList.empty(p), PCollectionList.empty(p));
+    tester.addEqualityGroup(PCollectionList.of(first).and(second));
+    // Constructors should all produce equivalent
+    tester.addEqualityGroup(
+        PCollectionList.of(first).and(second).and(third),
+        PCollectionList.of(first).and(second).and(third),
+        PCollectionList.<String>empty(p).and(first).and(second).and(third),
+        PCollectionList.of(ImmutableList.of(first, second, third)),
+        PCollectionList.of(first).and(ImmutableList.of(second, third)),
+        PCollectionList.of(ImmutableList.of(first, second)).and(third));
+    // Order is considered
+    tester.addEqualityGroup(PCollectionList.of(first).and(third).and(second));
+    tester.addEqualityGroup(PCollectionList.empty(TestPipeline.create()));
+
+    tester.testEquals();
+  }
+
+  @Test
+  public void testExpansionOrderWithDuplicates() {
+    TestPipeline p = TestPipeline.create();
+    BoundedCountingInput count = CountingInput.upTo(10L);
+    PCollection<Long> firstCount = p.apply("CountFirst", count);
+    PCollection<Long> secondCount = p.apply("CountSecond", count);
+
+    PCollectionList<Long> counts =
+        PCollectionList.of(firstCount).and(secondCount).and(firstCount).and(firstCount);
+
+    ImmutableList<PCollection<Long>> expectedOrder =
+        ImmutableList.of(firstCount, secondCount, firstCount, firstCount);
+    PCollectionList<Long> reconstructed = PCollectionList.empty(p);
+    assertThat(counts.expand(), hasSize(4));
+    for (int i = 0; i < 4; i++) {
+      PValue value = counts.expand().get(i).getValue();
+      assertThat(
+          "Index " + i + " should be equal", value,
+          Matchers.<PValue>equalTo(expectedOrder.get(i)));
+      reconstructed = reconstructed.and((PCollection<Long>) value);
+    }
+    assertThat(reconstructed, equalTo(counts));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/34373c21/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
index b5351da..7d767cf 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java
@@ -17,21 +17,31 @@
  */
 package org.apache.beam.sdk.values;
 
+import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.testing.EqualsTester;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.RunnableOnService;
 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.MapElements;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.hamcrest.Matchers;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -93,4 +103,64 @@ public final class PCollectionTupleTest implements Serializable {
     pipeline.run();
   }
 
+  @Test
+  public void testEquals() {
+    TestPipeline p = TestPipeline.create();
+    TupleTag<Long> longTag = new TupleTag<>();
+    PCollection<Long> longs = p.apply(CountingInput.unbounded());
+    TupleTag<String> strTag = new TupleTag<>();
+    PCollection<String> strs = p.apply(Create.of("foo", "bar"));
+
+    EqualsTester tester = new EqualsTester();
+    // Empty tuples in the same pipeline are equal
+    tester.addEqualityGroup(PCollectionTuple.empty(p), PCollectionTuple.empty(p));
+
+    tester.addEqualityGroup(PCollectionTuple.of(longTag, longs).and(strTag, strs),
+        PCollectionTuple.of(longTag, longs).and(strTag, strs));
+
+    tester.addEqualityGroup(PCollectionTuple.of(longTag, longs));
+    tester.addEqualityGroup(PCollectionTuple.of(strTag, strs));
+
+    TestPipeline otherPipeline = TestPipeline.create();
+    // Empty tuples in different pipelines are not equal
+    tester.addEqualityGroup(PCollectionTuple.empty(otherPipeline));
+    tester.testEquals();
+  }
+
+  @Test
+  public void testExpandHasMatchingTags() {
+    TupleTag<Integer> intTag = new TupleTag<>();
+    TupleTag<String> strTag = new TupleTag<>();
+    TupleTag<Long> longTag = new TupleTag<>();
+
+    Pipeline p = TestPipeline.create();
+    PCollection<Long> longs = p.apply(CountingInput.upTo(100L));
+    PCollection<String> strs = p.apply(Create.of("foo", "bar", "baz"));
+    PCollection<Integer> ints = longs.apply(MapElements.via(new SimpleFunction<Long, Integer>() {
+      @Override
+      public Integer apply(Long input) {
+        return input.intValue();
+      }
+    }));
+
+    Map<TupleTag<?>, PCollection<?>> pcsByTag =
+        ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
+            .put(strTag, strs)
+            .put(intTag, ints)
+            .put(longTag, longs)
+            .build();
+    PCollectionTuple tuple =
+        PCollectionTuple.of(intTag, ints).and(longTag, longs).and(strTag, strs);
+    assertThat(tuple.getAll(), equalTo(pcsByTag));
+    PCollectionTuple reconstructed = PCollectionTuple.empty(p);
+    for (TaggedPValue taggedValue : tuple.expand()) {
+      TupleTag<?> tag = taggedValue.getTag();
+      PValue value = taggedValue.getValue();
+      assertThat("The tag should map back to the value", tuple.get(tag), equalTo(value));
+      assertThat(value, Matchers.<PValue>equalTo(pcsByTag.get(tag)));
+      reconstructed = reconstructed.and(tag, (PCollection) value);
+    }
+
+    assertThat(reconstructed, equalTo(tuple));
+  }
 }