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

[01/50] incubator-beam git commit: Add access to original DoFn on ParDo

Repository: incubator-beam
Updated Branches:
  refs/heads/apex-runner c9f140666 -> 0a1b27895


Add access to original DoFn on 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/0db32631
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/0db32631
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/0db32631

Branch: refs/heads/apex-runner
Commit: 0db32631238b5ec5cacf91b8ecb81e8fda99963a
Parents: 44878e5
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 20:47:26 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:32:06 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/SimpleDoFnRunner.java     |   8 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   | 248 ++++++++++++-------
 2 files changed, 161 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0db32631/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 d5ecbc9..4eda376 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
@@ -79,7 +79,7 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
 
   private final TupleTag<OutputT> mainOutputTag;
 
-  private final boolean ignoresWindow;
+  private final boolean observesWindow;
 
   public SimpleDoFnRunner(
       PipelineOptions options,
@@ -92,8 +92,8 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
       AggregatorFactory aggregatorFactory,
       WindowingStrategy<?, ?> windowingStrategy) {
     this.fn = fn;
-    this.ignoresWindow =
-        !DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().usesSingleWindow();
+    this.observesWindow =
+        DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().observesWindow();
     this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn);
     this.outputManager = outputManager;
     this.mainOutputTag = mainOutputTag;
@@ -123,7 +123,7 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
 
   @Override
   public void processElement(WindowedValue<InputT> compressedElem) {
-    if (ignoresWindow) {
+    if (observesWindow) {
       invokeProcessElement(compressedElem);
     } else {
       for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0db32631/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index 93eb1ac..a3a306a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -27,8 +27,10 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
@@ -521,7 +523,7 @@ public class ParDo {
    */
   public static <InputT, OutputT> Bound<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
     validate(fn);
-    return of(adapt(fn), displayDataForFn(fn));
+    return new Unbound().of(fn, displayDataForFn(fn));
   }
 
   /**
@@ -538,17 +540,11 @@ public class ParDo {
    */
   @Deprecated
   public static <InputT, OutputT> Bound<InputT, OutputT> of(OldDoFn<InputT, OutputT> fn) {
-    return of(fn, displayDataForFn(fn));
-  }
-
-  private static <InputT, OutputT> Bound<InputT, OutputT> of(
-          OldDoFn<InputT, OutputT> fn, DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
-    return new Unbound().of(fn, fnDisplayData);
+    return new Unbound().of(fn, displayDataForFn(fn));
   }
 
   private static <T> DisplayData.ItemSpec<? extends Class<?>> displayDataForFn(T fn) {
-    return DisplayData.item("fn", fn.getClass())
-        .withLabel("Transform Function");
+    return DisplayData.item("fn", fn.getClass()).withLabel("Transform Function");
   }
 
   /**
@@ -647,21 +643,6 @@ public class ParDo {
     }
 
     /**
-     * Returns a new multi-output {@link ParDo} transform that's like
-     * this transform but with the specified main and side output
-     * tags. Does not modify this transform. The resulting transform
-     * is still incomplete.
-     *
-     * <p>See the discussion of Side Outputs above and on
-     * {@link ParDo#withOutputTags} for more explanation.
-     */
-    public <OutputT> UnboundMulti<OutputT> withOutputTags(TupleTag<OutputT> mainOutputTag,
-                                              TupleTagList sideOutputTags) {
-      return new UnboundMulti<>(
-          name, sideInputs, mainOutputTag, sideOutputTags);
-    }
-
-    /**
      * Returns a new {@link ParDo} {@link PTransform} that's like this
      * transform but which will invoke the given {@link DoFn}
      * function, and which has its input and output types bound. Does
@@ -671,7 +652,20 @@ public class ParDo {
      */
     public <InputT, OutputT> Bound<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
       validate(fn);
-      return of(adapt(fn), displayDataForFn(fn));
+      return of(fn, displayDataForFn(fn));
+    }
+
+    /**
+     * Returns a new multi-output {@link ParDo} transform that's like this transform but with the
+     * specified main and side output tags. Does not modify this transform. The resulting transform
+     * is still incomplete.
+     *
+     * <p>See the discussion of Side Outputs above and on {@link ParDo#withOutputTags} for more
+     * explanation.
+     */
+    public <OutputT> UnboundMulti<OutputT> withOutputTags(
+        TupleTag<OutputT> mainOutputTag, TupleTagList sideOutputTags) {
+      return new UnboundMulti<>(name, sideInputs, mainOutputTag, sideOutputTags);
     }
 
     /**
@@ -685,13 +679,13 @@ public class ParDo {
      * @deprecated please port your {@link OldDoFn} to a {@link DoFn}
      */
     @Deprecated
-    public <InputT, OutputT> Bound<InputT, OutputT> of(OldDoFn<InputT, OutputT> fn) {
-      return of(fn, displayDataForFn(fn));
+    public <InputT, OutputT> Bound<InputT, OutputT> of(OldDoFn<InputT, OutputT> oldFn) {
+      return of(oldFn, displayDataForFn(oldFn));
     }
 
     private <InputT, OutputT> Bound<InputT, OutputT> of(
-        OldDoFn<InputT, OutputT> fn, DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
-      return new Bound<>(name, sideInputs, fn, fnDisplayData);
+        Serializable originalFn, DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
+      return new Bound<>(name, originalFn, sideInputs, fnDisplayData);
     }
   }
 
@@ -711,17 +705,18 @@ public class ParDo {
       extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
     // Inherits name.
     private final List<PCollectionView<?>> sideInputs;
-    private final OldDoFn<InputT, OutputT> fn;
+    private final Serializable fn;
     private final DisplayData.ItemSpec<? extends Class<?>> fnDisplayData;
 
-    Bound(String name,
-          List<PCollectionView<?>> sideInputs,
-          OldDoFn<InputT, OutputT> fn,
-          DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
+    Bound(
+        String name,
+        Serializable fn,
+        List<PCollectionView<?>> sideInputs,
+        DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
       super(name);
-      this.sideInputs = sideInputs;
       this.fn = SerializableUtils.clone(fn);
       this.fnDisplayData = fnDisplayData;
+      this.sideInputs = sideInputs;
     }
 
     /**
@@ -746,35 +741,38 @@ public class ParDo {
      */
     public Bound<InputT, OutputT> withSideInputs(
         Iterable<? extends PCollectionView<?>> sideInputs) {
-      ImmutableList.Builder<PCollectionView<?>> builder = ImmutableList.builder();
-      builder.addAll(this.sideInputs);
-      builder.addAll(sideInputs);
-      return new Bound<>(name, builder.build(), fn, fnDisplayData);
+      return new Bound<>(
+          name,
+          fn,
+          ImmutableList.<PCollectionView<?>>builder()
+              .addAll(this.sideInputs)
+              .addAll(sideInputs)
+              .build(),
+          fnDisplayData);
     }
 
     /**
-     * Returns a new multi-output {@link ParDo} {@link PTransform}
-     * that's like this {@link PTransform} but with the specified main
-     * and side output tags. Does not modify this {@link PTransform}.
+     * Returns a new multi-output {@link ParDo} {@link PTransform} that's like this {@link
+     * PTransform} but with the specified main and side output tags. Does not modify this {@link
+     * PTransform}.
      *
-     * <p>See the discussion of Side Outputs above and on
-     * {@link ParDo#withOutputTags} for more explanation.
+     * <p>See the discussion of Side Outputs above and on {@link ParDo#withOutputTags} for more
+     * explanation.
      */
-    public BoundMulti<InputT, OutputT> withOutputTags(TupleTag<OutputT> mainOutputTag,
-                                           TupleTagList sideOutputTags) {
-      return new BoundMulti<>(
-          name, sideInputs, mainOutputTag, sideOutputTags, fn, fnDisplayData);
+    public BoundMulti<InputT, OutputT> withOutputTags(
+        TupleTag<OutputT> mainOutputTag, TupleTagList sideOutputTags) {
+      return new BoundMulti<>(name, fn, sideInputs, mainOutputTag, sideOutputTags, fnDisplayData);
     }
 
     @Override
     public PCollection<OutputT> apply(PCollection<? extends InputT> input) {
       checkArgument(
-          !isSplittable(fn), "Splittable DoFn not supported by the current runner");
+          !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner");
       return PCollection.<OutputT>createPrimitiveOutputInternal(
               input.getPipeline(),
               input.getWindowingStrategy(),
               input.isBounded())
-          .setTypeDescriptorInternal(fn.getOutputTypeDescriptor());
+          .setTypeDescriptorInternal(getOldFn().getOutputTypeDescriptor());
     }
 
     @Override
@@ -782,14 +780,14 @@ public class ParDo {
     protected Coder<OutputT> getDefaultOutputCoder(PCollection<? extends InputT> input)
         throws CannotProvideCoderException {
       return input.getPipeline().getCoderRegistry().getDefaultCoder(
-          fn.getOutputTypeDescriptor(),
-          fn.getInputTypeDescriptor(),
+          getOldFn().getOutputTypeDescriptor(),
+          getOldFn().getInputTypeDescriptor(),
           ((PCollection<InputT>) input).getCoder());
     }
 
     @Override
     protected String getKindString() {
-      Class<?> clazz = DoFnAdapters.getDoFnClass(fn);
+      Class<?> clazz = DoFnAdapters.getDoFnClass(getOldFn());
       if (clazz.isAnonymousClass()) {
         return "AnonymousParDo";
       } else {
@@ -807,10 +805,47 @@ public class ParDo {
     @Override
     public void populateDisplayData(Builder builder) {
       super.populateDisplayData(builder);
-      ParDo.populateDisplayData(builder, fn, fnDisplayData);
+      ParDo.populateDisplayData(builder, (HasDisplayData) fn, fnDisplayData);
     }
 
+    /**
+     * @deprecated this method to be converted to return {@link DoFn}. If you want to receive
+     * an {@link OldDoFn} you should (temporarily) use {@link #getOldFn}.
+     */
+    @Deprecated
     public OldDoFn<InputT, OutputT> getFn() {
+      return getOldFn();
+    }
+
+    /**
+     * @deprecated please migrate to {@link #getNewFn} until {@link #getFn} is migrated to return
+     * a {@link DoFn}.
+     */
+    @Deprecated
+    public OldDoFn<InputT, OutputT> getOldFn() {
+      if (fn instanceof OldDoFn) {
+        return (OldDoFn<InputT, OutputT>) fn;
+      } else {
+        return adapt((DoFn<InputT, OutputT>) fn);
+      }
+    }
+
+    public DoFn<InputT, OutputT> getNewFn() {
+      if (fn instanceof DoFn) {
+        return (DoFn<InputT, OutputT>) fn;
+      } else {
+        return ((OldDoFn<InputT, OutputT>) fn).toDoFn();
+      }
+    }
+
+    /**
+     * Returns the {@link OldDoFn} or {@link DoFn} used to create this transform.
+     *
+     * @deprecated for migration purposes only. There are some cases of {@link OldDoFn} that are not
+     *     fully supported by wrapping it into a {@link DoFn}, such as {@link RequiresWindowAccess}.
+     */
+    @Deprecated
+    public Object getOriginalFn() {
       return fn;
     }
 
@@ -870,12 +905,14 @@ public class ParDo {
      */
     public UnboundMulti<OutputT> withSideInputs(
         Iterable<? extends PCollectionView<?>> sideInputs) {
-      ImmutableList.Builder<PCollectionView<?>> builder = ImmutableList.builder();
-      builder.addAll(this.sideInputs);
-      builder.addAll(sideInputs);
       return new UnboundMulti<>(
-          name, builder.build(),
-          mainOutputTag, sideOutputTags);
+          name,
+          ImmutableList.<PCollectionView<?>>builder()
+              .addAll(this.sideInputs)
+              .addAll(sideInputs)
+              .build(),
+          mainOutputTag,
+          sideOutputTags);
     }
 
     /**
@@ -888,7 +925,7 @@ public class ParDo {
      */
     public <InputT> BoundMulti<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
       validate(fn);
-      return of(adapt(fn), displayDataForFn(fn));
+      return of(fn, displayDataForFn(fn));
     }
 
     /**
@@ -906,10 +943,9 @@ public class ParDo {
       return of(fn, displayDataForFn(fn));
     }
 
-    private <InputT> BoundMulti<InputT, OutputT> of(OldDoFn<InputT, OutputT> fn,
-        DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
-      return new BoundMulti<>(
-              name, sideInputs, mainOutputTag, sideOutputTags, fn, fnDisplayData);
+    private <InputT> BoundMulti<InputT, OutputT> of(
+        Serializable fn, DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
+      return new BoundMulti<>(name, fn, sideInputs, mainOutputTag, sideOutputTags, fnDisplayData);
     }
   }
 
@@ -930,15 +966,16 @@ public class ParDo {
     private final List<PCollectionView<?>> sideInputs;
     private final TupleTag<OutputT> mainOutputTag;
     private final TupleTagList sideOutputTags;
-    private final OldDoFn<InputT, OutputT> fn;
     private final DisplayData.ItemSpec<? extends Class<?>> fnDisplayData;
+    private final Serializable fn;
 
-    BoundMulti(String name,
-               List<PCollectionView<?>> sideInputs,
-               TupleTag<OutputT> mainOutputTag,
-               TupleTagList sideOutputTags,
-               OldDoFn<InputT, OutputT> fn,
-               DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
+    BoundMulti(
+        String name,
+        Serializable fn,
+        List<PCollectionView<?>> sideInputs,
+        TupleTag<OutputT> mainOutputTag,
+        TupleTagList sideOutputTags,
+        DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
       super(name);
       this.sideInputs = sideInputs;
       this.mainOutputTag = mainOutputTag;
@@ -961,28 +998,32 @@ public class ParDo {
     }
 
     /**
-     * Returns a new multi-output {@link ParDo} {@link PTransform}
-     * that's like this {@link PTransform} but with the specified additional side
-     * inputs. Does not modify this {@link PTransform}.
+     * Returns a new multi-output {@link ParDo} {@link PTransform} that's like this {@link
+     * PTransform} but with the specified additional side inputs. Does not modify this {@link
+     * PTransform}.
      *
-     * <p>See the discussion of Side Inputs above and on
-     * {@link ParDo#withSideInputs} for more explanation.
+     * <p>See the discussion of Side Inputs above and on {@link ParDo#withSideInputs} for more
+     * explanation.
      */
     public BoundMulti<InputT, OutputT> withSideInputs(
         Iterable<? extends PCollectionView<?>> sideInputs) {
-      ImmutableList.Builder<PCollectionView<?>> builder = ImmutableList.builder();
-      builder.addAll(this.sideInputs);
-      builder.addAll(sideInputs);
       return new BoundMulti<>(
-          name, builder.build(),
-          mainOutputTag, sideOutputTags, fn, fnDisplayData);
+          name,
+          fn,
+          ImmutableList.<PCollectionView<?>>builder()
+              .addAll(this.sideInputs)
+              .addAll(sideInputs)
+              .build(),
+          mainOutputTag,
+          sideOutputTags,
+          fnDisplayData);
     }
 
 
     @Override
     public PCollectionTuple apply(PCollection<? extends InputT> input) {
       checkArgument(
-          !isSplittable(fn), "Splittable DoFn not supported by the current runner");
+          !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner");
 
       PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal(
           input.getPipeline(),
@@ -993,7 +1034,7 @@ public class ParDo {
       // The fn will likely be an instance of an anonymous subclass
       // such as DoFn<Integer, String> { }, thus will have a high-fidelity
       // TypeDescriptor for the output type.
-      outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor());
+      outputs.get(mainOutputTag).setTypeDescriptorInternal(getOldFn().getOutputTypeDescriptor());
 
       return outputs;
     }
@@ -1012,13 +1053,13 @@ public class ParDo {
       Coder<InputT> inputCoder = ((PCollection<InputT>) input).getCoder();
       return input.getPipeline().getCoderRegistry().getDefaultCoder(
           output.getTypeDescriptor(),
-          fn.getInputTypeDescriptor(),
+          getOldFn().getInputTypeDescriptor(),
           inputCoder);
       }
 
     @Override
     protected String getKindString() {
-      Class<?> clazz = DoFnAdapters.getDoFnClass(fn);
+      Class<?> clazz = DoFnAdapters.getDoFnClass(getOldFn());
       if (clazz.isAnonymousClass()) {
         return "AnonymousParMultiDo";
       } else {
@@ -1029,11 +1070,37 @@ public class ParDo {
     @Override
     public void populateDisplayData(Builder builder) {
       super.populateDisplayData(builder);
-      ParDo.populateDisplayData(builder, fn, fnDisplayData);
+      ParDo.populateDisplayData(builder, (HasDisplayData) fn, fnDisplayData);
     }
 
+    /**
+     * @deprecated this method to be converted to return {@link DoFn}. If you want to receive
+     * an {@link OldDoFn} you should (temporarily) use {@link #getOldFn}.
+     */
+    @Deprecated
     public OldDoFn<InputT, OutputT> getFn() {
-      return fn;
+      return getOldFn();
+    }
+
+    /**
+     * @deprecated please migrate to {@link #getNewFn} until {@link #getFn} is migrated to return
+     * a {@link DoFn}.
+     */
+    @Deprecated
+    public OldDoFn<InputT, OutputT> getOldFn() {
+      if (fn instanceof OldDoFn) {
+        return (OldDoFn<InputT, OutputT>) fn;
+      } else {
+        return adapt((DoFn<InputT, OutputT>) fn);
+      }
+    }
+
+    public DoFn<InputT, OutputT> getNewFn() {
+      if (fn instanceof DoFn) {
+        return (DoFn<InputT, OutputT>) fn;
+      } else {
+        return ((OldDoFn<InputT, OutputT>) fn).toDoFn();
+      }
     }
 
     public TupleTag<OutputT> getMainOutputTag() {
@@ -1050,11 +1117,10 @@ public class ParDo {
   }
 
   private static void populateDisplayData(
-      DisplayData.Builder builder, OldDoFn<?, ?> fn,
+      DisplayData.Builder builder,
+      HasDisplayData fn,
       DisplayData.ItemSpec<? extends Class<?>> fnDisplayData) {
-    builder
-        .include("fn", fn)
-        .add(fnDisplayData);
+    builder.include("fn", fn).add(fnDisplayData);
   }
 
   private static boolean isSplittable(OldDoFn<?, ?> oldDoFn) {


[12/50] incubator-beam git commit: [BEAM-790] Refactor PipelineOptionsFactory validation into separate methods.

Posted by th...@apache.org.
[BEAM-790] Refactor PipelineOptionsFactory validation into separate methods.


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

Branch: refs/heads/apex-runner
Commit: b809883813a992167bfa4ad59759fb992c0694d2
Parents: ea8583c
Author: Pei He <pe...@google.com>
Authored: Thu Oct 20 16:32:45 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Oct 21 15:43:00 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     | 151 +++++++++++++------
 1 file changed, 105 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b8098838/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index 7206b11..eefe8c7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -958,7 +958,7 @@ public class PipelineOptionsFactory {
   /**
    * Validates that a given class conforms to the following properties:
    * <ul>
-   *   <li>Any property with the same name must have the same return type for all derived
+   *   <li>Any method with the same name must have the same return type for all derived
    *       interfaces of {@link PipelineOptions}.
    *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
    *       getter and setter method.
@@ -979,22 +979,37 @@ public class PipelineOptionsFactory {
   private static List<PropertyDescriptor> validateClass(Class<? extends PipelineOptions> iface,
       Set<Class<? extends PipelineOptions>> validatedPipelineOptionsInterfaces,
       Class<? extends PipelineOptions> klass) throws IntrospectionException {
-    Set<Method> methods = Sets.newHashSet(IGNORED_METHODS);
-    // Ignore synthetic methods
-    for (Method method : klass.getMethods()) {
-      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
-        methods.add(method);
-      }
-    }
-    // Ignore methods on the base PipelineOptions interface.
-    try {
-      methods.add(iface.getMethod("as", Class.class));
-      methods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class));
-    } catch (NoSuchMethodException | SecurityException e) {
-      throw new RuntimeException(e);
-    }
-
     // Verify that there are no methods with the same name with two different return types.
+    validateReturnType(iface);
+
+    SortedSet<Method> allInterfaceMethods = FluentIterable
+        .from(ReflectHelpers.getClosureOfMethodsOnInterfaces(
+            validatedPipelineOptionsInterfaces))
+        .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
+        .filter(NOT_SYNTHETIC_PREDICATE)
+        .toSortedSet(MethodComparator.INSTANCE);
+
+    List<PropertyDescriptor> descriptors = getPropertyDescriptors(allInterfaceMethods, iface);
+
+    // Verify that all method annotations are valid.
+    validateMethodAnnotations(allInterfaceMethods, descriptors);
+
+    // Verify that each property has a matching read and write method.
+    validateGettersSetters(iface, descriptors);
+
+    // Verify all methods are bean methods or known methods.
+    validateMethodsAreEitherBeanMethodOrKnownMethod(iface, klass, descriptors);
+
+    return descriptors;
+  }
+
+  /**
+   * Validates that any method with the same name must have the same return type for all derived
+   * interfaces of {@link PipelineOptions}.
+   *
+   * @param iface The interface to validate.
+   */
+  private static void validateReturnType(Class<? extends PipelineOptions> iface) {
     Iterable<Method> interfaceMethods = FluentIterable
         .from(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
         .filter(NOT_SYNTHETIC_PREDICATE)
@@ -1019,24 +1034,29 @@ public class PipelineOptionsFactory {
       }
     }
     throwForMultipleDefinitions(iface, multipleDefinitions);
+  }
 
-    // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
-    // that no setter has @JsonIgnore.
-    SortedSet<Method> allInterfaceMethods =
-        FluentIterable.from(
-                ReflectHelpers.getClosureOfMethodsOnInterfaces(
-                    validatedPipelineOptionsInterfaces))
-            .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
-            .filter(NOT_SYNTHETIC_PREDICATE)
-            .toSortedSet(MethodComparator.INSTANCE);
+  /**
+   * Validates that a given class conforms to the following properties:
+   * <ul>
+   *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
+   *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for
+   *       this property must be annotated with {@link JsonIgnore @JsonIgnore}.
+   * </ul>
+   *
+   * @param allInterfaceMethods All interface methods that derive from {@link PipelineOptions}.
+   * @param descriptors The list of {@link PropertyDescriptor}s representing all valid bean
+   * properties of {@code iface}.
+   */
+  private static void validateMethodAnnotations(
+      SortedSet<Method> allInterfaceMethods,
+      List<PropertyDescriptor> descriptors) {
     SortedSetMultimap<Method, Method> methodNameToAllMethodMap =
         TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
     for (Method method : allInterfaceMethods) {
       methodNameToAllMethodMap.put(method, method);
     }
 
-    List<PropertyDescriptor> descriptors = getPropertyDescriptors(allInterfaceMethods, iface);
-
     List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
     List<IgnoredSetter> ignoredSetters = new ArrayList<>();
 
@@ -1047,6 +1067,8 @@ public class PipelineOptionsFactory {
           || IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
         continue;
       }
+      // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
+      // that no setter has @JsonIgnore.
       SortedSet<Method> getters = methodNameToAllMethodMap.get(descriptor.getReadMethod());
       SortedSet<Method> gettersWithJsonIgnore = Sets.filter(getters, JsonIgnorePredicate.INSTANCE);
 
@@ -1073,8 +1095,8 @@ public class PipelineOptionsFactory {
               JsonIgnorePredicate.INSTANCE);
 
       Iterable<String> settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore)
-              .transform(MethodToDeclaringClassFunction.INSTANCE)
-              .transform(ReflectHelpers.CLASS_NAME);
+          .transform(MethodToDeclaringClassFunction.INSTANCE)
+          .transform(ReflectHelpers.CLASS_NAME);
 
       if (!settersWithJsonIgnore.isEmpty()) {
         IgnoredSetter ignored = new IgnoredSetter();
@@ -1085,12 +1107,22 @@ public class PipelineOptionsFactory {
     }
     throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters);
     throwForSettersWithJsonIgnore(ignoredSetters);
+  }
 
+  /**
+   * Validates that every bean property of the given interface must have both a getter and setter.
+   *
+   * @param iface The interface to validate.
+   * @param descriptors The list of {@link PropertyDescriptor}s representing all valid bean
+   * properties of {@code iface}.
+   */
+  private static void validateGettersSetters(
+      Class<? extends PipelineOptions> iface,
+      List<PropertyDescriptor> descriptors) {
     List<MissingBeanMethod> missingBeanMethods = new ArrayList<>();
-    // Verify that each property has a matching read and write method.
     for (PropertyDescriptor propertyDescriptor : descriptors) {
       if (!(IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod())
-        || propertyDescriptor.getReadMethod() != null)) {
+          || propertyDescriptor.getReadMethod() != null)) {
         MissingBeanMethod method = new MissingBeanMethod();
         method.property = propertyDescriptor;
         method.methodType = "getter";
@@ -1098,20 +1130,49 @@ public class PipelineOptionsFactory {
         continue;
       }
       if (!(IGNORED_METHODS.contains(propertyDescriptor.getReadMethod())
-              || propertyDescriptor.getWriteMethod() != null)) {
+          || propertyDescriptor.getWriteMethod() != null)) {
         MissingBeanMethod method = new MissingBeanMethod();
         method.property = propertyDescriptor;
         method.methodType = "setter";
         missingBeanMethods.add(method);
         continue;
       }
-      methods.add(propertyDescriptor.getReadMethod());
-      methods.add(propertyDescriptor.getWriteMethod());
     }
     throwForMissingBeanMethod(iface, missingBeanMethods);
-    final Set<String> knownMethods = Sets.newHashSet();
-    for (Method method : methods) {
-      knownMethods.add(method.getName());
+  }
+
+  /**
+   * Validates that every non-static or synthetic method is either a known method such as
+   * {@link PipelineOptions#as} or a bean property.
+   *
+   * @param iface The interface to validate.
+   * @param klass The proxy class representing the interface.
+   */
+  private static void validateMethodsAreEitherBeanMethodOrKnownMethod(
+      Class<? extends PipelineOptions> iface,
+      Class<? extends PipelineOptions> klass,
+      List<PropertyDescriptor> descriptors) {
+    Set<Method> knownMethods = Sets.newHashSet(IGNORED_METHODS);
+    // Ignore synthetic methods
+    for (Method method : klass.getMethods()) {
+      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
+        knownMethods.add(method);
+      }
+    }
+    // Ignore methods on the base PipelineOptions interface.
+    try {
+      knownMethods.add(iface.getMethod("as", Class.class));
+      knownMethods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class));
+    } catch (NoSuchMethodException | SecurityException e) {
+      throw new RuntimeException(e);
+    }
+    for (PropertyDescriptor descriptor : descriptors) {
+      knownMethods.add(descriptor.getReadMethod());
+      knownMethods.add(descriptor.getWriteMethod());
+    }
+    final Set<String> knownMethodsNames = Sets.newHashSet();
+    for (Method method : knownMethods) {
+      knownMethodsNames.add(method.getName());
     }
 
     // Verify that no additional methods are on an interface that aren't a bean property.
@@ -1120,20 +1181,18 @@ public class PipelineOptionsFactory {
     SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
     unknownMethods.addAll(
         Sets.filter(
-            Sets.difference(Sets.newHashSet(iface.getMethods()), methods),
+            Sets.difference(Sets.newHashSet(iface.getMethods()), knownMethods),
             Predicates.and(NOT_SYNTHETIC_PREDICATE,
-                           new Predicate<Method>() {
-                             @Override
-                               public boolean apply(@Nonnull Method input) {
-                                 return !knownMethods.contains(input.getName());
-                             }
-                           })));
+                new Predicate<Method>() {
+                  @Override
+                  public boolean apply(@Nonnull Method input) {
+                    return !knownMethodsNames.contains(input.getName());
+                  }
+                })));
     checkArgument(unknownMethods.isEmpty(),
         "Methods %s on [%s] do not conform to being bean properties.",
         FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
         iface.getName());
-
-    return descriptors;
   }
 
   private static class MultipleDefinitions {


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

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


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

Branch: refs/heads/apex-runner
Commit: 0a1b2789589ad834736d2998d00e43f554a56432
Parents: 6a971d6
Author: Thomas Weise <th...@apache.org>
Authored: Tue Oct 25 09:16:52 2016 -0700
Committer: Thomas Weise <th...@apache.org>
Committed: Tue Oct 25 09:16:52 2016 -0700

----------------------------------------------------------------------
 runners/apex/pom.xml                                               | 2 +-
 .../beam/runners/apex/translators/utils/ApexStreamTuple.java       | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0a1b2789/runners/apex/pom.xml
----------------------------------------------------------------------
diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml
index 8b62410..2239d03 100644
--- a/runners/apex/pom.xml
+++ b/runners/apex/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0a1b2789/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java
index a260a66..7f8b0fa 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java
@@ -134,7 +134,7 @@ public interface ApexStreamTuple<T> {
   /**
    * Coder for {@link ApexStreamTuple}.
    */
-  public static class ApexStreamTupleCoder<T> extends StandardCoder<ApexStreamTuple<T>> {
+  class ApexStreamTupleCoder<T> extends StandardCoder<ApexStreamTuple<T>> {
     private static final long serialVersionUID = 1L;
     final Coder<T> valueCoder;
 


[20/50] incubator-beam git commit: Move shared DelegatingAggregator out of OldDoFn

Posted by th...@apache.org.
Move shared DelegatingAggregator out of OldDoFn


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

Branch: refs/heads/apex-runner
Commit: 139437bdca8872a11f6a87a9f54347985523faf2
Parents: 0d500ef
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 11:45:38 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../sdk/transforms/DelegatingAggregator.java    | 125 +++++++++++++++++++
 .../org/apache/beam/sdk/transforms/DoFn.java    |  30 ++---
 .../org/apache/beam/sdk/transforms/OldDoFn.java |  97 --------------
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 4 files changed, 142 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/139437bd/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
new file mode 100644
index 0000000..d92bb71
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DelegatingAggregator.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.base.MoreObjects;
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+
+/**
+ * An {@link Aggregator} that delegates calls to {@link #addValue} to another aggregator.
+ *
+ * <p>This {@link Aggregator} is designed to be constructed without a delegate, at pipeline
+ * construction time, and serialized within a {@link DoFn}. The delegate aggregator to which it
+ * submits values must be provided by the runner at execution time.
+ *
+ * @param <AggInputT> the type of input element
+ * @param <AggOutputT> the type of output element
+ */
+class DelegatingAggregator<AggInputT, AggOutputT>
+    implements Aggregator<AggInputT, AggOutputT>, Serializable {
+  private final UUID id;
+
+  private final String name;
+
+  private final CombineFn<AggInputT, ?, AggOutputT> combineFn;
+
+  private Aggregator<AggInputT, ?> delegate;
+
+  public DelegatingAggregator(String name,
+      CombineFn<? super AggInputT, ?, AggOutputT> combiner) {
+    this.id = UUID.randomUUID();
+    this.name = checkNotNull(name, "name cannot be null");
+    // Safe contravariant cast
+    @SuppressWarnings("unchecked")
+    CombineFn<AggInputT, ?, AggOutputT> specificCombiner =
+        (CombineFn<AggInputT, ?, AggOutputT>) checkNotNull(combiner, "combineFn cannot be null");
+    this.combineFn = specificCombiner;
+  }
+
+  @Override
+  public void addValue(AggInputT value) {
+    if (delegate == null) {
+      throw new IllegalStateException(
+          String.format(
+              "addValue cannot be called on Aggregator outside of the execution of a %s.",
+              DoFn.class.getSimpleName()));
+    } else {
+      delegate.addValue(value);
+    }
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public CombineFn<AggInputT, ?, AggOutputT> getCombineFn() {
+    return combineFn;
+  }
+
+  /**
+   * Sets the current delegate of the Aggregator.
+   *
+   * @param delegate the delegate to set in this aggregator
+   */
+  public void setDelegate(Aggregator<AggInputT, ?> delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(getClass())
+        .add("name", name)
+        .add("combineFn", combineFn)
+        .toString();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, name, combineFn.getClass());
+  }
+
+  /**
+   * Indicates whether some other object is "equal to" this one.
+   *
+   * <p>{@code DelegatingAggregator} instances are equal if they have the same name, their
+   * CombineFns are the same class, and they have identical IDs.
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (o == this) {
+      return true;
+    }
+    if (o == null) {
+      return false;
+    }
+    if (o instanceof DelegatingAggregator) {
+      DelegatingAggregator<?, ?> that = (DelegatingAggregator<?, ?>) o;
+      return Objects.equals(this.id, that.id)
+          && Objects.equals(this.name, that.name)
+          && Objects.equals(this.combineFn.getClass(), that.combineFn.getClass());
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/139437bd/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 8b3aaf8..0531cbb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -35,8 +35,8 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
@@ -775,31 +775,31 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
   }
 
   /**
-   * Returns an {@link Aggregator} with aggregation logic specified by the
-   * {@link CombineFn} argument. The name provided must be unique across
-   * {@link Aggregator}s created within the {@link DoFn}. Aggregators can only be created
-   * during pipeline construction.
+   * Returns an {@link Aggregator} with aggregation logic specified by the {@link CombineFn}
+   * argument. The name provided must be unique across {@link Aggregator}s created within the {@link
+   * DoFn}. Aggregators can only be created during pipeline construction.
    *
    * @param name the name of the aggregator
    * @param combiner the {@link CombineFn} to use in the aggregator
-   * @return an aggregator for the provided name and combiner in the scope of
-   *         this {@link DoFn}
+   * @return an aggregator for the provided name and combiner in the scope of this {@link DoFn}
    * @throws NullPointerException if the name or combiner is null
-   * @throws IllegalArgumentException if the given name collides with another
-   *         aggregator in this scope
+   * @throws IllegalArgumentException if the given name collides with another aggregator in this
+   *     scope
    * @throws IllegalStateException if called during pipeline execution.
    */
-  public final <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
-      createAggregator(String name, Combine.CombineFn<? super AggInputT, ?, AggOutputT> combiner) {
+  public final <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+      String name, Combine.CombineFn<? super AggInputT, ?, AggOutputT> combiner) {
     checkNotNull(name, "name cannot be null");
     checkNotNull(combiner, "combiner cannot be null");
-    checkArgument(!aggregators.containsKey(name),
+    checkArgument(
+        !aggregators.containsKey(name),
         "Cannot create aggregator with name %s."
-        + " An Aggregator with that name already exists within this scope.",
+            + " An Aggregator with that name already exists within this scope.",
         name);
-    checkState(!aggregatorsAreFinal,
+    checkState(
+        !aggregatorsAreFinal,
         "Cannot create an aggregator during pipeline execution."
-        + " Aggregators should be registered during pipeline construction.");
+            + " Aggregators should be registered during pipeline construction.");
 
     DelegatingAggregator<AggInputT, AggOutputT> aggregator =
         new DelegatingAggregator<>(name, combiner);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/139437bd/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index 72c2965..b269f47 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -21,14 +21,11 @@ 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.io.Serializable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Objects;
-import java.util.UUID;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -505,100 +502,6 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
   }
 
   /**
-   * An {@link Aggregator} that delegates calls to addValue to another
-   * aggregator.
-   *
-   * @param <AggInputT> the type of input element
-   * @param <AggOutputT> the type of output element
-   */
-  static class DelegatingAggregator<AggInputT, AggOutputT> implements
-      Aggregator<AggInputT, AggOutputT>, Serializable {
-    private final UUID id;
-
-    private final String name;
-
-    private final CombineFn<AggInputT, ?, AggOutputT> combineFn;
-
-    private Aggregator<AggInputT, ?> delegate;
-
-    public DelegatingAggregator(String name,
-        CombineFn<? super AggInputT, ?, AggOutputT> combiner) {
-      this.id = UUID.randomUUID();
-      this.name = checkNotNull(name, "name cannot be null");
-      // Safe contravariant cast
-      @SuppressWarnings("unchecked")
-      CombineFn<AggInputT, ?, AggOutputT> specificCombiner =
-          (CombineFn<AggInputT, ?, AggOutputT>) checkNotNull(combiner, "combineFn cannot be null");
-      this.combineFn = specificCombiner;
-    }
-
-    @Override
-    public void addValue(AggInputT value) {
-      if (delegate == null) {
-        throw new IllegalStateException(
-            "addValue cannot be called on Aggregator outside of the execution of a OldDoFn.");
-      } else {
-        delegate.addValue(value);
-      }
-    }
-
-    @Override
-    public String getName() {
-      return name;
-    }
-
-    @Override
-    public CombineFn<AggInputT, ?, AggOutputT> getCombineFn() {
-      return combineFn;
-    }
-
-    /**
-     * Sets the current delegate of the Aggregator.
-     *
-     * @param delegate the delegate to set in this aggregator
-     */
-    public void setDelegate(Aggregator<AggInputT, ?> delegate) {
-      this.delegate = delegate;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(getClass())
-          .add("name", name)
-          .add("combineFn", combineFn)
-          .toString();
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(id, name, combineFn.getClass());
-    }
-
-    /**
-     * Indicates whether some other object is "equal to" this one.
-     *
-     * <p>{@code DelegatingAggregator} instances are equal if they have the same name, their
-     * CombineFns are the same class, and they have identical IDs.
-     */
-    @Override
-    public boolean equals(Object o) {
-      if (o == this) {
-        return true;
-      }
-      if (o == null) {
-        return false;
-      }
-      if (o instanceof DelegatingAggregator) {
-        DelegatingAggregator<?, ?> that = (DelegatingAggregator<?, ?>) o;
-        return Objects.equals(this.id, that.id)
-            && Objects.equals(this.name, that.name)
-            && Objects.equals(this.combineFn.getClass(), that.combineFn.getClass());
-      }
-      return false;
-    }
-  }
-
-  /**
    * A {@link Context} for an {@link OldDoFn} via a context for a proper {@link DoFn}.
    */
   private class AdaptedContext extends Context {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/139437bd/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
index c072fd7..f51a6b0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java
@@ -24,7 +24,6 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -35,7 +34,7 @@ import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
 /**
- * Tests for {@link OldDoFn.DelegatingAggregator}.
+ * Tests for {@link DelegatingAggregator}.
  */
 @RunWith(JUnit4.class)
 public class DoFnDelegatingAggregatorTest {
@@ -63,7 +62,7 @@ public class DoFnDelegatingAggregatorTest {
 
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("cannot be called");
-    thrown.expectMessage("OldDoFn");
+    thrown.expectMessage("DoFn");
 
     aggregator.addValue(21.2);
   }


[28/50] incubator-beam git commit: Make aggregator registration idempotent in FlinkRunner

Posted by th...@apache.org.
Make aggregator registration idempotent in FlinkRunner


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

Branch: refs/heads/apex-runner
Commit: 2089c5cd2662a2eeea39ac7ebd1bfd8bcdc1aa16
Parents: 1919d8b
Author: Kenneth Knowles <kl...@google.com>
Authored: Sun Oct 23 21:26:48 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:26:48 2016 -0700

----------------------------------------------------------------------
 .../flink/translation/functions/FlinkProcessContext.java  | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2089c5cd/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
index fa5eb1a..baf97cb 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.common.collect.Iterables;
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
@@ -40,6 +41,7 @@ import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
@@ -316,7 +318,13 @@ class FlinkProcessContext<InputT, OutputT>
   createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
     SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper =
         new SerializableFnAggregatorWrapper<>(combiner);
-    runtimeContext.addAccumulator(name, wrapper);
+    Accumulator<?, ?> existingAccum =
+        (Accumulator<AggInputT, Serializable>) runtimeContext.getAccumulator(name);
+    if (existingAccum != null) {
+      return wrapper;
+    } else {
+      runtimeContext.addAccumulator(name, wrapper);
+    }
     return wrapper;
   }
 }


[39/50] incubator-beam git commit: This closes #1157

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


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

Branch: refs/heads/apex-runner
Commit: c390a2a7ff1b3a58de213f85218eef689e712df4
Parents: 8dfadbf 2089c5c
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 09:10:04 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Oct 24 09:10:04 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/AggregatorFactory.java    |  39 ++++
 .../beam/runners/core/SimpleDoFnRunner.java     |   8 +-
 .../runners/direct/DoFnLifecycleManager.java    |  42 ++--
 .../beam/runners/direct/ParDoEvaluator.java     |   3 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |   6 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   5 +-
 ...leManagerRemovingTransformEvaluatorTest.java |  16 +-
 .../direct/DoFnLifecycleManagerTest.java        |  12 +-
 .../direct/DoFnLifecycleManagersTest.java       |  48 ++++-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  11 +
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  11 +
 .../beam/runners/direct/SplittableDoFnTest.java |   8 +-
 .../functions/FlinkProcessContext.java          |  10 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |  11 +-
 .../sdk/transforms/DelegatingAggregator.java    | 125 +++++++++++
 .../org/apache/beam/sdk/transforms/DoFn.java    |  55 +++--
 .../beam/sdk/transforms/DoFnAdapters.java       |   1 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  18 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 214 +++++++++----------
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |  16 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   2 +-
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  16 +-
 23 files changed, 463 insertions(+), 219 deletions(-)
----------------------------------------------------------------------



[23/50] incubator-beam git commit: Port direct runner to use new DoFn directly

Posted by th...@apache.org.
Port direct runner to use new DoFn directly


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

Branch: refs/heads/apex-runner
Commit: 1919d8b3a850bd146137652546da851ee461cd28
Parents: f0c8d30
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 20 20:55:00 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/DoFnLifecycleManager.java    | 42 +++++++++--------
 .../beam/runners/direct/ParDoEvaluator.java     |  3 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |  6 +--
 .../direct/ParDoSingleEvaluatorFactory.java     |  5 +-
 ...leManagerRemovingTransformEvaluatorTest.java | 16 +++----
 .../direct/DoFnLifecycleManagerTest.java        | 12 ++---
 .../direct/DoFnLifecycleManagersTest.java       | 48 ++++++++++++++++----
 .../direct/ParDoMultiEvaluatorFactoryTest.java  | 11 +++++
 .../direct/ParDoSingleEvaluatorFactoryTest.java | 11 +++++
 .../beam/runners/direct/SplittableDoFnTest.java |  8 +++-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 23 ++++++++--
 11 files changed, 130 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
index 0e15c18..23460b6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
@@ -26,7 +26,9 @@ import java.util.Collection;
 import java.util.Iterator;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn.Setup;
+import org.apache.beam.sdk.transforms.DoFn.Teardown;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,49 +37,49 @@ import org.slf4j.LoggerFactory;
  * Manages {@link DoFn} setup, teardown, and serialization.
  *
  * <p>{@link DoFnLifecycleManager} is similar to a {@link ThreadLocal} storing a {@link DoFn}, but
- * calls the {@link DoFn} {@link Setup} the first time the {@link DoFn} is obtained and {@link
- * Teardown} whenever the {@link DoFn} is removed, and provides a method for clearing all cached
- * {@link DoFn DoFns}.
+ * calls the {@link DoFn} {@link Setup @Setup} method the first time the {@link DoFn} is obtained
+ * and {@link Teardown @Teardown} whenever the {@link DoFn} is removed, and provides a method for
+ * clearing all cached {@link DoFn DoFns}.
  */
 class DoFnLifecycleManager {
   private static final Logger LOG = LoggerFactory.getLogger(DoFnLifecycleManager.class);
 
-  public static DoFnLifecycleManager of(OldDoFn<?, ?> original) {
+  public static DoFnLifecycleManager of(DoFn<?, ?> original) {
     return new DoFnLifecycleManager(original);
   }
 
-  private final LoadingCache<Thread, OldDoFn<?, ?>> outstanding;
+  private final LoadingCache<Thread, DoFn<?, ?>> outstanding;
 
-  private DoFnLifecycleManager(OldDoFn<?, ?> original) {
+  private DoFnLifecycleManager(DoFn<?, ?> original) {
     this.outstanding = CacheBuilder.newBuilder().build(new DeserializingCacheLoader(original));
   }
 
-  public OldDoFn<?, ?> get() throws Exception {
+  public DoFn<?, ?> get() throws Exception {
     Thread currentThread = Thread.currentThread();
     return outstanding.get(currentThread);
   }
 
   public void remove() throws Exception {
     Thread currentThread = Thread.currentThread();
-    OldDoFn<?, ?> fn = outstanding.asMap().remove(currentThread);
-    fn.teardown();
+    DoFn<?, ?> fn = outstanding.asMap().remove(currentThread);
+    DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
   }
 
   /**
    * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. Returns all exceptions
    * that were thrown while calling the remove methods.
    *
-   * <p>If the returned Collection is nonempty, an exception was thrown from at least one
-   * {@link DoFn#teardown()} method, and the {@link PipelineRunner} should throw an exception.
+   * <p>If the returned Collection is nonempty, an exception was thrown from at least one {@link
+   * DoFn.Teardown @Teardown} method, and the {@link PipelineRunner} should throw an exception.
    */
   public Collection<Exception> removeAll() throws Exception {
-    Iterator<OldDoFn<?, ?>> fns = outstanding.asMap().values().iterator();
+    Iterator<DoFn<?, ?>> fns = outstanding.asMap().values().iterator();
     Collection<Exception> thrown = new ArrayList<>();
     while (fns.hasNext()) {
-      OldDoFn<?, ?> fn = fns.next();
+      DoFn<?, ?> fn = fns.next();
       fns.remove();
       try {
-        fn.teardown();
+        DoFnInvokers.INSTANCE.invokerFor(fn).invokeTeardown();
       } catch (Exception e) {
         thrown.add(e);
       }
@@ -85,18 +87,18 @@ class DoFnLifecycleManager {
     return thrown;
   }
 
-  private class DeserializingCacheLoader extends CacheLoader<Thread, OldDoFn<?, ?>> {
+  private class DeserializingCacheLoader extends CacheLoader<Thread, DoFn<?, ?>> {
     private final byte[] original;
 
-    public DeserializingCacheLoader(OldDoFn<?, ?> original) {
+    public DeserializingCacheLoader(DoFn<?, ?> original) {
       this.original = SerializableUtils.serializeToByteArray(original);
     }
 
     @Override
-    public OldDoFn<?, ?> load(Thread key) throws Exception {
-      OldDoFn<?, ?> fn = (OldDoFn<?, ?>) SerializableUtils.deserializeFromByteArray(original,
+    public DoFn<?, ?> load(Thread key) throws Exception {
+      DoFn<?, ?> fn = (DoFn<?, ?>) SerializableUtils.deserializeFromByteArray(original,
           "DoFn Copy in thread " + key.getName());
-      fn.setup();
+      DoFnInvokers.INSTANCE.invokerFor(fn).invokeSetup();
       return fn;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/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 a59fb4d..b524dfa 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
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.direct;
 
 import com.google.common.collect.ImmutableList;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -45,7 +46,7 @@ class ParDoEvaluator<T> implements TransformEvaluator<T> {
       DirectStepContext stepContext,
       CommittedBundle<InputT> inputBundle,
       AppliedPTransform<PCollection<InputT>, ?, ?> application,
-      Object fn,
+      Serializable fn, // may be OldDoFn or DoFn
       List<PCollectionView<?>> sideInputs,
       TupleTag<OutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
index d909e8b..02469ff 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java
@@ -24,7 +24,7 @@ import java.util.Map;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
 import org.apache.beam.sdk.values.PCollection;
@@ -50,7 +50,7 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
           public DoFnLifecycleManager load(AppliedPTransform<?, ?, ?> key)
               throws Exception {
             BoundMulti<?, ?> bound = (BoundMulti<?, ?>) key.getTransform();
-            return DoFnLifecycleManager.of(bound.getFn());
+            return DoFnLifecycleManager.of(bound.getNewFn());
           }
         });
   }
@@ -87,7 +87,7 @@ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory {
               stepContext,
               inputBundle,
               application,
-              (OldDoFn) fnLocal.get(),
+              (DoFn) fnLocal.get(),
               application.getTransform().getSideInputs(),
               application.getTransform().getMainOutputTag(),
               application.getTransform().getSideOutputTags().getAll(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
index 1a06ea6..0584e41 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo.Bound;
 import org.apache.beam.sdk.values.PCollection;
@@ -52,7 +51,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
                   public DoFnLifecycleManager load(AppliedPTransform<?, ?, ?> key)
                       throws Exception {
                     Bound<?, ?> bound = (Bound<?, ?>) key.getTransform();
-                    return DoFnLifecycleManager.of(bound.getFn());
+                    return DoFnLifecycleManager.of(bound.getNewFn());
                   }
                 });
   }
@@ -92,7 +91,7 @@ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory {
               stepContext,
               inputBundle,
               application,
-              (OldDoFn) fnLocal.get(),
+              fnLocal.get(),
               application.getTransform().getSideInputs(),
               mainOutputTag,
               Collections.<TupleTag<?>>emptyList(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
index 2e4fee2..9e2732e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
@@ -27,7 +27,7 @@ import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.hamcrest.Matchers;
 import org.junit.Before;
@@ -50,7 +50,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void delegatesToUnderlying() throws Exception {
     RecordingTransformEvaluator underlying = new RecordingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     TransformEvaluator<Object> evaluator =
         DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager);
     WindowedValue<Object> first = WindowedValue.valueInGlobalWindow(new Object());
@@ -67,7 +67,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void removesOnExceptionInProcessElement() throws Exception {
     ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     assertThat(original, not(nullValue()));
     TransformEvaluator<Object> evaluator =
         DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager);
@@ -75,7 +75,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
     try {
       evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object()));
     } catch (Exception e) {
-      assertThat(lifecycleManager.get(), not(Matchers.<OldDoFn<?, ?>>theInstance(original)));
+      assertThat(lifecycleManager.get(), not(Matchers.<DoFn<?, ?>>theInstance(original)));
       return;
     }
     fail("Expected ThrowingTransformEvaluator to throw on method call");
@@ -84,7 +84,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   @Test
   public void removesOnExceptionInFinishBundle() throws Exception {
     ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator();
-    OldDoFn<?, ?> original = lifecycleManager.get();
+    DoFn<?, ?> original = lifecycleManager.get();
     // the LifecycleManager is set when the evaluator starts
     assertThat(original, not(nullValue()));
     TransformEvaluator<Object> evaluator =
@@ -94,7 +94,7 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
       evaluator.finishBundle();
     } catch (Exception e) {
       assertThat(lifecycleManager.get(),
-          Matchers.not(Matchers.<OldDoFn<?, ?>>theInstance(original)));
+          Matchers.not(Matchers.<DoFn<?, ?>>theInstance(original)));
       return;
     }
     fail("Expected ThrowingTransformEvaluator to throw on method call");
@@ -134,8 +134,8 @@ public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   }
 
 
-  private static class TestFn extends OldDoFn<Object, Object> {
-    @Override
+  private static class TestFn extends DoFn<Object, Object> {
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
index 1f0af99..aef9d29 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.hamcrest.Matchers;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -101,7 +101,7 @@ public class DoFnLifecycleManagerTest {
     assertThat(obtained.setupCalled, is(true));
     assertThat(obtained.teardownCalled, is(true));
 
-    assertThat(mgr.get(), not(Matchers.<OldDoFn<?, ?>>theInstance(obtained)));
+    assertThat(mgr.get(), not(Matchers.<DoFn<?, ?>>theInstance(obtained)));
   }
 
   @Test
@@ -142,11 +142,11 @@ public class DoFnLifecycleManagerTest {
   }
 
 
-  private static class TestFn extends OldDoFn<Object, Object> {
+  private static class TestFn extends DoFn<Object, Object> {
     boolean setupCalled = false;
     boolean teardownCalled = false;
 
-    @Override
+    @Setup
     public void setup() {
       checkState(!setupCalled);
       checkState(!teardownCalled);
@@ -154,11 +154,11 @@ public class DoFnLifecycleManagerTest {
       setupCalled = true;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
 
-    @Override
+    @Teardown
     public void teardown() {
       checkState(setupCalled);
       checkState(!teardownCalled);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
index 39a4a9d..a19ff99 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
@@ -18,12 +18,15 @@
 
 package org.apache.beam.runners.direct;
 
+import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
 
 import com.google.common.collect.ImmutableList;
 import java.util.ArrayList;
 import java.util.Collection;
-import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.UserCodeException;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
@@ -51,9 +54,15 @@ public class DoFnLifecycleManagersTest {
     third.get();
 
     final Collection<Matcher<? super Throwable>> suppressions = new ArrayList<>();
-    suppressions.add(new ThrowableMessageMatcher("foo"));
-    suppressions.add(new ThrowableMessageMatcher("bar"));
-    suppressions.add(new ThrowableMessageMatcher("baz"));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("foo"))));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("bar"))));
+    suppressions.add(allOf(
+        instanceOf(UserCodeException.class),
+        new CausedByMatcher(new ThrowableMessageMatcher("baz"))));
 
     thrown.expect(
         new BaseMatcher<Exception>() {
@@ -90,18 +99,18 @@ public class DoFnLifecycleManagersTest {
     DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third));
   }
 
-  private static class ThrowsInCleanupFn extends OldDoFn<Object, Object> {
+  private static class ThrowsInCleanupFn extends DoFn<Object, Object> {
     private final String message;
 
     private ThrowsInCleanupFn(String message) {
       this.message = message;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
 
-    @Override
+    @Teardown
     public void teardown() throws Exception {
       throw new Exception(message);
     }
@@ -130,9 +139,32 @@ public class DoFnLifecycleManagersTest {
     }
   }
 
+  private static class CausedByMatcher extends BaseMatcher<Throwable> {
+    private final Matcher<Throwable> causeMatcher;
+
+    public CausedByMatcher(
+        Matcher<Throwable> causeMatcher) {
+      this.causeMatcher = causeMatcher;
+    }
 
-  private static class EmptyFn extends OldDoFn<Object, Object> {
     @Override
+    public boolean matches(Object item) {
+      if (!(item instanceof UserCodeException)) {
+        return false;
+      }
+      UserCodeException that = (UserCodeException) item;
+      return causeMatcher.matches(that.getCause());
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("a throwable with a cause ").appendDescriptionOf(causeMatcher);
+    }
+  }
+
+
+  private static class EmptyFn extends DoFn<Object, Object> {
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
index 88e1484..8b0070b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java
@@ -57,6 +57,7 @@ import org.apache.beam.sdk.values.TupleTagList;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -236,6 +237,11 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
             WindowedValue.valueInGlobalWindow("bazam", PaneInfo.ON_TIME_AND_ONLY_FIRING)));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStatePutsStateInResult() throws Exception {
     TestPipeline p = TestPipeline.create();
@@ -321,6 +327,11 @@ public class ParDoMultiEvaluatorFactoryTest implements Serializable {
         containsInAnyOrder("foo", "bara", "bazam"));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
     TestPipeline p = TestPipeline.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
index 6a02e40..e562b28 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java
@@ -54,6 +54,7 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -166,6 +167,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
     assertThat(result.getAggregatorChanges(), equalTo(mutator));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStatePutsStateInResult() throws Exception {
     TestPipeline p = TestPipeline.create();
@@ -238,6 +244,11 @@ public class ParDoSingleEvaluatorFactoryTest implements Serializable {
         containsInAnyOrder("foo", "bara", "bazam"));
   }
 
+  /**
+   * This test ignored, as today testing of GroupByKey is all the state that needs testing.
+   * This should be ported to state when ready.
+   */
+  @Ignore("State is not supported until BEAM-25. GroupByKey tests the needed functionality.")
   @Test
   public void finishBundleWithStateAndTimersPutsTimersInResult() throws Exception {
     TestPipeline p = TestPipeline.create();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
index 84a0cd9..c164ce6 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
@@ -34,7 +34,6 @@ 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.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
@@ -46,6 +45,7 @@ import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.MutableDateTime;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -140,6 +140,9 @@ public class SplittableDoFnTest {
     }
   }
 
+  @Ignore(
+      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
+          + "It must be implemented as a primitive.")
   @Test
   public void testPairWithIndexBasic() throws ClassNotFoundException {
     Pipeline p = TestPipeline.create();
@@ -164,6 +167,9 @@ public class SplittableDoFnTest {
     p.run();
   }
 
+  @Ignore(
+      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
+          + "It must be implemented as a primitive.")
   @Test
   public void testPairWithIndexWindowedTimestamped() throws ClassNotFoundException {
     // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1919d8b3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index a9f26a4..f16e0b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -73,6 +73,10 @@ import org.joda.time.Instant;
 public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDisplayData {
 
   public DoFn<InputT, OutputT> toDoFn() {
+    DoFn<InputT, OutputT> doFn = DoFnAdapters.getDoFn(this);
+    if (doFn != null) {
+      return doFn;
+    }
     if (this instanceof RequiresWindowAccess) {
       // No parameters as it just accesses `this`
       return new AdaptedRequiresWindowAccessDoFn();
@@ -553,8 +557,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
     private final DoFn<InputT, OutputT>.ProcessContext newContext;
 
-    public AdaptedProcessContext(
-        DoFn<InputT, OutputT>.ProcessContext newContext) {
+    public AdaptedProcessContext(DoFn<InputT, OutputT>.ProcessContext newContext) {
       this.newContext = newContext;
     }
 
@@ -632,21 +635,31 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
   private class AdaptedDoFn extends DoFn<InputT, OutputT> {
 
+    @Setup
+    public void setup() throws Exception {
+      OldDoFn.this.setup();
+    }
+
     @StartBundle
-    public void startBundle(DoFn.Context c) throws Exception {
+    public void startBundle(Context c) throws Exception {
       OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
     }
 
     @ProcessElement
-    public void processElement(DoFn.ProcessContext c) throws Exception {
+    public void processElement(ProcessContext c) throws Exception {
       OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c));
     }
 
     @FinishBundle
-    public void finishBundle(DoFn.Context c) throws Exception {
+    public void finishBundle(Context c) throws Exception {
       OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
     }
 
+    @Teardown
+    public void teardown() throws Exception {
+      OldDoFn.this.teardown();
+    }
+
     @Override
     public Duration getAllowedTimestampSkew() {
       return OldDoFn.this.getAllowedTimestampSkew();


[18/50] incubator-beam git commit: Add setupDelegatingAggregators for DoFn (for now)

Posted by th...@apache.org.
Add setupDelegatingAggregators for DoFn (for now)


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

Branch: refs/heads/apex-runner
Commit: c2e751f49d72968f2478931cdb884fd4af173610
Parents: 08dd149
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 11:53:29 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/SimpleDoFnRunner.java     |  1 +
 .../org/apache/beam/sdk/transforms/DoFn.java    | 24 ++++++++++++++++++++
 2 files changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c2e751f4/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 0360bc2..1cf56a6 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
@@ -228,6 +228,7 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
       this.stepContext = stepContext;
       this.aggregatorFactory = aggregatorFactory;
       this.windowFn = windowFn;
+      super.setupDelegateAggregators();
     }
 
     //////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c2e751f4/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 0531cbb..11ca853 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -214,6 +214,30 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
     protected abstract <AggInputT, AggOutputT>
         Aggregator<AggInputT, AggOutputT> createAggregator(
             String name, CombineFn<AggInputT, ?, AggOutputT> combiner);
+
+    /**
+     * Sets up {@link Aggregator}s created by the {@link DoFn} so they are usable within this context.
+     *
+     * <p>This method should be called by runners before the {@link StartBundle @StartBundle} method.
+     */
+    @Experimental(Kind.AGGREGATOR)
+    protected final void setupDelegateAggregators() {
+      for (DelegatingAggregator<?, ?> aggregator : aggregators.values()) {
+        setupDelegateAggregator(aggregator);
+      }
+
+      aggregatorsAreFinal = true;
+    }
+
+    private <AggInputT, AggOutputT> void setupDelegateAggregator(
+        DelegatingAggregator<AggInputT, AggOutputT> aggregator) {
+
+      Aggregator<AggInputT, AggOutputT> delegate = createAggregator(
+          aggregator.getName(), aggregator.getCombineFn());
+
+      aggregator.setDelegate(delegate);
+    }
+
   }
 
   /**


[38/50] incubator-beam git commit: [maven-release-plugin] prepare for next development iteration

Posted by th...@apache.org.
[maven-release-plugin] prepare for next development iteration


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

Branch: refs/heads/apex-runner
Commit: 8dfadbf01acc4fed6f604fe531e87730bab2b2bc
Parents: c26b63b
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 17:36:25 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 17:36:25 2016 +0200

----------------------------------------------------------------------
 examples/java/pom.xml                       | 2 +-
 examples/java8/pom.xml                      | 2 +-
 examples/pom.xml                            | 2 +-
 pom.xml                                     | 4 ++--
 runners/core-java/pom.xml                   | 2 +-
 runners/direct-java/pom.xml                 | 2 +-
 runners/flink/examples/pom.xml              | 2 +-
 runners/flink/pom.xml                       | 2 +-
 runners/flink/runner/pom.xml                | 2 +-
 runners/google-cloud-dataflow-java/pom.xml  | 2 +-
 runners/pom.xml                             | 2 +-
 runners/spark/pom.xml                       | 2 +-
 sdks/java/build-tools/pom.xml               | 2 +-
 sdks/java/core/pom.xml                      | 2 +-
 sdks/java/extensions/join-library/pom.xml   | 2 +-
 sdks/java/extensions/pom.xml                | 2 +-
 sdks/java/io/google-cloud-platform/pom.xml  | 2 +-
 sdks/java/io/hdfs/pom.xml                   | 2 +-
 sdks/java/io/jdbc/pom.xml                   | 2 +-
 sdks/java/io/jms/pom.xml                    | 2 +-
 sdks/java/io/kafka/pom.xml                  | 2 +-
 sdks/java/io/kinesis/pom.xml                | 2 +-
 sdks/java/io/mongodb/pom.xml                | 2 +-
 sdks/java/io/pom.xml                        | 2 +-
 sdks/java/java8tests/pom.xml                | 2 +-
 sdks/java/maven-archetypes/examples/pom.xml | 2 +-
 sdks/java/maven-archetypes/pom.xml          | 2 +-
 sdks/java/maven-archetypes/starter/pom.xml  | 2 +-
 sdks/java/microbenchmarks/pom.xml           | 2 +-
 sdks/java/pom.xml                           | 2 +-
 sdks/pom.xml                                | 2 +-
 31 files changed, 32 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 37cb15a..fc82ed4 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-examples-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index 44cf918..e6408dc 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-examples-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 2efb146..2820473 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6a24d98..2338f55 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,7 +34,7 @@
   <url>http://beam.incubator.apache.org</url>
   <inceptionYear>2016</inceptionYear>
 
-  <version>0.3.0-incubating-SNAPSHOT</version>
+  <version>0.4.0-incubating-SNAPSHOT</version>
 
   <licenses>
     <license>
@@ -48,7 +48,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary</url>
-    <tag>release-0.3.0-incubating</tag>
+    <tag>release-0.2.0-incubating</tag>
   </scm>
 
   <issueManagement>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index d84c420..5f7ec08 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index aec0e71..c182d15 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index b8489fc..a0cf676 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 68e82d2..e012c4b 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 8759591..283d060 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-flink-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 0f2d3b2..c5dd274 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 605c3b2..68cf29c 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index a246c19..ccec3c6 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/pom.xml b/sdks/java/build-tools/pom.xml
index 279555e..cc27bea 100644
--- a/sdks/java/build-tools/pom.xml
+++ b/sdks/java/build-tools/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index d44a494..9937cb8 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/extensions/join-library/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml
index e4acf82..4687554 100644
--- a/sdks/java/extensions/join-library/pom.xml
+++ b/sdks/java/extensions/join-library/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-extensions-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/extensions/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index 419b0ce..4328d3d 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index 1a598bb..054ee21 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 0ec542c..22c3187 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml
index 75eb5ed..3f513ca 100644
--- a/sdks/java/io/jdbc/pom.xml
+++ b/sdks/java/io/jdbc/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/jms/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml
index 7113434..5a74b34 100644
--- a/sdks/java/io/jms/pom.xml
+++ b/sdks/java/io/jms/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml
index e5c3285..f2b8326 100644
--- a/sdks/java/io/kafka/pom.xml
+++ b/sdks/java/io/kafka/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml
index 06817da..0f65d67 100644
--- a/sdks/java/io/kinesis/pom.xml
+++ b/sdks/java/io/kinesis/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml
index 5555173..ad0609d 100644
--- a/sdks/java/io/mongodb/pom.xml
+++ b/sdks/java/io/mongodb/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-io-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/io/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml
index 82cf8d0..3750202 100644
--- a/sdks/java/io/pom.xml
+++ b/sdks/java/io/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/java8tests/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
index 1fe41d7..13f5fce 100644
--- a/sdks/java/java8tests/pom.xml
+++ b/sdks/java/java8tests/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/examples/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml
index dcdf94e..e36c4fc 100644
--- a/sdks/java/maven-archetypes/examples/pom.xml
+++ b/sdks/java/maven-archetypes/examples/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-maven-archetypes-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml
index dd2a16a..02bb150 100644
--- a/sdks/java/maven-archetypes/pom.xml
+++ b/sdks/java/maven-archetypes/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/maven-archetypes/starter/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/pom.xml b/sdks/java/maven-archetypes/starter/pom.xml
index 45b60a6..c38f80c 100644
--- a/sdks/java/maven-archetypes/starter/pom.xml
+++ b/sdks/java/maven-archetypes/starter/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-maven-archetypes-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/microbenchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/pom.xml b/sdks/java/microbenchmarks/pom.xml
index 4d8d922..5c6cf4d 100644
--- a/sdks/java/microbenchmarks/pom.xml
+++ b/sdks/java/microbenchmarks/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 4576e4f..eae5a58 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8dfadbf0/sdks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/pom.xml b/sdks/pom.xml
index aa9cbed..235d102 100644
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.3.0-incubating-SNAPSHOT</version>
+    <version>0.4.0-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[06/50] incubator-beam git commit: Switch the DirectRunner to use new DoFnRunner code path

Posted by th...@apache.org.
Switch the DirectRunner to use new DoFnRunner code path

This does not yet switch to run new DoFn directly, but
to use the generic DoFnRunner dispatch in preparation
for switching over to the original DoFn instead of the
wrapper as an OldDoFn.


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

Branch: refs/heads/apex-runner
Commit: 522646122f43b2ae502f45f8c97859a7660cc86f
Parents: 0db3263
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 21:31:25 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:33:00 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/runners/direct/ParDoEvaluator.java  | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/52264612/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 bbe7827..a59fb4d 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
@@ -31,7 +31,6 @@ import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.util.ReadyCheckingSideInputReader;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -46,7 +45,7 @@ class ParDoEvaluator<T> implements TransformEvaluator<T> {
       DirectStepContext stepContext,
       CommittedBundle<InputT> inputBundle,
       AppliedPTransform<PCollection<InputT>, ?, ?> application,
-      OldDoFn<InputT, OutputT> fn,
+      Object fn,
       List<PCollectionView<?>> sideInputs,
       TupleTag<OutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,


[37/50] incubator-beam git commit: [maven-release-plugin] prepare branch release-0.3.0-incubating

Posted by th...@apache.org.
[maven-release-plugin] prepare branch release-0.3.0-incubating


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

Branch: refs/heads/apex-runner
Commit: c26b63baecde9105d32e3ba03874eb782162ecda
Parents: 69ae8bd
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 17:36:16 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 17:36:16 2016 +0200

----------------------------------------------------------------------
 pom.xml | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c26b63ba/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index de4bd63..6a24d98 100644
--- a/pom.xml
+++ b/pom.xml
@@ -48,7 +48,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary</url>
-    <tag>release-0.2.0-incubating</tag>
+    <tag>release-0.3.0-incubating</tag>
   </scm>
 
   <issueManagement>
@@ -92,13 +92,13 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <beam.javadoc_opts/>
+    <beam.javadoc_opts />
 
     <!-- Disable integration tests by default -->
     <skipITs>true</skipITs>
 
     <!-- Do not add additional surefire arguments by default -->
-    <beamSurefireArgline/>
+    <beamSurefireArgline />
 
     <!-- If updating dependencies, please update any relevant javadoc offlineLinks -->
     <avro.version>1.8.1</avro.version>
@@ -1023,7 +1023,7 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <ignore/>
+                    <ignore />
                   </action>
                 </pluginExecution>
                 <pluginExecution>
@@ -1037,7 +1037,7 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <ignore/>
+                    <ignore />
                   </action>
                 </pluginExecution>
               </pluginExecutions>


[10/50] incubator-beam git commit: Fix typo and dependencies in beam-runners-direct-java

Posted by th...@apache.org.
Fix typo and dependencies in beam-runners-direct-java


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

Branch: refs/heads/apex-runner
Commit: 5f946c6c00cfea0b38b64db6da33bcf2d2783fa3
Parents: 18e5d12
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 19:26:35 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Oct 21 15:25:54 2016 -0700

----------------------------------------------------------------------
 pom.xml                                    |  4 ++--
 runners/direct-java/pom.xml                | 11 +++++++++--
 runners/google-cloud-dataflow-java/pom.xml |  2 +-
 3 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f946c6c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1263827..de4bd63 100644
--- a/pom.xml
+++ b/pom.xml
@@ -691,13 +691,13 @@
         <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-core</artifactId>
         <version>${project.version}</version>
-        <type>test-jar</type>
+        <classifier>tests</classifier>
         <scope>test</scope>
       </dependency>
 
       <dependency>
         <groupId>org.apache.beam</groupId>
-        <artifactId>beam-sdks-java-core</artifactId>
+        <artifactId>beam-runners-core-java</artifactId>
         <version>${project.version}</version>
         <classifier>tests</classifier>
         <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f946c6c/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 6cb1838..aec0e71 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -83,7 +83,7 @@
               <failIfNoTests>true</failIfNoTests>
               <dependenciesToScan>
                 <dependency>org.apache.beam:beam-sdks-java-core</dependency>
-                <dependency>org.apache.beam:beam-runners-java-core</dependency>
+                <dependency>org.apache.beam:beam-runners-core-java</dependency>
               </dependenciesToScan>
               <systemPropertyVariables>
                 <beamTestPipelineOptions>
@@ -331,7 +331,14 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <type>test-jar</type>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-java</artifactId>
+      <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5f946c6c/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index b035028..0f2d3b2 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -402,7 +402,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <type>test-jar</type>
+      <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
 


[08/50] incubator-beam git commit: Use DisplayData delegate in Dataflow Read transform override

Posted by th...@apache.org.
Use DisplayData delegate in Dataflow Read transform override


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

Branch: refs/heads/apex-runner
Commit: 4869ef19fa70769831dacc2ac332ec0a11b3f9d8
Parents: 1533511
Author: Scott Wegner <sw...@google.com>
Authored: Fri Oct 21 11:41:41 2016 -0700
Committer: Scott Wegner <sw...@google.com>
Committed: Fri Oct 21 12:32:42 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4869ef19/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 7bf270d..89c364e 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
@@ -2248,10 +2248,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
 
       @Override
       public void populateDisplayData(DisplayData.Builder builder) {
-        super.populateDisplayData(builder);
-        builder
-            .add(DisplayData.item("source", source.getClass()))
-            .include("source", source);
+        builder.delegate(source);
       }
 
       public UnboundedSource<T, ?> getSource() {


[05/50] incubator-beam git commit: Add OldDoFn.toDoFn() adapter

Posted by th...@apache.org.
Add OldDoFn.toDoFn() adapter


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

Branch: refs/heads/apex-runner
Commit: 44878e57eafc3ee6c437b8537dd1f88024006a51
Parents: fe0b7bf
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 20:44:52 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:32:06 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 183 +++++++++++++++++++
 1 file changed, 183 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/44878e57/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index a445c7d..912bf24 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -75,6 +75,23 @@ import org.joda.time.Instant;
 @Deprecated
 public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDisplayData {
 
+  public DoFn<InputT, OutputT> toDoFn() {
+    if (this instanceof RequiresWindowAccess) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Cannot convert %s to %s because it implements %s."
+                  + " Please convert your %s to a %s directly.",
+              getClass(),
+              DoFn.class.getSimpleName(),
+              RequiresWindowAccess.class.getSimpleName(),
+              OldDoFn.class.getSimpleName(),
+              DoFn.class.getSimpleName()));
+    }
+
+    // No parameters as it just accesses `this`
+    return new AdaptedDoFn();
+  }
+
   /**
    * Information accessible to all methods in this {@code OldDoFn}.
    * Used primarily to output elements.
@@ -587,4 +604,170 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
       return false;
     }
   }
+
+  /**
+   * A {@link Context} for an {@link OldDoFn} via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedContext extends Context {
+
+    private final DoFn<InputT, OutputT>.Context newContext;
+
+    public AdaptedContext(
+        DoFn<InputT, OutputT>.Context newContext) {
+      this.newContext = newContext;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return newContext.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      newContext.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      newContext.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      newContext.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      newContext.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return null;
+    }
+  }
+
+  /**
+   * A {@link ProcessContext} for an {@link OldDoFn} via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedProcessContext extends ProcessContext {
+
+    private final DoFn<InputT, OutputT>.ProcessContext newContext;
+
+    public AdaptedProcessContext(
+        DoFn<InputT, OutputT>.ProcessContext newContext) {
+      this.newContext = newContext;
+    }
+
+    @Override
+    public InputT element() {
+      return newContext.element();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      return newContext.sideInput(view);
+    }
+
+    @Override
+    public Instant timestamp() {
+      return newContext.timestamp();
+    }
+
+    @Override
+    public BoundedWindow window() {
+      throw new UnsupportedOperationException(String.format(
+          "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s",
+          OldDoFn.class.getSimpleName(),
+          OldDoFn.ProcessContext.class.getSimpleName(),
+          OldDoFn.class.getSimpleName(),
+          DoFn.class.getSimpleName()));
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return newContext.pane();
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      throw new UnsupportedOperationException(String.format(
+          "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s",
+          OldDoFn.class.getSimpleName(),
+          OldDoFn.ProcessContext.class.getSimpleName(),
+          OldDoFn.class.getSimpleName(),
+          DoFn.class.getSimpleName()));
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return newContext.getPipelineOptions();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      newContext.output(output);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      newContext.outputWithTimestamp(output, timestamp);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      newContext.sideOutput(tag, output);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      newContext.sideOutputWithTimestamp(tag, output, timestamp);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return null;
+    }
+  }
+
+  private class AdaptedDoFn extends DoFn<InputT, OutputT> {
+
+    @StartBundle
+    public void startBundle(DoFn.Context c) throws Exception {
+      OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @ProcessElement
+    public void processElement(DoFn.ProcessContext c) throws Exception {
+      OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c));
+    }
+
+    @FinishBundle
+    public void finishBundle(DoFn.Context c) throws Exception {
+      OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @Override
+    public Duration getAllowedTimestampSkew() {
+      return OldDoFn.this.getAllowedTimestampSkew();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      OldDoFn.this.populateDisplayData(builder);
+    }
+
+    @Override
+    protected TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return OldDoFn.this.getInputTypeDescriptor();
+    }
+
+    @Override
+    protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return OldDoFn.this.getOutputTypeDescriptor();
+    }
+  }
 }


[22/50] incubator-beam git commit: Port ReifyTimestampAndWindowsDoFn to RequiresWindowAccess

Posted by th...@apache.org.
Port ReifyTimestampAndWindowsDoFn to RequiresWindowAccess

This should become either a DoFn or probably more appropriately,
just something internal to runners that actually require it to be
manifested as a DoFn at all.

As an intermediate migration step, this lessens the level to which
it depends on unsupported APIs.


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

Branch: refs/heads/apex-runner
Commit: b2350417f73ae6c34f849ff0e93d5bd93df3088d
Parents: 164ee56
Author: Kenneth Knowles <kl...@google.com>
Authored: Sun Oct 23 19:45:41 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/util/ReifyTimestampAndWindowsDoFn.java | 16 +++++-----------
 1 file changed, 5 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b2350417/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
index 8f3f540..6da4da0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.util;
 
 import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
 import org.apache.beam.sdk.values.KV;
 
 /**
@@ -28,20 +29,13 @@ import org.apache.beam.sdk.values.KV;
  * @param <V> the type of the values of the input {@code PCollection}
  */
 @SystemDoFnInternal
-public class ReifyTimestampAndWindowsDoFn<K, V>
-    extends OldDoFn<KV<K, V>, KV<K, WindowedValue<V>>> {
+public class ReifyTimestampAndWindowsDoFn<K, V> extends OldDoFn<KV<K, V>, KV<K, WindowedValue<V>>>
+    implements RequiresWindowAccess {
   @Override
-  public void processElement(ProcessContext c)
-      throws Exception {
+  public void processElement(ProcessContext c) throws Exception {
     KV<K, V> kv = c.element();
     K key = kv.getKey();
     V value = kv.getValue();
-    c.output(KV.of(
-        key,
-        WindowedValue.of(
-            value,
-            c.timestamp(),
-            c.windowingInternals().windows(),
-            c.pane())));
+    c.output(KV.of(key, WindowedValue.of(value, c.timestamp(), c.window(), c.pane())));
   }
 }


[26/50] incubator-beam git commit: Fix binding of aggregator delegation in DoFnAdapters

Posted by th...@apache.org.
Fix binding of aggregator delegation in DoFnAdapters


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

Branch: refs/heads/apex-runner
Commit: 2ab955d5501c87adea3fd17ad5dd1ad73be13364
Parents: 4e185d0
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 12:44:47 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2ab955d5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
index 0b0d207..ca724cd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
@@ -278,6 +278,7 @@ public class DoFnAdapters {
     private ContextAdapter(DoFn<InputT, OutputT> fn, OldDoFn<InputT, OutputT>.Context context) {
       fn.super();
       this.context = context;
+      super.setupDelegateAggregators();
     }
 
     @Override


[45/50] incubator-beam git commit: Add timestamps to maven output on Travis CI

Posted by th...@apache.org.
Add timestamps to maven output on Travis CI


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

Branch: refs/heads/apex-runner
Commit: 7f82a573d00a5a30331b7bbb8757e55f4a2d93ae
Parents: 44b9f4e
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 14:34:17 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 15:04:14 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7f82a573/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index a9ba6d1..277f883 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -48,7 +48,9 @@ before_install:
   - cat ~/.m2/settings.xml
   - sed -i.bak -e 's|https://nexus.codehaus.org/snapshots/|https://oss.sonatype.org/content/repositories/codehaus-snapshots/|g' ~/.m2/settings.xml
   - cat ~/.m2/settings.xml
-  - echo "MAVEN_OPTS='-Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal'" > ~/.mavenrc
+  - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
+  - echo $'MAVEN_OPTS="$MAVEN_OPTS -Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS"' >> ~/.mavenrc
+  - cat ~/.mavenrc
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
   - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
   - export BEAM_SUREFIRE_ARGLINE="-Xmx512m"


[27/50] incubator-beam git commit: Make DoFnTester aggregator initialization idempotent

Posted by th...@apache.org.
Make DoFnTester aggregator initialization idempotent


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

Branch: refs/heads/apex-runner
Commit: 043ebecacf7a8e96939b025afa8480c6df2f3b41
Parents: 2ab955d
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 13:35:29 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/transforms/DoFnTester.java    | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/043ebeca/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 302bb02..7995719 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
@@ -543,6 +543,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     private <AinT, AccT, AoutT> Aggregator<AinT, AoutT> aggregator(
         final String name,
         final CombineFn<AinT, AccT, AoutT> combiner) {
+
       Aggregator<AinT, AoutT> aggregator = new Aggregator<AinT, AoutT>() {
         @Override
         public void addValue(AinT value) {
@@ -561,7 +562,22 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
           return combiner;
         }
       };
-      accumulators.put(name, combiner.createAccumulator());
+
+      // Aggregator instantiation is idempotent
+      if (accumulators.containsKey(name)) {
+        Class<?> currentAccumClass = accumulators.get(name).getClass();
+        Class<?> createAccumClass = combiner.createAccumulator().getClass();
+        checkState(
+            currentAccumClass.isAssignableFrom(createAccumClass),
+            "Aggregator %s already initialized with accumulator type %s "
+                + "but was re-initialized with accumulator type %s",
+            name,
+            currentAccumClass,
+            createAccumClass);
+
+      } else {
+        accumulators.put(name, combiner.createAccumulator());
+      }
       return aggregator;
     }
 


[29/50] incubator-beam git commit: Fix ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers()

Posted by th...@apache.org.
Fix ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers()

Before, it was only removing the end-of-window timer. Now, it also
removed that GC 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/59f042c8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/59f042c8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/59f042c8

Branch: refs/heads/apex-runner
Commit: 59f042c85a7265d95620afd5aea565416cd35ff9
Parents: caf1fb7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 10:36:20 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 10:39:07 2016 +0200

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/59f042c8/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
index 78c4e0b..66fb27c 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
@@ -943,7 +943,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> impleme
     directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
     Instant gc = garbageCollectionTime(directContext.window());
     if (gc.isAfter(eow)) {
-      directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+      directContext.timers().deleteTimer(gc, TimeDomain.EVENT_TIME);
     }
   }
 


[21/50] incubator-beam git commit: Fix NPE in SimpleDoFnRunner

Posted by th...@apache.org.
Fix NPE in SimpleDoFnRunner


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

Branch: refs/heads/apex-runner
Commit: 0d500efe3b2d4248e2889dfcf36423aa473ee0a5
Parents: b235041
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 11:30:33 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0d500efe/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 4eda376..0360bc2 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
@@ -19,6 +19,7 @@ package org.apache.beam.runners.core;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.util.Collection;
@@ -511,7 +512,7 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
 
     @Override
     public BoundedWindow window() {
-      return null;
+      return Iterables.getOnlyElement(windows);
     }
 
     @Override


[11/50] incubator-beam git commit: [BEAM-755] Fix typo and dependencies in beam-runners-direct-java

Posted by th...@apache.org.
[BEAM-755] Fix typo and dependencies in beam-runners-direct-java

This closes #1140


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

Branch: refs/heads/apex-runner
Commit: ea8583c4f45e56a6ecb4b6dbe7eb255e5f27693d
Parents: 18e5d12 5f946c6
Author: Luke Cwik <lc...@google.com>
Authored: Fri Oct 21 15:26:17 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Oct 21 15:26:17 2016 -0700

----------------------------------------------------------------------
 pom.xml                                    |  4 ++--
 runners/direct-java/pom.xml                | 11 +++++++++--
 runners/google-cloud-dataflow-java/pom.xml |  2 +-
 3 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-beam git commit: Choose BigQuery Write implementation based on Input PCollection

Posted by th...@apache.org.
Choose BigQuery Write implementation based on Input PCollection

Stop using PipelineOptions, and instead use the boundedness of the input
to choose how to write to BigQuery. This means that runners that don't
use the streaming flag still function appropriately.

Fixes BEAM-746


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

Branch: refs/heads/apex-runner
Commit: 75d137b81ae240ad1a2e8942627738a6871581c1
Parents: 6d9d8bc
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 12 17:58:57 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:37:41 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  7 ++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 35 +++++++++++++++-----
 2 files changed, 31 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75d137b8/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 5626067..50c5ae9 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -119,6 +119,7 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
@@ -1748,9 +1749,9 @@ public class BigQueryIO {
         BigQueryOptions options = p.getOptions().as(BigQueryOptions.class);
         BigQueryServices bqServices = getBigQueryServices();
 
-        // In a streaming job, or when a tablespec function is defined, we use StreamWithDeDup
-        // and BigQuery's streaming import API.
-        if (options.isStreaming() || tableRefFunction != null) {
+        // When writing an Unbounded PCollection, or when a tablespec function is defined, we use
+        // StreamWithDeDup and BigQuery's streaming import API.
+        if (input.isBounded() == IsBounded.UNBOUNDED || tableRefFunction != null) {
           return input.apply(
               new StreamWithDeDup(getTable(), tableRefFunction, getSchema(), bqServices));
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/75d137b8/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 74c35a6..51a69a2 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
@@ -91,6 +91,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.TableRowJsonCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.io.CountingSource;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryQuerySource;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryTableSource;
@@ -120,8 +121,10 @@ 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.DoFnTester;
+import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -1190,13 +1193,12 @@ public class BigQueryIOTest implements Serializable {
     assertThat(displayData, hasDisplayItem("validation", false));
   }
 
-  private void testWriteValidatesDataset(boolean streaming) throws Exception {
+  private void testWriteValidatesDataset(boolean unbounded) throws Exception {
     String projectId = "someproject";
     String datasetId = "somedataset";
 
     BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
     options.setProject(projectId);
-    options.setStreaming(streaming);
 
     FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
         .withJobService(mockJobService)
@@ -1210,17 +1212,34 @@ public class BigQueryIOTest implements Serializable {
     tableRef.setDatasetId(datasetId);
     tableRef.setTableId("sometable");
 
+    PCollection<TableRow> tableRows;
+    if (unbounded) {
+      tableRows =
+          p.apply(CountingInput.unbounded())
+              .apply(
+                  MapElements.via(
+                      new SimpleFunction<Long, TableRow>() {
+                        @Override
+                        public TableRow apply(Long input) {
+                          return null;
+                        }
+                      }))
+              .setCoder(TableRowJsonCoder.of());
+    } else {
+      tableRows = p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()));
+    }
+
     thrown.expect(RuntimeException.class);
     // Message will be one of following depending on the execution environment.
     thrown.expectMessage(
         Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
             .or(Matchers.containsString("BigQuery dataset not found for table")));
-    p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
-     .apply(BigQueryIO.Write
-         .to(tableRef)
-         .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
-         .withSchema(new TableSchema())
-         .withTestServices(fakeBqServices));
+    tableRows
+        .apply(
+            BigQueryIO.Write.to(tableRef)
+                .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+                .withSchema(new TableSchema())
+                .withTestServices(fakeBqServices));
   }
 
   @Test


[14/50] incubator-beam git commit: [BEAM-794] Differ combining in case of merging windows with sideInputs.

Posted by th...@apache.org.
[BEAM-794] Differ combining in case of merging windows with sideInputs.


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

Branch: refs/heads/apex-runner
Commit: a7cc8206cbbc6ac10e71a0563da2fea4c708277b
Parents: 4c90582
Author: Sela <an...@paypal.com>
Authored: Fri Oct 21 16:00:57 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Sat Oct 22 12:23:33 2016 +0300

----------------------------------------------------------------------
 .../apache/beam/runners/spark/SparkRunner.java  | 33 +++++++++++++++++++-
 1 file changed, 32 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a7cc8206/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 cad53be..b17c38c 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
@@ -19,6 +19,7 @@
 package org.apache.beam.runners.spark;
 
 import java.util.Collection;
+import java.util.List;
 import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
 import org.apache.beam.runners.spark.translation.EvaluationContext;
 import org.apache.beam.runners.spark.translation.SparkContextFactory;
@@ -34,11 +35,13 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+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;
@@ -206,7 +209,7 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
         @SuppressWarnings("unchecked")
         Class<PTransform<?, ?>> transformClass =
             (Class<PTransform<?, ?>>) node.getTransform().getClass();
-        if (translator.hasTranslation(transformClass)) {
+        if (translator.hasTranslation(transformClass) && !shouldDefer(node)) {
           LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName());
           LOG.debug("Composite transform class: '{}'", transformClass);
           doVisitTransform(node);
@@ -216,6 +219,34 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
       return CompositeBehavior.ENTER_TRANSFORM;
     }
 
+    private boolean shouldDefer(TransformTreeNode node) {
+      PInput input = node.getInput();
+      // if the input is not a PCollection, or it is but with non merging windows, don't defer.
+      if (!(input instanceof PCollection)
+          || ((PCollection) input).getWindowingStrategy().getWindowFn().isNonMerging()) {
+        return false;
+      }
+      // so far we know that the input is a PCollection with merging windows.
+      // check for sideInput in case of a Combine transform.
+      PTransform<?, ?> transform = node.getTransform();
+      boolean hasSideInput = false;
+      if (transform instanceof Combine.PerKey) {
+        List<PCollectionView<?>> sideInputs = ((Combine.PerKey<?, ?, ?>) transform).getSideInputs();
+        hasSideInput = sideInputs != null && !sideInputs.isEmpty();
+      } else if (transform instanceof Combine.Globally) {
+        List<PCollectionView<?>> sideInputs = ((Combine.Globally<?, ?>) transform).getSideInputs();
+        hasSideInput = sideInputs != null && !sideInputs.isEmpty();
+      }
+      // defer if sideInputs are defined.
+      if (hasSideInput) {
+        LOG.info("Deferring combine transformation {} for job {}", transform,
+            ctxt.getPipeline().getOptions().getJobName());
+        return true;
+      }
+      // default.
+      return false;
+    }
+
     @Override
     public void visitPrimitiveTransform(TransformTreeNode node) {
       doVisitTransform(node);


[15/50] incubator-beam git commit: This closes #1153

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


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

Branch: refs/heads/apex-runner
Commit: a9a41eb94bee1200c93a735bbe54e80a5d776e3e
Parents: 4c90582 a7cc820
Author: Sela <an...@paypal.com>
Authored: Sat Oct 22 12:24:21 2016 +0300
Committer: Sela <an...@paypal.com>
Committed: Sat Oct 22 12:24:21 2016 +0300

----------------------------------------------------------------------
 .../apache/beam/runners/spark/SparkRunner.java  | 33 +++++++++++++++++++-
 1 file changed, 32 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/apex-runner
Commit: 7de8d571d6a86de0dfdbb1756e101c72319f5db4
Parents: d285e85 6291a6f
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 18:03:29 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 18:03:29 2016 -0700

----------------------------------------------------------------------
 pom.xml         | 3 ---
 runners/pom.xml | 1 +
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------



[03/50] incubator-beam git commit: Rename SimpleDoFnRunner to SimpleOldDoFnRunner and remove public access

Posted by th...@apache.org.
Rename SimpleDoFnRunner to SimpleOldDoFnRunner and remove public access


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

Branch: refs/heads/apex-runner
Commit: 8ff6c9d9ae0294b60164a36c096a6c6d59047908
Parents: f043865
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 20:08:26 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:32:06 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunners.java   |   2 +-
 .../beam/runners/core/SimpleDoFnRunner.java     | 521 -------------------
 .../beam/runners/core/SimpleOldDoFnRunner.java  | 521 +++++++++++++++++++
 .../beam/runners/core/SimpleDoFnRunnerTest.java |  88 ----
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  88 ++++
 5 files changed, 610 insertions(+), 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8ff6c9d9/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 41eb9d1..13cd161 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
@@ -62,7 +62,7 @@ public class DoFnRunners {
       StepContext stepContext,
       AggregatorFactory aggregatorFactory,
       WindowingStrategy<?, ?> windowingStrategy) {
-    return new SimpleDoFnRunner<>(
+    return new SimpleOldDoFnRunner<>(
         options,
         fn,
         sideInputReader,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8ff6c9d9/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
deleted file mode 100644
index 8f25705..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ /dev/null
@@ -1,521 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.core;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import org.apache.beam.runners.core.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.OldDoFn;
-import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
-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.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.TimerInternals;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingInternals;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-
-/**
- * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in.
- *
- * @param <InputT> the type of the {@link OldDoFn} (main) input elements
- * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
- */
-public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-
-  /** The {@link OldDoFn} being run. */
-  private final OldDoFn<InputT, OutputT> fn;
-  /** The context used for running the {@link OldDoFn}. */
-  private final DoFnContext<InputT, OutputT> context;
-
-  public SimpleDoFnRunner(PipelineOptions options, OldDoFn<InputT, OutputT> fn,
-      SideInputReader sideInputReader,
-      OutputManager outputManager,
-      TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
-      AggregatorFactory aggregatorFactory, WindowingStrategy<?, ?> windowingStrategy) {
-    this.fn = fn;
-    this.context = new DoFnContext<>(
-        options,
-        fn,
-        sideInputReader,
-        outputManager,
-        mainOutputTag,
-        sideOutputTags,
-        stepContext,
-        aggregatorFactory,
-        windowingStrategy == null ? null : windowingStrategy.getWindowFn());
-  }
-
-  @Override
-  public void startBundle() {
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.startBundle(context);
-    } catch (Throwable t) {
-      // Exception in user code.
-      throw wrapUserCodeException(t);
-    }
-  }
-
-  @Override
-  public void processElement(WindowedValue<InputT> elem) {
-    if (elem.getWindows().size() <= 1
-        || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
-            && context.sideInputReader.isEmpty())) {
-      invokeProcessElement(elem);
-    } else {
-      // We could modify the windowed value (and the processContext) to
-      // avoid repeated allocations, but this is more straightforward.
-      for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
-        invokeProcessElement(windowedValue);
-      }
-    }
-  }
-
-  private void invokeProcessElement(WindowedValue<InputT> elem) {
-    final OldDoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.processElement(processContext);
-    } catch (Exception ex) {
-      throw wrapUserCodeException(ex);
-    }
-  }
-
-  @Override
-  public void finishBundle() {
-    // This can contain user code. Wrap it in case it throws an exception.
-    try {
-      fn.finishBundle(context);
-    } catch (Throwable t) {
-      // Exception in user code.
-      throw wrapUserCodeException(t);
-    }
-  }
-
-  /**
-   * Returns a new {@link OldDoFn.ProcessContext} for the given element.
-   */
-  private OldDoFn<InputT, OutputT>.ProcessContext createProcessContext(
-      WindowedValue<InputT> elem) {
-    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
-  }
-
-  private RuntimeException wrapUserCodeException(Throwable t) {
-    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
-  }
-
-  private boolean isSystemDoFn() {
-    return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
-  }
-
-  /**
-   * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}.
-   *
-   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
-   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
-   */
-  private static class DoFnContext<InputT, OutputT> extends OldDoFn<InputT, OutputT>.Context {
-    private static final int MAX_SIDE_OUTPUTS = 1000;
-
-    final PipelineOptions options;
-    final OldDoFn<InputT, OutputT> fn;
-    final SideInputReader sideInputReader;
-    final OutputManager outputManager;
-    final TupleTag<OutputT> mainOutputTag;
-    final StepContext stepContext;
-    final AggregatorFactory aggregatorFactory;
-    final WindowFn<?, ?> windowFn;
-
-    /**
-     * The set of known output tags, some of which may be undeclared, so we can throw an
-     * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
-     */
-    private Set<TupleTag<?>> outputTags;
-
-    public DoFnContext(PipelineOptions options,
-                       OldDoFn<InputT, OutputT> fn,
-                       SideInputReader sideInputReader,
-                       OutputManager outputManager,
-                       TupleTag<OutputT> mainOutputTag,
-                       List<TupleTag<?>> sideOutputTags,
-                       StepContext stepContext,
-                       AggregatorFactory aggregatorFactory,
-                       WindowFn<?, ?> windowFn) {
-      fn.super();
-      this.options = options;
-      this.fn = fn;
-      this.sideInputReader = sideInputReader;
-      this.outputManager = outputManager;
-      this.mainOutputTag = mainOutputTag;
-      this.outputTags = Sets.newHashSet();
-
-      outputTags.add(mainOutputTag);
-      for (TupleTag<?> sideOutputTag : sideOutputTags) {
-        outputTags.add(sideOutputTag);
-      }
-
-      this.stepContext = stepContext;
-      this.aggregatorFactory = aggregatorFactory;
-      this.windowFn = windowFn;
-      super.setupDelegateAggregators();
-    }
-
-    //////////////////////////////////////////////////////////////////////////////
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return options;
-    }
-
-    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
-        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
-      final Instant inputTimestamp = timestamp;
-
-      if (timestamp == null) {
-        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      }
-
-      if (windows == null) {
-        try {
-          // The windowFn can never succeed at accessing the element, so its type does not
-          // matter here
-          @SuppressWarnings("unchecked")
-          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
-          windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
-            @Override
-            public Object element() {
-              throw new UnsupportedOperationException(
-                  "WindowFn attempted to access input element when none was available");
-            }
-
-            @Override
-            public Instant timestamp() {
-              if (inputTimestamp == null) {
-                throw new UnsupportedOperationException(
-                    "WindowFn attempted to access input timestamp when none was available");
-              }
-              return inputTimestamp;
-            }
-
-            @Override
-            public W window() {
-              throw new UnsupportedOperationException(
-                  "WindowFn attempted to access input windows when none were available");
-            }
-          });
-        } catch (Exception e) {
-          throw UserCodeException.wrap(e);
-        }
-      }
-
-      return WindowedValue.of(output, timestamp, windows, pane);
-    }
-
-    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-      if (!sideInputReader.contains(view)) {
-        throw new IllegalArgumentException("calling sideInput() with unknown view");
-      }
-      BoundedWindow sideInputWindow =
-          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
-      return sideInputReader.get(view, sideInputWindow);
-    }
-
-    void outputWindowedValue(
-        OutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
-    }
-
-    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
-      outputManager.output(mainOutputTag, windowedElem);
-      if (stepContext != null) {
-        stepContext.noteOutput(windowedElem);
-      }
-    }
-
-    private <T> void sideOutputWindowedValue(TupleTag<T> tag,
-                                               T output,
-                                               Instant timestamp,
-                                               Collection<? extends BoundedWindow> windows,
-                                               PaneInfo pane) {
-      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
-    }
-
-    private <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
-      if (!outputTags.contains(tag)) {
-        // This tag wasn't declared nor was it seen before during this execution.
-        // Thus, this must be a new, undeclared and unconsumed output.
-        // To prevent likely user errors, enforce the limit on the number of side
-        // outputs.
-        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
-          throw new IllegalArgumentException(
-              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
-        }
-        outputTags.add(tag);
-      }
-
-      outputManager.output(tag, windowedElem);
-      if (stepContext != null) {
-        stepContext.noteSideOutput(tag, windowedElem);
-      }
-    }
-
-    // Following implementations of output, outputWithTimestamp, and sideOutput
-    // are only accessible in OldDoFn.startBundle and OldDoFn.finishBundle, and will be shadowed by
-    // ProcessContext's versions in OldDoFn.processElement.
-    @Override
-    public void output(OutputT output) {
-      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
-      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
-      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
-    }
-
-    @Override
-    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
-        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null");
-      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
-    }
-  }
-
-  /**
-   * A concrete implementation of {@link OldDoFn.ProcessContext} used for running a {@link OldDoFn}
-   * over a single element.
-   *
-   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
-   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
-   */
-  private static class DoFnProcessContext<InputT, OutputT>
-      extends OldDoFn<InputT, OutputT>.ProcessContext {
-
-
-    final OldDoFn<InputT, OutputT> fn;
-    final DoFnContext<InputT, OutputT> context;
-    final WindowedValue<InputT> windowedValue;
-
-    public DoFnProcessContext(OldDoFn<InputT, OutputT> fn,
-                              DoFnContext<InputT, OutputT> context,
-                              WindowedValue<InputT> windowedValue) {
-      fn.super();
-      this.fn = fn;
-      this.context = context;
-      this.windowedValue = windowedValue;
-    }
-
-    @Override
-    public PipelineOptions getPipelineOptions() {
-      return context.getPipelineOptions();
-    }
-
-    @Override
-    public InputT element() {
-      return windowedValue.getValue();
-    }
-
-    @Override
-    public <T> T sideInput(PCollectionView<T> view) {
-      checkNotNull(view, "View passed to sideInput cannot be null");
-      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
-      BoundedWindow window;
-      if (!windowIter.hasNext()) {
-        if (context.windowFn instanceof GlobalWindows) {
-          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
-          // without windows
-          window = GlobalWindow.INSTANCE;
-        } else {
-          throw new IllegalStateException(
-              "sideInput called when main input element is not in any windows");
-        }
-      } else {
-        window = windowIter.next();
-        if (windowIter.hasNext()) {
-          throw new IllegalStateException(
-              "sideInput called when main input element is in multiple windows");
-        }
-      }
-      return context.sideInput(view, window);
-    }
-
-    @Override
-    public BoundedWindow window() {
-      if (!(fn instanceof OldDoFn.RequiresWindowAccess)) {
-        throw new UnsupportedOperationException(
-            "window() is only available in the context of a OldDoFn marked as"
-                + "RequiresWindowAccess.");
-      }
-      return Iterables.getOnlyElement(windows());
-    }
-
-    @Override
-    public PaneInfo pane() {
-      return windowedValue.getPane();
-    }
-
-    @Override
-    public void output(OutputT output) {
-      context.outputWindowedValue(windowedValue.withValue(output));
-    }
-
-    @Override
-    public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      checkTimestamp(timestamp);
-      context.outputWindowedValue(output, timestamp,
-          windowedValue.getWindows(), windowedValue.getPane());
-    }
-
-    void outputWindowedValue(
-        OutputT output,
-        Instant timestamp,
-        Collection<? extends BoundedWindow> windows,
-        PaneInfo pane) {
-      context.outputWindowedValue(output, timestamp, windows, pane);
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
-      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
-    }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
-      checkTimestamp(timestamp);
-      context.sideOutputWindowedValue(
-          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
-    }
-
-    @Override
-    public Instant timestamp() {
-      return windowedValue.getTimestamp();
-    }
-
-    public Collection<? extends BoundedWindow> windows() {
-      return windowedValue.getWindows();
-    }
-
-    private void checkTimestamp(Instant timestamp) {
-      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
-        throw new IllegalArgumentException(String.format(
-            "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
-            + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
-            + "OldDoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
-            timestamp, windowedValue.getTimestamp(),
-            PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
-      }
-    }
-
-    @Override
-    public WindowingInternals<InputT, OutputT> windowingInternals() {
-      return new WindowingInternals<InputT, OutputT>() {
-        @Override
-        public void outputWindowedValue(OutputT output, Instant timestamp,
-            Collection<? extends BoundedWindow> windows, PaneInfo pane) {
-          context.outputWindowedValue(output, timestamp, windows, pane);
-        }
-
-        @Override
-        public Collection<? extends BoundedWindow> windows() {
-          return windowedValue.getWindows();
-        }
-
-        @Override
-        public PaneInfo pane() {
-          return windowedValue.getPane();
-        }
-
-        @Override
-        public TimerInternals timerInternals() {
-          return context.stepContext.timerInternals();
-        }
-
-        @Override
-        public <T> void writePCollectionViewData(
-            TupleTag<?> tag,
-            Iterable<WindowedValue<T>> data,
-            Coder<T> elemCoder) throws IOException {
-          @SuppressWarnings("unchecked")
-          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
-
-          context.stepContext.writePCollectionViewData(
-              tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
-              window(), windowCoder);
-        }
-
-        @Override
-        public StateInternals<?> stateInternals() {
-          return context.stepContext.stateInternals();
-        }
-
-        @Override
-        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
-          return context.sideInput(view, mainInputWindow);
-        }
-      };
-    }
-
-    @Override
-    protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
-        createAggregatorInternal(
-            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
-      return context.createAggregatorInternal(name, combiner);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8ff6c9d9/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
new file mode 100644
index 0000000..1298fc8
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java
@@ -0,0 +1,521 @@
+/*
+ * 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 com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.OldDoFn;
+import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess;
+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.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.TimerInternals;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+/**
+ * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+ * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+ */
+class SimpleOldDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+  /** The {@link OldDoFn} being run. */
+  private final OldDoFn<InputT, OutputT> fn;
+  /** The context used for running the {@link OldDoFn}. */
+  private final DoFnContext<InputT, OutputT> context;
+
+  public SimpleOldDoFnRunner(PipelineOptions options, OldDoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
+      AggregatorFactory aggregatorFactory, WindowingStrategy<?, ?> windowingStrategy) {
+    this.fn = fn;
+    this.context = new DoFnContext<>(
+        options,
+        fn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy == null ? null : windowingStrategy.getWindowFn());
+  }
+
+  @Override
+  public void startBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.startBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> elem) {
+    if (elem.getWindows().size() <= 1
+        || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
+            && context.sideInputReader.isEmpty())) {
+      invokeProcessElement(elem);
+    } else {
+      // We could modify the windowed value (and the processContext) to
+      // avoid repeated allocations, but this is more straightforward.
+      for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
+        invokeProcessElement(windowedValue);
+      }
+    }
+  }
+
+  private void invokeProcessElement(WindowedValue<InputT> elem) {
+    final OldDoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.processElement(processContext);
+    } catch (Exception ex) {
+      throw wrapUserCodeException(ex);
+    }
+  }
+
+  @Override
+  public void finishBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      fn.finishBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  /**
+   * Returns a new {@link OldDoFn.ProcessContext} for the given element.
+   */
+  private OldDoFn<InputT, OutputT>.ProcessContext createProcessContext(
+      WindowedValue<InputT> elem) {
+    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
+  }
+
+  private RuntimeException wrapUserCodeException(Throwable t) {
+    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
+  }
+
+  private boolean isSystemDoFn() {
+    return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+  }
+
+  /**
+   * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}.
+   *
+   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+   */
+  private static class DoFnContext<InputT, OutputT> extends OldDoFn<InputT, OutputT>.Context {
+    private static final int MAX_SIDE_OUTPUTS = 1000;
+
+    final PipelineOptions options;
+    final OldDoFn<InputT, OutputT> fn;
+    final SideInputReader sideInputReader;
+    final OutputManager outputManager;
+    final TupleTag<OutputT> mainOutputTag;
+    final StepContext stepContext;
+    final AggregatorFactory aggregatorFactory;
+    final WindowFn<?, ?> windowFn;
+
+    /**
+     * The set of known output tags, some of which may be undeclared, so we can throw an
+     * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
+     */
+    private Set<TupleTag<?>> outputTags;
+
+    public DoFnContext(PipelineOptions options,
+                       OldDoFn<InputT, OutputT> fn,
+                       SideInputReader sideInputReader,
+                       OutputManager outputManager,
+                       TupleTag<OutputT> mainOutputTag,
+                       List<TupleTag<?>> sideOutputTags,
+                       StepContext stepContext,
+                       AggregatorFactory aggregatorFactory,
+                       WindowFn<?, ?> windowFn) {
+      fn.super();
+      this.options = options;
+      this.fn = fn;
+      this.sideInputReader = sideInputReader;
+      this.outputManager = outputManager;
+      this.mainOutputTag = mainOutputTag;
+      this.outputTags = Sets.newHashSet();
+
+      outputTags.add(mainOutputTag);
+      for (TupleTag<?> sideOutputTag : sideOutputTags) {
+        outputTags.add(sideOutputTag);
+      }
+
+      this.stepContext = stepContext;
+      this.aggregatorFactory = aggregatorFactory;
+      this.windowFn = windowFn;
+      super.setupDelegateAggregators();
+    }
+
+    //////////////////////////////////////////////////////////////////////////////
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
+        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
+      final Instant inputTimestamp = timestamp;
+
+      if (timestamp == null) {
+        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      }
+
+      if (windows == null) {
+        try {
+          // The windowFn can never succeed at accessing the element, so its type does not
+          // matter here
+          @SuppressWarnings("unchecked")
+          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
+          windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
+            @Override
+            public Object element() {
+              throw new UnsupportedOperationException(
+                  "WindowFn attempted to access input element when none was available");
+            }
+
+            @Override
+            public Instant timestamp() {
+              if (inputTimestamp == null) {
+                throw new UnsupportedOperationException(
+                    "WindowFn attempted to access input timestamp when none was available");
+              }
+              return inputTimestamp;
+            }
+
+            @Override
+            public W window() {
+              throw new UnsupportedOperationException(
+                  "WindowFn attempted to access input windows when none were available");
+            }
+          });
+        } catch (Exception e) {
+          throw UserCodeException.wrap(e);
+        }
+      }
+
+      return WindowedValue.of(output, timestamp, windows, pane);
+    }
+
+    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+      if (!sideInputReader.contains(view)) {
+        throw new IllegalArgumentException("calling sideInput() with unknown view");
+      }
+      BoundedWindow sideInputWindow =
+          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+      return sideInputReader.get(view, sideInputWindow);
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
+      outputManager.output(mainOutputTag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteOutput(windowedElem);
+      }
+    }
+
+    private <T> void sideOutputWindowedValue(TupleTag<T> tag,
+                                               T output,
+                                               Instant timestamp,
+                                               Collection<? extends BoundedWindow> windows,
+                                               PaneInfo pane) {
+      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    private <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
+      if (!outputTags.contains(tag)) {
+        // This tag wasn't declared nor was it seen before during this execution.
+        // Thus, this must be a new, undeclared and unconsumed output.
+        // To prevent likely user errors, enforce the limit on the number of side
+        // outputs.
+        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
+          throw new IllegalArgumentException(
+              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
+        }
+        outputTags.add(tag);
+      }
+
+      outputManager.output(tag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteSideOutput(tag, windowedElem);
+      }
+    }
+
+    // Following implementations of output, outputWithTimestamp, and sideOutput
+    // are only accessible in OldDoFn.startBundle and OldDoFn.finishBundle, and will be shadowed by
+    // ProcessContext's versions in OldDoFn.processElement.
+    @Override
+    public void output(OutputT output) {
+      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null");
+      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
+    }
+  }
+
+  /**
+   * A concrete implementation of {@link OldDoFn.ProcessContext} used for running a {@link OldDoFn}
+   * over a single element.
+   *
+   * @param <InputT> the type of the {@link OldDoFn} (main) input elements
+   * @param <OutputT> the type of the {@link OldDoFn} (main) output elements
+   */
+  private static class DoFnProcessContext<InputT, OutputT>
+      extends OldDoFn<InputT, OutputT>.ProcessContext {
+
+
+    final OldDoFn<InputT, OutputT> fn;
+    final DoFnContext<InputT, OutputT> context;
+    final WindowedValue<InputT> windowedValue;
+
+    public DoFnProcessContext(OldDoFn<InputT, OutputT> fn,
+                              DoFnContext<InputT, OutputT> context,
+                              WindowedValue<InputT> windowedValue) {
+      fn.super();
+      this.fn = fn;
+      this.context = context;
+      this.windowedValue = windowedValue;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public InputT element() {
+      return windowedValue.getValue();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      checkNotNull(view, "View passed to sideInput cannot be null");
+      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
+      BoundedWindow window;
+      if (!windowIter.hasNext()) {
+        if (context.windowFn instanceof GlobalWindows) {
+          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
+          // without windows
+          window = GlobalWindow.INSTANCE;
+        } else {
+          throw new IllegalStateException(
+              "sideInput called when main input element is not in any windows");
+        }
+      } else {
+        window = windowIter.next();
+        if (windowIter.hasNext()) {
+          throw new IllegalStateException(
+              "sideInput called when main input element is in multiple windows");
+        }
+      }
+      return context.sideInput(view, window);
+    }
+
+    @Override
+    public BoundedWindow window() {
+      if (!(fn instanceof OldDoFn.RequiresWindowAccess)) {
+        throw new UnsupportedOperationException(
+            "window() is only available in the context of a OldDoFn marked as"
+                + "RequiresWindowAccess.");
+      }
+      return Iterables.getOnlyElement(windows());
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return windowedValue.getPane();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.outputWindowedValue(windowedValue.withValue(output));
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      checkTimestamp(timestamp);
+      context.outputWindowedValue(output, timestamp,
+          windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      context.outputWindowedValue(output, timestamp, windows, pane);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkTimestamp(timestamp);
+      context.sideOutputWindowedValue(
+          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    @Override
+    public Instant timestamp() {
+      return windowedValue.getTimestamp();
+    }
+
+    public Collection<? extends BoundedWindow> windows() {
+      return windowedValue.getWindows();
+    }
+
+    private void checkTimestamp(Instant timestamp) {
+      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
+        throw new IllegalArgumentException(String.format(
+            "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+            + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+            + "OldDoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
+            timestamp, windowedValue.getTimestamp(),
+            PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
+      }
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return new WindowingInternals<InputT, OutputT>() {
+        @Override
+        public void outputWindowedValue(OutputT output, Instant timestamp,
+            Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+          context.outputWindowedValue(output, timestamp, windows, pane);
+        }
+
+        @Override
+        public Collection<? extends BoundedWindow> windows() {
+          return windowedValue.getWindows();
+        }
+
+        @Override
+        public PaneInfo pane() {
+          return windowedValue.getPane();
+        }
+
+        @Override
+        public TimerInternals timerInternals() {
+          return context.stepContext.timerInternals();
+        }
+
+        @Override
+        public <T> void writePCollectionViewData(
+            TupleTag<?> tag,
+            Iterable<WindowedValue<T>> data,
+            Coder<T> elemCoder) throws IOException {
+          @SuppressWarnings("unchecked")
+          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
+
+          context.stepContext.writePCollectionViewData(
+              tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
+              window(), windowCoder);
+        }
+
+        @Override
+        public StateInternals<?> stateInternals() {
+          return context.stepContext.stateInternals();
+        }
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+          return context.sideInput(view, mainInputWindow);
+        }
+      };
+    }
+
+    @Override
+    protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
+        createAggregatorInternal(
+            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
+      return context.createAggregatorInternal(name, combiner);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8ff6c9d9/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
deleted file mode 100644
index 773f26a..0000000
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.core;
-
-import static org.hamcrest.Matchers.is;
-import static org.mockito.Mockito.mock;
-
-import java.util.Arrays;
-import java.util.List;
-
-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;
-import org.apache.beam.sdk.values.TupleTag;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link SimpleDoFnRunner} functionality.
- */
-@RunWith(JUnit4.class)
-public class SimpleDoFnRunnerTest {
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Test
-  public void testExceptionsWrappedAsUserCodeException() {
-    ThrowingDoFn fn = new ThrowingDoFn();
-    DoFnRunner<String, String> runner = createRunner(fn);
-
-    thrown.expect(UserCodeException.class);
-    thrown.expectCause(is(fn.exceptionToThrow));
-
-    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
-  }
-
-  @Test
-  public void testSystemDoFnInternalExceptionsNotWrapped() {
-    ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
-    DoFnRunner<String, String> runner = createRunner(fn);
-
-    thrown.expect(is(fn.exceptionToThrow));
-
-    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
-  }
-
-  private DoFnRunner<String, String> createRunner(OldDoFn<String, String> fn) {
-    // Pass in only necessary parameters for the test
-    List<TupleTag<?>> sideOutputTags = Arrays.asList();
-    StepContext context = mock(StepContext.class);
-    return new SimpleDoFnRunner<>(
-          null, fn, null, null, null, sideOutputTags, context, null, null);
-  }
-
-  static class ThrowingDoFn extends OldDoFn<String, String> {
-    final Exception exceptionToThrow =
-        new UnsupportedOperationException("Expected exception");
-
-    @Override
-    public void processElement(ProcessContext c) throws Exception {
-      throw exceptionToThrow;
-    }
-  }
-
-  @SystemDoFnInternal
-  static class ThrowingSystemDoFn extends ThrowingDoFn {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8ff6c9d9/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
new file mode 100644
index 0000000..0e23dcb
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.is;
+import static org.mockito.Mockito.mock;
+
+import java.util.Arrays;
+import java.util.List;
+
+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;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link SimpleOldDoFnRunner} functionality.
+ */
+@RunWith(JUnit4.class)
+public class SimpleOldDoFnRunnerTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testExceptionsWrappedAsUserCodeException() {
+    ThrowingDoFn fn = new ThrowingDoFn();
+    DoFnRunner<String, String> runner = createRunner(fn);
+
+    thrown.expect(UserCodeException.class);
+    thrown.expectCause(is(fn.exceptionToThrow));
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+  }
+
+  @Test
+  public void testSystemDoFnInternalExceptionsNotWrapped() {
+    ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
+    DoFnRunner<String, String> runner = createRunner(fn);
+
+    thrown.expect(is(fn.exceptionToThrow));
+
+    runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+  }
+
+  private DoFnRunner<String, String> createRunner(OldDoFn<String, String> fn) {
+    // Pass in only necessary parameters for the test
+    List<TupleTag<?>> sideOutputTags = Arrays.asList();
+    StepContext context = mock(StepContext.class);
+    return new SimpleOldDoFnRunner<>(
+          null, fn, null, null, null, sideOutputTags, context, null, null);
+  }
+
+  static class ThrowingDoFn extends OldDoFn<String, String> {
+    final Exception exceptionToThrow =
+        new UnsupportedOperationException("Expected exception");
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      throw exceptionToThrow;
+    }
+  }
+
+  @SystemDoFnInternal
+  static class ThrowingSystemDoFn extends ThrowingDoFn {
+  }
+}


[09/50] incubator-beam git commit: Closes #1155

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


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

Branch: refs/heads/apex-runner
Commit: 18e5d128b7918f743c57d8c735452b559bea25e8
Parents: 1533511 4869ef1
Author: bchambers <bc...@google.com>
Authored: Fri Oct 21 13:27:37 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Fri Oct 21 13:27:37 2016 -0700

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



[13/50] incubator-beam git commit: [BEAM-790] Refactor PipelineOptionsFactory validation.

Posted by th...@apache.org.
[BEAM-790] Refactor PipelineOptionsFactory validation.

This closes #1151


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

Branch: refs/heads/apex-runner
Commit: 4c9058236e97434a31596e8e955f09eb8929c43a
Parents: ea8583c b809883
Author: Luke Cwik <lc...@google.com>
Authored: Fri Oct 21 15:43:26 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Oct 21 15:43:26 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     | 151 +++++++++++++------
 1 file changed, 105 insertions(+), 46 deletions(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/apex-runner
Commit: 6a971d6b13a035f74ce1b2d2efc6d37f84a2373c
Parents: c9f1406 7de8d57
Author: Thomas Weise <th...@apache.org>
Authored: Tue Oct 25 09:07:50 2016 -0700
Committer: Thomas Weise <th...@apache.org>
Committed: Tue Oct 25 09:07:50 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |  10 +-
 examples/java/pom.xml                           |   2 +-
 .../beam/examples/DebuggingWordCount.java       |   2 +-
 .../apache/beam/examples/WindowedWordCount.java |   2 +-
 .../org/apache/beam/examples/WordCount.java     |   4 +-
 .../common/ExampleBigQueryTableOptions.java     |   2 +-
 ...xamplePubsubTopicAndSubscriptionOptions.java |   2 +-
 .../common/ExamplePubsubTopicOptions.java       |   2 +-
 .../beam/examples/complete/AutoComplete.java    |   2 +-
 .../apache/beam/examples/complete/TfIdf.java    |   2 +-
 .../examples/complete/TopWikipediaSessions.java |   2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |   2 +-
 .../cookbook/CombinePerKeyExamples.java         |   2 +-
 .../beam/examples/cookbook/DeDupExample.java    |   4 +-
 .../beam/examples/cookbook/FilterExamples.java  |   2 +-
 .../beam/examples/cookbook/JoinExamples.java    |   2 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |   2 +-
 .../beam/examples/WindowedWordCountIT.java      |  11 +
 .../examples/cookbook/BigQueryTornadoesIT.java  |  14 +-
 examples/java8/pom.xml                          |   2 +-
 .../beam/examples/complete/game/GameStats.java  |   2 +-
 .../examples/complete/game/HourlyTeamScore.java |   2 +-
 .../examples/complete/game/LeaderBoard.java     |   2 +-
 .../beam/examples/complete/game/UserScore.java  |   2 +-
 examples/pom.xml                                |   2 +-
 pom.xml                                         |  17 +-
 runners/core-java/pom.xml                       |   2 +-
 .../beam/runners/core/AggregatorFactory.java    |  39 ++
 .../apache/beam/runners/core/DoFnRunner.java    |   6 +-
 .../apache/beam/runners/core/DoFnRunners.java   | 191 ++++--
 .../runners/core/ElementByteSizeObservable.java |   5 +-
 .../runners/core/GBKIntoKeyedWorkItems.java     |  17 +-
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   5 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   5 +
 .../runners/core/ReduceFnContextFactory.java    |   2 +-
 .../beam/runners/core/ReduceFnRunner.java       |  39 +-
 .../beam/runners/core/SimpleDoFnRunner.java     | 337 ++++++----
 .../beam/runners/core/SimpleOldDoFnRunner.java  | 521 +++++++++++++++
 .../apache/beam/runners/core/TriggerRunner.java | 247 -------
 .../core/UnboundedReadFromBoundedSource.java    |   2 +-
 .../core/triggers/AfterAllStateMachine.java     |   5 +
 .../AfterDelayFromFirstElementStateMachine.java |  17 +-
 .../core/triggers/AfterEachStateMachine.java    |   5 +
 .../core/triggers/AfterFirstStateMachine.java   |   6 +
 .../core/triggers/AfterPaneStateMachine.java    |   7 +
 .../triggers/AfterWatermarkStateMachine.java    |   4 +-
 .../runners/core/triggers/FinishedTriggers.java |   8 +-
 .../core/triggers/TriggerStateMachine.java      |   4 +-
 .../triggers/TriggerStateMachineRunner.java     |   2 +-
 .../core/triggers/TriggerStateMachines.java     | 215 +++++++
 .../beam/runners/core/ReduceFnRunnerTest.java   | 281 ++++----
 .../beam/runners/core/ReduceFnTester.java       | 157 +++--
 .../beam/runners/core/SimpleDoFnRunnerTest.java |  88 ---
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  88 +++
 .../beam/runners/core/SplittableParDoTest.java  |  14 +-
 .../core/triggers/TriggerStateMachinesTest.java | 199 ++++++
 runners/direct-java/pom.xml                     |  18 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   3 +-
 .../beam/runners/direct/DirectMetrics.java      |   7 +
 .../beam/runners/direct/DirectRunner.java       |   2 +-
 .../runners/direct/DoFnLifecycleManager.java    |  42 +-
 .../runners/direct/ForwardingPTransform.java    |   2 +-
 .../beam/runners/direct/ParDoEvaluator.java     |   4 +-
 .../direct/ParDoMultiEvaluatorFactory.java      |   6 +-
 .../direct/ParDoSingleEvaluatorFactory.java     |   5 +-
 .../direct/TestStreamEvaluatorFactory.java      |   2 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |  15 +-
 .../beam/runners/direct/WatermarkManager.java   |   4 +-
 .../beam/runners/direct/DirectRunnerTest.java   |  32 -
 ...leManagerRemovingTransformEvaluatorTest.java |  16 +-
 .../direct/DoFnLifecycleManagerTest.java        |  12 +-
 .../direct/DoFnLifecycleManagersTest.java       |  48 +-
 .../direct/ForwardingPTransformTest.java        |   7 +-
 .../direct/ParDoMultiEvaluatorFactoryTest.java  |  11 +
 .../direct/ParDoSingleEvaluatorFactoryTest.java |  11 +
 .../beam/runners/direct/SplittableDoFnTest.java |   8 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |  52 ++
 .../runners/direct/WatermarkManagerTest.java    |   2 +-
 runners/flink/examples/pom.xml                  |   2 +-
 runners/flink/pom.xml                           |   2 +-
 runners/flink/runner/pom.xml                    |   2 +-
 .../flink/FlinkDetachedRunnerResult.java        |  76 +++
 .../apache/beam/runners/flink/FlinkRunner.java  |   9 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |  11 +-
 .../beam/runners/flink/TestFlinkRunner.java     |   9 +-
 .../functions/FlinkProcessContext.java          |  10 +-
 runners/google-cloud-dataflow-java/pom.xml      |   4 +-
 .../dataflow/DataflowPipelineTranslator.java    | 133 ++--
 .../beam/runners/dataflow/DataflowRunner.java   |  20 +-
 .../DataflowUnboundedReadFromBoundedSource.java |   4 +-
 .../dataflow/internal/ReadTranslator.java       |   2 +-
 .../options/DataflowPipelineDebugOptions.java   |   4 +-
 .../options/DataflowPipelineOptions.java        |   2 +-
 .../DataflowPipelineWorkerPoolOptions.java      |   8 +-
 .../options/DataflowProfilingOptions.java       |   2 +-
 .../options/DataflowWorkerLoggingOptions.java   |   4 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    |  30 +-
 .../runners/dataflow/util/MonitoringUtil.java   |   2 +-
 .../beam/runners/dataflow/util/Stager.java      |   2 +-
 .../DataflowPipelineTranslatorTest.java         |  38 +-
 .../runners/dataflow/DataflowRunnerTest.java    |   6 +-
 .../dataflow/util/MonitoringUtilTest.java       |   4 +-
 runners/pom.xml                                 |   3 +-
 runners/spark/pom.xml                           |  75 ++-
 .../runners/spark/SparkPipelineOptions.java     |   6 +-
 .../apache/beam/runners/spark/SparkRunner.java  |  48 +-
 .../beam/runners/spark/TestSparkRunner.java     |   4 +-
 .../beam/runners/spark/examples/WordCount.java  |   2 +-
 .../apache/beam/runners/spark/io/SourceRDD.java |   6 +-
 .../runners/spark/translation/DoFnFunction.java |  74 +--
 .../spark/translation/EvaluationContext.java    |   6 +-
 .../translation/GroupCombineFunctions.java      | 235 ++++---
 .../spark/translation/MultiDoFnFunction.java    |  85 ++-
 .../translation/SparkAbstractCombineFn.java     | 134 ++++
 .../spark/translation/SparkContextFactory.java  |   3 +-
 .../spark/translation/SparkGlobalCombineFn.java | 260 ++++++++
 .../spark/translation/SparkKeyedCombineFn.java  | 273 ++++++++
 .../spark/translation/SparkProcessContext.java  | 160 ++++-
 .../spark/translation/TransformTranslator.java  | 143 ++--
 .../spark/translation/TranslationUtils.java     |  28 +-
 .../streaming/StreamingEvaluationContext.java   |  33 +-
 .../streaming/StreamingTransformTranslator.java | 126 ++--
 .../runners/spark/util/BroadcastHelper.java     |  26 -
 .../spark/util/SparkSideInputReader.java        |  95 +++
 .../apache/beam/runners/spark/DeDupTest.java    |  58 --
 .../beam/runners/spark/EmptyInputTest.java      |  74 ---
 .../beam/runners/spark/SimpleWordCountTest.java | 107 ---
 .../apache/beam/runners/spark/TfIdfTest.java    | 259 --------
 .../spark/translation/CombineGloballyTest.java  | 101 ---
 .../spark/translation/CombinePerKeyTest.java    |  78 ---
 .../spark/translation/DoFnOutputTest.java       |  65 --
 .../translation/MultiOutputWordCountTest.java   | 174 -----
 .../spark/translation/SerializationTest.java    | 199 ------
 .../spark/translation/SideEffectsTest.java      |  20 +-
 .../translation/SparkPipelineOptionsTest.java   |  42 --
 .../translation/TransformTranslatorTest.java    | 104 ---
 .../translation/WindowedWordCountTest.java      | 116 ----
 .../streaming/EmptyStreamAssertionTest.java     |  10 +-
 .../ResumeFromCheckpointStreamingTest.java      |   2 +-
 .../streaming/SimpleStreamingWordCountTest.java |   1 -
 .../streaming/utils/PAssertStreaming.java       |   6 +-
 sdks/java/build-tools/pom.xml                   |   2 +-
 .../src/main/resources/beam/checkstyle.xml      |   9 +-
 .../src/main/resources/beam/suppressions.xml    |  12 +-
 sdks/java/core/pom.xml                          |   2 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  12 +-
 .../org/apache/beam/sdk/PipelineResult.java     |   4 +-
 .../beam/sdk/annotations/Experimental.java      |   4 +-
 .../sdk/coders/CannotProvideCoderException.java |   2 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |  26 +-
 .../apache/beam/sdk/coders/CoderFactory.java    |   4 +-
 .../apache/beam/sdk/coders/CoderProvider.java   |   2 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   4 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |   6 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   4 +
 .../java/org/apache/beam/sdk/io/AvroSource.java |  97 ++-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |   9 +-
 .../apache/beam/sdk/io/CompressedSource.java    |   8 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  14 +-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   4 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |   5 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   4 +-
 .../sdk/io/SerializableAvroCodecFactory.java    |   2 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |   8 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |   6 +-
 .../org/apache/beam/sdk/options/GcpOptions.java |   8 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |   4 +-
 .../beam/sdk/options/GoogleApiDebugOptions.java |   2 +-
 .../beam/sdk/options/PipelineOptions.java       |  12 +-
 .../sdk/options/PipelineOptionsFactory.java     | 196 ++++--
 .../sdk/options/ProxyInvocationHandler.java     | 178 ++---
 .../apache/beam/sdk/options/ValueProvider.java  |  25 +-
 .../sdk/runners/PipelineRunnerRegistrar.java    |   2 +-
 .../beam/sdk/testing/BigqueryMatcher.java       | 239 +++++++
 .../org/apache/beam/sdk/testing/PAssert.java    |   4 +-
 .../apache/beam/sdk/transforms/Aggregator.java  |  11 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  82 ++-
 .../beam/sdk/transforms/CombineFnBase.java      |  17 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |  33 +-
 .../beam/sdk/transforms/CombineWithContext.java |   3 +-
 .../sdk/transforms/DelegatingAggregator.java    | 125 ++++
 .../org/apache/beam/sdk/transforms/DoFn.java    | 135 +++-
 .../beam/sdk/transforms/DoFnAdapters.java       |   5 +-
 .../apache/beam/sdk/transforms/DoFnTester.java  |  18 +-
 .../beam/sdk/transforms/FlatMapElements.java    |   6 +-
 .../apache/beam/sdk/transforms/MapElements.java |   8 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 301 +++++++--
 .../org/apache/beam/sdk/transforms/ParDo.java   | 277 +++++---
 .../apache/beam/sdk/transforms/Partition.java   |   4 +-
 .../sdk/transforms/SerializableFunction.java    |   2 +-
 .../sdk/transforms/display/DisplayData.java     | 518 ++++++++++-----
 .../sdk/transforms/reflect/DoFnInvokers.java    |  21 +-
 .../sdk/transforms/reflect/DoFnSignature.java   | 280 +++++++-
 .../sdk/transforms/reflect/DoFnSignatures.java  | 259 +++++++-
 .../beam/sdk/transforms/windowing/AfterAll.java |   2 +-
 .../windowing/AfterDelayFromFirstElement.java   |  15 +
 .../sdk/transforms/windowing/AfterEach.java     |   2 +-
 .../sdk/transforms/windowing/AfterFirst.java    |   2 +-
 .../sdk/transforms/windowing/AfterPane.java     |   7 +
 .../AfterSynchronizedProcessingTime.java        |   6 +-
 .../transforms/windowing/AfterWatermark.java    |  12 +-
 .../transforms/windowing/DefaultTrigger.java    |   2 +-
 .../beam/sdk/transforms/windowing/Never.java    |   8 +-
 .../transforms/windowing/OrFinallyTrigger.java  |  17 +-
 .../beam/sdk/transforms/windowing/PaneInfo.java |   2 +-
 .../sdk/transforms/windowing/Repeatedly.java    |  10 +-
 .../beam/sdk/transforms/windowing/Trigger.java  |  15 +-
 .../beam/sdk/transforms/windowing/Window.java   |   2 +-
 .../beam/sdk/transforms/windowing/WindowFn.java |   2 +-
 ...AttemptAndTimeBoundedExponentialBackOff.java |   2 +-
 .../apache/beam/sdk/util/CredentialFactory.java |   2 +-
 .../apache/beam/sdk/util/FinishedTriggers.java  |   8 +-
 .../apache/beam/sdk/util/IOChannelFactory.java  |   2 +-
 .../org/apache/beam/sdk/util/PathValidator.java |   6 +-
 .../beam/sdk/util/PerKeyCombineFnRunner.java    |  26 +-
 .../sdk/util/ReifyTimestampAndWindowsDoFn.java  |  16 +-
 .../apache/beam/sdk/util/ReshuffleTrigger.java  |   2 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |   2 +-
 .../java/org/apache/beam/sdk/util/Timer.java    |  56 ++
 .../apache/beam/sdk/util/TimerInternals.java    |   4 +-
 .../org/apache/beam/sdk/util/TimerSpec.java     |  30 +
 .../org/apache/beam/sdk/util/TimerSpecs.java    |  41 ++
 .../java/org/apache/beam/sdk/util/Timers.java   |  10 +-
 .../apache/beam/sdk/util/common/Reiterable.java |   2 +-
 .../apache/beam/sdk/util/common/Reiterator.java |   2 +-
 .../CopyOnAccessInMemoryStateInternals.java     |   2 +-
 .../beam/sdk/util/state/ReadableState.java      |  10 +-
 .../beam/sdk/util/state/StateContext.java       |   6 +-
 .../apache/beam/sdk/values/PCollectionView.java |   8 +-
 .../java/org/apache/beam/sdk/values/PInput.java |   6 +-
 .../org/apache/beam/sdk/values/POutput.java     |   8 +-
 .../java/org/apache/beam/sdk/values/PValue.java |   4 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   4 +-
 .../beam/sdk/coders/NullableCoderTest.java      |  11 +
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  12 +-
 .../org/apache/beam/sdk/io/AvroSourceTest.java  |  43 ++
 .../io/BoundedReadFromUnboundedSourceTest.java  |   4 +-
 .../beam/sdk/io/CompressedSourceTest.java       |   4 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |  30 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |  10 +-
 .../io/SerializableAvroCodecFactoryTest.java    |   8 +-
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  25 +
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   6 +-
 .../sdk/options/PipelineOptionsFactoryTest.java |  69 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |  49 +-
 .../options/PipelineOptionsValidatorTest.java   |  18 +-
 .../sdk/options/ProxyInvocationHandlerTest.java |  76 ++-
 .../beam/sdk/options/ValueProviderTest.java     |  13 +-
 .../beam/sdk/testing/BigqueryMatcherTest.java   | 176 +++++
 .../beam/sdk/transforms/CombineFnsTest.java     |   7 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   4 +-
 .../DoFnDelegatingAggregatorTest.java           |   5 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |  24 +-
 .../transforms/display/DisplayDataMatchers.java | 141 ++--
 .../display/DisplayDataMatchersTest.java        |  67 +-
 .../sdk/transforms/display/DisplayDataTest.java | 367 +++++++----
 .../DoFnSignaturesSplittableDoFnTest.java       |   6 +-
 .../transforms/reflect/DoFnSignaturesTest.java  | 645 ++++++++++++++++++-
 .../reflect/DoFnSignaturesTestUtils.java        |   5 +-
 .../sdk/transforms/windowing/WindowTest.java    |   4 +-
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |  24 +-
 .../beam/sdk/util/ExecutableTriggerTest.java    | 127 ++++
 .../sdk/util/FinishedTriggersBitSetTest.java    |  55 ++
 .../sdk/util/FinishedTriggersProperties.java    | 110 ++++
 .../beam/sdk/util/FinishedTriggersSetTest.java  |  60 ++
 .../beam/sdk/util/ReshuffleTriggerTest.java     |  67 ++
 sdks/java/extensions/join-library/pom.xml       |   2 +-
 sdks/java/extensions/pom.xml                    |   2 +-
 sdks/java/io/google-cloud-platform/pom.xml      |   2 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  12 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |   2 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |   2 +-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  39 +-
 sdks/java/io/hdfs/pom.xml                       |   2 +-
 sdks/java/io/jdbc/pom.xml                       |   2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  70 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  39 +-
 sdks/java/io/jms/pom.xml                        |   2 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  14 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  26 +-
 sdks/java/io/kinesis/pom.xml                    |   2 +-
 sdks/java/io/mongodb/pom.xml                    |   2 +-
 sdks/java/io/pom.xml                            |   2 +-
 sdks/java/java8tests/pom.xml                    |   2 +-
 .../PipelineOptionsFactoryJava8Test.java        |   8 +-
 sdks/java/maven-archetypes/examples/pom.xml     |   2 +-
 .../main/resources/archetype-resources/pom.xml  |   8 +-
 .../src/main/java/DebuggingWordCount.java       |   2 +-
 .../src/main/java/WindowedWordCount.java        |   2 +-
 .../src/main/java/WordCount.java                |   2 +-
 sdks/java/maven-archetypes/pom.xml              |   2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |   2 +-
 .../main/resources/archetype-resources/pom.xml  |   2 +-
 .../resources/projects/basic/reference/pom.xml  |   2 +-
 sdks/java/microbenchmarks/pom.xml               |   2 +-
 sdks/java/pom.xml                               |   2 +-
 sdks/pom.xml                                    |   2 +-
 298 files changed, 8342 insertions(+), 4136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6a971d6b/runners/pom.xml
----------------------------------------------------------------------
diff --cc runners/pom.xml
index ff800d1,b2b613f..203b951
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@@ -36,8 -36,8 +36,9 @@@
      <module>core-java</module>
      <module>direct-java</module>
      <module>flink</module>
+     <module>google-cloud-dataflow-java</module>
      <module>spark</module>
 +    <module>apex</module>
    </modules>
  
    <profiles>


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

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


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

Branch: refs/heads/apex-runner
Commit: b437c6b9e19754dc7624d043d23a4704329b83f7
Parents: c390a2a 59f042c
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 24 10:23:12 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Oct 24 10:23:12 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/core/ReduceFnRunner.java |  2 +-
 .../org/apache/beam/sdk/util/state/ReadableState.java     | 10 +++-------
 2 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[30/50] incubator-beam git commit: Fix Javadoc of ReadableState

Posted by th...@apache.org.
Fix Javadoc of ReadableState


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

Branch: refs/heads/apex-runner
Commit: caf1fb705e785cb312cedc334efa73e081469da0
Parents: a9a41eb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 24 10:35:41 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 24 10:39:07 2016 +0200

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/state/ReadableState.java     | 10 +++-------
 1 file changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/caf1fb70/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
index ea8a231..3b4cb7b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java
@@ -21,14 +21,10 @@ import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 
 /**
- * A {@code StateContents} is produced by the read methods on all {@link State} objects.
- * Calling {@link #read} returns the associated value.
+ * {@link State} that can be read via {@link #read()}.
  *
- * <p>This class is similar to {@link java.util.concurrent.Future}, but each invocation of
- * {@link #read} need not return the same value.
- *
- * <p>Getting the {@code StateContents} from a read method indicates the desire to eventually
- * read a value. Depending on the runner this may or may not immediately start the read.
+ * <p>Use {@link #readLater()} for marking several states for prefetching. Runners
+ * can potentially batch these into one read.
  *
  * @param <T> The type of value returned by {@link #read}.
  */


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

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


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

Branch: refs/heads/apex-runner
Commit: d285e85da7d24e7cfc3ede951f00248a11f21b76
Parents: 44b9f4e 7f82a57
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 15:04:15 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 15:04:15 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[19/50] incubator-beam git commit: Port some of ParDoTest to new DoFn

Posted by th...@apache.org.
Port some of ParDoTest to new DoFn


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

Branch: refs/heads/apex-runner
Commit: 3094017956913b583a9bd8be5ce685683b591669
Parents: c2e751f
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 12:35:03 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/transforms/DoFn.java   |  7 ++++---
 .../java/org/apache/beam/sdk/util/StringUtils.java  |  2 +-
 .../org/apache/beam/sdk/transforms/ParDoTest.java   | 16 +++++++---------
 3 files changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30940179/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 11ca853..018877f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -35,7 +35,6 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
@@ -216,9 +215,11 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
             String name, CombineFn<AggInputT, ?, AggOutputT> combiner);
 
     /**
-     * Sets up {@link Aggregator}s created by the {@link DoFn} so they are usable within this context.
+     * Sets up {@link Aggregator}s created by the {@link DoFn} so they are usable within this
+     * context.
      *
-     * <p>This method should be called by runners before the {@link StartBundle @StartBundle} method.
+     * <p>This method should be called by runners before the {@link StartBundle @StartBundle}
+     * method.
      */
     @Experimental(Kind.AGGREGATOR)
     protected final void setupDelegateAggregators() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30940179/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
index 4f81eef..1c52c1f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
@@ -96,7 +96,7 @@ public class StringUtils {
   }
 
   private static final String[] STANDARD_NAME_SUFFIXES =
-      new String[]{"OldDoFn", "Fn"};
+      new String[]{"OldDoFn", "DoFn", "Fn"};
 
   /**
    * Pattern to match a non-anonymous inner class.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/30940179/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 52244a0..d3ea9fb 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
@@ -51,12 +51,12 @@ import org.apache.beam.sdk.testing.NeedsRunner;
 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.OldDoFn.RequiresWindowAccess;
 import org.apache.beam.sdk.transforms.ParDo.Bound;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
 import org.apache.beam.sdk.transforms.display.DisplayDataMatchers;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
@@ -90,13 +90,11 @@ public class ParDoTest implements Serializable {
   @Rule
   public transient ExpectedException thrown = ExpectedException.none();
 
-  private static class PrintingOldDoFn extends OldDoFn<String, String> implements
-      RequiresWindowAccess {
-
-    @Override
-    public void processElement(ProcessContext c) {
+  private static class PrintingDoFn extends DoFn<String, String> {
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
       c.output(c.element() + ":" + c.timestamp().getMillis()
-          + ":" + c.window().maxTimestamp().getMillis());
+          + ":" + window.maxTimestamp().getMillis());
     }
   }
 
@@ -848,7 +846,7 @@ public class ParDoTest implements Serializable {
           output5.getName());
     }
 
-    assertEquals("ParDo(Printing)", ParDo.of(new PrintingOldDoFn()).getName());
+    assertEquals("ParDo(Printing)", ParDo.of(new PrintingDoFn()).getName());
 
     assertEquals(
         "ParMultiDo(SideOutputDummy)",
@@ -1381,7 +1379,7 @@ public class ParDoTest implements Serializable {
                     System.out.println("Finish: 3");
                   }
                 }))
-        .apply(ParDo.of(new PrintingOldDoFn()));
+        .apply(ParDo.of(new PrintingDoFn()));
 
     PAssert.that(output).satisfies(new Checker());
 


[43/50] incubator-beam git commit: OffsetBasedSource: allow empty sources

Posted by th...@apache.org.
OffsetBasedSource: allow empty sources

As one example, empty files exist.


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

Branch: refs/heads/apex-runner
Commit: 7e9ad9954e50bd01fba4cda84c182af895b2c23f
Parents: dfaf2a8
Author: Dan Halperin <dh...@google.com>
Authored: Fri Oct 21 12:21:45 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 13:16:50 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  4 +--
 .../beam/sdk/io/OffsetBasedSourceTest.java      | 30 +++++++++++++++-----
 2 files changed, 25 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7e9ad995/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
index 6e49cc3..e9a398d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
@@ -146,8 +146,8 @@ public abstract class OffsetBasedSource<T> extends BoundedSource<T> {
         this.endOffset >= 0,
         "End offset has value %s, must be non-negative", this.endOffset);
     checkArgument(
-        this.startOffset < this.endOffset,
-        "Start offset %s must be before end offset %s",
+        this.startOffset <= this.endOffset,
+        "Start offset %s may not be larger than end offset %s",
         this.startOffset, this.endOffset);
     checkArgument(
         this.minBundleSize >= 0,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7e9ad995/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
index 923b4b4..6584e5d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
@@ -62,7 +62,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+    public boolean producesSortedKeys(PipelineOptions options) {
       return false;
     }
 
@@ -85,7 +85,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public OffsetBasedReader<Integer> createReader(PipelineOptions options) throws IOException {
+    public CoarseRangeReader createReader(PipelineOptions options) {
       return new CoarseRangeReader(this);
     }
   }
@@ -105,7 +105,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean startImpl() throws IOException {
+    public boolean startImpl() {
       current = getCurrentSource().getStartOffset();
       while (current % granularity != 0) {
         ++current;
@@ -114,7 +114,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public boolean advanceImpl() throws IOException {
+    public boolean advanceImpl() {
       ++current;
       return true;
     }
@@ -130,7 +130,7 @@ public class OffsetBasedSourceTest {
     }
 
     @Override
-    public void close() throws IOException { }
+    public void close() { }
   }
 
   public static void assertSplitsAre(List<? extends OffsetBasedSource<?>> splits,
@@ -211,7 +211,7 @@ public class OffsetBasedSourceTest {
     // in the face of that.
     PipelineOptions options = PipelineOptionsFactory.create();
     CoarseRangeSource source = new CoarseRangeSource(13, 35, 1, 10);
-    try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
+    try (CoarseRangeReader reader = source.createReader(options)) {
       List<Integer> items = new ArrayList<>();
 
       assertEquals(0.0, reader.getFractionConsumed(), 1e-6);
@@ -304,7 +304,7 @@ public class OffsetBasedSourceTest {
   public void testSplitAtFraction() throws IOException {
     PipelineOptions options = PipelineOptionsFactory.create();
     CoarseRangeSource source = new CoarseRangeSource(13, 35, 1, 10);
-    try (CoarseRangeReader reader = (CoarseRangeReader) source.createReader(options)) {
+    try (CoarseRangeReader reader = source.createReader(options)) {
       List<Integer> originalItems = new ArrayList<>();
       assertTrue(reader.start());
       originalItems.add(reader.getCurrent());
@@ -341,4 +341,20 @@ public class OffsetBasedSourceTest {
     CoarseRangeSource original = new CoarseRangeSource(13, 35, 1, 10);
     assertSplitAtFractionExhaustive(original, options);
   }
+
+  @Test
+  public void testEmptyOffsetRange() throws Exception {
+    CoarseRangeSource empty = new CoarseRangeSource(0, 0, 1, 1);
+    try (CoarseRangeReader reader = empty.createReader(PipelineOptionsFactory.create())) {
+      assertEquals(0, reader.getSplitPointsConsumed());
+      assertEquals(OffsetBasedReader.SPLIT_POINTS_UNKNOWN, reader.getSplitPointsRemaining());
+      assertEquals(0.0, reader.getFractionConsumed(), 0.0001);
+
+      assertFalse(reader.start());
+
+      assertEquals(0, reader.getSplitPointsConsumed());
+      assertEquals(0, reader.getSplitPointsRemaining());
+      assertEquals(1.0, reader.getFractionConsumed(), 0.0001);
+    }
+  }
 }


[16/50] incubator-beam git commit: Add adapter from OldDoFn.RequiresWindowAccess to DoFn

Posted by th...@apache.org.
Add adapter from OldDoFn.RequiresWindowAccess to DoFn


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

Branch: refs/heads/apex-runner
Commit: 164ee56b41e01c0ee637eff24e23a814b5885e6f
Parents: a9a41eb
Author: Kenneth Knowles <kl...@google.com>
Authored: Sun Oct 23 19:45:18 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:50 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 89 +++++++++++++++++---
 1 file changed, 76 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/164ee56b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index e3cfc38..72c2965 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -77,19 +77,12 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
 
   public DoFn<InputT, OutputT> toDoFn() {
     if (this instanceof RequiresWindowAccess) {
-      throw new UnsupportedOperationException(
-          String.format(
-              "Cannot convert %s to %s because it implements %s."
-                  + " Please convert your %s to a %s directly.",
-              getClass(),
-              DoFn.class.getSimpleName(),
-              RequiresWindowAccess.class.getSimpleName(),
-              OldDoFn.class.getSimpleName(),
-              DoFn.class.getSimpleName()));
-    }
-
-    // No parameters as it just accesses `this`
-    return new AdaptedDoFn();
+      // No parameters as it just accesses `this`
+      return new AdaptedRequiresWindowAccessDoFn();
+    } else {
+      // No parameters as it just accesses `this`
+      return new AdaptedDoFn();
+    }
   }
 
   /**
@@ -770,4 +763,74 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
       return OldDoFn.this.getOutputTypeDescriptor();
     }
   }
+
+  /**
+   * A {@link ProcessContext} for an {@link OldDoFn} that implements
+   * {@link OldDoFn.RequiresWindowAcccess}, via a context for a proper {@link DoFn}.
+   */
+  private class AdaptedRequiresWindowAccessProcessContext extends AdaptedProcessContext {
+
+    private final BoundedWindow window;
+
+    public AdaptedRequiresWindowAccessProcessContext(
+        DoFn<InputT, OutputT>.ProcessContext newContext,
+        BoundedWindow window) {
+      super(newContext);
+      this.window = window;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return window;
+    }
+  }
+
+  private class AdaptedRequiresWindowAccessDoFn extends DoFn<InputT, OutputT> {
+
+    @Setup
+    public void setup() throws Exception {
+      OldDoFn.this.setup();
+    }
+
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) throws Exception {
+      OldDoFn.this.processElement(
+          OldDoFn.this.new AdaptedRequiresWindowAccessProcessContext(c, window));
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c));
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      OldDoFn.this.teardown();
+    }
+
+    @Override
+    public Duration getAllowedTimestampSkew() {
+      return OldDoFn.this.getAllowedTimestampSkew();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      OldDoFn.this.populateDisplayData(builder);
+    }
+
+    @Override
+    protected TypeDescriptor<InputT> getInputTypeDescriptor() {
+      return OldDoFn.this.getInputTypeDescriptor();
+    }
+
+    @Override
+    protected TypeDescriptor<OutputT> getOutputTypeDescriptor() {
+      return OldDoFn.this.getOutputTypeDescriptor();
+    }
+  }
 }


[17/50] incubator-beam git commit: Move AggregatorFactory to runners-core and deprecate SDK version

Posted by th...@apache.org.
Move AggregatorFactory to runners-core and deprecate SDK version


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

Branch: refs/heads/apex-runner
Commit: 08dd14981bad95a029be8ac758a6091c55850200
Parents: 139437b
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 11:49:07 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 19:52:51 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/core/AggregatorFactory.java    | 39 ++++++++++++++++++++
 .../apache/beam/sdk/transforms/Aggregator.java  | 11 ++----
 2 files changed, 42 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/08dd1498/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
new file mode 100644
index 0000000..153d30d
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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 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.
+ */
+public 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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/08dd1498/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 e8f6247..13bf322 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
@@ -73,8 +73,10 @@ public interface Aggregator<InputT, OutputT> {
   CombineFn<InputT, ?, OutputT> getCombineFn();
 
   /**
-   * A factory for creating aggregators.
+   * @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}.
@@ -87,11 +89,4 @@ public interface Aggregator<InputT, OutputT> {
         Class<?> fnClass, ExecutionContext.StepContext stepContext,
         String aggregatorName, CombineFn<InputT, AccumT, OutputT> combine);
   }
-
-  // TODO: Consider the following additional API conveniences:
-  // - In addition to createAggregator(), consider adding getAggregator() to
-  //   avoid the need to store the aggregator locally in a DoFn, i.e., create
-  //   if not already present.
-  // - Add a shortcut for the most common aggregator:
-  //   c.createAggregator("name", new Sum.SumIntegerFn()).
 }


[42/50] incubator-beam git commit: Closes #1169

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


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

Branch: refs/heads/apex-runner
Commit: dfaf2a896029aac62d9db1f0000141206d42209d
Parents: b437c6b 5d01184
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 12:22:29 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 12:22:29 2016 -0700

----------------------------------------------------------------------
 .../examples/src/main/resources/archetype-resources/pom.xml  | 8 ++++----
 .../starter/src/main/resources/archetype-resources/pom.xml   | 2 +-
 .../src/test/resources/projects/basic/reference/pom.xml      | 2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[25/50] incubator-beam git commit: Fix binding of aggregator creating in OldDoFn

Posted by th...@apache.org.
Fix binding of aggregator creating in OldDoFn


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

Branch: refs/heads/apex-runner
Commit: 4e185d0b0a7ec4c096380a25b9cbe4703621ec6b
Parents: 3094017
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 12:44:30 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4e185d0b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
index b269f47..a9f26a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
@@ -511,6 +511,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     public AdaptedContext(
         DoFn<InputT, OutputT>.Context newContext) {
       this.newContext = newContext;
+      super.setupDelegateAggregators();
     }
 
     @Override
@@ -541,7 +542,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return null;
+      return newContext.createAggregator(name, combiner);
     }
   }
 
@@ -625,7 +626,7 @@ public abstract class OldDoFn<InputT, OutputT> implements Serializable, HasDispl
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      return null;
+      return newContext.createAggregator(name, combiner);
     }
   }
 


[35/50] incubator-beam git commit: Add property name to RuntimeValueProvider

Posted by th...@apache.org.
Add property name to RuntimeValueProvider


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

Branch: refs/heads/apex-runner
Commit: 462a6417e925e2defb42d87191fb97353637ce7e
Parents: 43beb1e
Author: sammcveety <sa...@gmail.com>
Authored: Fri Oct 14 13:41:33 2016 -0400
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 08:15:05 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ProxyInvocationHandler.java   |  7 +++++--
 .../org/apache/beam/sdk/options/ValueProvider.java | 17 ++++++++++++++---
 .../apache/beam/sdk/options/ValueProviderTest.java |  7 +++++++
 3 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index 3e74916..f7c6da2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -493,12 +493,15 @@ class ProxyInvocationHandler implements InvocationHandler {
       }
     }
     if (method.getReturnType().equals(ValueProvider.class)) {
+      String propertyName = gettersToPropertyNames.get(method.getName());
       return defaultObject == null
         ? new RuntimeValueProvider(
-          method.getName(), (Class<? extends PipelineOptions>) method.getDeclaringClass(),
+          method.getName(), propertyName,
+          (Class<? extends PipelineOptions>) method.getDeclaringClass(),
           proxy.getOptionsId())
         : new RuntimeValueProvider(
-          method.getName(), (Class<? extends PipelineOptions>) method.getDeclaringClass(),
+          method.getName(), propertyName,
+          (Class<? extends PipelineOptions>) method.getDeclaringClass(),
           defaultObject, proxy.getOptionsId());
     } else if (defaultObject != null) {
       return defaultObject;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
index 04b6926..3f35a18 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
@@ -108,6 +108,7 @@ public interface ValueProvider<T> {
 
     private final Class<? extends PipelineOptions> klass;
     private final String methodName;
+    private final String propertyName;
     @Nullable
     private final T defaultValue;
     private final Long optionsId;
@@ -116,9 +117,10 @@ public interface ValueProvider<T> {
      * Creates a {@link RuntimeValueProvider} that will query the provided
      * {@code optionsId} for a value.
      */
-    RuntimeValueProvider(String methodName, Class<? extends PipelineOptions> klass,
-                         Long optionsId) {
+    RuntimeValueProvider(String methodName, String propertyName,
+                         Class<? extends PipelineOptions> klass, Long optionsId) {
       this.methodName = methodName;
+      this.propertyName = propertyName;
       this.klass = klass;
       this.defaultValue = null;
       this.optionsId = optionsId;
@@ -128,9 +130,11 @@ public interface ValueProvider<T> {
      * Creates a {@link RuntimeValueProvider} that will query the provided
      * {@code optionsId} for a value, or use the default if no value is available.
      */
-    RuntimeValueProvider(String methodName, Class<? extends PipelineOptions> klass,
+    RuntimeValueProvider(String methodName, String propertyName,
+                         Class<? extends PipelineOptions> klass,
       T defaultValue, Long optionsId) {
       this.methodName = methodName;
+      this.propertyName = propertyName;
       this.klass = klass;
       this.defaultValue = defaultValue;
       this.optionsId = optionsId;
@@ -167,6 +171,13 @@ public interface ValueProvider<T> {
       PipelineOptions options = optionsMap.get(optionsId);
       return options != null;
     }
+
+    /**
+     * Returns the property name that corresponds to this provider.
+     */
+    public String propertyName() {
+      return propertyName;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/462a6417/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
index 0d93b31..c9eb479 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
@@ -98,6 +98,13 @@ public class ValueProviderTest {
   }
 
   @Test
+  public void testRuntimePropertyName() {
+    TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
+    ValueProvider<String> provider = options.getFoo();
+    assertEquals("foo", ((RuntimeValueProvider) provider).propertyName());
+  }
+
+  @Test
   public void testDefaultRuntimeProvider() {
     TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
     ValueProvider<String> provider = options.getBar();


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

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


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

Branch: refs/heads/apex-runner
Commit: 6d9d8bc6fc2795fa25a9120324e652b9bec20ca8
Parents: a9a41eb e43228c
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 07:07:44 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:07:44 2016 -0700

----------------------------------------------------------------------
 .../streaming/StreamingEvaluationContext.java            | 11 ++++++++---
 .../translation/streaming/EmptyStreamAssertionTest.java  | 10 +++++++---
 .../streaming/ResumeFromCheckpointStreamingTest.java     |  2 +-
 .../streaming/SimpleStreamingWordCountTest.java          |  1 -
 .../translation/streaming/utils/PAssertStreaming.java    |  6 +++---
 5 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[04/50] incubator-beam git commit: Simplify DoFnRunners to be somewhat compositional

Posted by th...@apache.org.
Simplify DoFnRunners to be somewhat compositional


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

Branch: refs/heads/apex-runner
Commit: f043865ea9d842b85e21687c4503067a530960e6
Parents: 2d1df8b
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Oct 10 14:30:53 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:32:06 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunners.java   | 67 +++++++++-----------
 1 file changed, 29 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f043865e/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 7726374..41eb9d1 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
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -80,31 +81,15 @@ public class DoFnRunners {
    */
   public static <K, InputT, OutputT, W extends BoundedWindow>
       DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> lateDataDroppingRunner(
-          PipelineOptions options,
-          ReduceFnExecutor<K, InputT, OutputT, W> reduceFnExecutor,
-          SideInputReader sideInputReader,
-          OutputManager outputManager,
-          TupleTag<KV<K, OutputT>> mainOutputTag,
-          List<TupleTag<?>> sideOutputTags,
+          DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> wrappedRunner,
           StepContext stepContext,
-          AggregatorFactory aggregatorFactory,
-          WindowingStrategy<?, W> windowingStrategy) {
-    DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> simpleDoFnRunner =
-        simpleRunner(
-            options,
-            reduceFnExecutor.asDoFn(),
-            sideInputReader,
-            outputManager,
-            mainOutputTag,
-            sideOutputTags,
-            stepContext,
-            aggregatorFactory,
-            windowingStrategy);
+          WindowingStrategy<?, W> windowingStrategy,
+          Aggregator<Long, Long> droppedDueToLatenessAggregator) {
     return new LateDataDroppingDoFnRunner<>(
-        simpleDoFnRunner,
+        wrappedRunner,
         windowingStrategy,
         stepContext.timerInternals(),
-        reduceFnExecutor.getDroppedDueToLatenessAggregator());
+        droppedDueToLatenessAggregator);
   }
 
 
@@ -118,23 +103,8 @@ public class DoFnRunners {
       StepContext stepContext,
       AggregatorFactory aggregatorFactory,
       WindowingStrategy<?, ?> windowingStrategy) {
-    if (doFn instanceof ReduceFnExecutor) {
-      @SuppressWarnings("rawtypes")
-      ReduceFnExecutor fn = (ReduceFnExecutor) doFn;
-      @SuppressWarnings({"unchecked", "cast", "rawtypes"})
-      DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
-          options,
-          fn,
-          sideInputReader,
-          outputManager,
-          (TupleTag) mainOutputTag,
-          sideOutputTags,
-          stepContext,
-          aggregatorFactory,
-          (WindowingStrategy) windowingStrategy);
-      return runner;
-    }
-    return simpleRunner(
+
+    DoFnRunner<InputT, OutputT> doFnRunner = simpleRunner(
         options,
         doFn,
         sideInputReader,
@@ -144,5 +114,26 @@ public class DoFnRunners {
         stepContext,
         aggregatorFactory,
         windowingStrategy);
+
+    if (!(doFn instanceof ReduceFnExecutor)) {
+      return doFnRunner;
+    } else {
+      // When a DoFn is a ReduceFnExecutor, we know it has to have an aggregator for dropped
+      // elements and we also learn that for some K and V,
+      //   InputT = KeyedWorkItem<K, V>
+      //   OutputT = KV<K, V>
+
+      Aggregator<Long, Long> droppedDueToLatenessAggregator =
+          ((ReduceFnExecutor<?, ?, ?, ?>) doFn).getDroppedDueToLatenessAggregator();
+
+      @SuppressWarnings({"unchecked", "cast", "rawtypes"})
+      DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
+          (DoFnRunner) doFnRunner,
+          stepContext,
+          (WindowingStrategy) windowingStrategy,
+          droppedDueToLatenessAggregator);
+
+      return runner;
+    }
   }
 }


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

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


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

Branch: refs/heads/apex-runner
Commit: 153351126357da91d7adeab8339025914e18920c
Parents: 465c7d1 5226461
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Oct 20 19:52:50 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 19:52:50 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunners.java   | 189 +++++--
 .../beam/runners/core/SimpleDoFnRunner.java     | 337 +++++++-----
 .../beam/runners/core/SimpleOldDoFnRunner.java  | 521 +++++++++++++++++++
 .../beam/runners/core/SimpleDoFnRunnerTest.java |  88 ----
 .../runners/core/SimpleOldDoFnRunnerTest.java   |  88 ++++
 .../beam/runners/direct/ParDoEvaluator.java     |   3 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java | 183 +++++++
 .../org/apache/beam/sdk/transforms/ParDo.java   | 248 +++++----
 8 files changed, 1299 insertions(+), 358 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/15335112/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java
----------------------------------------------------------------------


[31/50] incubator-beam git commit: [BEAM-769] Spark: Make graceful stop the default.

Posted by th...@apache.org.
[BEAM-769] Spark: Make graceful stop the default.

streaming tests fail on "nothing processed" if runtime env. is slow because timeout
is hit before processing is done.

Keep "pumping-in" the last batch in a mocked stream to handle overflowing batches in case of a
graceful stop.

Change tests accordingly.


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

Branch: refs/heads/apex-runner
Commit: e43228c92cd9dd8a81a28940d419b721a2aeb2d8
Parents: a9a41eb
Author: Sela <an...@paypal.com>
Authored: Fri Oct 21 01:20:33 2016 +0300
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:07:27 2016 -0700

----------------------------------------------------------------------
 .../streaming/StreamingEvaluationContext.java            | 11 ++++++++---
 .../translation/streaming/EmptyStreamAssertionTest.java  | 10 +++++++---
 .../streaming/ResumeFromCheckpointStreamingTest.java     |  2 +-
 .../streaming/SimpleStreamingWordCountTest.java          |  1 -
 .../translation/streaming/utils/PAssertStreaming.java    |  6 +++---
 5 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
index 2652f2b..49afa26 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java
@@ -92,6 +92,7 @@ public class StreamingEvaluationContext extends EvaluationContext {
             WindowedValue.getValueOnlyCoder(coder);
         // create the DStream from queue
         Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>();
+        JavaRDD<WindowedValue<T>> lastRDD = null;
         for (Iterable<T> v : values) {
           Iterable<WindowedValue<T>> windowedValues =
               Iterables.transform(v, WindowingHelpers.<T>windowValueFunction());
@@ -99,10 +100,14 @@ public class StreamingEvaluationContext extends EvaluationContext {
               CoderHelpers.toByteArrays(windowedValues, windowCoder)).map(
                   CoderHelpers.fromByteFunction(windowCoder));
           rddQueue.offer(rdd);
+          lastRDD = rdd;
         }
-        // create dstream from queue, one at a time, no defaults
-        // mainly for unit test so no reason to have this configurable
-        dStream = jssc.queueStream(rddQueue, true);
+        // create dstream from queue, one at a time,
+        // with last as default in case batches repeat (graceful stops for example).
+        // if the stream is empty, avoid creating a default empty RDD.
+        // mainly for unit test so no reason to have this configurable.
+        dStream = lastRDD != null ? jssc.queueStream(rddQueue, true, lastRDD)
+            : jssc.queueStream(rddQueue, true);
       }
       return dStream;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
index 1560c66..4f2a7c6 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java
@@ -34,7 +34,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
-
+import org.junit.rules.TemporaryFolder;
 
 
 /**
@@ -48,11 +48,15 @@ public class EmptyStreamAssertionTest implements Serializable {
           + "     but: was <0>";
 
   @Rule
+  public TemporaryFolder checkpointParentDir = new TemporaryFolder();
+
+  @Rule
   public TestOptionsForStreaming commonOptions = new TestOptionsForStreaming();
 
   @Test
-  public void testFixedWindows() throws Exception {
-    SparkPipelineOptions options = commonOptions.getOptions();
+  public void testAssertion() throws Exception {
+    SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
+        checkpointParentDir.newFolder(getClass().getSimpleName()));
     Duration windowDuration = new Duration(options.getBatchIntervalMillis());
 
     Pipeline pipeline = Pipeline.create(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
index fc7fa34..fd1d11a 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
@@ -157,7 +157,7 @@ public class ResumeFromCheckpointStreamingTest {
 
     // requires a graceful stop so that checkpointing of the first run would finish successfully
     // before stopping and attempting to resume.
-    return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED, true);
+    return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
index 8f2dde3..4bc9a3d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java
@@ -62,7 +62,6 @@ public class SimpleStreamingWordCountTest implements Serializable {
 
   @Test
   public void testFixedWindows() throws Exception {
-
     SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(
         checkpointParentDir.newFolder(getClass().getSimpleName()));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e43228c9/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
index 3bf1ef6..496735d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java
@@ -81,13 +81,13 @@ public final class PAssertStreaming implements Serializable {
   }
 
   /**
-   * Default to stop immediately, useful for most tests except for the once that may require
-   * to finish writing checkpoints for example.
+   * Default to stop gracefully so that tests will finish processing even if slower for reasons
+   * such as a slow runtime environment.
    */
   public static <T> EvaluationResult runAndAssertContents(Pipeline p,
                                                           PCollection<T> actual,
                                                           T[] expected) {
-    return runAndAssertContents(p, actual, expected, false);
+    return runAndAssertContents(p, actual, expected, true);
   }
 
   private static class AssertDoFn<T> extends OldDoFn<Iterable<T>, Void> {


[24/50] incubator-beam git commit: Fix flipped conditional in SimpleDoFnRunner

Posted by th...@apache.org.
Fix flipped conditional in SimpleDoFnRunner


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

Branch: refs/heads/apex-runner
Commit: f0c8d30d61c631fea642becde38b9cc52e873b5e
Parents: 043ebec
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Oct 21 13:53:53 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sun Oct 23 21:04:17 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0c8d30d/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 1cf56a6..dec9905 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
@@ -125,11 +125,11 @@ public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, Out
   @Override
   public void processElement(WindowedValue<InputT> compressedElem) {
     if (observesWindow) {
-      invokeProcessElement(compressedElem);
-    } else {
       for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {
         invokeProcessElement(elem);
       }
+    } else {
+      invokeProcessElement(compressedElem);
     }
   }
 


[02/50] incubator-beam git commit: Add SimpleDoFnRunner for new DoFn and construction via DoFnRunners

Posted by th...@apache.org.
Add SimpleDoFnRunner for new DoFn and construction via DoFnRunners


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

Branch: refs/heads/apex-runner
Commit: fe0b7bff6bacbd4c5920dc547a014dd6d8d2eabb
Parents: 8ff6c9d
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Oct 19 20:38:23 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Oct 20 18:32:06 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/core/DoFnRunners.java   | 120 +++-
 .../beam/runners/core/SimpleDoFnRunner.java     | 584 +++++++++++++++++++
 2 files changed, 703 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fe0b7bff/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 13cd161..15f7b7e 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
@@ -18,11 +18,12 @@
 package org.apache.beam.runners.core;
 
 import java.util.List;
-
 import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor;
+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.Aggregator.AggregatorFactory;
+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;
@@ -48,6 +49,35 @@ public class DoFnRunners {
   }
 
   /**
+   * Returns an implementation of {@link DoFnRunner} that for a {@link DoFn}.
+   *
+   * <p>If the {@link DoFn} observes the window, this runner will explode the windows of a
+   * compressed {@link WindowedValue}. It is the responsibility of the runner to perform any key
+   * partitioning needed, etc.
+   */
+  static <InputT, OutputT> DoFnRunner<InputT, OutputT> simpleRunner(
+      PipelineOptions options,
+      DoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    return new SimpleDoFnRunner<>(
+        options,
+        fn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy);
+  }
+
+  /**
    * Returns a basic implementation of {@link DoFnRunner} that works for most {@link OldDoFn DoFns}.
    *
    * <p>It invokes {@link OldDoFn#processElement} for each input.
@@ -92,7 +122,45 @@ public class DoFnRunners {
         droppedDueToLatenessAggregator);
   }
 
+  /**
+   * Creates a {@link DoFnRunner} for the provided {@link DoFn}.
+   */
+  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
+      PipelineOptions options,
+      DoFn<InputT, OutputT> doFn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
 
+    // Unlike for OldDoFn, there is no ReduceFnExecutor that is a new DoFn,
+    // and window-exploded processing is achieved within the simple runner
+    return simpleRunner(
+        options,
+        doFn,
+        sideInputReader,
+        outputManager,
+        mainOutputTag,
+        sideOutputTags,
+        stepContext,
+        aggregatorFactory,
+        windowingStrategy);
+  }
+
+  /**
+   * Creates a {@link DoFnRunner} for the provided {@link OldDoFn}.
+   *
+   * <p>In particular, if the {@link OldDoFn} is a {@link ReduceFnExecutor}, a specialized
+   * implementation detail of streaming {@link GroupAlsoByWindow}, then it will create a special
+   * runner that operates on {@link KeyedWorkItem KeyedWorkItems}, drops late data and counts
+   * dropped elements.
+   *
+   * @deprecated please port uses of {@link OldDoFn} to use {@link DoFn}
+   */
+  @Deprecated
   public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
       PipelineOptions options,
       OldDoFn<InputT, OutputT> doFn,
@@ -136,4 +204,54 @@ public class DoFnRunners {
       return runner;
     }
   }
+
+  /**
+   * Creates the right kind of {@link DoFnRunner} for an object that can be either a {@link DoFn} or
+   * {@link OldDoFn}. This can be used so that the client need not explicitly reference either such
+   * class, but merely deserialize a payload and pass it to this method.
+   *
+   * @deprecated for migration purposes only for services where users may still submit either {@link
+   *     OldDoFn} or {@link DoFn}. If you know that you have a {@link DoFn} then you should use the
+   *     variant for that instead.
+   */
+  @Deprecated
+  public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
+      PipelineOptions options,
+      Object deserializedFn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    if (deserializedFn instanceof DoFn) {
+      return createDefault(
+          options,
+          (DoFn) deserializedFn,
+          sideInputReader,
+          outputManager,
+          mainOutputTag,
+          sideOutputTags,
+          stepContext,
+          aggregatorFactory,
+          windowingStrategy);
+    } else if (deserializedFn instanceof OldDoFn) {
+      return createDefault(
+          options,
+          (OldDoFn) deserializedFn,
+          sideInputReader,
+          outputManager,
+          mainOutputTag,
+          sideOutputTags,
+          stepContext,
+          aggregatorFactory,
+          windowingStrategy);
+    } else {
+      throw new IllegalArgumentException(String.format("Cannot create %s for %s of class %s",
+          DoFnRunner.class.getSimpleName(),
+          deserializedFn,
+          deserializedFn.getClass()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/fe0b7bff/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
new file mode 100644
index 0000000..d5ecbc9
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -0,0 +1,584 @@
+/*
+ * 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 com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.InputProvider;
+import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.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.TimerInternals;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+/**
+ * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the {@link DoFn} (main) input elements
+ * @param <OutputT> the type of the {@link DoFn} (main) output elements
+ */
+public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+  /** The {@link DoFn} being run. */
+  private final DoFn<InputT, OutputT> fn;
+
+  /** The {@link DoFnInvoker} being run. */
+  private final DoFnInvoker<InputT, OutputT> invoker;
+
+  /** The context used for running the {@link DoFn}. */
+  private final DoFnContext<InputT, OutputT> context;
+
+  private final OutputManager outputManager;
+
+  private final TupleTag<OutputT> mainOutputTag;
+
+  private final boolean ignoresWindow;
+
+  public SimpleDoFnRunner(
+      PipelineOptions options,
+      DoFn<InputT, OutputT> fn,
+      SideInputReader sideInputReader,
+      OutputManager outputManager,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      StepContext stepContext,
+      AggregatorFactory aggregatorFactory,
+      WindowingStrategy<?, ?> windowingStrategy) {
+    this.fn = fn;
+    this.ignoresWindow =
+        !DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().usesSingleWindow();
+    this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn);
+    this.outputManager = outputManager;
+    this.mainOutputTag = mainOutputTag;
+    this.context =
+        new DoFnContext<>(
+            options,
+            fn,
+            sideInputReader,
+            outputManager,
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            aggregatorFactory,
+            windowingStrategy == null ? null : windowingStrategy.getWindowFn());
+  }
+
+  @Override
+  public void startBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeStartBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  @Override
+  public void processElement(WindowedValue<InputT> compressedElem) {
+    if (ignoresWindow) {
+      invokeProcessElement(compressedElem);
+    } else {
+      for (WindowedValue<InputT> elem : compressedElem.explodeWindows()) {
+        invokeProcessElement(elem);
+      }
+    }
+  }
+
+  private void invokeProcessElement(WindowedValue<InputT> elem) {
+    final DoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
+
+    // Note that if the element must be exploded into all its windows, that has to be done outside
+    // of this runner.
+    final DoFn.ExtraContextFactory<InputT, OutputT> extraContextFactory =
+        createExtraContextFactory(elem);
+
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeProcessElement(processContext, extraContextFactory);
+    } catch (Exception ex) {
+      throw wrapUserCodeException(ex);
+    }
+  }
+
+  @Override
+  public void finishBundle() {
+    // This can contain user code. Wrap it in case it throws an exception.
+    try {
+      invoker.invokeFinishBundle(context);
+    } catch (Throwable t) {
+      // Exception in user code.
+      throw wrapUserCodeException(t);
+    }
+  }
+
+  /** Returns a new {@link DoFn.ProcessContext} for the given element. */
+  private DoFn<InputT, OutputT>.ProcessContext createProcessContext(WindowedValue<InputT> elem) {
+    return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
+  }
+
+  private DoFn.ExtraContextFactory<InputT, OutputT> createExtraContextFactory(
+      WindowedValue<InputT> elem) {
+    return new DoFnExtraContextFactory<InputT, OutputT>(elem.getWindows(), elem.getPane());
+  }
+
+  private RuntimeException wrapUserCodeException(Throwable t) {
+    throw UserCodeException.wrapIf(!isSystemDoFn(), t);
+  }
+
+  private boolean isSystemDoFn() {
+    return invoker.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+  }
+
+  /**
+   * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}.
+   *
+   * @param <InputT> the type of the {@link DoFn} (main) input elements
+   * @param <OutputT> the type of the {@link DoFn} (main) output elements
+   */
+  private static class DoFnContext<InputT, OutputT> extends DoFn<InputT, OutputT>.Context {
+    private static final int MAX_SIDE_OUTPUTS = 1000;
+
+    final PipelineOptions options;
+    final DoFn<InputT, OutputT> fn;
+    final SideInputReader sideInputReader;
+    final OutputManager outputManager;
+    final TupleTag<OutputT> mainOutputTag;
+    final StepContext stepContext;
+    final AggregatorFactory aggregatorFactory;
+    final WindowFn<?, ?> windowFn;
+
+    /**
+     * The set of known output tags, some of which may be undeclared, so we can throw an exception
+     * when it exceeds {@link #MAX_SIDE_OUTPUTS}.
+     */
+    private Set<TupleTag<?>> outputTags;
+
+    public DoFnContext(
+        PipelineOptions options,
+        DoFn<InputT, OutputT> fn,
+        SideInputReader sideInputReader,
+        OutputManager outputManager,
+        TupleTag<OutputT> mainOutputTag,
+        List<TupleTag<?>> sideOutputTags,
+        StepContext stepContext,
+        AggregatorFactory aggregatorFactory,
+        WindowFn<?, ?> windowFn) {
+      fn.super();
+      this.options = options;
+      this.fn = fn;
+      this.sideInputReader = sideInputReader;
+      this.outputManager = outputManager;
+      this.mainOutputTag = mainOutputTag;
+      this.outputTags = Sets.newHashSet();
+
+      outputTags.add(mainOutputTag);
+      for (TupleTag<?> sideOutputTag : sideOutputTags) {
+        outputTags.add(sideOutputTag);
+      }
+
+      this.stepContext = stepContext;
+      this.aggregatorFactory = aggregatorFactory;
+      this.windowFn = windowFn;
+    }
+
+    //////////////////////////////////////////////////////////////////////////////
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
+        T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
+      final Instant inputTimestamp = timestamp;
+
+      if (timestamp == null) {
+        timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      }
+
+      if (windows == null) {
+        try {
+          // The windowFn can never succeed at accessing the element, so its type does not
+          // matter here
+          @SuppressWarnings("unchecked")
+          WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
+          windows =
+              objectWindowFn.assignWindows(
+                  objectWindowFn.new AssignContext() {
+                    @Override
+                    public Object element() {
+                      throw new UnsupportedOperationException(
+                          "WindowFn attempted to access input element when none was available");
+                    }
+
+                    @Override
+                    public Instant timestamp() {
+                      if (inputTimestamp == null) {
+                        throw new UnsupportedOperationException(
+                            "WindowFn attempted to access input timestamp when none was available");
+                      }
+                      return inputTimestamp;
+                    }
+
+                    @Override
+                    public W window() {
+                      throw new UnsupportedOperationException(
+                          "WindowFn attempted to access input windows when none were available");
+                    }
+                  });
+        } catch (Exception e) {
+          throw UserCodeException.wrap(e);
+        }
+      }
+
+      return WindowedValue.of(output, timestamp, windows, pane);
+    }
+
+    public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+      if (!sideInputReader.contains(view)) {
+        throw new IllegalArgumentException("calling sideInput() with unknown view");
+      }
+      BoundedWindow sideInputWindow =
+          view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+      return sideInputReader.get(view, sideInputWindow);
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
+      outputManager.output(mainOutputTag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteOutput(windowedElem);
+      }
+    }
+
+    private <T> void sideOutputWindowedValue(
+        TupleTag<T> tag,
+        T output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
+    }
+
+    private <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
+      if (!outputTags.contains(tag)) {
+        // This tag wasn't declared nor was it seen before during this execution.
+        // Thus, this must be a new, undeclared and unconsumed output.
+        // To prevent likely user errors, enforce the limit on the number of side
+        // outputs.
+        if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
+          throw new IllegalArgumentException(
+              "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
+        }
+        outputTags.add(tag);
+      }
+
+      outputManager.output(tag, windowedElem);
+      if (stepContext != null) {
+        stepContext.noteSideOutput(tag, windowedElem);
+      }
+    }
+
+    // Following implementations of output, outputWithTimestamp, and sideOutput
+    // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by
+    // ProcessContext's versions in DoFn.processElement.
+    @Override
+    public void output(OutputT output) {
+      outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      checkNotNull(combiner, "Combiner passed to createAggregator cannot be null");
+      return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner);
+    }
+  }
+
+  /**
+   * A concrete implementation of {@link DoFn.ProcessContext} used for running a {@link DoFn} over a
+   * single element.
+   *
+   * @param <InputT> the type of the {@link DoFn} (main) input elements
+   * @param <OutputT> the type of the {@link DoFn} (main) output elements
+   */
+  private static class DoFnProcessContext<InputT, OutputT>
+      extends DoFn<InputT, OutputT>.ProcessContext {
+
+    final DoFn<InputT, OutputT> fn;
+    final DoFnContext<InputT, OutputT> context;
+    final WindowedValue<InputT> windowedValue;
+
+    public DoFnProcessContext(
+        DoFn<InputT, OutputT> fn,
+        DoFnContext<InputT, OutputT> context,
+        WindowedValue<InputT> windowedValue) {
+      fn.super();
+      this.fn = fn;
+      this.context = context;
+      this.windowedValue = windowedValue;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return context.getPipelineOptions();
+    }
+
+    @Override
+    public InputT element() {
+      return windowedValue.getValue();
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      checkNotNull(view, "View passed to sideInput cannot be null");
+      Iterator<? extends BoundedWindow> windowIter = windows().iterator();
+      BoundedWindow window;
+      if (!windowIter.hasNext()) {
+        if (context.windowFn instanceof GlobalWindows) {
+          // TODO: Remove this once GroupByKeyOnly no longer outputs elements
+          // without windows
+          window = GlobalWindow.INSTANCE;
+        } else {
+          throw new IllegalStateException(
+              "sideInput called when main input element is not in any windows");
+        }
+      } else {
+        window = windowIter.next();
+        if (windowIter.hasNext()) {
+          throw new IllegalStateException(
+              "sideInput called when main input element is in multiple windows");
+        }
+      }
+      return context.sideInput(view, window);
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return windowedValue.getPane();
+    }
+
+    @Override
+    public void output(OutputT output) {
+      context.outputWindowedValue(windowedValue.withValue(output));
+    }
+
+    @Override
+    public void outputWithTimestamp(OutputT output, Instant timestamp) {
+      checkTimestamp(timestamp);
+      context.outputWindowedValue(
+          output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    void outputWindowedValue(
+        OutputT output,
+        Instant timestamp,
+        Collection<? extends BoundedWindow> windows,
+        PaneInfo pane) {
+      context.outputWindowedValue(output, timestamp, windows, pane);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkTimestamp(timestamp);
+      context.sideOutputWindowedValue(
+          tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+    }
+
+    @Override
+    public Instant timestamp() {
+      return windowedValue.getTimestamp();
+    }
+
+    public Collection<? extends BoundedWindow> windows() {
+      return windowedValue.getWindows();
+    }
+
+    private void checkTimestamp(Instant timestamp) {
+      if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
+        throw new IllegalArgumentException(
+            String.format(
+                "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+                    + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+                    + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed "
+                    + "skew.",
+                timestamp,
+                windowedValue.getTimestamp(),
+                PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
+      }
+    }
+
+    @Override
+    protected <AggregatorInputT, AggregatorOutputT>
+        Aggregator<AggregatorInputT, AggregatorOutputT> createAggregator(
+            String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
+      return context.createAggregator(name, combiner);
+    }
+  }
+
+  private class DoFnExtraContextFactory<InputT, OutputT>
+      implements DoFn.ExtraContextFactory<InputT, OutputT> {
+
+    /** The windows of the current element. */
+    private final Collection<? extends BoundedWindow> windows;
+
+    /** The pane of the current element. */
+    private final PaneInfo pane;
+
+    public DoFnExtraContextFactory(Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+      this.windows = windows;
+      this.pane = pane;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return null;
+    }
+
+    @Override
+    public InputProvider<InputT> inputProvider() {
+      throw new UnsupportedOperationException("InputProvider parameters are not supported.");
+    }
+
+    @Override
+    public OutputReceiver<OutputT> outputReceiver() {
+      throw new UnsupportedOperationException("OutputReceiver parameters are not supported.");
+    }
+
+    @Override
+    public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+      throw new UnsupportedOperationException("RestrictionTracker parameters are not supported.");
+    }
+
+    @Override
+    public WindowingInternals<InputT, OutputT> windowingInternals() {
+      return new WindowingInternals<InputT, OutputT>() {
+        @Override
+        public Collection<? extends BoundedWindow> windows() {
+          return windows;
+        }
+
+        @Override
+        public PaneInfo pane() {
+          return pane;
+        }
+
+        @Override
+        public TimerInternals timerInternals() {
+          return context.stepContext.timerInternals();
+        }
+
+        @Override
+        public <T> void writePCollectionViewData(
+            TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder)
+            throws IOException {
+          @SuppressWarnings("unchecked")
+          Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
+
+          context.stepContext.writePCollectionViewData(
+              tag,
+              data,
+              IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
+              window(),
+              windowCoder);
+        }
+
+        @Override
+        public StateInternals<?> stateInternals() {
+          return context.stepContext.stateInternals();
+        }
+
+        @Override
+        public void outputWindowedValue(
+            OutputT output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {}
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+          return context.sideInput(view, mainInputWindow);
+        }
+      };
+    }
+  }
+}


[41/50] incubator-beam git commit: [BEAM-806] update archetype poms in master

Posted by th...@apache.org.
[BEAM-806] update archetype poms in master

Per the linked JIRA, maven release:branch did not do the right thing.


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

Branch: refs/heads/apex-runner
Commit: 5d011847a658ff1cbd1e4eb0e0c581ceecbfc2c0
Parents: b437c6b
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 11:12:41 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 11:12:41 2016 -0700

----------------------------------------------------------------------
 .../examples/src/main/resources/archetype-resources/pom.xml  | 8 ++++----
 .../starter/src/main/resources/archetype-resources/pom.xml   | 2 +-
 .../src/test/resources/projects/basic/reference/pom.xml      | 2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
index 147c2dc..7926578 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
@@ -108,21 +108,21 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- Adds a dependency on a specific version of the Dataflow runnner. -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-direct-java</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
       <scope>runtime</scope>
     </dependency>
 
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
       <scope>runtime</scope>
     </dependency>
 
@@ -130,7 +130,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
index 0f7d3de..5287d1f 100644
--- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml
@@ -55,7 +55,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- slf4j API frontend binding with JUL backend -->

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5d011847/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
index 66091fe..cce2f99 100644
--- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
+++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml
@@ -55,7 +55,7 @@
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
-      <version>0.3.0-incubating-SNAPSHOT</version>
+      <version>0.4.0-incubating-SNAPSHOT</version>
     </dependency>
 
     <!-- slf4j API frontend binding with JUL backend -->


[47/50] incubator-beam git commit: pom.xml: make Dataflow runner like every other runner

Posted by th...@apache.org.
pom.xml: make Dataflow runner like every other runner

Pom dependencies no longer require it to be treated specially.


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

Branch: refs/heads/apex-runner
Commit: 6291a6f674a98b8898d807e72ebf2ad5cc5d3ce7
Parents: d285e85
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 13:16:01 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 18:03:28 2016 -0700

----------------------------------------------------------------------
 pom.xml         | 3 ---
 runners/pom.xml | 1 +
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6291a6f6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2338f55..df388d4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -139,9 +139,6 @@
 
   <modules>
     <module>sdks</module>
-    <!-- Expose Dataflow runner as top level module to satisfy dependencies in sdks/java/maven-archetypes 
-      and examples/java. Until these are refactored out, we need to modify the build order. -->
-    <module>runners/google-cloud-dataflow-java</module>
     <module>runners</module>
     <!-- sdks/java/maven-archetypes has several dependencies on the DataflowPipelineRunner. 
       Until these are refactored out or a released artifact exists, we need to modify the 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6291a6f6/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 68cf29c..b2b613f 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -36,6 +36,7 @@
     <module>core-java</module>
     <module>direct-java</module>
     <module>flink</module>
+    <module>google-cloud-dataflow-java</module>
     <module>spark</module>
   </modules>
 


[36/50] incubator-beam git commit: Closes #1107

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


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

Branch: refs/heads/apex-runner
Commit: 69ae8bd7656437497c2371deaaa26793f17989e2
Parents: 43beb1e 462a641
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 08:15:06 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 08:15:06 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ProxyInvocationHandler.java   |  7 +++++--
 .../org/apache/beam/sdk/options/ValueProvider.java | 17 ++++++++++++++---
 .../apache/beam/sdk/options/ValueProviderTest.java |  7 +++++++
 3 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[33/50] incubator-beam git commit: Closes #1091

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


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

Branch: refs/heads/apex-runner
Commit: 43beb1ea337630121bb5972e31f6cff9b04004c1
Parents: 6d9d8bc 75d137b
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 07:37:41 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 07:37:41 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  7 ++--
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 35 +++++++++++++++-----
 2 files changed, 31 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[44/50] incubator-beam git commit: Closes #1168

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


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

Branch: refs/heads/apex-runner
Commit: 44b9f4ebbc3229d8a9ec46b0c377e57ad0176dbe
Parents: dfaf2a8 7e9ad99
Author: Dan Halperin <dh...@google.com>
Authored: Mon Oct 24 13:16:51 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Oct 24 13:16:51 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/io/OffsetBasedSource.java   |  4 +--
 .../beam/sdk/io/OffsetBasedSourceTest.java      | 30 +++++++++++++++-----
 2 files changed, 25 insertions(+), 9 deletions(-)
----------------------------------------------------------------------